From f26f0b66261c6ccea6de5824ef7997f108f3e7b7 Mon Sep 17 00:00:00 2001 From: "Colin P. McCabe" Date: Mon, 29 Jul 2024 15:57:40 -0700 Subject: [PATCH 001/123] tests/kafkatest/version.py: Add 3.9.0 as DEV_VERSION --- tests/kafkatest/version.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index bacbedbb48..35d93c5370 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -122,7 +122,7 @@ def get_version(node=None): return DEV_BRANCH DEV_BRANCH = KafkaVersion("dev") -DEV_VERSION = KafkaVersion("3.8.0-SNAPSHOT") +DEV_VERSION = KafkaVersion("3.9.0-SNAPSHOT") # This should match the LATEST_PRODUCTION version defined in MetadataVersion.java LATEST_STABLE_METADATA_VERSION = "3.8" @@ -274,3 +274,7 @@ def get_version(node=None): # 3.8.x version V_3_8_0 = KafkaVersion("3.8.0") LATEST_3_8 = V_3_8_0 + +# 3.9.x version +V_3_9_0 = KafkaVersion("3.9.0") +LATEST_3_9 = V_3_9_0 From a4ea9aec73a3da470d079bad82d92810cac49d55 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Tue, 30 Jul 2024 05:17:15 +0200 Subject: [PATCH 002/123] KAFKA-16448: Add ErrorHandlerContext in production exception handler (#16433) This PR is part of KIP-1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing. This PR expose the new ErrorHandlerContext as a parameter to the Production exception handler and deprecate the previous handle signature. Co-authored-by: Dabz Co-authored-by: loicgreffier Reviewers: Bruno Cadonna , Matthias J. Sax --- .../DefaultProductionExceptionHandler.java | 8 + .../streams/errors/ErrorHandlerContext.java | 28 --- .../errors/ProductionExceptionHandler.java | 49 ++++- .../internals/DefaultErrorHandlerContext.java | 16 -- .../processor/internals/CorruptedRecord.java | 2 +- .../internals/GlobalStateManagerImpl.java | 3 +- .../internals/GlobalStateUpdateTask.java | 3 +- .../processor/internals/ProcessorAdapter.java | 3 +- .../internals/ProcessorContextImpl.java | 3 +- .../processor/internals/ProcessorNode.java | 2 - .../internals/ProcessorRecordContext.java | 18 +- .../internals/RecordCollectorImpl.java | 188 +++++++++++++----- .../processor/internals/RecordQueue.java | 2 +- .../streams/processor/internals/SinkNode.java | 3 +- .../processor/internals/StampedRecord.java | 18 +- .../processor/internals/StreamTask.java | 6 +- ...aysContinueProductionExceptionHandler.java | 44 ---- ...essingExceptionHandlerIntegrationTest.java | 2 - .../internals/ProcessorNodeTest.java | 7 +- .../internals/RecordCollectorTest.java | 187 ++++++++++++++--- .../test/InternalMockProcessorContext.java | 7 + 21 files changed, 367 insertions(+), 232 deletions(-) delete mode 100644 streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java index 33a95f4b44..0896114cf2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java @@ -25,12 +25,20 @@ * happens while attempting to produce result records. */ public class DefaultProductionExceptionHandler implements ProductionExceptionHandler { + @Deprecated @Override public ProductionExceptionHandlerResponse handle(final ProducerRecord record, final Exception exception) { return ProductionExceptionHandlerResponse.FAIL; } + @Override + public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { + return ProductionExceptionHandlerResponse.FAIL; + } + @Override public void configure(final Map configs) { // ignore diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java index 0c50547549..6c5e4f1959 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java @@ -89,34 +89,6 @@ public interface ErrorHandlerContext { */ Headers headers(); - /** - * Return the non-deserialized byte[] of the input message key if the context has been triggered by a message. - * - *

If this method is invoked within a {@link Punctuator#punctuate(long) - * punctuation callback}, or while processing a record that was forwarded by a punctuation - * callback, it will return {@code null}. - * - *

If this method is invoked in a sub-topology due to a repartition, the returned key would be one sent - * to the repartition topic. - * - * @return the raw byte of the key of the source message - */ - byte[] sourceRawKey(); - - /** - * Return the non-deserialized byte[] of the input message value if the context has been triggered by a message. - * - *

If this method is invoked within a {@link Punctuator#punctuate(long) - * punctuation callback}, or while processing a record that was forwarded by a punctuation - * callback, it will return {@code null}. - * - *

If this method is invoked in a sub-topology due to a repartition, the returned value would be one sent - * to the repartition topic. - * - * @return the raw byte of the value of the source message - */ - byte[] sourceRawValue(); - /** * Return the current processor node ID. * diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java index 6ae0170bfc..25aa00f7a9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java @@ -30,9 +30,28 @@ public interface ProductionExceptionHandler extends Configurable { * * @param record The record that failed to produce * @param exception The exception that occurred during production + * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead. */ - ProductionExceptionHandlerResponse handle(final ProducerRecord record, - final Exception exception); + @Deprecated + default ProductionExceptionHandlerResponse handle(final ProducerRecord record, + final Exception exception) { + throw new UnsupportedOperationException(); + } + + /** + * Inspect a record that we attempted to produce, and the exception that resulted + * from attempting to produce it and determine whether or not to continue processing. + * + * @param context The error handler context metadata + * @param record The record that failed to produce + * @param exception The exception that occurred during production + */ + @SuppressWarnings("deprecation") + default ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { + return handle(record, exception); + } /** * Handles serialization exception and determine if the process should continue. The default implementation is to @@ -40,12 +59,31 @@ ProductionExceptionHandlerResponse handle(final ProducerRecord r * * @param record the record that failed to serialize * @param exception the exception that occurred during serialization + * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead. */ + @Deprecated default ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, final Exception exception) { return ProductionExceptionHandlerResponse.FAIL; } + /** + * Handles serialization exception and determine if the process should continue. The default implementation is to + * fail the process. + * + * @param context the error handler context metadata + * @param record the record that failed to serialize + * @param exception the exception that occurred during serialization + * @param origin the origin of the serialization exception + */ + @SuppressWarnings("deprecation") + default ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { + return handleSerializationException(record, exception); + } + enum ProductionExceptionHandlerResponse { /* continue processing */ CONTINUE(0, "CONTINUE"), @@ -68,4 +106,11 @@ enum ProductionExceptionHandlerResponse { this.name = name; } } + + enum SerializationExceptionOrigin { + /* serialization exception occurred during serialization of the key */ + KEY, + /* serialization exception occurred during serialization of the value */ + VALUE + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java index ff79860d77..aa066fb6da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java @@ -28,8 +28,6 @@ public class DefaultErrorHandlerContext implements ErrorHandlerContext { private final int partition; private final long offset; private final Headers headers; - private final byte[] sourceRawKey; - private final byte[] sourceRawValue; private final String processorNodeId; private final TaskId taskId; @@ -37,16 +35,12 @@ public DefaultErrorHandlerContext(final String topic, final int partition, final long offset, final Headers headers, - final byte[] sourceRawKey, - final byte[] sourceRawValue, final String processorNodeId, final TaskId taskId) { this.topic = topic; this.partition = partition; this.offset = offset; this.headers = headers; - this.sourceRawKey = sourceRawKey; - this.sourceRawValue = sourceRawValue; this.processorNodeId = processorNodeId; this.taskId = taskId; } @@ -71,16 +65,6 @@ public Headers headers() { return headers; } - @Override - public byte[] sourceRawKey() { - return sourceRawKey; - } - - @Override - public byte[] sourceRawValue() { - return sourceRawValue; - } - @Override public String processorNodeId() { return processorNodeId; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java index 1bc8cb5109..d31a29883c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java @@ -28,7 +28,7 @@ public class CorruptedRecord extends StampedRecord { CorruptedRecord(final ConsumerRecord rawRecord) { - super(rawRecord, ConsumerRecord.NO_TIMESTAMP, rawRecord); + super(rawRecord, ConsumerRecord.NO_TIMESTAMP); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index b1263ddc58..6b7214a9ed 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -307,8 +307,7 @@ private void reprocessState(final List topicPartitions, record.offset(), record.partition(), record.topic(), - record.headers(), - record); + record.headers()); globalProcessorContext.setRecordContext(recordContext); try { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index 1713efb52a..12a6beedbc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -113,8 +113,7 @@ public void update(final ConsumerRecord record) { deserialized.offset(), deserialized.partition(), deserialized.topic(), - deserialized.headers(), - record); + deserialized.headers()); processorContext.setRecordContext(recordContext); processorContext.setCurrentNode(sourceNodeAndDeserializer.sourceNode()); final Record toProcess = new Record<>( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java index a5d88f5a7f..79db3847cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java @@ -66,8 +66,7 @@ public void process(final Record record) { context.offset(), context.partition(), context.topic(), - record.headers(), - processorRecordContext.rawRecord() + record.headers() )); delegate.process(record.key(), record.value()); } finally { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 6a79434b62..b484d26f0f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -261,8 +261,7 @@ public void forward(final Record record, final String childName) { recordContext.offset(), recordContext.partition(), recordContext.topic(), - record.headers(), - recordContext.rawRecord()); + record.headers()); } if (childName == null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index eabf9e3d5c..c30d42ba74 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -209,8 +209,6 @@ public void process(final Record record) { internalProcessorContext.partition(), internalProcessorContext.offset(), internalProcessorContext.headers(), - internalProcessorContext.recordContext().rawRecord().key(), - internalProcessorContext.recordContext().rawRecord().value(), internalProcessorContext.currentNode().name(), internalProcessorContext.taskId()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java index 3d1ce0529e..839baaad87 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; @@ -38,28 +37,17 @@ public class ProcessorRecordContext implements RecordContext, RecordMetadata { private final String topic; private final int partition; private final Headers headers; - private final ConsumerRecord rawRecord; public ProcessorRecordContext(final long timestamp, final long offset, final int partition, final String topic, final Headers headers) { - this(timestamp, offset, partition, topic, headers, null); - } - - public ProcessorRecordContext(final long timestamp, - final long offset, - final int partition, - final String topic, - final Headers headers, - final ConsumerRecord rawRecord) { this.timestamp = timestamp; this.offset = offset; this.topic = topic; this.partition = partition; this.headers = Objects.requireNonNull(headers); - this.rawRecord = rawRecord; } @Override @@ -87,10 +75,6 @@ public Headers headers() { return headers; } - public ConsumerRecord rawRecord() { - return rawRecord; - } - public long residentMemorySizeEstimate() { long size = 0; size += Long.BYTES; // value.context.timestamp @@ -189,7 +173,7 @@ public static ProcessorRecordContext deserialize(final ByteBuffer buffer) { headers = new RecordHeaders(headerArr); } - return new ProcessorRecordContext(timestamp, offset, partition, topic, headers, null); + return new ProcessorRecordContext(timestamp, offset, partition, topic, headers); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 16d67666cc..35097153a5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -44,6 +44,7 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; +import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -198,57 +199,47 @@ public void send(final String topic, final byte[] valBytes; try { keyBytes = keySerializer.serialize(topic, headers, key); - valBytes = valueSerializer.serialize(topic, headers, value); } catch (final ClassCastException exception) { - final String keyClass = key == null ? "unknown because key is null" : key.getClass().getName(); - final String valueClass = value == null ? "unknown because value is null" : value.getClass().getName(); - throw new StreamsException( - String.format( - "ClassCastException while producing data to topic %s. " + - "A serializer (key: %s / value: %s) is not compatible to the actual key or value type " + - "(key type: %s / value type: %s). " + - "Change the default Serdes in StreamConfig or provide correct Serdes via method parameters " + - "(for example if using the DSL, `#to(String topic, Produced produced)` with " + - "`Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).", - topic, - keySerializer.getClass().getName(), - valueSerializer.getClass().getName(), - keyClass, - valueClass), + throw createStreamsExceptionForKeyClassCastException( + topic, + key, + keySerializer, exception); } catch (final Exception exception) { - final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); - final ProductionExceptionHandler.ProductionExceptionHandlerResponse response; - - log.debug(String.format("Error serializing record to topic %s", topic), exception); - - try { - response = productionExceptionHandler.handleSerializationException(record, exception); - } catch (final Exception e) { - log.error("Fatal when handling serialization exception", e); - recordSendError(topic, e, null); - return; - } - - if (response == ProductionExceptionHandlerResponse.FAIL) { - throw new StreamsException( - String.format( - "Unable to serialize record. ProducerRecord(topic=[%s], partition=[%d], timestamp=[%d]", - topic, - partition, - timestamp), - exception - ); - } - - log.warn("Unable to serialize record, continue processing. " + - "ProducerRecord(topic=[{}], partition=[{}], timestamp=[{}])", - topic, - partition, - timestamp); - - droppedRecordsSensor.record(); + handleException( + ProductionExceptionHandler.SerializationExceptionOrigin.KEY, + topic, + key, + value, + headers, + partition, + timestamp, + processorNodeId, + context, + exception); + return; + } + try { + valBytes = valueSerializer.serialize(topic, headers, value); + } catch (final ClassCastException exception) { + throw createStreamsExceptionForValueClassCastException( + topic, + value, + valueSerializer, + exception); + } catch (final Exception exception) { + handleException( + ProductionExceptionHandler.SerializationExceptionOrigin.VALUE, + topic, + key, + value, + headers, + partition, + timestamp, + processorNodeId, + context, + exception); return; } @@ -285,7 +276,7 @@ public void send(final String topic, topicProducedSensor.record(bytesProduced, context.currentSystemTimeMs()); } } else { - recordSendError(topic, exception, serializedRecord); + recordSendError(topic, exception, serializedRecord, context, processorNodeId); // KAFKA-7510 only put message key and value in TRACE level log so we don't leak data by default log.trace("Failed record: (key {} value {} timestamp {}) topic=[{}] partition=[{}]", key, value, timestamp, topic, partition); @@ -293,7 +284,96 @@ public void send(final String topic, }); } - private void recordSendError(final String topic, final Exception exception, final ProducerRecord serializedRecord) { + private void handleException(final ProductionExceptionHandler.SerializationExceptionOrigin origin, + final String topic, + final K key, + final V value, + final Headers headers, + final Integer partition, + final Long timestamp, + final String processorNodeId, + final InternalProcessorContext context, + final Exception exception) { + final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); + final ProductionExceptionHandlerResponse response; + + log.debug(String.format("Error serializing record to topic %s", topic), exception); + + try { + final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( + context.recordContext().topic(), + context.recordContext().partition(), + context.recordContext().offset(), + context.recordContext().headers(), + processorNodeId, + taskId); + response = productionExceptionHandler.handleSerializationException(errorHandlerContext, record, exception, origin); + } catch (final Exception e) { + log.error("Fatal when handling serialization exception", e); + recordSendError(topic, e, null, context, processorNodeId); + return; + } + + if (response == ProductionExceptionHandlerResponse.FAIL) { + throw new StreamsException( + String.format( + "Unable to serialize record. ProducerRecord(topic=[%s], partition=[%d], timestamp=[%d]", + topic, + partition, + timestamp), + exception + ); + } + + log.warn("Unable to serialize record, continue processing. " + + "ProducerRecord(topic=[{}], partition=[{}], timestamp=[{}])", + topic, + partition, + timestamp); + + droppedRecordsSensor.record(); + } + private StreamsException createStreamsExceptionForKeyClassCastException(final String topic, + final K key, + final Serializer keySerializer, + final ClassCastException exception) { + final String keyClass = key == null ? "unknown because key is null" : key.getClass().getName(); + return new StreamsException( + String.format( + "ClassCastException while producing data to topic %s. " + + "The key serializer %s is not compatible to the actual key type: %s. " + + "Change the default key serde in StreamConfig or provide the correct key serde via method parameters " + + "(for example if using the DSL, `#to(String topic, Produced produced)` with " + + "`Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).", + topic, + keySerializer.getClass().getName(), + keyClass), + exception); + } + + private StreamsException createStreamsExceptionForValueClassCastException(final String topic, + final V value, + final Serializer valueSerializer, + final ClassCastException exception) { + final String valueClass = value == null ? "unknown because value is null" : value.getClass().getName(); + return new StreamsException( + String.format( + "ClassCastException while producing data to topic %s. " + + "The value serializer %s is not compatible to the actual value type: %s. " + + "Change the default value serde in StreamConfig or provide the correct value serde via method parameters " + + "(for example if using the DSL, `#to(String topic, Produced produced)` with " + + "`Produced.valueSerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).", + topic, + valueSerializer.getClass().getName(), + valueClass), + exception); + } + + private void recordSendError(final String topic, + final Exception exception, + final ProducerRecord serializedRecord, + final InternalProcessorContext context, + final String processorNodeId) { String errorMessage = String.format(SEND_EXCEPTION_MESSAGE, topic, taskId, exception.toString()); if (isFatalException(exception)) { @@ -314,7 +394,15 @@ private void recordSendError(final String topic, final Exception exception, fina "`delivery.timeout.ms` to a larger value to wait longer for such scenarios and avoid timeout errors"; sendException.set(new TaskCorruptedException(Collections.singleton(taskId))); } else { - if (productionExceptionHandler.handle(serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { + final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( + context.recordContext().topic(), + context.recordContext().partition(), + context.recordContext().offset(), + context.recordContext().headers(), + processorNodeId, + taskId); + + if (productionExceptionHandler.handle(errorHandlerContext, serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent."; sendException.set(new StreamsException(errorMessage, exception)); } else { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index a3c9ea67f0..a6b30a07ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -230,7 +230,7 @@ private void updateHead() { droppedRecordsSensor.record(); continue; } - headRecord = new StampedRecord(deserialized, timestamp, raw); + headRecord = new StampedRecord(deserialized, timestamp); headRecordSizeInBytes = consumerRecordSizeInBytes(raw); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index 871cb2284e..6e79616d30 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -88,8 +88,7 @@ public void process(final Record record) { context.offset(), context.partition(), context.topic(), - record.headers(), - context.recordContext().rawRecord() + record.headers() ); final String topic = topicExtractor.extract(key, value, contextForExtraction); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java index d82cd98ed7..71e3ca2e3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java @@ -20,11 +20,9 @@ import org.apache.kafka.common.header.Headers; public class StampedRecord extends Stamped> { - private final ConsumerRecord rawRecord; - public StampedRecord(final ConsumerRecord record, final long timestamp, final ConsumerRecord rawRecord) { + public StampedRecord(final ConsumerRecord record, final long timestamp) { super(record, timestamp); - this.rawRecord = rawRecord; } public String topic() { @@ -51,20 +49,6 @@ public Headers headers() { return value.headers(); } - public ConsumerRecord rawRecord() { - return rawRecord; - } - - @Override - public boolean equals(final Object other) { - return super.equals(other); - } - - @Override - public int hashCode() { - return super.hashCode(); - } - @Override public String toString() { return value.toString() + ", timestamp = " + timestamp; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 30b9038aa6..8cbe5780b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -844,8 +844,7 @@ private void doProcess(final long wallClockTime) { record.offset(), record.partition(), record.topic(), - record.headers(), - record.rawRecord() + record.headers() ); updateProcessorContext(currNode, wallClockTime, recordContext); @@ -906,8 +905,7 @@ public void punctuate(final ProcessorNode node, -1L, -1, null, - new RecordHeaders(), - null + new RecordHeaders() ); updateProcessorContext(node, time.milliseconds(), recordContext); diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java deleted file mode 100644 index be1e98e4a7..0000000000 --- a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.errors; - -import org.apache.kafka.clients.producer.ProducerRecord; - -import java.util.Map; - -/** - * Production exception handler that always instructs streams to continue when an exception - * happens while attempting to produce result records. - */ -public class AlwaysContinueProductionExceptionHandler implements ProductionExceptionHandler { - @Override - public ProductionExceptionHandlerResponse handle(final ProducerRecord record, - final Exception exception) { - return ProductionExceptionHandlerResponse.CONTINUE; - } - - @Override - public ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, - final Exception exception) { - return ProductionExceptionHandlerResponse.CONTINUE; - } - - @Override - public void configure(final Map configs) { - // ignore - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index bb29ac64f0..6c1a64344e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -176,8 +176,6 @@ public void configure(final Map configs) { } private static void assertProcessingExceptionHandlerInputs(final ErrorHandlerContext context, final Record record, final Exception exception) { - assertTrue(Arrays.asList("ID123-2-ERR", "ID123-5-ERR").contains(new String(context.sourceRawKey()))); - assertTrue(Arrays.asList("ID123-A2", "ID123-A5").contains(new String(context.sourceRawValue()))); assertTrue(Arrays.asList("ID123-2-ERR", "ID123-5-ERR").contains((String) record.key())); assertTrue(Arrays.asList("ID123-A2", "ID123-A5").contains((String) record.value())); assertEquals("TOPIC_NAME", context.topic()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 6be033cb2d..df3f927686 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.InvalidOffsetException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.internals.RecordHeaders; @@ -58,7 +57,6 @@ import java.util.Set; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.ROLLUP_VALUE; -import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -315,8 +313,7 @@ private InternalProcessorContext mockInternalProcessorContext() OFFSET, PARTITION, TOPIC, - new RecordHeaders(), - new ConsumerRecord<>(TOPIC, PARTITION, OFFSET, KEY.getBytes(), VALUE.getBytes()))); + new RecordHeaders())); when(internalProcessorContext.currentNode()).thenReturn(new ProcessorNode<>(NAME)); return internalProcessorContext; @@ -337,8 +334,6 @@ public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHa assertEquals(internalProcessorContext.topic(), context.topic()); assertEquals(internalProcessorContext.partition(), context.partition()); assertEquals(internalProcessorContext.offset(), context.offset()); - assertArrayEquals(internalProcessorContext.recordContext().rawRecord().key(), context.sourceRawKey()); - assertArrayEquals(internalProcessorContext.recordContext().rawRecord().value(), context.sourceRawValue()); assertEquals(internalProcessorContext.currentNode().name(), context.processorNodeId()); assertEquals(internalProcessorContext.taskId(), context.taskId()); assertEquals(KEY, record.key()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index 5222759bc5..2623102885 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -48,8 +48,8 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.errors.AlwaysContinueProductionExceptionHandler; import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; +import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.ProductionExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskMigratedException; @@ -96,6 +96,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -918,9 +919,9 @@ public void shouldThrowInformativeStreamsExceptionOnKeyClassCastException() { expected.getMessage(), equalTo( "ClassCastException while producing data to topic topic. " + - "A serializer (key: org.apache.kafka.common.serialization.LongSerializer / value: org.apache.kafka.common.serialization.StringSerializer) " + - "is not compatible to the actual key or value type (key type: java.lang.String / value type: java.lang.String). " + - "Change the default Serdes in StreamConfig or provide correct Serdes via method parameters " + + "The key serializer org.apache.kafka.common.serialization.LongSerializer " + + "is not compatible to the actual key type: java.lang.String. " + + "Change the default key serde in StreamConfig or provide the correct key serde via method parameters " + "(for example if using the DSL, `#to(String topic, Produced produced)` with `Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).") ); } @@ -946,9 +947,9 @@ public void shouldThrowInformativeStreamsExceptionOnKeyAndNullValueClassCastExce expected.getMessage(), equalTo( "ClassCastException while producing data to topic topic. " + - "A serializer (key: org.apache.kafka.common.serialization.LongSerializer / value: org.apache.kafka.common.serialization.StringSerializer) " + - "is not compatible to the actual key or value type (key type: java.lang.String / value type: unknown because value is null). " + - "Change the default Serdes in StreamConfig or provide correct Serdes via method parameters " + + "The key serializer org.apache.kafka.common.serialization.LongSerializer " + + "is not compatible to the actual key type: java.lang.String. " + + "Change the default key serde in StreamConfig or provide the correct key serde via method parameters " + "(for example if using the DSL, `#to(String topic, Produced produced)` with `Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).") ); } @@ -974,10 +975,10 @@ public void shouldThrowInformativeStreamsExceptionOnValueClassCastException() { expected.getMessage(), equalTo( "ClassCastException while producing data to topic topic. " + - "A serializer (key: org.apache.kafka.common.serialization.StringSerializer / value: org.apache.kafka.common.serialization.LongSerializer) " + - "is not compatible to the actual key or value type (key type: java.lang.String / value type: java.lang.String). " + - "Change the default Serdes in StreamConfig or provide correct Serdes via method parameters " + - "(for example if using the DSL, `#to(String topic, Produced produced)` with `Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).") + "The value serializer org.apache.kafka.common.serialization.LongSerializer " + + "is not compatible to the actual value type: java.lang.String. " + + "Change the default value serde in StreamConfig or provide the correct value serde via method parameters " + + "(for example if using the DSL, `#to(String topic, Produced produced)` with `Produced.valueSerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).") ); } @@ -1002,10 +1003,10 @@ public void shouldThrowInformativeStreamsExceptionOnValueAndNullKeyClassCastExce expected.getMessage(), equalTo( "ClassCastException while producing data to topic topic. " + - "A serializer (key: org.apache.kafka.common.serialization.StringSerializer / value: org.apache.kafka.common.serialization.LongSerializer) " + - "is not compatible to the actual key or value type (key type: unknown because key is null / value type: java.lang.String). " + - "Change the default Serdes in StreamConfig or provide correct Serdes via method parameters " + - "(for example if using the DSL, `#to(String topic, Produced produced)` with `Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).") + "The value serializer org.apache.kafka.common.serialization.LongSerializer " + + "is not compatible to the actual value type: java.lang.String. " + + "Change the default value serde in StreamConfig or provide the correct value serde via method parameters " + + "(for example if using the DSL, `#to(String topic, Produced produced)` with `Produced.valueSerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).") ); } @@ -1171,11 +1172,11 @@ public void shouldThrowStreamsExceptionOnSubsequentSendIfASendFailsWithDefaultEx topology ); - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, null, null, streamPartitioner); + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); final StreamsException thrown = assertThrows( StreamsException.class, - () -> collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, null, null, streamPartitioner) + () -> collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner) ); assertEquals(exception, thrown.getCause()); assertThat( @@ -1198,7 +1199,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfASendFailsWithDefaultE topology ); - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, null, null, streamPartitioner); + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); final StreamsException thrown = assertThrows(StreamsException.class, collector::flush); assertEquals(exception, thrown.getCause()); @@ -1222,7 +1223,7 @@ public void shouldThrowStreamsExceptionOnSubsequentCloseIfASendFailsWithDefaultE topology ); - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, null, null, streamPartitioner); + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); final StreamsException thrown = assertThrows(StreamsException.class, collector::closeClean); assertEquals(exception, thrown.getCause()); @@ -1241,7 +1242,7 @@ public void shouldThrowStreamsExceptionOnSubsequentSendIfFatalEvenWithContinueEx logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new AlwaysContinueProductionExceptionHandler(), + new ProductionExceptionHandlerMock(ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE), streamsMetrics, topology ); @@ -1268,7 +1269,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfFatalEvenWithContinueE logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new AlwaysContinueProductionExceptionHandler(), + new ProductionExceptionHandlerMock(ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE), streamsMetrics, topology ); @@ -1292,7 +1293,7 @@ public void shouldThrowStreamsExceptionOnSubsequentCloseIfFatalEvenWithContinueE logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new AlwaysContinueProductionExceptionHandler(), + new ProductionExceptionHandlerMock(ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE), streamsMetrics, topology ); @@ -1314,8 +1315,13 @@ public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContin final RecordCollector collector = new RecordCollectorImpl( logContext, taskId, - getExceptionalStreamsProducerOnSend(new Exception()), - new AlwaysContinueProductionExceptionHandler(), + getExceptionalStreamsProducerOnSend(new RuntimeException("KABOOM!")), + new ProductionExceptionHandlerMock( + ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE, + context, + sinkNodeName, + taskId + ), streamsMetrics, topology ); @@ -1324,7 +1330,7 @@ public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContin LogCaptureAppender.createAndRegister(RecordCollectorImpl.class)) { logCaptureAppender.setThreshold(Level.INFO); - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, null, null, streamPartitioner); + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); collector.flush(); final List messages = logCaptureAppender.getMessages(); @@ -1348,7 +1354,7 @@ public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContin )); assertEquals(1.0, metric.metricValue()); - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, null, null, streamPartitioner); + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); collector.flush(); collector.closeClean(); } @@ -1365,7 +1371,7 @@ public void shouldThrowStreamsExceptionOnUnknownTopicOrPartitionExceptionWithDef topology ); - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, null, null, streamPartitioner); + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); // With default handler which returns FAIL, flush() throws StreamsException with TimeoutException cause, // otherwise it would throw a TaskCorruptedException with null cause @@ -1386,12 +1392,17 @@ public void shouldNotThrowTaskCorruptedExceptionOnUnknownTopicOrPartitionExcepti logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new AlwaysContinueProductionExceptionHandler(), + new ProductionExceptionHandlerMock( + ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE, + context, + sinkNodeName, + taskId + ), streamsMetrics, topology ); - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, null, null, streamPartitioner); + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); assertDoesNotThrow(collector::flush); } @@ -1527,7 +1538,13 @@ public void shouldThrowStreamsExceptionUsingDefaultExceptionHandler() { @Test public void shouldDropRecordExceptionUsingAlwaysContinueExceptionHandler() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { - final RecordCollector collector = newRecordCollector(new AlwaysContinueProductionExceptionHandler()); + final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( + ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE, + context, + sinkNodeName, + taskId, + ProductionExceptionHandler.SerializationExceptionOrigin.KEY + )); collector.initialize(); collector.send(topic, "key", "val", null, 0, null, errorSerializer, stringSerializer, sinkNodeName, context); @@ -1547,11 +1564,53 @@ public void shouldDropRecordExceptionUsingAlwaysContinueExceptionHandler() { } } + @Test + public void shouldThrowStreamsExceptionWhenValueSerializationFailedAndProductionExceptionHandlerRepliesWithFail() { + try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { + final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( + ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL, + context, + sinkNodeName, + taskId, + ProductionExceptionHandler.SerializationExceptionOrigin.VALUE + )); + collector.initialize(); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> collector.send(topic, "key", "val", null, 0, null, stringSerializer, errorSerializer, sinkNodeName, context)); + + assertInstanceOf(RuntimeException.class, exception.getCause()); + assertEquals("KABOOM!", exception.getCause().getMessage()); + } + } + + @Test + public void shouldThrowStreamsExceptionWhenKeySerializationFailedAndProductionExceptionHandlerRepliesWithFail() { + try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { + final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( + ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL, + context, + sinkNodeName, + taskId, + ProductionExceptionHandler.SerializationExceptionOrigin.KEY + )); + collector.initialize(); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> collector.send(topic, "key", "val", null, 0, null, errorSerializer, stringSerializer, sinkNodeName, context)); + + assertInstanceOf(RuntimeException.class, exception.getCause()); + assertEquals("KABOOM!", exception.getCause().getMessage()); + } + } + @SuppressWarnings({"unchecked", "rawtypes"}) @Test public void shouldNotCallProductionExceptionHandlerOnClassCastException() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { - final RecordCollector collector = newRecordCollector(new AlwaysContinueProductionExceptionHandler()); + final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock(ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE)); collector.initialize(); assertThat(mockProducer.history().isEmpty(), equalTo(true)); @@ -1637,7 +1696,7 @@ private static class ErrorStringSerializer extends StringSerializer { @Override public byte[] serialize(final String topic, final Headers headers, final String data) { - throw new SerializationException("Not Supported"); + throw new SerializationException("KABOOM!"); } } @@ -1705,4 +1764,68 @@ public byte[] serialize(final String topic, final Headers headers, final String return serialize(topic, data); } } + + public static class ProductionExceptionHandlerMock implements ProductionExceptionHandler { + private final ProductionExceptionHandlerResponse response; + private InternalProcessorContext expectedContext; + private String expectedProcessorNodeId; + private TaskId expectedTaskId; + private SerializationExceptionOrigin expectedSerializationExceptionOrigin; + + public ProductionExceptionHandlerMock(final ProductionExceptionHandlerResponse response) { + this.response = response; + } + + public ProductionExceptionHandlerMock(final ProductionExceptionHandlerResponse response, + final InternalProcessorContext context, + final String processorNodeId, + final TaskId taskId) { + this(response); + this.expectedContext = context; + this.expectedProcessorNodeId = processorNodeId; + this.expectedTaskId = taskId; + } + + public ProductionExceptionHandlerMock(final ProductionExceptionHandlerResponse response, + final InternalProcessorContext context, + final String processorNodeId, + final TaskId taskId, + final SerializationExceptionOrigin origin) { + this(response, context, processorNodeId, taskId); + this.expectedSerializationExceptionOrigin = origin; + } + + @Override + public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { + assertInputs(context, exception); + return response; + } + + @Override + public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { + assertInputs(context, exception); + assertEquals(expectedSerializationExceptionOrigin, origin); + return response; + } + + @Override + public void configure(final Map configs) { + // do nothing + } + + private void assertInputs(final ErrorHandlerContext context, final Exception exception) { + assertEquals(expectedContext.recordContext().topic(), context.topic()); + assertEquals(expectedContext.recordContext().partition(), context.partition()); + assertEquals(expectedContext.recordContext().offset(), context.offset()); + assertEquals(expectedProcessorNodeId, context.processorNodeId()); + assertEquals(expectedTaskId, context.taskId()); + assertInstanceOf(RuntimeException.class, exception); + assertEquals("KABOOM!", exception.getMessage()); + } + } } diff --git a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java index 237d1c665b..ff9080a2de 100644 --- a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java @@ -239,6 +239,13 @@ public InternalMockProcessorContext(final File stateDir, appConfigs(), IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, false); + this.recordContext = new ProcessorRecordContext( + 0, + 0, + 0, + "topic", + new RecordHeaders() + ); } @Override From 10d9f7872d8fad81cd95f327ba6140994cf35576 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Tue, 30 Jul 2024 05:33:33 +0200 Subject: [PATCH 003/123] KAFKA-16448: Add ErrorHandlerContext in deserialization exception handler (#16432) This PR is part of KIP1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing. This PR expose the new ErrorHandlerContext as a parameter to the Deserialization exception handlers and deprecate the previous handle signature. Co-authored-by: Dabz Co-authored-by: loicgreffier Reviewers: Bruno Cadonna , Matthias J. Sax --- .../DeserializationExceptionHandler.java | 26 +++- .../LogAndContinueExceptionHandler.java | 14 ++ .../errors/LogAndFailExceptionHandler.java | 14 ++ .../internals/DefaultErrorHandlerContext.java | 12 +- .../processor/internals/ProcessorNode.java | 1 + .../internals/RecordDeserializer.java | 24 +++- .../internals/RecordDeserializerTest.java | 131 +++++++++++++++++- 7 files changed, 205 insertions(+), 17 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java index 95ccfeced8..0d64611de6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java @@ -16,9 +16,9 @@ */ package org.apache.kafka.streams.errors; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.Configurable; +import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; /** @@ -37,11 +37,27 @@ public interface DeserializationExceptionHandler extends Configurable { * @param context processor context * @param record record that failed deserialization * @param exception the actual exception + * @deprecated Since 3.9. Use Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} + */ + @Deprecated + default DeserializationHandlerResponse handle(final ProcessorContext context, + final ConsumerRecord record, + final Exception exception) { + throw new UnsupportedOperationException(); + } + + /** + * Inspect a record and the exception received. + * + * @param context error handler context + * @param record record that failed deserialization + * @param exception the actual exception */ - @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. - DeserializationHandlerResponse handle(final ProcessorContext context, - final ConsumerRecord record, - final Exception exception); + default DeserializationHandlerResponse handle(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { + return handle(((DefaultErrorHandlerContext) context).processorContext().orElse(null), record, exception); + } /** * Enumeration that describes the response from the exception handler. diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java index a468be2e67..a93b7c9951 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java @@ -32,6 +32,7 @@ public class LogAndContinueExceptionHandler implements DeserializationExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class); + @Deprecated @Override public DeserializationHandlerResponse handle(final ProcessorContext context, final ConsumerRecord record, @@ -45,6 +46,19 @@ public DeserializationHandlerResponse handle(final ProcessorContext context, return DeserializationHandlerResponse.CONTINUE; } + @Override + public DeserializationHandlerResponse handle(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { + + log.warn("Exception caught during Deserialization, " + + "taskId: {}, topic: {}, partition: {}, offset: {}", + context.taskId(), record.topic(), record.partition(), record.offset(), + exception); + + return DeserializationHandlerResponse.CONTINUE; + } + @Override public void configure(final Map configs) { // ignore diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java index 3f0fd48652..5fdda623bd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java @@ -33,6 +33,7 @@ public class LogAndFailExceptionHandler implements DeserializationExceptionHandl private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class); @Override + @Deprecated public DeserializationHandlerResponse handle(final ProcessorContext context, final ConsumerRecord record, final Exception exception) { @@ -45,6 +46,19 @@ public DeserializationHandlerResponse handle(final ProcessorContext context, return DeserializationHandlerResponse.FAIL; } + @Override + public DeserializationHandlerResponse handle(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { + + log.error("Exception caught during Deserialization, " + + "taskId: {}, topic: {}, partition: {}, offset: {}", + context.taskId(), record.topic(), record.partition(), record.offset(), + exception); + + return DeserializationHandlerResponse.FAIL; + } + @Override public void configure(final Map configs) { // ignore diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java index aa066fb6da..c907ff3eb8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java @@ -18,8 +18,11 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.streams.errors.ErrorHandlerContext; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.TaskId; +import java.util.Optional; + /** * Default implementation of {@link ErrorHandlerContext} that provides access to the metadata of the record that caused the error. */ @@ -30,8 +33,10 @@ public class DefaultErrorHandlerContext implements ErrorHandlerContext { private final Headers headers; private final String processorNodeId; private final TaskId taskId; + private ProcessorContext processorContext; - public DefaultErrorHandlerContext(final String topic, + public DefaultErrorHandlerContext(final ProcessorContext processorContext, + final String topic, final int partition, final long offset, final Headers headers, @@ -43,6 +48,7 @@ public DefaultErrorHandlerContext(final String topic, this.headers = headers; this.processorNodeId = processorNodeId; this.taskId = taskId; + this.processorContext = processorContext; } @Override @@ -74,4 +80,8 @@ public String processorNodeId() { public TaskId taskId() { return taskId; } + + public Optional processorContext() { + return Optional.ofNullable(processorContext); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index c30d42ba74..65eec47cb1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -205,6 +205,7 @@ public void process(final Record record) { throw e; } catch (final Exception e) { final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( + null, internalProcessorContext.topic(), internalProcessorContext.partition(), internalProcessorContext.offset(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 2680c179b5..8ee2dc014e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.slf4j.Logger; @@ -69,7 +70,7 @@ ConsumerRecord deserialize(final ProcessorContext processo Optional.empty() ); } catch (final Exception deserializationException) { - handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor); + handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, sourceNode().name()); return null; // 'handleDeserializationFailure' would either throw or swallow -- if we swallow we need to skip the record by returning 'null' } } @@ -80,12 +81,27 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa final ConsumerRecord rawRecord, final Logger log, final Sensor droppedRecordsSensor) { + handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, null); + } + + public static void handleDeserializationFailure(final DeserializationExceptionHandler deserializationExceptionHandler, + final ProcessorContext processorContext, + final Exception deserializationException, + final ConsumerRecord rawRecord, + final Logger log, + final Sensor droppedRecordsSensor, + final String sourceNodeName) { final DeserializationExceptionHandler.DeserializationHandlerResponse response; try { - response = deserializationExceptionHandler.handle( + final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( (InternalProcessorContext) processorContext, - rawRecord, - deserializationException); + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + rawRecord.headers(), + sourceNodeName, + processorContext.taskId()); + response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException); } catch (final Exception fatalUserException) { log.error( "Deserialization error callback failed after deserialization error for record {}", diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index ac973c1789..23f364fc6a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -24,16 +24,28 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.ErrorHandlerContext; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.test.InternalMockProcessorContext; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import java.util.Map; import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; public class RecordDeserializerTest { - - private final RecordHeaders headers = new RecordHeaders(new Header[] {new RecordHeader("key", "value".getBytes())}); + private final String sourceNodeName = "source-node"; + private final TaskId taskId = new TaskId(0, 0); + private final RecordHeaders headers = new RecordHeaders(new Header[]{new RecordHeader("key", "value".getBytes())}); private final ConsumerRecord rawRecord = new ConsumerRecord<>("topic", 1, 1, @@ -46,13 +58,17 @@ public class RecordDeserializerTest { headers, Optional.empty()); + private final InternalProcessorContext context = new InternalMockProcessorContext<>(); + @Test public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { final RecordDeserializer recordDeserializer = new RecordDeserializer( new TheSourceNode( + sourceNodeName, false, false, - "key", "value" + "key", + "value" ), null, new LogContext(), @@ -69,17 +85,82 @@ public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { assertEquals(rawRecord.headers(), record.headers()); } + @ParameterizedTest + @CsvSource({ + "true, true", + "true, false", + "false, true", + }) + public void shouldThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandlerRepliesWithFail(final boolean keyThrowsException, + final boolean valueThrowsException) { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + sourceNodeName, + keyThrowsException, + valueThrowsException, + "key", + "value" + ), + new DeserializationExceptionHandlerMock( + DeserializationExceptionHandler.DeserializationHandlerResponse.FAIL, + rawRecord, + sourceNodeName, + taskId + ), + new LogContext(), + new Metrics().sensor("dropped-records") + ); + + final StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); + assertEquals(e.getMessage(), "Deserialization exception handler is set " + + "to fail upon a deserialization error. " + + "If you would rather have the streaming pipeline " + + "continue after a deserialization error, please set the " + + "default.deserialization.exception.handler appropriately."); + } + + @ParameterizedTest + @CsvSource({ + "true, true", + "true, false", + "false, true" + }) + public void shouldNotThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandlerRepliesWithContinue(final boolean keyThrowsException, + final boolean valueThrowsException) { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + sourceNodeName, + keyThrowsException, + valueThrowsException, + "key", + "value" + ), + new DeserializationExceptionHandlerMock( + DeserializationExceptionHandler.DeserializationHandlerResponse.CONTINUE, + rawRecord, + sourceNodeName, + taskId + ), + new LogContext(), + new Metrics().sensor("dropped-records") + ); + + final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); + assertNull(record); + } + static class TheSourceNode extends SourceNode { private final boolean keyThrowsException; private final boolean valueThrowsException; private final Object key; private final Object value; - TheSourceNode(final boolean keyThrowsException, + TheSourceNode(final String name, + final boolean keyThrowsException, final boolean valueThrowsException, final Object key, final Object value) { - super("", null, null); + super(name, null, null); this.keyThrowsException = keyThrowsException; this.valueThrowsException = valueThrowsException; this.key = key; @@ -89,7 +170,7 @@ static class TheSourceNode extends SourceNode { @Override public Object deserializeKey(final String topic, final Headers headers, final byte[] data) { if (keyThrowsException) { - throw new RuntimeException(); + throw new RuntimeException("KABOOM!"); } return key; } @@ -97,10 +178,46 @@ public Object deserializeKey(final String topic, final Headers headers, final by @Override public Object deserializeValue(final String topic, final Headers headers, final byte[] data) { if (valueThrowsException) { - throw new RuntimeException(); + throw new RuntimeException("KABOOM!"); } return value; } } + public static class DeserializationExceptionHandlerMock implements DeserializationExceptionHandler { + private final DeserializationHandlerResponse response; + private final ConsumerRecord expectedRecord; + private final String expectedProcessorNodeId; + private final TaskId expectedTaskId; + + public DeserializationExceptionHandlerMock(final DeserializationHandlerResponse response, + final ConsumerRecord record, + final String processorNodeId, + final TaskId taskId) { + this.response = response; + this.expectedRecord = record; + this.expectedProcessorNodeId = processorNodeId; + this.expectedTaskId = taskId; + } + + @Override + public DeserializationHandlerResponse handle(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { + assertEquals(expectedRecord.topic(), context.topic()); + assertEquals(expectedRecord.partition(), context.partition()); + assertEquals(expectedRecord.offset(), context.offset()); + assertEquals(expectedProcessorNodeId, context.processorNodeId()); + assertEquals(expectedTaskId, context.taskId()); + assertEquals(expectedRecord, record); + assertInstanceOf(RuntimeException.class, exception); + assertEquals("KABOOM!", exception.getMessage()); + return response; + } + + @Override + public void configure(final Map configs) { + // do nothing + } + } } From b8532070f79ff4d13f61e4cd766b9190299aab75 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Mon, 29 Jul 2024 21:07:49 -0700 Subject: [PATCH 004/123] HOTFIX: fix compilation error --- .../kafka/streams/processor/internals/ProcessorNode.java | 2 +- .../streams/processor/internals/RecordCollectorImpl.java | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 65eec47cb1..763edc9a04 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -205,7 +205,7 @@ public void process(final Record record) { throw e; } catch (final Exception e) { final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( - null, + null, // only required to pass for DeserializationExceptionHandler internalProcessorContext.topic(), internalProcessorContext.partition(), internalProcessorContext.offset(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 35097153a5..de4afc2c92 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -301,12 +301,14 @@ private void handleException(final ProductionExceptionHandler.Serializati try { final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( + null, // only required to pass for DeserializationExceptionHandler context.recordContext().topic(), context.recordContext().partition(), context.recordContext().offset(), context.recordContext().headers(), processorNodeId, - taskId); + taskId + ); response = productionExceptionHandler.handleSerializationException(errorHandlerContext, record, exception, origin); } catch (final Exception e) { log.error("Fatal when handling serialization exception", e); @@ -395,12 +397,14 @@ private void recordSendError(final String topic, sendException.set(new TaskCorruptedException(Collections.singleton(taskId))); } else { final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( + null, // only required to pass for DeserializationExceptionHandler context.recordContext().topic(), context.recordContext().partition(), context.recordContext().offset(), context.recordContext().headers(), processorNodeId, - taskId); + taskId + ); if (productionExceptionHandler.handle(errorHandlerContext, serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent."; From bc243ab1e8fc764a8993f9c3c3858614913882bd Mon Sep 17 00:00:00 2001 From: Josep Prat Date: Tue, 30 Jul 2024 09:19:48 +0200 Subject: [PATCH 005/123] MINOR: Add 3.8.0 to system tests (#16714) Reviewers: Manikumar Reddy --- gradle/dependencies.gradle | 2 ++ tests/docker/Dockerfile | 2 ++ vagrant/base.sh | 3 ++- 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 835981f6fb..a2845d0107 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -135,6 +135,7 @@ versions += [ kafka_35: "3.5.2", kafka_36: "3.6.2", kafka_37: "3.7.1", + kafka_38: "3.8.0", lz4: "1.8.0", mavenArtifact: "3.9.6", metrics: "2.2.0", @@ -228,6 +229,7 @@ libs += [ kafkaStreams_35: "org.apache.kafka:kafka-streams:$versions.kafka_35", kafkaStreams_36: "org.apache.kafka:kafka-streams:$versions.kafka_36", kafkaStreams_37: "org.apache.kafka:kafka-streams:$versions.kafka_37", + kafkaStreams_38: "org.apache.kafka:kafka-streams:$versions.kafka_38", lz4: "org.lz4:lz4-java:$versions.lz4", metrics: "com.yammer.metrics:metrics-core:$versions.metrics", dropwizardMetrics: "io.dropwizard.metrics:metrics-core:$versions.dropwizardMetrics", diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index 1c2006646b..bca82534cd 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -100,6 +100,7 @@ RUN mkdir -p "/opt/kafka-3.4.1" && chmod a+rw /opt/kafka-3.4.1 && curl -s "$KAFK RUN mkdir -p "/opt/kafka-3.5.2" && chmod a+rw /opt/kafka-3.5.2 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.5.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.5.2" RUN mkdir -p "/opt/kafka-3.6.2" && chmod a+rw /opt/kafka-3.6.2 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.6.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.6.2" RUN mkdir -p "/opt/kafka-3.7.1" && chmod a+rw /opt/kafka-3.7.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.7.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.7.1" +RUN mkdir -p "/opt/kafka-3.8.0" && chmod a+rw /opt/kafka-3.8.0 && curl -s "$KAFKA_MIRROR/kafka_2.12-3.8.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-3.8.0" # Streams test dependencies @@ -126,6 +127,7 @@ RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.4.1-test.jar" -o /opt/kafka-3.4.1/lib RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.5.2-test.jar" -o /opt/kafka-3.5.2/libs/kafka-streams-3.5.2-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.6.2-test.jar" -o /opt/kafka-3.6.2/libs/kafka-streams-3.6.2-test.jar RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.7.1-test.jar" -o /opt/kafka-3.7.1/libs/kafka-streams-3.7.1-test.jar +RUN curl -s "$KAFKA_MIRROR/kafka-streams-3.8.0-test.jar" -o /opt/kafka-3.8.0/libs/kafka-streams-3.8.0-test.jar # The version of Kibosh to use for testing. # If you update this, also update vagrant/base.sh diff --git a/vagrant/base.sh b/vagrant/base.sh index b05e3502be..b896b610a0 100755 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -164,7 +164,8 @@ get_kafka 3.6.2 2.12 chmod a+rw /opt/kafka-3.6.2 get_kafka 3.7.1 2.12 chmod a+rw /opt/kafka-3.7.1 - +get_kafka 3.8.0 2.12 +chmod a+rw /opt/kafka-3.8.0 # For EC2 nodes, we want to use /mnt, which should have the local disk. On local # VMs, we can just create it if it doesn't exist and use it like we'd use From 0969789973ee17b5744c0212828148542cb75a51 Mon Sep 17 00:00:00 2001 From: PaulRMellor <47596553+PaulRMellor@users.noreply.github.com> Date: Tue, 30 Jul 2024 13:46:00 +0100 Subject: [PATCH 006/123] KAFKA-15469: Add documentation for configuration providers (#16650) Reviewers: Mickael Maison --- docs/configuration.html | 122 +++++++++++++++++++- docs/toc.html | 241 +++++++++++++++++++++------------------- 2 files changed, 248 insertions(+), 115 deletions(-) diff --git a/docs/configuration.html b/docs/configuration.html index 3f2b012148..cd12dd3ea9 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -308,6 +308,126 @@

Below are the configuration properties for Tiered Storage. - +

+ + 3.11 Configuration Providers +

+ +

+ Use configuration providers to load configuration data from external sources. This might include sensitive information, such as passwords, API keys, or other credentials. +

+ +

You have the following options:

+ + +

+ To use a configuration provider, specify it in your configuration using the config.providers property. +

+ +

+ + Using Configuration Providers +

+

Configuration providers allow you to pass parameters and retrieve configuration data from various sources.

+

To specify configuration providers, you use a comma-separated list of aliases and the fully-qualified class names that implement the configuration providers:

+
config.providers=provider1,provider2
+config.providers.provider1.class=com.example.Provider1
+config.providers.provider2.class=com.example.Provider2
+

Each provider can have its own set of parameters, which are passed in a specific format:

+
config.providers.<provider_alias>.param.<name>=<value>
+

The ConfigProvider interface serves as a base for all configuration providers. + Custom implementations of this interface can be created to retrieve configuration data from various sources. + You can package the implementation as a JAR file, add the JAR to your classpath, and reference the provider's class in your configuration.

+

Example custom provider configuration

+
config.providers=customProvider
+config.providers.customProvider.class=com.example.customProvider
+config.providers.customProvider.param.param1=value1
+config.providers.customProvider.param.param2=value2
+ +

+ + DirectoryConfigProvider +

+

The DirectoryConfigProvider retrieves configuration data from files stored in a specified directory.

+

Each file represents a key, and its content is the value. This provider is useful for loading multiple configuration files and for organizing configuration data into separate files.

+

To restrict the files that the DirectoryConfigProvider can access, use the allowed.paths parameter. + This parameter accepts a comma-separated list of paths that the provider is allowed to access. If not set, all paths are allowed.

+

Example DirectoryConfigProvider configuration

+
config.providers=dirProvider
+config.providers.dirProvider.class=org.apache.kafka.common.config.provider.DirectoryConfigProvider
+config.providers.dirProvider.param.allowed.paths=/path/to/dir1,/path/to/dir2
+

+ To reference a value supplied by the DirectoryConfigProvider, use the correct placeholder syntax: +

+
${dirProvider:<path_to_file>:<file_name>}
+ +

+ + EnvVarConfigProvider +

+

The EnvVarConfigProvider retrieves configuration data from environment variables.

+

No specific parameters are required, as it reads directly from the specified environment variables.

+

This provider is useful for configuring applications running in containers, for example, to load certificates or JAAS configuration from environment variables mapped from secrets.

+

To restrict which environment variables the EnvVarConfigProvider can access, use the allowlist.pattern parameter. This parameter accepts a regular expression that environment variable names must match to be used by the provider.

+

Example EnvVarConfigProvider configuration

+
config.providers=envVarProvider
+config.providers.envVarProvider.class=org.apache.kafka.common.config.provider.EnvVarConfigProvider
+config.providers.envVarProvider.param.allowlist.pattern=^MY_ENVAR1_.*
+

+ To reference a value supplied by the EnvVarConfigProvider, use the correct placeholder syntax: +

+
${envVarProvider:<enVar_name>}
+ +

+ + FileConfigProvider +

+

The FileConfigProvider retrieves configuration data from a single properties file.

+

This provider is useful for loading configuration data from mounted files.

+

To restrict the file paths that the FileConfigProvider can access, use the allowed.paths parameter. This parameter accepts a comma-separated list of paths that the provider is allowed to access. If not set, all paths are allowed.

+

Example FileConfigProvider configuration

+
config.providers=fileProvider
+config.providers.fileProvider.class=org.apache.kafka.common.config.provider.FileConfigProvider
+config.providers.fileProvider.param.allowed.paths=/path/to/config1,/path/to/config2
+

+ To reference a value supplied by the FileConfigProvider, use the correct placeholder syntax: +

+
${fileProvider:<path_and_filename>:<property>}
+ +

+ + Example: Referencing files +

+

+ Here’s an example that uses a file configuration provider with Kafka Connect to provide authentication credentials to a database for a connector. +

+

+ First, create a connector-credentials.properties configuration file with the following credentials: +

+
dbUsername=my-username
+dbPassword=my-password
+

+ Specify a FileConfigProvider in the Kafka Connect configuration: +

+

Example Kafka Connect configuration with a FileConfigProvider

+
config.providers=fileProvider
+config.providers.fileProvider.class=org.apache.kafka.common.config.provider.FileConfigProvider
+

Next, reference the properties from the file in the connector configuration.

+

Example connector configuration referencing file properties

+
database.user=${fileProvider:/path/to/connector-credentials.properties:dbUsername}
+database.password=${fileProvider:/path/to/connector-credentials.properties:dbPassword}
+

At runtime, the configuration provider reads and extracts the values from the properties file.

+
diff --git a/docs/toc.html b/docs/toc.html index 2413491148..c05de6d17f 100644 --- a/docs/toc.html +++ b/docs/toc.html @@ -29,7 +29,7 @@
  • 1.5 Upgrading
  • 1.6 Docker -
  • +
  • 2. APIs -
  • +
  • 3. Configuration +
  • 3.6 Kafka Streams Configs
  • 3.7 AdminClient Configs
  • 3.8 MirrorMaker Configs
  • 3.9 System Properties -
  • 3.10 Tiered Storage Configs
  • +
  • 3.10 Tiered Storage Configs +
  • 3.11 Configuration Providers + -
  • +
  • 4. Design -
  • +
  • 5. Implementation -
  • +
  • 6. Operations -
  • +
  • 7. Security -
  • +
  • 8. Kafka Connect -
  • +
  • 9. Kafka Streams -
  • + From b8c54c3f3839bc02263925ba649a5a2cb4b99a63 Mon Sep 17 00:00:00 2001 From: Josep Prat Date: Tue, 30 Jul 2024 19:04:38 +0200 Subject: [PATCH 007/123] KAFKA-17214: Add 3.8.0 version to streams system tests (#16728) * KAFKA-17214: Add 3.8.0 version to streams system tests Reviewers: Bill Bejeck --- build.gradle | 16 + settings.gradle | 1 + .../apache/kafka/streams/StreamsConfig.java | 10 +- .../streams/internals/UpgradeFromValues.java | 3 +- .../assignment/AssignorConfiguration.java | 2 + .../kafka/streams/tests/SmokeTestClient.java | 299 ++++++++ .../kafka/streams/tests/SmokeTestDriver.java | 670 ++++++++++++++++++ .../kafka/streams/tests/SmokeTestUtil.java | 131 ++++ .../kafka/streams/tests/StreamsSmokeTest.java | 100 +++ .../streams/tests/StreamsUpgradeTest.java | 120 ++++ .../streams_application_upgrade_test.py | 5 +- .../streams_broker_compatibility_test.py | 11 +- .../tests/streams/streams_upgrade_test.py | 6 +- 13 files changed, 1363 insertions(+), 11 deletions(-) create mode 100644 streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java create mode 100644 streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java create mode 100644 streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java create mode 100644 streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java create mode 100644 streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java diff --git a/build.gradle b/build.gradle index 7028c409f8..a15e00c0a6 100644 --- a/build.gradle +++ b/build.gradle @@ -2436,6 +2436,7 @@ project(':streams') { ':streams:upgrade-system-tests-35:test', ':streams:upgrade-system-tests-36:test', ':streams:upgrade-system-tests-37:test', + ':streams:upgrade-system-tests-38:test', ':streams:examples:test' ] ) @@ -2935,6 +2936,21 @@ project(':streams:upgrade-system-tests-37') { } } +project(':streams:upgrade-system-tests-38') { + base { + archivesName = "kafka-streams-upgrade-system-tests-38" + } + + dependencies { + testImplementation libs.kafkaStreams_38 + testRuntimeOnly libs.junitJupiter + } + + systemTestLibs { + dependsOn testJar + } +} + project(':jmh-benchmarks') { apply plugin: 'io.github.goooler.shadow' diff --git a/settings.gradle b/settings.gradle index 09bc70605d..92df42212d 100644 --- a/settings.gradle +++ b/settings.gradle @@ -98,6 +98,7 @@ include 'clients', 'streams:upgrade-system-tests-35', 'streams:upgrade-system-tests-36', 'streams:upgrade-system-tests-37', + 'streams:upgrade-system-tests-38', 'tools', 'tools:tools-api', 'transaction-coordinator', diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index eab567d525..d1ffd6a587 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -428,6 +428,13 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String UPGRADE_FROM_37 = UpgradeFromValues.UPGRADE_FROM_37.toString(); + /** + * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 3.8.x}. + */ + @SuppressWarnings("WeakerAccess") + public static final String UPGRADE_FROM_38 = UpgradeFromValues.UPGRADE_FROM_38.toString(); + + /** * Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for at-least-once processing guarantees. */ @@ -794,7 +801,8 @@ public class StreamsConfig extends AbstractConfig { UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" + UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" + UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + - UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "(for upgrading from the corresponding old version)."; + UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "\", \"" + + UPGRADE_FROM_38 + "(for upgrading from the corresponding old version)."; /** {@code windowstore.changelog.additional.retention.ms} */ @SuppressWarnings("WeakerAccess") diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java b/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java index 2bf19da39b..7d65205c55 100644 --- a/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/internals/UpgradeFromValues.java @@ -39,7 +39,8 @@ public enum UpgradeFromValues { UPGRADE_FROM_34("3.4"), UPGRADE_FROM_35("3.5"), UPGRADE_FROM_36("3.6"), - UPGRADE_FROM_37("3.7"); + UPGRADE_FROM_37("3.7"), + UPGRADE_FROM_38("3.8"); private final String value; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java index 3fc522f85b..28257eacfc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java @@ -129,6 +129,7 @@ public RebalanceProtocol rebalanceProtocol() { case UPGRADE_FROM_35: case UPGRADE_FROM_36: case UPGRADE_FROM_37: + case UPGRADE_FROM_38: // we need to add new version when new "upgrade.from" values become available // This config is for explicitly sending FK response to a requested partition @@ -190,6 +191,7 @@ public int configuredMetadataVersion(final int priorVersion) { case UPGRADE_FROM_35: case UPGRADE_FROM_36: case UPGRADE_FROM_37: + case UPGRADE_FROM_38: // we need to add new version when new "upgrade.from" values become available // This config is for explicitly sending FK response to a requested partition diff --git a/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java new file mode 100644 index 0000000000..dc0ad4d560 --- /dev/null +++ b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.Grouped; +import org.apache.kafka.streams.kstream.KGroupedStream; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Suppressed.BufferConfig; +import org.apache.kafka.streams.kstream.TimeWindows; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowStore; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.streams.kstream.Suppressed.untilWindowCloses; + +public class SmokeTestClient extends SmokeTestUtil { + + private final String name; + + private KafkaStreams streams; + private boolean uncaughtException = false; + private boolean started; + private volatile boolean closed; + + private static void addShutdownHook(final String name, final Runnable runnable) { + if (name != null) { + Runtime.getRuntime().addShutdownHook(KafkaThread.nonDaemon(name, runnable)); + } else { + Runtime.getRuntime().addShutdownHook(new Thread(runnable)); + } + } + + private static File tempDirectory() { + final String prefix = "kafka-"; + final File file; + try { + file = Files.createTempDirectory(prefix).toFile(); + } catch (final IOException ex) { + throw new RuntimeException("Failed to create a temp dir", ex); + } + file.deleteOnExit(); + + addShutdownHook("delete-temp-file-shutdown-hook", () -> { + try { + Utils.delete(file); + } catch (final IOException e) { + System.out.println("Error deleting " + file.getAbsolutePath()); + e.printStackTrace(System.out); + } + }); + + return file; + } + + public SmokeTestClient(final String name) { + this.name = name; + } + + public boolean started() { + return started; + } + + public boolean closed() { + return closed; + } + + public void start(final Properties streamsProperties) { + final Topology build = getTopology(); + streams = new KafkaStreams(build, getStreamsConfig(streamsProperties)); + + final CountDownLatch countDownLatch = new CountDownLatch(1); + streams.setStateListener((newState, oldState) -> { + System.out.printf("%s %s: %s -> %s%n", name, Instant.now(), oldState, newState); + if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { + started = true; + countDownLatch.countDown(); + } + + if (newState == KafkaStreams.State.NOT_RUNNING) { + closed = true; + } + }); + + streams.setUncaughtExceptionHandler(e -> { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + System.out.println(name + ": FATAL: An unexpected exception is encountered: " + e); + e.printStackTrace(System.out); + uncaughtException = true; + return StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; + }); + + addShutdownHook("streams-shutdown-hook", this::close); + + streams.start(); + try { + if (!countDownLatch.await(1, TimeUnit.MINUTES)) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't start in one minute"); + } + } catch (final InterruptedException e) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: " + e); + e.printStackTrace(System.out); + } + System.out.println(name + ": SMOKE-TEST-CLIENT-STARTED"); + System.out.println(name + " started at " + Instant.now()); + } + + public void closeAsync() { + streams.close(Duration.ZERO); + } + + public void close() { + final boolean closed = streams.close(Duration.ofMinutes(1)); + + if (closed && !uncaughtException) { + System.out.println(name + ": SMOKE-TEST-CLIENT-CLOSED"); + } else if (closed) { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION"); + } else { + System.out.println(name + ": SMOKE-TEST-CLIENT-EXCEPTION: Didn't close"); + } + } + + private Properties getStreamsConfig(final Properties props) { + final Properties fullProps = new Properties(props); + fullProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); + fullProps.put(StreamsConfig.CLIENT_ID_CONFIG, "SmokeTest-" + name); + fullProps.put(StreamsConfig.STATE_DIR_CONFIG, tempDirectory().getAbsolutePath()); + fullProps.putAll(props); + return fullProps; + } + + public Topology getTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); + final KStream source = builder.stream("data", stringIntConsumed); + source.filterNot((k, v) -> k.equals("flush")) + .to("echo", Produced.with(stringSerde, intSerde)); + final KStream data = source.filter((key, value) -> value == null || value != END); + data.process(SmokeTestUtil.printProcessorSupplier("data", name)); + + // min + final KGroupedStream groupedData = data.groupByKey(Grouped.with(stringSerde, intSerde)); + + final KTable, Integer> minAggregation = groupedData + .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofDays(1), Duration.ofMinutes(1))) + .aggregate( + () -> Integer.MAX_VALUE, + (aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate, + Materialized + .>as("uwin-min") + .withValueSerde(intSerde) + .withRetention(Duration.ofHours(25)) + ); + + streamify(minAggregation, "min-raw"); + + streamify(minAggregation.suppress(untilWindowCloses(BufferConfig.unbounded())), "min-suppressed"); + + minAggregation + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("min", Produced.with(stringSerde, intSerde)); + + final KTable, Integer> smallWindowSum = groupedData + .windowedBy(TimeWindows.ofSizeAndGrace(Duration.ofSeconds(2), Duration.ofSeconds(30)).advanceBy(Duration.ofSeconds(1))) + .reduce(Integer::sum); + + streamify(smallWindowSum, "sws-raw"); + streamify(smallWindowSum.suppress(untilWindowCloses(BufferConfig.unbounded())), "sws-suppressed"); + + final KTable minTable = builder.table( + "min", + Consumed.with(stringSerde, intSerde), + Materialized.as("minStoreName")); + + minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min", name)); + + // max + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .aggregate( + () -> Integer.MIN_VALUE, + (aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate, + Materialized.>as("uwin-max").withValueSerde(intSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("max", Produced.with(stringSerde, intSerde)); + + final KTable maxTable = builder.table( + "max", + Consumed.with(stringSerde, intSerde), + Materialized.as("maxStoreName")); + maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max", name)); + + // sum + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .aggregate( + () -> 0L, + (aggKey, value, aggregate) -> (long) value + aggregate, + Materialized.>as("win-sum").withValueSerde(longSerde)) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("sum", Produced.with(stringSerde, longSerde)); + + final Consumed stringLongConsumed = Consumed.with(stringSerde, longSerde); + final KTable sumTable = builder.table("sum", stringLongConsumed); + sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum", name)); + + // cnt + groupedData + .windowedBy(TimeWindows.ofSizeWithNoGrace(Duration.ofDays(2))) + .count(Materialized.as("uwin-cnt")) + .toStream(new Unwindow<>()) + .filterNot((k, v) -> k.equals("flush")) + .to("cnt", Produced.with(stringSerde, longSerde)); + + final KTable cntTable = builder.table( + "cnt", + Consumed.with(stringSerde, longSerde), + Materialized.as("cntStoreName")); + cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt", name)); + + // dif + maxTable + .join( + minTable, + (value1, value2) -> value1 - value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("dif", Produced.with(stringSerde, intSerde)); + + // avg + sumTable + .join( + cntTable, + (value1, value2) -> (double) value1 / (double) value2) + .toStream() + .filterNot((k, v) -> k.equals("flush")) + .to("avg", Produced.with(stringSerde, doubleSerde)); + + // test repartition + final Agg agg = new Agg(); + cntTable.groupBy(agg.selector(), Grouped.with(stringSerde, longSerde)) + .aggregate(agg.init(), agg.adder(), agg.remover(), + Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) + .withKeySerde(Serdes.String()) + .withValueSerde(Serdes.Long())) + .toStream() + .to("tagg", Produced.with(stringSerde, longSerde)); + + return builder.build(); + } + + private static void streamify(final KTable, Integer> windowedTable, final String topic) { + windowedTable + .toStream() + .filterNot((k, v) -> k.key().equals("flush")) + .map((key, value) -> new KeyValue<>(key.toString(), value)) + .to(topic, Produced.with(stringSerde, intSerde)); + } +} diff --git a/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java new file mode 100644 index 0000000000..dbacbb9625 --- /dev/null +++ b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -0,0 +1,670 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.apache.kafka.common.utils.Utils.mkEntry; + +public class SmokeTestDriver extends SmokeTestUtil { + private static final String[] NUMERIC_VALUE_TOPICS = { + "data", + "echo", + "max", + "min", "min-suppressed", "min-raw", + "dif", + "sum", + "sws-raw", "sws-suppressed", + "cnt", + "avg", + "tagg" + }; + private static final String[] STRING_VALUE_TOPICS = { + "fk" + }; + + private static final String[] TOPICS = new String[NUMERIC_VALUE_TOPICS.length + STRING_VALUE_TOPICS.length]; + static { + System.arraycopy(NUMERIC_VALUE_TOPICS, 0, TOPICS, 0, NUMERIC_VALUE_TOPICS.length); + System.arraycopy(STRING_VALUE_TOPICS, 0, TOPICS, NUMERIC_VALUE_TOPICS.length, STRING_VALUE_TOPICS.length); + } + + private static final int MAX_RECORD_EMPTY_RETRIES = 30; + + private static class ValueList { + public final String key; + private final int[] values; + private int index; + + ValueList(final int min, final int max) { + key = min + "-" + max; + + values = new int[max - min + 1]; + for (int i = 0; i < values.length; i++) { + values[i] = min + i; + } + // We want to randomize the order of data to test not completely predictable processing order + // However, values are also use as a timestamp of the record. (TODO: separate data and timestamp) + // We keep some correlation of time and order. Thus, the shuffling is done with a sliding window + shuffle(values, 10); + + index = 0; + } + + int next() { + return (index < values.length) ? values[index++] : -1; + } + } + + public static String[] topics() { + return Arrays.copyOf(TOPICS, TOPICS.length); + } + + static void generatePerpetually(final String kafka, + final int numKeys, + final int maxRecordsPerKey) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + } + + final Random rand = new Random(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (true) { + final int index = rand.nextInt(numKeys); + final String key = data[index].key; + final int value = data[index].next(); + + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + producer.send(record); + + final ProducerRecord fkRecord = + new ProducerRecord<>( + "fk", + intSerde.serializer().serialize("", value), + stringSerde.serializer().serialize("", key) + ); + producer.send(fkRecord); + + numRecordsProduced++; + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(2); + } + } + } + + public static Map> generate(final String kafka, + final int numKeys, + final int maxRecordsPerKey, + final Duration timeToSpend) { + final Properties producerProps = generatorProperties(kafka); + + int numRecordsProduced = 0; + + final Map> allData = new HashMap<>(); + final ValueList[] data = new ValueList[numKeys]; + for (int i = 0; i < numKeys; i++) { + data[i] = new ValueList(i, i + maxRecordsPerKey - 1); + allData.put(data[i].key, new HashSet<>()); + } + final Random rand = new Random(); + + int remaining = data.length; + + final long recordPauseTime = timeToSpend.toMillis() / numKeys / maxRecordsPerKey; + + final List> dataNeedRetry = new ArrayList<>(); + final List> fkNeedRetry = new ArrayList<>(); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + while (remaining > 0) { + final int index = rand.nextInt(remaining); + final String key = data[index].key; + final int value = data[index].next(); + + if (value < 0) { + remaining--; + data[index] = data[remaining]; + } else { + final ProducerRecord record = + new ProducerRecord<>( + "data", + stringSerde.serializer().serialize("", key), + intSerde.serializer().serialize("", value) + ); + + producer.send(record, new TestCallback(record, dataNeedRetry)); + + final ProducerRecord fkRecord = + new ProducerRecord<>( + "fk", + intSerde.serializer().serialize("", value), + stringSerde.serializer().serialize("", key) + ); + + producer.send(fkRecord, new TestCallback(fkRecord, fkNeedRetry)); + + numRecordsProduced++; + allData.get(key).add(value); + if (numRecordsProduced % 100 == 0) { + System.out.println(Instant.now() + " " + numRecordsProduced + " records produced"); + } + Utils.sleep(Math.max(recordPauseTime, 2)); + } + } + producer.flush(); + + retry(producer, dataNeedRetry, stringSerde); + retry(producer, fkNeedRetry, intSerde); + + flush(producer, + "data", + stringSerde.serializer().serialize("", "flush"), + intSerde.serializer().serialize("", 0) + ); + flush(producer, + "fk", + intSerde.serializer().serialize("", 0), + stringSerde.serializer().serialize("", "flush") + ); + } + return Collections.unmodifiableMap(allData); + } + + private static void retry(final KafkaProducer producer, + List> needRetry, + final Serde keySerde) { + int remainingRetries = 5; + while (!needRetry.isEmpty()) { + final List> needRetry2 = new ArrayList<>(); + for (final ProducerRecord record : needRetry) { + System.out.println( + "retry producing " + keySerde.deserializer().deserialize("", record.key())); + producer.send(record, new TestCallback(record, needRetry2)); + } + producer.flush(); + needRetry = needRetry2; + if (--remainingRetries == 0 && !needRetry.isEmpty()) { + System.err.println("Failed to produce all records after multiple retries"); + Exit.exit(1); + } + } + } + + private static void flush(final KafkaProducer producer, + final String topic, + final byte[] keyBytes, + final byte[] valBytes) { + // now that we've sent everything, we'll send some final records with a timestamp high enough to flush out + // all suppressed records. + final List partitions = producer.partitionsFor(topic); + for (final PartitionInfo partition : partitions) { + producer.send(new ProducerRecord<>( + partition.topic(), + partition.partition(), + System.currentTimeMillis() + Duration.ofDays(2).toMillis(), + keyBytes, + valBytes + )); + } + } + + private static Properties generatorProperties(final String kafka) { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + return producerProps; + } + + private static class TestCallback implements Callback { + private final ProducerRecord originalRecord; + private final List> needRetry; + + TestCallback(final ProducerRecord originalRecord, + final List> needRetry) { + this.originalRecord = originalRecord; + this.needRetry = needRetry; + } + + @Override + public void onCompletion(final RecordMetadata metadata, final Exception exception) { + if (exception != null) { + if (exception instanceof TimeoutException) { + needRetry.add(originalRecord); + } else { + exception.printStackTrace(); + Exit.exit(1); + } + } + } + } + + private static void shuffle(final int[] data, @SuppressWarnings("SameParameterValue") final int windowSize) { + final Random rand = new Random(); + for (int i = 0; i < data.length; i++) { + // we shuffle data within windowSize + final int j = rand.nextInt(Math.min(data.length - i, windowSize)) + i; + + // swap + final int tmp = data[i]; + data[i] = data[j]; + data[j] = tmp; + } + } + + public static class NumberDeserializer implements Deserializer { + @Override + public Number deserialize(final String topic, final byte[] data) { + final Number value; + switch (topic) { + case "data": + case "echo": + case "min": + case "min-raw": + case "min-suppressed": + case "sws-raw": + case "sws-suppressed": + case "max": + case "dif": + value = intSerde.deserializer().deserialize(topic, data); + break; + case "sum": + case "cnt": + case "tagg": + value = longSerde.deserializer().deserialize(topic, data); + break; + case "avg": + value = doubleSerde.deserializer().deserialize(topic, data); + break; + default: + throw new RuntimeException("unknown topic: " + topic); + } + return value; + } + } + + public static VerificationResult verify(final String kafka, + final Map> inputs, + final int maxRecordsPerKey) { + final Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class); + props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final List partitions = getAllPartitions(consumer, NUMERIC_VALUE_TOPICS); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + final int recordsGenerated = inputs.size() * maxRecordsPerKey; + int recordsProcessed = 0; + final Map processed = + Stream.of(NUMERIC_VALUE_TOPICS) + .collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0))); + + final Map>>> events = new HashMap<>(); + + VerificationResult verificationResult = new VerificationResult(false, "no results yet"); + int retry = 0; + final long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { + final ConsumerRecords records = consumer.poll(Duration.ofSeconds(5)); + if (records.isEmpty() && recordsProcessed >= recordsGenerated) { + verificationResult = verifyAll(inputs, events, false); + if (verificationResult.passed()) { + break; + } else if (retry++ > MAX_RECORD_EMPTY_RETRIES) { + System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries."); + break; + } else { + System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry); + } + } else { + System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry."); + + retry = 0; + for (final ConsumerRecord record : records) { + final String key = record.key(); + + final String topic = record.topic(); + processed.get(topic).incrementAndGet(); + + if (topic.equals("echo")) { + recordsProcessed++; + if (recordsProcessed % 100 == 0) { + System.out.println("Echo records processed = " + recordsProcessed); + } + } + + events.computeIfAbsent(topic, t -> new HashMap<>()) + .computeIfAbsent(key, k -> new LinkedList<>()) + .add(record); + } + + System.out.println(processed); + } + } + consumer.close(); + final long finished = System.currentTimeMillis() - start; + System.out.println("Verification time=" + finished); + System.out.println("-------------------"); + System.out.println("Result Verification"); + System.out.println("-------------------"); + System.out.println("recordGenerated=" + recordsGenerated); + System.out.println("recordProcessed=" + recordsProcessed); + + if (recordsProcessed > recordsGenerated) { + System.out.println("PROCESSED-MORE-THAN-GENERATED"); + } else if (recordsProcessed < recordsGenerated) { + System.out.println("PROCESSED-LESS-THAN-GENERATED"); + } + + boolean success; + + final Map> received = + events.get("echo") + .entrySet() + .stream() + .map(entry -> mkEntry( + entry.getKey(), + entry.getValue().stream().map(ConsumerRecord::value).collect(Collectors.toSet())) + ) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + success = inputs.equals(received); + + if (success) { + System.out.println("ALL-RECORDS-DELIVERED"); + } else { + int missedCount = 0; + for (final Map.Entry> entry : inputs.entrySet()) { + missedCount += received.get(entry.getKey()).size(); + } + System.out.println("missedRecords=" + missedCount); + } + + // give it one more try if it's not already passing. + if (!verificationResult.passed()) { + verificationResult = verifyAll(inputs, events, true); + } + success &= verificationResult.passed(); + + System.out.println(verificationResult.result()); + + System.out.println(success ? "SUCCESS" : "FAILURE"); + return verificationResult; + } + + public static class VerificationResult { + private final boolean passed; + private final String result; + + VerificationResult(final boolean passed, final String result) { + this.passed = passed; + this.result = result; + } + + public boolean passed() { + return passed; + } + + public String result() { + return result; + } + } + + private static VerificationResult verifyAll(final Map> inputs, + final Map>>> events, + final boolean printResults) { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + boolean pass; + try (final PrintStream resultStream = new PrintStream(byteArrayOutputStream)) { + pass = verifyTAgg(resultStream, inputs, events.get("tagg"), printResults); + pass &= verifySuppressed(resultStream, "min-suppressed", events, printResults); + pass &= verify(resultStream, "min-suppressed", inputs, events, windowedKey -> { + final String unwindowedKey = windowedKey.substring(1, windowedKey.length() - 1).replaceAll("@.*", ""); + return getMin(unwindowedKey); + }, printResults); + pass &= verifySuppressed(resultStream, "sws-suppressed", events, printResults); + pass &= verify(resultStream, "min", inputs, events, SmokeTestDriver::getMin, printResults); + pass &= verify(resultStream, "max", inputs, events, SmokeTestDriver::getMax, printResults); + pass &= verify(resultStream, "dif", inputs, events, key -> getMax(key).intValue() - getMin(key).intValue(), printResults); + pass &= verify(resultStream, "sum", inputs, events, SmokeTestDriver::getSum, printResults); + pass &= verify(resultStream, "cnt", inputs, events, key1 -> getMax(key1).intValue() - getMin(key1).intValue() + 1L, printResults); + pass &= verify(resultStream, "avg", inputs, events, SmokeTestDriver::getAvg, printResults); + } + return new VerificationResult(pass, new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8)); + } + + private static boolean verify(final PrintStream resultStream, + final String topic, + final Map> inputData, + final Map>>> events, + final Function keyToExpectation, + final boolean printResults) { + final Map>> observedInputEvents = events.get("data"); + final Map>> outputEvents = events.getOrDefault(topic, emptyMap()); + if (outputEvents.isEmpty()) { + resultStream.println(topic + " is empty"); + return false; + } else { + resultStream.printf("verifying %s with %d keys%n", topic, outputEvents.size()); + + if (outputEvents.size() != inputData.size()) { + resultStream.printf("fail: resultCount=%d expectedCount=%s%n\tresult=%s%n\texpected=%s%n", + outputEvents.size(), inputData.size(), outputEvents.keySet(), inputData.keySet()); + return false; + } + for (final Map.Entry>> entry : outputEvents.entrySet()) { + final String key = entry.getKey(); + final Number expected = keyToExpectation.apply(key); + final Number actual = entry.getValue().getLast().value(); + if (!expected.equals(actual)) { + resultStream.printf("%s fail: key=%s actual=%s expected=%s%n", topic, key, actual, expected); + + if (printResults) { + resultStream.printf("\t inputEvents=%n%s%n\t" + + "echoEvents=%n%s%n\tmaxEvents=%n%s%n\tminEvents=%n%s%n\tdifEvents=%n%s%n\tcntEvents=%n%s%n\ttaggEvents=%n%s%n", + indent("\t\t", observedInputEvents.get(key)), + indent("\t\t", events.getOrDefault("echo", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("max", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("min", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("dif", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("cnt", emptyMap()).getOrDefault(key, new LinkedList<>())), + indent("\t\t", events.getOrDefault("tagg", emptyMap()).getOrDefault(key, new LinkedList<>()))); + + if (!Utils.mkSet("echo", "max", "min", "dif", "cnt", "tagg").contains(topic)) + resultStream.printf("%sEvents=%n%s%n", topic, indent("\t\t", entry.getValue())); + } + + return false; + } + } + return true; + } + } + + + private static boolean verifySuppressed(final PrintStream resultStream, + @SuppressWarnings("SameParameterValue") final String topic, + final Map>>> events, + final boolean printResults) { + resultStream.println("verifying suppressed " + topic); + final Map>> topicEvents = events.getOrDefault(topic, emptyMap()); + for (final Map.Entry>> entry : topicEvents.entrySet()) { + if (entry.getValue().size() != 1) { + final String unsuppressedTopic = topic.replace("-suppressed", "-raw"); + final String key = entry.getKey(); + final String unwindowedKey = key.substring(1, key.length() - 1).replaceAll("@.*", ""); + resultStream.printf("fail: key=%s%n\tnon-unique result:%n%s%n", + key, + indent("\t\t", entry.getValue())); + + if (printResults) + resultStream.printf("\tresultEvents:%n%s%n\tinputEvents:%n%s%n", + indent("\t\t", events.get(unsuppressedTopic).get(key)), + indent("\t\t", events.get("data").get(unwindowedKey))); + + return false; + } + } + return true; + } + + private static String indent(@SuppressWarnings("SameParameterValue") final String prefix, + final Iterable> list) { + final StringBuilder stringBuilder = new StringBuilder(); + for (final ConsumerRecord record : list) { + stringBuilder.append(prefix).append(record).append('\n'); + } + return stringBuilder.toString(); + } + + private static Long getSum(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) * (max - min + 1L) / 2L; + } + + private static Double getAvg(final String key) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + return ((long) min + max) / 2.0; + } + + + private static boolean verifyTAgg(final PrintStream resultStream, + final Map> allData, + final Map>> taggEvents, + final boolean printResults) { + if (taggEvents == null) { + resultStream.println("tagg is missing"); + return false; + } else if (taggEvents.isEmpty()) { + resultStream.println("tagg is empty"); + return false; + } else { + resultStream.println("verifying tagg"); + + // generate expected answer + final Map expected = new HashMap<>(); + for (final String key : allData.keySet()) { + final int min = getMin(key).intValue(); + final int max = getMax(key).intValue(); + final String cnt = Long.toString(max - min + 1L); + + expected.put(cnt, expected.getOrDefault(cnt, 0L) + 1); + } + + // check the result + for (final Map.Entry>> entry : taggEvents.entrySet()) { + final String key = entry.getKey(); + Long expectedCount = expected.remove(key); + if (expectedCount == null) { + expectedCount = 0L; + } + + if (entry.getValue().getLast().value().longValue() != expectedCount) { + resultStream.println("fail: key=" + key + " tagg=" + entry.getValue() + " expected=" + expectedCount); + + if (printResults) + resultStream.println("\t taggEvents: " + entry.getValue()); + return false; + } + } + + } + return true; + } + + private static Number getMin(final String key) { + return Integer.parseInt(key.split("-")[0]); + } + + private static Number getMax(final String key) { + return Integer.parseInt(key.split("-")[1]); + } + + private static List getAllPartitions(final KafkaConsumer consumer, final String... topics) { + final List partitions = new ArrayList<>(); + + for (final String topic : topics) { + for (final PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} diff --git a/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java new file mode 100644 index 0000000000..0ba228a381 --- /dev/null +++ b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; + +import java.time.Instant; + +public class SmokeTestUtil { + + static final int END = Integer.MAX_VALUE; + + static ProcessorSupplier printProcessorSupplier(final String topic) { + return printProcessorSupplier(topic, ""); + } + + static ProcessorSupplier printProcessorSupplier(final String topic, final String name) { + return () -> new ContextualProcessor() { + private int numRecordsProcessed = 0; + private long smallestOffset = Long.MAX_VALUE; + private long largestOffset = Long.MIN_VALUE; + + @Override + public void init(final ProcessorContext context) { + super.init(context); + System.out.println("[3.8] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.flush(); + numRecordsProcessed = 0; + smallestOffset = Long.MAX_VALUE; + largestOffset = Long.MIN_VALUE; + } + + @Override + public void process(final Record record) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.printf("%s: %s%n", name, Instant.now()); + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + + context().recordMetadata().ifPresent(recordMetadata -> { + if (smallestOffset > recordMetadata.offset()) { + smallestOffset = recordMetadata.offset(); + } + if (largestOffset < recordMetadata.offset()) { + largestOffset = recordMetadata.offset(); + } + }); + } + + @Override + public void close() { + System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.println("processed " + numRecordsProcessed + " records"); + final long processed; + if (largestOffset >= smallestOffset) { + processed = 1L + largestOffset - smallestOffset; + } else { + processed = 0L; + } + System.out.println("offset " + smallestOffset + " to " + largestOffset + " -> processed " + processed); + System.out.flush(); + } + }; + } + + public static final class Unwindow implements KeyValueMapper, V, K> { + @Override + public K apply(final Windowed winKey, final V value) { + return winKey.key(); + } + } + + public static class Agg { + + KeyValueMapper> selector() { + return (key, value) -> new KeyValue<>(value == null ? null : Long.toString(value), 1L); + } + + public Initializer init() { + return () -> 0L; + } + + Aggregator adder() { + return (aggKey, value, aggregate) -> aggregate + value; + } + + Aggregator remover() { + return (aggKey, value, aggregate) -> aggregate - value; + } + } + + public static Serde stringSerde = Serdes.String(); + + public static Serde intSerde = Serdes.Integer(); + + static Serde longSerde = Serdes.Long(); + + static Serde doubleSerde = Serdes.Double(); + + public static void sleep(final long duration) { + try { + Thread.sleep(duration); + } catch (final Exception ignore) { } + } + +} diff --git a/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java new file mode 100644 index 0000000000..5803b2fbd0 --- /dev/null +++ b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; + +import static org.apache.kafka.streams.tests.SmokeTestDriver.generate; +import static org.apache.kafka.streams.tests.SmokeTestDriver.generatePerpetually; + +public class StreamsSmokeTest { + + /** + * args ::= kafka propFileName command disableAutoTerminate + * command := "run" | "process" + * + * @param args + */ + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); + Exit.exit(1); + } + + final String propFileName = args[0]; + final String command = args[1]; + final boolean disableAutoTerminate = args.length > 2; + + final Properties streamsProperties = Utils.loadProps(propFileName); + final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + + if (kafka == null) { + System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG); + Exit.exit(1); + } + + if ("process".equals(command)) { + if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE_V2.equals(processingGuarantee)) { + + System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE_V2); + + Exit.exit(1); + } + } + + System.out.println("StreamsTest instance started (StreamsSmokeTest)"); + System.out.println("command=" + command); + System.out.println("props=" + streamsProperties); + System.out.println("disableAutoTerminate=" + disableAutoTerminate); + + switch (command) { + case "run": + // this starts the driver (data generation and result verification) + final int numKeys = 10; + final int maxRecordsPerKey = 500; + if (disableAutoTerminate) { + generatePerpetually(kafka, numKeys, maxRecordsPerKey); + } else { + // slow down data production to span 30 seconds so that system tests have time to + // do their bounces, etc. + final Map> allData = + generate(kafka, numKeys, maxRecordsPerKey, Duration.ofSeconds(30)); + SmokeTestDriver.verify(kafka, allData, maxRecordsPerKey); + } + break; + case "process": + // this starts the stream processing app + new SmokeTestClient(UUID.randomUUID().toString()).start(streamsProperties); + break; + default: + System.out.println("unknown command: " + command); + } + } + +} diff --git a/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java new file mode 100644 index 0000000000..157f2ae412 --- /dev/null +++ b/streams/upgrade-system-tests-38/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.tests; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; + +import java.util.Properties; + +import static org.apache.kafka.streams.tests.SmokeTestUtil.intSerde; +import static org.apache.kafka.streams.tests.SmokeTestUtil.stringSerde; + + +public class StreamsUpgradeTest { + + @SuppressWarnings("unchecked") + public static void main(final String[] args) throws Exception { + if (args.length < 1) { + System.err.println("StreamsUpgradeTest requires one argument (properties-file) but provided none"); + } + final String propFileName = args[0]; + + final Properties streamsProperties = Utils.loadProps(propFileName); + + System.out.println("StreamsTest instance started (StreamsUpgradeTest v3.7)"); + System.out.println("props=" + streamsProperties); + + final StreamsBuilder builder = new StreamsBuilder(); + final KTable dataTable = builder.table( + "data", Consumed.with(stringSerde, intSerde)); + final KStream dataStream = dataTable.toStream(); + dataStream.process(printProcessorSupplier("data")); + dataStream.to("echo"); + + final boolean runFkJoin = Boolean.parseBoolean(streamsProperties.getProperty( + "test.run_fk_join", + "false")); + if (runFkJoin) { + try { + final KTable fkTable = builder.table( + "fk", Consumed.with(intSerde, stringSerde)); + buildFKTable(dataStream, fkTable); + } catch (final Exception e) { + System.err.println("Caught " + e.getMessage()); + } + } + + final Properties config = new Properties(); + config.setProperty( + StreamsConfig.APPLICATION_ID_CONFIG, + "StreamsUpgradeTest"); + config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + config.putAll(streamsProperties); + + final KafkaStreams streams = new KafkaStreams(builder.build(), config); + streams.start(); + + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + streams.close(); + System.out.println("UPGRADE-TEST-CLIENT-CLOSED"); + System.out.flush(); + })); + } + + private static void buildFKTable(final KStream primaryTable, + final KTable otherTable) { + final KStream kStream = primaryTable.toTable() + .join(otherTable, v -> v, (k0, v0) -> v0) + .toStream(); + kStream.process(printProcessorSupplier("fk")); + kStream.to("fk-result", Produced.with(stringSerde, stringSerde)); + } + + private static ProcessorSupplier printProcessorSupplier(final String topic) { + return () -> new ContextualProcessor() { + private int numRecordsProcessed = 0; + + @Override + public void init(final ProcessorContext context) { + System.out.println("[3.8] initializing processor: topic=" + topic + "taskId=" + context.taskId()); + numRecordsProcessed = 0; + } + + @Override + public void process(final Record record) { + numRecordsProcessed++; + if (numRecordsProcessed % 100 == 0) { + System.out.println("processed " + numRecordsProcessed + " records from topic=" + topic); + } + } + + @Override + public void close() {} + }; + } +} diff --git a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py index 46234af1f7..f728470d86 100644 --- a/tests/kafkatest/tests/streams/streams_application_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_application_upgrade_test.py @@ -22,13 +22,14 @@ from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService from kafkatest.services.zookeeper import ZookeeperService from kafkatest.version import LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ - LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, DEV_VERSION, KafkaVersion + LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, DEV_VERSION, KafkaVersion smoke_test_versions = [str(LATEST_2_2), str(LATEST_2_3), str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), - str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7)] + str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), + str(LATEST_3_8)] class StreamsUpgradeTest(Test): """ diff --git a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py index 0a6773ba66..e70b9a667e 100644 --- a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py @@ -24,7 +24,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.version import LATEST_0_11_0, LATEST_0_10_2, LATEST_0_10_1, LATEST_0_10_0, LATEST_1_0, LATEST_1_1, \ LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ - LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, KafkaVersion + LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, KafkaVersion class StreamsBrokerCompatibility(Test): @@ -69,7 +69,8 @@ def setUp(self): str(LATEST_2_1),str(LATEST_2_2),str(LATEST_2_3),str(LATEST_2_4), str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8), str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3), - str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)]) + str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7), + str(LATEST_3_8)]) def test_compatible_brokers_eos_disabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() @@ -91,7 +92,8 @@ def test_compatible_brokers_eos_disabled(self, broker_version): str(LATEST_2_1),str(LATEST_2_2),str(LATEST_2_3),str(LATEST_2_4), str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8), str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3), - str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)]) + str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7), + str(LATEST_3_8)]) def test_compatible_brokers_eos_alpha_enabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() @@ -111,7 +113,8 @@ def test_compatible_brokers_eos_alpha_enabled(self, broker_version): @cluster(num_nodes=4) @matrix(broker_version=[str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8), str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3), - str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7)]) + str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7), + str(LATEST_3_8)]) def test_compatible_brokers_eos_v2_enabled(self, broker_version): self.kafka.set_version(KafkaVersion(broker_version)) self.kafka.start() diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index d1607aee70..7b0ff2f341 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -26,7 +26,7 @@ from kafkatest.tests.streams.utils import extract_generation_from_logs, extract_generation_id from kafkatest.version import LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, LATEST_1_1, \ LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ - LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, DEV_BRANCH, DEV_VERSION, \ + LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, DEV_BRANCH, DEV_VERSION, \ KafkaVersion # broker 0.10.0 is not compatible with newer Kafka Streams versions @@ -36,7 +36,7 @@ str(LATEST_2_4), str(LATEST_2_5), str(LATEST_2_6), str(LATEST_2_7), str(LATEST_2_8), str(LATEST_3_0), str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), - str(LATEST_3_7), str(DEV_BRANCH)] + str(LATEST_3_7), str(LATEST_3_8), str(DEV_BRANCH)] metadata_1_versions = [str(LATEST_0_10_0)] metadata_2_versions = [str(LATEST_0_10_1), str(LATEST_0_10_2), str(LATEST_0_11_0), str(LATEST_1_0), str(LATEST_1_1), @@ -46,7 +46,7 @@ # -> https://issues.apache.org/jira/browse/KAFKA-14646 # thus, we cannot test two bounce rolling upgrade because we know it's broken # instead we add version 2.4...3.3 to the `metadata_2_versions` upgrade list -fk_join_versions = [str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7)] +fk_join_versions = [str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8)] """ From 3d2ea547d8253115b77756624756de21e243bb39 Mon Sep 17 00:00:00 2001 From: Josep Prat Date: Tue, 30 Jul 2024 19:39:10 +0200 Subject: [PATCH 008/123] KAFKA-17214: Add 3.8.0 version to core and client system tests (#16726) Reviewers: Greg Harris --- .../tests/client/client_compatibility_features_test.py | 3 ++- .../client/client_compatibility_produce_consume_test.py | 3 ++- .../tests/core/compatibility_test_new_broker_test.py | 3 ++- tests/kafkatest/tests/core/downgrade_test.py | 4 +++- tests/kafkatest/tests/core/upgrade_test.py | 5 ++++- 5 files changed, 13 insertions(+), 5 deletions(-) diff --git a/tests/kafkatest/tests/client/client_compatibility_features_test.py b/tests/kafkatest/tests/client/client_compatibility_features_test.py index f29f1df0b4..2384c4d658 100644 --- a/tests/kafkatest/tests/client/client_compatibility_features_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_features_test.py @@ -29,7 +29,7 @@ from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, \ LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, \ LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, \ - V_0_11_0_0, V_0_10_1_0, KafkaVersion + LATEST_3_8, V_0_11_0_0, V_0_10_1_0, KafkaVersion def get_broker_features(broker_version): features = {} @@ -141,6 +141,7 @@ def invoke_compatibility_program(self, features): @parametrize(broker_version=str(LATEST_3_5)) @parametrize(broker_version=str(LATEST_3_6)) @parametrize(broker_version=str(LATEST_3_7)) + @parametrize(broker_version=str(LATEST_3_8)) def run_compatibility_test(self, broker_version, metadata_quorum=quorum.zk): if self.zk: self.zk.start() diff --git a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py index afa69eb660..6a73443796 100644 --- a/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py +++ b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py @@ -26,7 +26,7 @@ from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, LATEST_1_0, \ LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, \ LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, \ - KafkaVersion + LATEST_3_8, KafkaVersion class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest): """ @@ -82,6 +82,7 @@ def min_cluster_size(self): @parametrize(broker_version=str(LATEST_3_5)) @parametrize(broker_version=str(LATEST_3_6)) @parametrize(broker_version=str(LATEST_3_7)) + @parametrize(broker_version=str(LATEST_3_8)) def test_produce_consume(self, broker_version, metadata_quorum=quorum.zk): print("running producer_consumer_compat with broker_version = %s" % broker_version, flush=True) self.kafka.set_version(KafkaVersion(broker_version)) diff --git a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py index d25f9992c6..35edaffc64 100644 --- a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py +++ b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py @@ -24,7 +24,7 @@ from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, LATEST_0_11_0, \ LATEST_1_0, LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, \ LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, \ - LATEST_3_7, DEV_BRANCH, KafkaVersion + LATEST_3_7, LATEST_3_8, DEV_BRANCH, KafkaVersion # Compatibility tests for moving to a new broker (e.g., 0.10.x) and using a mix of old and new clients (e.g., 0.9.x) class ClientCompatibilityTestNewBroker(ProduceConsumeValidateTest): @@ -65,6 +65,7 @@ def setUp(self): @matrix(producer_version=[str(LATEST_3_5)], consumer_version=[str(LATEST_3_5)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_3_6)], consumer_version=[str(LATEST_3_6)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_3_7)], consumer_version=[str(LATEST_3_7)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) + @matrix(producer_version=[str(LATEST_3_8)], consumer_version=[str(LATEST_3_8)], compression_types=[["none"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_2_1)], consumer_version=[str(LATEST_2_1)], compression_types=[["zstd"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_2_0)], consumer_version=[str(LATEST_2_0)], compression_types=[["snappy"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) @matrix(producer_version=[str(LATEST_1_1)], consumer_version=[str(LATEST_1_1)], compression_types=[["lz4"]], timestamp_type=[str("CreateTime")], metadata_quorum=quorum.all_non_upgrade) diff --git a/tests/kafkatest/tests/core/downgrade_test.py b/tests/kafkatest/tests/core/downgrade_test.py index a2ada3d868..68f102e874 100644 --- a/tests/kafkatest/tests/core/downgrade_test.py +++ b/tests/kafkatest/tests/core/downgrade_test.py @@ -21,7 +21,7 @@ from kafkatest.tests.end_to_end import EndToEndTest from kafkatest.version import LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, \ LATEST_2_6, LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, LATEST_3_7, DEV_BRANCH, KafkaVersion + LATEST_3_6, LATEST_3_7, LATEST_3_8, DEV_BRANCH, KafkaVersion class TestDowngrade(EndToEndTest): PARTITIONS = 3 @@ -81,6 +81,8 @@ def wait_until_rejoin(self): timeout_sec=60, backoff_sec=1, err_msg="Replicas did not rejoin the ISR in a reasonable amount of time") @cluster(num_nodes=7) + @parametrize(version=str(LATEST_3_8), compression_types=["snappy"]) + @parametrize(version=str(LATEST_3_8), compression_types=["zstd"], security_protocol="SASL_SSL") @parametrize(version=str(LATEST_3_7), compression_types=["snappy"]) @parametrize(version=str(LATEST_3_7), compression_types=["zstd"], security_protocol="SASL_SSL") @matrix(version=[str(LATEST_3_7)], compression_types=[["none"]], static_membership=[False, True]) diff --git a/tests/kafkatest/tests/core/upgrade_test.py b/tests/kafkatest/tests/core/upgrade_test.py index 8ba1d0a02d..f4548b3c87 100644 --- a/tests/kafkatest/tests/core/upgrade_test.py +++ b/tests/kafkatest/tests/core/upgrade_test.py @@ -27,7 +27,7 @@ from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, \ LATEST_0_11_0, LATEST_1_0, LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, \ LATEST_2_6, LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, LATEST_3_7, V_0_11_0_0, V_2_8_0, V_3_0_0, DEV_BRANCH, KafkaVersion + LATEST_3_6, LATEST_3_7, LATEST_3_8, V_0_11_0_0, V_2_8_0, V_3_0_0, DEV_BRANCH, KafkaVersion from kafkatest.services.kafka.util import new_jdk_not_supported class TestUpgrade(ProduceConsumeValidateTest): @@ -94,6 +94,9 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None): self.wait_until_rejoin() @cluster(num_nodes=6) + @parametrize(from_kafka_version=str(LATEST_3_8), to_message_format_version=None, compression_types=["none"]) + @parametrize(from_kafka_version=str(LATEST_3_8), to_message_format_version=None, compression_types=["lz4"]) + @parametrize(from_kafka_version=str(LATEST_3_8), to_message_format_version=None, compression_types=["snappy"]) @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["none"]) @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["lz4"]) @parametrize(from_kafka_version=str(LATEST_3_7), to_message_format_version=None, compression_types=["snappy"]) From 0370a6464b0ace99a34e3a34a18596eaf57c4d54 Mon Sep 17 00:00:00 2001 From: Josep Prat Date: Tue, 30 Jul 2024 21:45:20 +0200 Subject: [PATCH 009/123] MINOR: Add text and link to blog in announcement template email (#16734) Reviewers: Igor Soarez --- release/templates.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/release/templates.py b/release/templates.py index 7173ea9153..5eb3fc74f9 100644 --- a/release/templates.py +++ b/release/templates.py @@ -88,6 +88,9 @@ def release_announcement_email(release_version, contributors): https://www.apache.org/dist/kafka/{release_version}/RELEASE_NOTES.html +An overview of the release can be found in our announcement blog post: +https://kafka.apache.org/blog + You can download the source and binary release (Scala ) from: https://kafka.apache.org/downloads#{release_version} From c8dc09c2659bb6309d97c692c907c076898b4aeb Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 31 Jul 2024 07:58:07 +0200 Subject: [PATCH 010/123] KAFKA-16448: Handle fatal user exception during processing error (#16675) This PR is part of KIP-1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing. This PR catch the exceptions thrown while handling a processing exception Co-authored-by: Dabz Co-authored-by: loicgreffier Reviewers: Bruno Cadonna , Matthias J. Sax --- .../processor/internals/ProcessorNode.java | 8 +- ...essingExceptionHandlerIntegrationTest.java | 146 ++++++++++++++++++ .../internals/ProcessorNodeTest.java | 31 +++- 3 files changed, 179 insertions(+), 6 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 763edc9a04..175c9e104e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -213,9 +213,13 @@ public void process(final Record record) { internalProcessorContext.currentNode().name(), internalProcessorContext.taskId()); - final ProcessingExceptionHandler.ProcessingHandlerResponse response = processingExceptionHandler - .handle(errorHandlerContext, record, e); + final ProcessingExceptionHandler.ProcessingHandlerResponse response; + try { + response = processingExceptionHandler.handle(errorHandlerContext, record, e); + } catch (final Exception fatalUserException) { + throw new FailedProcessingException(fatalUserException); + } if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { log.error("Processing exception handler is set to fail upon" + " a processing error. If you would rather have the streaming pipeline" + diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index 6c1a64344e..61b5ed16bb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -45,10 +45,12 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertIterableEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -149,9 +151,150 @@ public void shouldContinueWhenProcessingExceptionOccurs() { } } + @Test + public void shouldStopOnFailedProcessorWhenProcessingExceptionOccursInFailProcessingExceptionHandler() { + final KeyValue event = new KeyValue<>("ID123-1", "ID123-A1"); + final KeyValue eventError = new KeyValue<>("ID123-2-ERR", "ID123-A2"); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + final AtomicBoolean isExecuted = new AtomicBoolean(false); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .map(KeyValue::new) + .mapValues(value -> value) + .process(runtimeErrorProcessorSupplierMock()) + .map((k, v) -> { + isExecuted.set(true); + return KeyValue.pair(k, v); + }) + .process(processor); + + final Properties properties = new Properties(); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + isExecuted.set(false); + inputTopic.pipeInput(event.key, event.value, Instant.EPOCH); + assertTrue(isExecuted.get()); + isExecuted.set(false); + final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); + assertTrue(e.getMessage().contains("Exception caught in process. " + + "taskId=0_0, processor=KSTREAM-SOURCE-0000000000, topic=TOPIC_NAME, " + + "partition=0, offset=1, stacktrace=java.lang.RuntimeException: " + + "Exception should be handled by processing exception handler")); + assertFalse(isExecuted.get()); + } + } + + @Test + public void shouldStopOnFailedProcessorWhenProcessingExceptionOccursInContinueProcessingExceptionHandler() { + final KeyValue event = new KeyValue<>("ID123-1", "ID123-A1"); + final KeyValue eventFalse = new KeyValue<>("ID123-2-ERR", "ID123-A2"); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + final AtomicBoolean isExecuted = new AtomicBoolean(false); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .map(KeyValue::new) + .mapValues(value -> value) + .process(runtimeErrorProcessorSupplierMock()) + .map((k, v) -> { + isExecuted.set(true); + return KeyValue.pair(k, v); + }) + .process(processor); + + final Properties properties = new Properties(); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + isExecuted.set(false); + inputTopic.pipeInput(event.key, event.value, Instant.EPOCH); + assertTrue(isExecuted.get()); + isExecuted.set(false); + inputTopic.pipeInput(eventFalse.key, eventFalse.value, Instant.EPOCH); + assertFalse(isExecuted.get()); + } + } + + @Test + public void shouldStopProcessingWhenFatalUserExceptionInFailProcessingExceptionHandler() { + final KeyValue event = new KeyValue<>("ID123-1", "ID123-A1"); + final KeyValue eventError = new KeyValue<>("ID123-ERR-FATAL", "ID123-A2"); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + final AtomicBoolean isExecuted = new AtomicBoolean(false); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .map(KeyValue::new) + .mapValues(value -> value) + .process(runtimeErrorProcessorSupplierMock()) + .map((k, v) -> { + isExecuted.set(true); + return KeyValue.pair(k, v); + }) + .process(processor); + + final Properties properties = new Properties(); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + isExecuted.set(false); + inputTopic.pipeInput(event.key, event.value, Instant.EPOCH); + assertTrue(isExecuted.get()); + isExecuted.set(false); + final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); + assertEquals("KABOOM!", e.getCause().getMessage()); + assertFalse(isExecuted.get()); + } + } + + @Test + public void shouldStopProcessingWhenFatalUserExceptionInContinueProcessingExceptionHandler() { + final KeyValue event = new KeyValue<>("ID123-1", "ID123-A1"); + final KeyValue eventError = new KeyValue<>("ID123-ERR-FATAL", "ID123-A2"); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + final AtomicBoolean isExecuted = new AtomicBoolean(false); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .map(KeyValue::new) + .mapValues(value -> value) + .process(runtimeErrorProcessorSupplierMock()) + .map((k, v) -> { + isExecuted.set(true); + return KeyValue.pair(k, v); + }) + .process(processor); + + final Properties properties = new Properties(); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + isExecuted.set(false); + inputTopic.pipeInput(event.key, event.value, Instant.EPOCH); + assertTrue(isExecuted.get()); + isExecuted.set(false); + final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); + assertEquals("KABOOM!", e.getCause().getMessage()); + assertFalse(isExecuted.get()); + } + } + public static class ContinueProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { @Override public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + if (((String) record.key()).contains("FATAL")) { + throw new RuntimeException("KABOOM!"); + } assertProcessingExceptionHandlerInputs(context, record, exception); return ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE; } @@ -165,6 +308,9 @@ public void configure(final Map configs) { public static class FailProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { @Override public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + if (((String) record.key()).contains("FATAL")) { + throw new RuntimeException("KABOOM!"); + } assertProcessingExceptionHandlerInputs(context, record, exception); return ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index df3f927686..9fe9244e0a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -60,6 +60,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -100,7 +101,7 @@ public void shouldThrowFailedProcessingExceptionWhenProcessingExceptionHandlerRe new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); - node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL, internalProcessorContext)); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL, internalProcessorContext, false)); final FailedProcessingException failedProcessingException = assertThrows(FailedProcessingException.class, () -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); @@ -116,7 +117,7 @@ public void shouldNotThrowFailedProcessingExceptionWhenProcessingExceptionHandle new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); - node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE, internalProcessorContext)); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE, internalProcessorContext, false)); assertDoesNotThrow(() -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); } @@ -146,6 +147,21 @@ public void shouldNotHandleInternalExceptionsThrownDuringProcessing(final String verify(processingExceptionHandler, never()).handle(any(), any(), any()); } + @Test + public void shouldThrowFailedProcessingExceptionWhenProcessingExceptionHandlerThrowsAnException() { + final ProcessorNode node = + new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); + + final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE, internalProcessorContext, true)); + + final FailedProcessingException failedProcessingException = assertThrows(FailedProcessingException.class, + () -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); + + assertInstanceOf(RuntimeException.class, failedProcessingException.getCause()); + assertEquals("KABOOM!", failedProcessingException.getCause().getMessage()); + } + private static class ExceptionalProcessor implements Processor { @Override public void init(final ProcessorContext context) { @@ -323,10 +339,14 @@ public static class ProcessingExceptionHandlerMock implements ProcessingExceptio private final ProcessingExceptionHandler.ProcessingHandlerResponse response; private final InternalProcessorContext internalProcessorContext; + private final boolean shouldThrowException; + public ProcessingExceptionHandlerMock(final ProcessingExceptionHandler.ProcessingHandlerResponse response, - final InternalProcessorContext internalProcessorContext) { + final InternalProcessorContext internalProcessorContext, + final boolean shouldThrowException) { this.response = response; this.internalProcessorContext = internalProcessorContext; + this.shouldThrowException = shouldThrowException; } @Override @@ -338,9 +358,12 @@ public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHa assertEquals(internalProcessorContext.taskId(), context.taskId()); assertEquals(KEY, record.key()); assertEquals(VALUE, record.value()); - assertTrue(exception instanceof RuntimeException); + assertInstanceOf(RuntimeException.class, exception); assertEquals("Processing exception should be caught and handled by the processing exception handler.", exception.getMessage()); + if (shouldThrowException) { + throw new RuntimeException("KABOOM!"); + } return response; } From fbdfd0d59630a3d712acf406a96cfd200c093ea0 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Wed, 31 Jul 2024 15:18:14 +0800 Subject: [PATCH 011/123] KAFKA-16666 Migrate OffsetMessageFormatter to tools module (#16689) Reviewers: Chia-Ping Tsai --- build.gradle | 2 +- checkstyle/import-control.xml | 1 + .../group/GroupMetadataManager.scala | 1 + .../consumer/ConsoleConsumerOptions.java | 4 + .../consumer/OffsetsMessageFormatter.java | 126 +++++++++++++++ .../consumer/ConsoleConsumerOptionsTest.java | 56 +++---- .../tools/consumer/ConsoleConsumerTest.java | 100 ++++++++++-- .../consumer/OffsetMessageFormatterTest.java | 153 ++++++++++++++++++ 8 files changed, 400 insertions(+), 43 deletions(-) create mode 100644 tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java create mode 100644 tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java diff --git a/build.gradle b/build.gradle index a15e00c0a6..51f9659e58 100644 --- a/build.gradle +++ b/build.gradle @@ -2117,6 +2117,7 @@ project(':tools') { implementation project(':connect:runtime') implementation project(':tools:tools-api') implementation project(':transaction-coordinator') + implementation project(':group-coordinator') implementation libs.argparse4j implementation libs.jacksonDatabind implementation libs.jacksonDataformatCsv @@ -2140,7 +2141,6 @@ project(':tools') { testImplementation project(':connect:runtime') testImplementation project(':connect:runtime').sourceSets.test.output testImplementation project(':storage:storage-api').sourceSets.main.output - testImplementation project(':group-coordinator') testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation libs.mockitoJunitJupiter // supports MockitoExtension diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 08c45d0aa3..a5784ef935 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -301,6 +301,7 @@ + diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index 064df2f607..5bb7216ab8 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -1243,6 +1243,7 @@ object GroupMetadataManager { // Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false. // (specify --formatter "kafka.coordinator.group.GroupMetadataManager\$OffsetsMessageFormatter" when consuming __consumer_offsets) + @Deprecated class OffsetsMessageFormatter extends MessageFormatter { def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream): Unit = { Option(consumerRecord.key).map(key => GroupMetadataManager.readMessageKey(ByteBuffer.wrap(key))).foreach { diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java index 0a3a008b7a..455ca885a5 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java @@ -368,6 +368,10 @@ private static String convertDeprecatedClass(String className) { System.err.println("WARNING: kafka.coordinator.transaction.TransactionLog$TransactionLogMessageFormatter is deprecated and will be removed in the next major release. " + "Please use org.apache.kafka.tools.consumer.TransactionLogMessageFormatter instead"); return className; + case "kafka.coordinator.group.GroupMetadataManager$OffsetsMessageFormatter": + System.err.println("WARNING: kafka.coordinator.group.GroupMetadataManager$OffsetsMessageFormatter is deprecated and will be removed in the next major release. " + + "Please use org.apache.kafka.tools.consumer.OffsetsMessageFormatter instead"); + return className; default: return className; } diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java b/tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java new file mode 100644 index 0000000000..62dcb871c8 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.consumer; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.MessageFormatter; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKeyJsonConverter; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValueJsonConverter; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.NullNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; + +import java.io.IOException; +import java.io.PrintStream; +import java.nio.ByteBuffer; +import java.util.Objects; +import java.util.Optional; + +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false. + */ +public class OffsetsMessageFormatter implements MessageFormatter { + + private static final String VERSION = "version"; + private static final String DATA = "data"; + private static final String KEY = "key"; + private static final String VALUE = "value"; + private static final String UNKNOWN = "unknown"; + + @Override + public void writeTo(ConsumerRecord consumerRecord, PrintStream output) { + ObjectNode json = new ObjectNode(JsonNodeFactory.instance); + + byte[] key = consumerRecord.key(); + if (Objects.nonNull(key)) { + short keyVersion = ByteBuffer.wrap(key).getShort(); + JsonNode dataNode = readToGroupMetadataKey(ByteBuffer.wrap(key)) + .map(logKey -> transferMetadataToJsonNode(logKey, keyVersion)) + .orElseGet(() -> new TextNode(UNKNOWN)); + // Only print if the message is an offset record. + if (dataNode instanceof NullNode) { + return; + } + json.putObject(KEY) + .put(VERSION, keyVersion) + .set(DATA, dataNode); + } else { + json.set(KEY, NullNode.getInstance()); + } + + byte[] value = consumerRecord.value(); + if (Objects.nonNull(value)) { + short valueVersion = ByteBuffer.wrap(value).getShort(); + JsonNode dataNode = readToOffsetCommitValue(ByteBuffer.wrap(value)) + .map(logValue -> OffsetCommitValueJsonConverter.write(logValue, valueVersion)) + .orElseGet(() -> new TextNode(UNKNOWN)); + json.putObject(VALUE) + .put(VERSION, valueVersion) + .set(DATA, dataNode); + } else { + json.set(VALUE, NullNode.getInstance()); + } + + try { + output.write(json.toString().getBytes(UTF_8)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private Optional readToGroupMetadataKey(ByteBuffer byteBuffer) { + short version = byteBuffer.getShort(); + if (version >= OffsetCommitKey.LOWEST_SUPPORTED_VERSION + && version <= OffsetCommitKey.HIGHEST_SUPPORTED_VERSION) { + return Optional.of(new OffsetCommitKey(new ByteBufferAccessor(byteBuffer), version)); + } else if (version >= GroupMetadataKey.LOWEST_SUPPORTED_VERSION && version <= GroupMetadataKey.HIGHEST_SUPPORTED_VERSION) { + return Optional.of(new GroupMetadataKey(new ByteBufferAccessor(byteBuffer), version)); + } else { + return Optional.empty(); + } + } + + private static JsonNode transferMetadataToJsonNode(ApiMessage logKey, short keyVersion) { + if (logKey instanceof OffsetCommitKey) { + return OffsetCommitKeyJsonConverter.write((OffsetCommitKey) logKey, keyVersion); + } else if (logKey instanceof GroupMetadataKey) { + return NullNode.getInstance(); + } else { + return new TextNode(UNKNOWN); + } + } + + private Optional readToOffsetCommitValue(ByteBuffer byteBuffer) { + short version = byteBuffer.getShort(); + if (version >= OffsetCommitValue.LOWEST_SUPPORTED_VERSION + && version <= OffsetCommitValue.HIGHEST_SUPPORTED_VERSION) { + return Optional.of(new OffsetCommitValue(new ByteBufferAccessor(byteBuffer), version)); + } else { + return Optional.empty(); + } + } +} diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java index 383b39700c..7c84a10fd4 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java @@ -649,50 +649,50 @@ public void testParseTimeoutMs() throws Exception { @Test public void testParseDeprecatedFormatter() throws Exception { - String[] deprecatedDefaultMessageFormatter = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--partition", "0", - "--formatter", "kafka.tools.DefaultMessageFormatter", - }; + String[] deprecatedDefaultMessageFormatter = generateArgsForFormatter("kafka.tools.DefaultMessageFormatter"); assertInstanceOf(DefaultMessageFormatter.class, new ConsoleConsumerOptions(deprecatedDefaultMessageFormatter).formatter()); - String[] deprecatedLoggingMessageFormatter = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--partition", "0", - "--formatter", "kafka.tools.LoggingMessageFormatter", - }; + String[] deprecatedLoggingMessageFormatter = generateArgsForFormatter("kafka.tools.LoggingMessageFormatter"); assertInstanceOf(LoggingMessageFormatter.class, new ConsoleConsumerOptions(deprecatedLoggingMessageFormatter).formatter()); - String[] deprecatedNoOpMessageFormatter = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--partition", "0", - "--formatter", "kafka.tools.NoOpMessageFormatter", - }; + String[] deprecatedNoOpMessageFormatter = generateArgsForFormatter("kafka.tools.NoOpMessageFormatter"); assertInstanceOf(NoOpMessageFormatter.class, new ConsoleConsumerOptions(deprecatedNoOpMessageFormatter).formatter()); } @SuppressWarnings("deprecation") @Test public void testNewAndDeprecateTransactionLogMessageFormatter() throws Exception { - String[] deprecatedTransactionLogMessageFormatter = new String[]{ - "--bootstrap-server", "localhost:9092", - "--topic", "test", - "--partition", "0", - "--formatter", "kafka.coordinator.transaction.TransactionLog$TransactionLogMessageFormatter", - }; + String[] deprecatedTransactionLogMessageFormatter = + generateArgsForFormatter("kafka.coordinator.transaction.TransactionLog$TransactionLogMessageFormatter"); assertInstanceOf(kafka.coordinator.transaction.TransactionLog.TransactionLogMessageFormatter.class, new ConsoleConsumerOptions(deprecatedTransactionLogMessageFormatter).formatter()); - String[] transactionLogMessageFormatter = new String[]{ + String[] transactionLogMessageFormatter = + generateArgsForFormatter("org.apache.kafka.tools.consumer.TransactionLogMessageFormatter"); + assertInstanceOf(TransactionLogMessageFormatter.class, + new ConsoleConsumerOptions(transactionLogMessageFormatter).formatter()); + } + + @SuppressWarnings("deprecation") + @Test + public void testNewAndDeprecateOffsetsMessageFormatter() throws Exception { + String[] deprecatedOffsetsMessageFormatter = + generateArgsForFormatter("kafka.coordinator.group.GroupMetadataManager$OffsetsMessageFormatter"); + assertInstanceOf(kafka.coordinator.group.GroupMetadataManager.OffsetsMessageFormatter.class, + new ConsoleConsumerOptions(deprecatedOffsetsMessageFormatter).formatter()); + + String[] offsetsMessageFormatter = + generateArgsForFormatter("org.apache.kafka.tools.consumer.OffsetsMessageFormatter"); + assertInstanceOf(OffsetsMessageFormatter.class, + new ConsoleConsumerOptions(offsetsMessageFormatter).formatter()); + } + + private String[] generateArgsForFormatter(String formatter) { + return new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", "--partition", "0", - "--formatter", "org.apache.kafka.tools.consumer.TransactionLogMessageFormatter", + "--formatter", formatter, }; - assertInstanceOf(TransactionLogMessageFormatter.class, - new ConsoleConsumerOptions(transactionLogMessageFormatter).formatter()); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java index d849a8f531..6e378b3a98 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java @@ -39,6 +39,10 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKeyJsonConverter; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValueJsonConverter; import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey; import org.apache.kafka.coordinator.transaction.generated.TransactionLogKeyJsonConverter; import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue; @@ -66,6 +70,7 @@ import static java.util.Collections.singleton; import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.ISOLATION_LEVEL_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; @@ -91,6 +96,7 @@ public class ConsoleConsumerTest { private final String topic = "test-topic"; + private final String groupId = "test-group"; private final String transactionId = "transactional-id"; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -289,7 +295,7 @@ public void testTransactionLogMessageFormatter(ClusterInstance cluster) throws E NewTopic newTopic = new NewTopic(topic, 1, (short) 1); admin.createTopics(singleton(newTopic)); - produceMessages(cluster); + produceMessagesWithTxn(cluster); String[] transactionLogMessageFormatter = new String[]{ "--bootstrap-server", cluster.bootstrapServers(), @@ -299,7 +305,7 @@ public void testTransactionLogMessageFormatter(ClusterInstance cluster) throws E }; ConsoleConsumerOptions options = new ConsoleConsumerOptions(transactionLogMessageFormatter); - ConsoleConsumer.ConsumerWrapper consumerWrapper = new ConsoleConsumer.ConsumerWrapper(options, createConsumer(cluster)); + ConsoleConsumer.ConsumerWrapper consumerWrapper = new ConsoleConsumer.ConsumerWrapper(options, createTxnConsumer(cluster)); try (ByteArrayOutputStream out = new ByteArrayOutputStream(); PrintStream output = new PrintStream(out)) { @@ -325,8 +331,52 @@ public void testTransactionLogMessageFormatter(ClusterInstance cluster) throws E } } - private void produceMessages(ClusterInstance cluster) { - try (Producer producer = createProducer(cluster)) { + @ClusterTest(brokers = 3) + public void testOffsetsMessageFormatter(ClusterInstance cluster) throws Exception { + try (Admin admin = cluster.createAdminClient()) { + + NewTopic newTopic = new NewTopic(topic, 1, (short) 1); + admin.createTopics(singleton(newTopic)); + produceMessages(cluster); + + String[] offsetsMessageFormatter = new String[]{ + "--bootstrap-server", cluster.bootstrapServers(), + "--topic", Topic.GROUP_METADATA_TOPIC_NAME, + "--formatter", "org.apache.kafka.tools.consumer.OffsetsMessageFormatter" + }; + + ConsoleConsumerOptions options = new ConsoleConsumerOptions(offsetsMessageFormatter); + ConsoleConsumer.ConsumerWrapper consumerWrapper = new ConsoleConsumer.ConsumerWrapper(options, createOffsetConsumer(cluster)); + + try (ByteArrayOutputStream out = new ByteArrayOutputStream(); + PrintStream output = new PrintStream(out)) { + ConsoleConsumer.process(1, options.formatter(), consumerWrapper, output, true); + + JsonNode jsonNode = objectMapper.reader().readTree(out.toByteArray()); + JsonNode keyNode = jsonNode.get("key"); + + OffsetCommitKey offsetCommitKey = + OffsetCommitKeyJsonConverter.read(keyNode.get("data"), OffsetCommitKey.HIGHEST_SUPPORTED_VERSION); + assertNotNull(offsetCommitKey); + assertEquals(Topic.GROUP_METADATA_TOPIC_NAME, offsetCommitKey.topic()); + assertEquals(groupId, offsetCommitKey.group()); + + JsonNode valueNode = jsonNode.get("value"); + OffsetCommitValue offsetCommitValue = + OffsetCommitValueJsonConverter.read(valueNode.get("data"), OffsetCommitValue.HIGHEST_SUPPORTED_VERSION); + assertNotNull(offsetCommitValue); + assertEquals(0, offsetCommitValue.offset()); + assertEquals(-1, offsetCommitValue.leaderEpoch()); + assertNotNull(offsetCommitValue.metadata()); + assertEquals(-1, offsetCommitValue.expireTimestamp()); + } finally { + consumerWrapper.cleanup(); + } + } + } + + private void produceMessagesWithTxn(ClusterInstance cluster) { + try (Producer producer = createTxnProducer(cluster)) { producer.initTransactions(); producer.beginTransaction(); producer.send(new ProducerRecord<>(topic, new byte[1_000 * 100])); @@ -334,26 +384,48 @@ private void produceMessages(ClusterInstance cluster) { } } - private Producer createProducer(ClusterInstance cluster) { - Properties props = new Properties(); - props.put(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + private void produceMessages(ClusterInstance cluster) { + try (Producer producer = new KafkaProducer<>(producerProps(cluster))) { + producer.send(new ProducerRecord<>(topic, new byte[1_000 * 100])); + } + } + + private Producer createTxnProducer(ClusterInstance cluster) { + Properties props = producerProps(cluster); props.put(ENABLE_IDEMPOTENCE_CONFIG, "true"); props.put(ACKS_CONFIG, "all"); props.put(TRANSACTIONAL_ID_CONFIG, transactionId); - props.put(KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); - props.put(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); return new KafkaProducer<>(props); } - private Consumer createConsumer(ClusterInstance cluster) { + private Consumer createTxnConsumer(ClusterInstance cluster) { + Properties props = consumerProps(cluster); + props.put(ISOLATION_LEVEL_CONFIG, "read_committed"); + props.put(AUTO_OFFSET_RESET_CONFIG, "earliest"); + return new KafkaConsumer<>(props); + } + + private Consumer createOffsetConsumer(ClusterInstance cluster) { + Properties props = consumerProps(cluster); + props.put(EXCLUDE_INTERNAL_TOPICS_CONFIG, "false"); + return new KafkaConsumer<>(props); + } + + private Properties producerProps(ClusterInstance cluster) { + Properties props = new Properties(); + props.put(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + props.put(KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + props.put(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + return props; + } + + private Properties consumerProps(ClusterInstance cluster) { Properties props = new Properties(); props.put(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); props.put(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); props.put(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); props.put(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, RangeAssignor.class.getName()); - props.put(ISOLATION_LEVEL_CONFIG, "read_committed"); - props.put(AUTO_OFFSET_RESET_CONFIG, "earliest"); - props.put(GROUP_ID_CONFIG, "test-group"); - return new KafkaConsumer<>(props); + props.put(GROUP_ID_CONFIG, groupId); + return props; } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java new file mode 100644 index 0000000000..f2c4a8e3e3 --- /dev/null +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.consumer; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.MessageFormatter; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.Collections; +import java.util.Optional; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class OffsetMessageFormatterTest { + + private static final OffsetCommitKey OFFSET_COMMIT_KEY = new OffsetCommitKey() + .setGroup("group-id") + .setTopic("foo") + .setPartition(1); + private static final OffsetCommitValue OFFSET_COMMIT_VALUE = new OffsetCommitValue() + .setOffset(100L) + .setLeaderEpoch(10) + .setMetadata("metadata") + .setCommitTimestamp(1234L) + .setExpireTimestamp(-1L); + private static final GroupMetadataKey GROUP_METADATA_KEY = new GroupMetadataKey().setGroup("group-id"); + private static final GroupMetadataValue GROUP_METADATA_VALUE = new GroupMetadataValue() + .setProtocolType("consumer") + .setGeneration(1) + .setProtocol("range") + .setLeader("leader") + .setMembers(Collections.emptyList()); + private static final String TOPIC = "TOPIC"; + + private static Stream parameters() { + return Stream.of( + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 10, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 10, OFFSET_COMMIT_VALUE).array(), + "{\"key\":{\"version\":10,\"data\":\"unknown\"},\"value\":{\"version\":10,\"data\":\"unknown\"}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_VALUE).array(), + "{\"key\":{\"version\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," + + "\"value\":{\"version\":0,\"data\":{\"offset\":100,\"metadata\":\"metadata\"," + + "\"commitTimestamp\":1234}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_VALUE).array(), + "{\"key\":{\"version\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," + + "\"value\":{\"version\":1,\"data\":{\"offset\":100,\"metadata\":\"metadata\"," + + "\"commitTimestamp\":1234,\"expireTimestamp\":-1}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 2, OFFSET_COMMIT_VALUE).array(), + "{\"key\":{\"version\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," + + "\"value\":{\"version\":2,\"data\":{\"offset\":100,\"metadata\":\"metadata\"," + + "\"commitTimestamp\":1234}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 3, OFFSET_COMMIT_VALUE).array(), + "{\"key\":{\"version\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," + + "\"value\":{\"version\":3,\"data\":{\"offset\":100,\"leaderEpoch\":10," + + "\"metadata\":\"metadata\",\"commitTimestamp\":1234}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 4, OFFSET_COMMIT_VALUE).array(), + "{\"key\":{\"version\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," + + "\"value\":{\"version\":4,\"data\":{\"offset\":100,\"leaderEpoch\":10," + + "\"metadata\":\"metadata\",\"commitTimestamp\":1234}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 5, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 4, OFFSET_COMMIT_VALUE).array(), + "{\"key\":{\"version\":5,\"data\":\"unknown\"},\"value\":{\"version\":4," + + "\"data\":{\"offset\":100,\"leaderEpoch\":10,\"metadata\":\"metadata\"," + + "\"commitTimestamp\":1234}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 5, OFFSET_COMMIT_VALUE).array(), + "{\"key\":{\"version\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," + + "\"value\":{\"version\":5,\"data\":\"unknown\"}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(), + null, + "{\"key\":{\"version\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," + + "\"value\":null}"), + Arguments.of( + null, + MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_VALUE).array(), + "{\"key\":null,\"value\":{\"version\":1,\"data\":{\"offset\":100,\"metadata\":\"metadata\"," + + "\"commitTimestamp\":1234,\"expireTimestamp\":-1}}}"), + Arguments.of(null, null, "{\"key\":null,\"value\":null}"), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_VALUE).array(), + "" + ) + ); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testTransactionLogMessageFormatter(byte[] keyBuffer, byte[] valueBuffer, String expectedOutput) { + ConsumerRecord record = new ConsumerRecord<>( + TOPIC, 0, 0, + 0L, TimestampType.CREATE_TIME, 0, + 0, keyBuffer, valueBuffer, + new RecordHeaders(), Optional.empty()); + + try (MessageFormatter formatter = new OffsetsMessageFormatter()) { + formatter.configure(emptyMap()); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + formatter.writeTo(record, new PrintStream(out)); + assertEquals(expectedOutput, out.toString()); + } + } +} From ccb04acb56b5a29f6de4dfc7b98897404eb14c0c Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Wed, 31 Jul 2024 10:29:02 -0700 Subject: [PATCH 012/123] Revert "KAFKA-16508: Streams custom handler should handle the timeout exceptions (#16450)" (#16738) This reverts commit 15a4501bded513822485dd85fa6258e16f1571ca. We consider this change backward incompatible and will fix forward for 4.0 release via KIP-1065, but need to revert for 3.9 release. Reviewers: Josep Prat , Bill Bejeck --- .../internals/RecordCollectorImpl.java | 13 +- .../CustomHandlerIntegrationTest.java | 166 ------------------ .../internals/RecordCollectorTest.java | 50 ------ 3 files changed, 1 insertion(+), 228 deletions(-) delete mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/CustomHandlerIntegrationTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index de4afc2c92..42b8d4f082 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -34,7 +34,6 @@ import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownServerException; -import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.Serializer; @@ -389,7 +388,7 @@ private void recordSendError(final String topic, "indicating the task may be migrated out"; sendException.set(new TaskMigratedException(errorMessage, exception)); } else { - if (isRetriable(exception)) { + if (exception instanceof RetriableException) { errorMessage += "\nThe broker is either slow or in bad state (like not having enough replicas) in responding the request, " + "or the connection to broker was interrupted sending the request or receiving the response. " + "\nConsider overwriting `max.block.ms` and /or " + @@ -419,16 +418,6 @@ private void recordSendError(final String topic, log.error(errorMessage, exception); } - /** - * The `TimeoutException` with root cause `UnknownTopicOrPartitionException` is considered as non-retriable - * (despite `TimeoutException` being a subclass of `RetriableException`, this particular case is explicitly excluded). - */ - private boolean isRetriable(final Exception exception) { - return exception instanceof RetriableException && - (!(exception instanceof TimeoutException) || exception.getCause() == null - || !(exception.getCause() instanceof UnknownTopicOrPartitionException)); - } - private boolean isFatalException(final Exception exception) { final boolean securityException = exception instanceof AuthenticationException || exception instanceof AuthorizationException || diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/CustomHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/CustomHandlerIntegrationTest.java deleted file mode 100644 index 3eea2ec7d8..0000000000 --- a/streams/src/test/java/org/apache/kafka/streams/integration/CustomHandlerIntegrationTest.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.integration; - -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KafkaStreams.State; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.Topology; -import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; -import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; -import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; -import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.test.TestUtils; - -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInfo; -import org.junit.jupiter.api.Timeout; - -import java.io.IOException; -import java.util.Collections; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicReference; - -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; -import static org.junit.jupiter.api.Assertions.assertInstanceOf; - - -@Timeout(600) -@Tag("integration") -public class CustomHandlerIntegrationTest { - private static final int NUM_BROKERS = 1; - private static final int NUM_THREADS = 2; - public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, - Utils.mkProperties(Collections.singletonMap("auto.create.topics.enable", "false"))); - - @BeforeAll - public static void startCluster() throws IOException { - CLUSTER.start(); - } - - @AfterAll - public static void closeCluster() { - CLUSTER.stop(); - } - - private final long timeout = 60000; - - // topic name - private static final String STREAM_INPUT = "STREAM_INPUT"; - private static final String NON_EXISTING_TOPIC = "non_existing_topic"; - - private KafkaStreams kafkaStreams; - AtomicReference caughtException; - Topology topology; - private String appId; - - @BeforeEach - public void before(final TestInfo testInfo) throws InterruptedException { - final StreamsBuilder builder = new StreamsBuilder(); - CLUSTER.createTopics(STREAM_INPUT); - caughtException = new AtomicReference<>(); - final String safeTestName = safeUniqueTestName(testInfo); - appId = "app-" + safeTestName; - - - builder.stream(STREAM_INPUT, Consumed.with(Serdes.Integer(), Serdes.String())) - .to(NON_EXISTING_TOPIC, Produced.with(Serdes.Integer(), Serdes.String())); - produceRecords(); - topology = builder.build(); - } - - @AfterEach - public void after() throws InterruptedException { - CLUSTER.deleteTopics(STREAM_INPUT); - if (kafkaStreams != null) { - kafkaStreams.close(); - kafkaStreams.cleanUp(); - } - } - - private void produceRecords() { - final Properties props = TestUtils.producerConfig( - CLUSTER.bootstrapServers(), - IntegerSerializer.class, - StringSerializer.class, - new Properties()); - IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp( - STREAM_INPUT, - Collections.singletonList(new KeyValue<>(1, "A")), - props, - CLUSTER.time.milliseconds() + 2 - ); - } - - private Properties getCommonProperties() { - final Properties streamsConfiguration = new Properties(); - streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, appId); - streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class); - streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); - streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, NUM_THREADS); - return streamsConfiguration; - } - - private void closeApplication(final Properties streamsConfiguration) throws Exception { - kafkaStreams.close(); - kafkaStreams.cleanUp(); - IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration); - } - - @Test - public void shouldThrowStreamsExceptionWithMissingTopicAndDefaultExceptionHandler() throws Exception { - final Properties streamsConfiguration = getCommonProperties(); - kafkaStreams = new KafkaStreams(topology, streamsConfiguration); - kafkaStreams.setUncaughtExceptionHandler(e -> { - caughtException.set(e); - return StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; - }); - kafkaStreams.start(); - TestUtils.waitForCondition( - () -> kafkaStreams.state() == State.RUNNING, - timeout, - () -> "Kafka Streams application did not reach state RUNNING in " + timeout + " ms"); - while (true) { - if (caughtException.get() != null) { - final Throwable throwable = caughtException.get(); - assertInstanceOf(StreamsException.class, throwable); - assertInstanceOf(TimeoutException.class, throwable.getCause()); - assertInstanceOf(UnknownTopicOrPartitionException.class, throwable.getCause().getCause()); - closeApplication(streamsConfiguration); - break; - } else { - Thread.sleep(100); - } - } - } -} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index 2623102885..735a7b7910 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -34,7 +34,6 @@ import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; @@ -93,7 +92,6 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.core.IsInstanceOf.instanceOf; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -1359,54 +1357,6 @@ public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContin collector.closeClean(); } - @Test - public void shouldThrowStreamsExceptionOnUnknownTopicOrPartitionExceptionWithDefaultExceptionHandler() { - final KafkaException exception = new TimeoutException("KABOOM!", new UnknownTopicOrPartitionException()); - final RecordCollector collector = new RecordCollectorImpl( - logContext, - taskId, - getExceptionalStreamsProducerOnSend(exception), - productionExceptionHandler, - streamsMetrics, - topology - ); - - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); - - // With default handler which returns FAIL, flush() throws StreamsException with TimeoutException cause, - // otherwise it would throw a TaskCorruptedException with null cause - final StreamsException thrown = assertThrows(StreamsException.class, collector::flush); - assertEquals(exception, thrown.getCause()); - assertThat( - thrown.getMessage(), - equalTo("Error encountered sending record to topic topic for task 0_0 due to:" + - "\norg.apache.kafka.common.errors.TimeoutException: KABOOM!" + - "\nException handler choose to FAIL the processing, no more records would be sent.") - ); - } - - @Test - public void shouldNotThrowTaskCorruptedExceptionOnUnknownTopicOrPartitionExceptionUsingAlwaysContinueExceptionHandler() { - final KafkaException exception = new TimeoutException("KABOOM!", new UnknownTopicOrPartitionException()); - final RecordCollector collector = new RecordCollectorImpl( - logContext, - taskId, - getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock( - ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE, - context, - sinkNodeName, - taskId - ), - streamsMetrics, - topology - ); - - collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); - - assertDoesNotThrow(collector::flush); - } - @Test public void shouldNotAbortTxnOnEOSCloseDirtyIfNothingSent() { final AtomicBoolean functionCalled = new AtomicBoolean(false); From aaed1bdd8911be329123c7f675deb6875ebf55bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20GREFFIER?= Date: Wed, 31 Jul 2024 22:24:15 +0200 Subject: [PATCH 013/123] KAFKA-16448: Unify class cast exception handling for both key and value (#16736) Part of KIP-1033. Minor code cleanup. Reviewers: Matthias J. Sax --- .../internals/RecordCollectorImpl.java | 52 ++++++++----------- 1 file changed, 22 insertions(+), 30 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 42b8d4f082..7a8b77b8a5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -52,9 +52,11 @@ import org.slf4j.Logger; +import java.text.MessageFormat; import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -199,7 +201,8 @@ public void send(final String topic, try { keyBytes = keySerializer.serialize(topic, headers, key); } catch (final ClassCastException exception) { - throw createStreamsExceptionForKeyClassCastException( + throw createStreamsExceptionForClassCastException( + ProductionExceptionHandler.SerializationExceptionOrigin.KEY, topic, key, keySerializer, @@ -222,7 +225,8 @@ public void send(final String topic, try { valBytes = valueSerializer.serialize(topic, headers, value); } catch (final ClassCastException exception) { - throw createStreamsExceptionForValueClassCastException( + throw createStreamsExceptionForClassCastException( + ProductionExceptionHandler.SerializationExceptionOrigin.VALUE, topic, value, valueSerializer, @@ -334,39 +338,27 @@ private void handleException(final ProductionExceptionHandler.Serializati droppedRecordsSensor.record(); } - private StreamsException createStreamsExceptionForKeyClassCastException(final String topic, - final K key, - final Serializer keySerializer, - final ClassCastException exception) { - final String keyClass = key == null ? "unknown because key is null" : key.getClass().getName(); - return new StreamsException( - String.format( - "ClassCastException while producing data to topic %s. " + - "The key serializer %s is not compatible to the actual key type: %s. " + - "Change the default key serde in StreamConfig or provide the correct key serde via method parameters " + - "(for example if using the DSL, `#to(String topic, Produced produced)` with " + - "`Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).", - topic, - keySerializer.getClass().getName(), - keyClass), - exception); - } - private StreamsException createStreamsExceptionForValueClassCastException(final String topic, - final V value, - final Serializer valueSerializer, - final ClassCastException exception) { - final String valueClass = value == null ? "unknown because value is null" : value.getClass().getName(); + private StreamsException createStreamsExceptionForClassCastException(final ProductionExceptionHandler.SerializationExceptionOrigin origin, + final String topic, + final KV keyOrValue, + final Serializer keyOrValueSerializer, + final ClassCastException exception) { + final String keyOrValueClass = keyOrValue == null + ? String.format("unknown because %s is null", origin.toString().toLowerCase(Locale.ROOT)) : keyOrValue.getClass().getName(); + return new StreamsException( + MessageFormat.format( String.format( "ClassCastException while producing data to topic %s. " + - "The value serializer %s is not compatible to the actual value type: %s. " + - "Change the default value serde in StreamConfig or provide the correct value serde via method parameters " + - "(for example if using the DSL, `#to(String topic, Produced produced)` with " + - "`Produced.valueSerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).", + "The {0} serializer %s is not compatible to the actual {0} type: %s. " + + "Change the default {0} serde in StreamConfig or provide the correct {0} serde via method parameters " + + "(for example if using the DSL, `#to(String topic, Produced produced)` with " + + "`Produced.{0}Serde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).", topic, - valueSerializer.getClass().getName(), - valueClass), + keyOrValueSerializer.getClass().getName(), + keyOrValueClass), + origin.toString().toLowerCase(Locale.ROOT)), exception); } From 2c957a6e5c1d3ee9c8251e11694040b85662faf9 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 30 Jul 2024 17:10:40 -0700 Subject: [PATCH 014/123] MINOR: simplify code which calles `Punctuator.punctuate()` (#16725) Reviewers: Bill Bejeck --- .../kafka/streams/processor/internals/ProcessorNode.java | 7 +------ .../kafka/streams/processor/internals/StreamTask.java | 4 ++-- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 175c9e104e..eaed7c6b8d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -24,7 +24,6 @@ import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.errors.internals.FailedProcessingException; -import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.api.FixedKeyProcessor; import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; import org.apache.kafka.streams.processor.api.InternalFixedKeyRecordFactory; @@ -203,7 +202,7 @@ public void process(final Record record) { } catch (final FailedProcessingException | TaskCorruptedException | TaskMigratedException e) { // Rethrow exceptions that should not be handled here throw e; - } catch (final Exception e) { + } catch (final RuntimeException e) { final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( null, // only required to pass for DeserializationExceptionHandler internalProcessorContext.topic(), @@ -232,10 +231,6 @@ public void process(final Record record) { } } - public void punctuate(final long timestamp, final Punctuator punctuator) { - punctuator.punctuate(timestamp); - } - public boolean isTerminalNode() { return children.isEmpty(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 8cbe5780b9..8b253c6e16 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -806,7 +806,7 @@ record = null; } catch (final StreamsException exception) { record = null; throw exception; - } catch (final Exception e) { + } catch (final RuntimeException e) { handleException(e); } finally { processorContext.setCurrentNode(null); @@ -914,7 +914,7 @@ public void punctuate(final ProcessorNode node, } try { - maybeMeasureLatency(() -> node.punctuate(timestamp, punctuator), time, punctuateLatencySensor); + maybeMeasureLatency(() -> punctuator.punctuate(timestamp), time, punctuateLatencySensor); } catch (final StreamsException e) { throw e; } catch (final RuntimeException e) { From 578fef23558830800b9e1ec561e6ec8448134d82 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Thu, 1 Aug 2024 00:53:47 +0200 Subject: [PATCH 015/123] KAFKA-16448: Handle processing exceptions in punctuate (#16300) This PR is part of KIP-1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing. This PR actually catches processing exceptions from punctuate. Co-authored-by: Dabz Co-authored-by: loicgreffier Reviewers: Bruno Cadonna , Matthias J. Sax --- checkstyle/suppressions.xml | 2 +- .../processor/internals/StreamTask.java | 44 +++++- .../processor/internals/StreamTaskTest.java | 137 +++++++++++++++++- 3 files changed, 176 insertions(+), 7 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index ba28341d1a..8c96ce7123 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -193,7 +193,7 @@ + files="(KafkaStreams|KStreamImpl|KTableImpl|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread|IQv2StoreIntegrationTest|KStreamImplTest|RocksDBStore|StreamTask).java"/> diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 8b253c6e16..6f2edd442b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -29,12 +29,14 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.TopologyConfig.TaskConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.errors.TopologyException; +import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.errors.internals.FailedProcessingException; import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.PunctuationType; @@ -63,6 +65,7 @@ import java.util.stream.Collectors; import static java.util.Collections.singleton; +import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeRecordSensor; @@ -101,6 +104,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator, private final Sensor restoreRemainingSensor; private final Sensor punctuateLatencySensor; private final Sensor bufferedRecordsSensor; + private final Sensor droppedRecordsSensor; private final Map e2eLatencySensors = new HashMap<>(); private final RecordQueueCreator recordQueueCreator; @@ -160,6 +164,7 @@ public StreamTask(final TaskId id, processLatencySensor = TaskMetrics.processLatencySensor(threadId, taskId, streamsMetrics); punctuateLatencySensor = TaskMetrics.punctuateSensor(threadId, taskId, streamsMetrics); bufferedRecordsSensor = TaskMetrics.activeBufferedRecordsSensor(threadId, taskId, streamsMetrics); + droppedRecordsSensor = TaskMetrics.droppedRecordsSensor(threadId, taskId, streamsMetrics); for (final String terminalNodeName : topology.terminalNodes()) { e2eLatencySensors.put( @@ -915,15 +920,48 @@ public void punctuate(final ProcessorNode node, try { maybeMeasureLatency(() -> punctuator.punctuate(timestamp), time, punctuateLatencySensor); - } catch (final StreamsException e) { + } catch (final FailedProcessingException e) { + throw createStreamsException(node.name(), e.getCause()); + } catch (final TaskCorruptedException | TaskMigratedException e) { throw e; - } catch (final RuntimeException e) { - throw new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e); + } catch (final Exception e) { + final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( + null, + recordContext.topic(), + recordContext.partition(), + recordContext.offset(), + recordContext.headers(), + node.name(), + id() + ); + + final ProcessingExceptionHandler.ProcessingHandlerResponse response; + + try { + response = processingExceptionHandler.handle(errorHandlerContext, null, e); + } catch (final Exception fatalUserException) { + throw new FailedProcessingException(fatalUserException); + } + + if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + log.error("Processing exception handler is set to fail upon" + + " a processing error. If you would rather have the streaming pipeline" + + " continue after a processing error, please set the " + + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately."); + + throw createStreamsException(node.name(), e); + } else { + droppedRecordsSensor.record(); + } } finally { processorContext.setCurrentNode(null); } } + private StreamsException createStreamsException(final String processorName, final Throwable cause) { + return new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, processorName), cause); + } + @SuppressWarnings("unchecked") private void updateProcessorContext(final ProcessorNode currNode, final long wallClockTime, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 817ffe1f74..a8771c2153 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.InvalidOffsetException; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetResetStrategy; @@ -45,14 +46,19 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.TopologyConfig; +import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; +import org.apache.kafka.streams.errors.LogAndContinueProcessingExceptionHandler; import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; +import org.apache.kafka.streams.errors.LogAndFailProcessingExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.errors.TopologyException; +import org.apache.kafka.streams.errors.internals.FailedProcessingException; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; @@ -121,6 +127,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -240,13 +247,18 @@ private static StreamsConfig createConfig(final String enforcedProcessingValue) } private static StreamsConfig createConfig(final String eosConfig, final String enforcedProcessingValue) { - return createConfig(eosConfig, enforcedProcessingValue, LogAndFailExceptionHandler.class.getName()); + return createConfig(eosConfig, enforcedProcessingValue, LogAndFailExceptionHandler.class.getName(), LogAndFailProcessingExceptionHandler.class.getName()); + } + + private static StreamsConfig createConfig(final String eosConfig, final String enforcedProcessingValue, final String deserializationExceptionHandler) { + return createConfig(eosConfig, enforcedProcessingValue, deserializationExceptionHandler, LogAndFailProcessingExceptionHandler.class.getName()); } private static StreamsConfig createConfig( final String eosConfig, final String enforcedProcessingValue, - final String deserializationExceptionHandler) { + final String deserializationExceptionHandler, + final String processingExceptionHandler) { final String canonicalPath; try { canonicalPath = BASE_DIR.getCanonicalPath(); @@ -262,7 +274,8 @@ private static StreamsConfig createConfig( mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()), mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig), mkEntry(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, enforcedProcessingValue), - mkEntry(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, deserializationExceptionHandler) + mkEntry(StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, deserializationExceptionHandler), + mkEntry(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, processingExceptionHandler) ))); } @@ -2647,6 +2660,124 @@ public void shouldNotCheckpointAfterRestorationWhenExactlyOnceEnabled() { verify(recordCollector, never()).offsets(); } + @Test + public void shouldPunctuateNotHandleFailProcessingExceptionAndThrowStreamsException() { + when(stateManager.taskId()).thenReturn(taskId); + when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); + task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", + LogAndFailExceptionHandler.class.getName(), LogAndContinueProcessingExceptionHandler.class.getName())); + + final StreamsException streamsException = assertThrows(StreamsException.class, () -> + task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + throw new FailedProcessingException( + new RuntimeException("KABOOM!") + ); + }) + ); + + assertInstanceOf(RuntimeException.class, streamsException.getCause()); + assertEquals("KABOOM!", streamsException.getCause().getMessage()); + } + + @Test + public void shouldPunctuateNotHandleTaskCorruptedExceptionAndThrowItAsIs() { + when(stateManager.taskId()).thenReturn(taskId); + when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); + task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", + LogAndFailExceptionHandler.class.getName(), LogAndContinueProcessingExceptionHandler.class.getName())); + + final Set tasksIds = new HashSet<>(); + tasksIds.add(new TaskId(0, 0)); + final TaskCorruptedException expectedException = new TaskCorruptedException(tasksIds, new InvalidOffsetException("Invalid offset") { + @Override + public Set partitions() { + return new HashSet<>(Collections.singletonList(new TopicPartition("topic", 0))); + } + }); + + final TaskCorruptedException taskCorruptedException = assertThrows(TaskCorruptedException.class, () -> + task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + throw expectedException; + }) + ); + + assertEquals(expectedException, taskCorruptedException); + } + + @Test + public void shouldPunctuateNotHandleTaskMigratedExceptionAndThrowItAsIs() { + when(stateManager.taskId()).thenReturn(taskId); + when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); + task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", + LogAndFailExceptionHandler.class.getName(), LogAndContinueProcessingExceptionHandler.class.getName())); + + final TaskMigratedException expectedException = new TaskMigratedException("TaskMigratedException", new RuntimeException("Task migrated cause")); + + final TaskMigratedException taskCorruptedException = assertThrows(TaskMigratedException.class, () -> + task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + throw expectedException; + }) + ); + + assertEquals(expectedException, taskCorruptedException); + } + + @Test + public void shouldPunctuateNotThrowStreamsExceptionWhenProcessingExceptionHandlerRepliesWithContinue() { + when(stateManager.taskId()).thenReturn(taskId); + when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); + task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", + LogAndFailExceptionHandler.class.getName(), LogAndContinueProcessingExceptionHandler.class.getName())); + + assertDoesNotThrow(() -> + task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + throw new KafkaException("KABOOM!"); + }) + ); + } + + @Test + public void shouldPunctuateThrowStreamsExceptionWhenProcessingExceptionHandlerRepliesWithFail() { + when(stateManager.taskId()).thenReturn(taskId); + when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); + task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", + LogAndFailExceptionHandler.class.getName(), LogAndFailProcessingExceptionHandler.class.getName())); + + final StreamsException streamsException = assertThrows(StreamsException.class, + () -> task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + throw new KafkaException("KABOOM!"); + })); + + assertInstanceOf(KafkaException.class, streamsException.getCause()); + assertEquals("KABOOM!", streamsException.getCause().getMessage()); + } + + @Test + public void shouldPunctuateThrowFailedProcessingExceptionWhenProcessingExceptionHandlerThrowsAnException() { + when(stateManager.taskId()).thenReturn(taskId); + when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); + task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", + LogAndFailExceptionHandler.class.getName(), ProcessingExceptionHandlerMock.class.getName())); + + final FailedProcessingException streamsException = assertThrows(FailedProcessingException.class, + () -> task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + throw new KafkaException("KABOOM!"); + })); + + assertInstanceOf(RuntimeException.class, streamsException.getCause()); + assertEquals("KABOOM from ProcessingExceptionHandlerMock!", streamsException.getCause().getMessage()); + } + + public static class ProcessingExceptionHandlerMock implements ProcessingExceptionHandler { + @Override + public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + throw new RuntimeException("KABOOM from ProcessingExceptionHandlerMock!"); + } + @Override + public void configure(final Map configs) { + // No-op + } + } private ProcessorStateManager mockStateManager() { final ProcessorStateManager manager = mock(ProcessorStateManager.class); From 25f04804cd4873c46b63ee7bb56d4b1d53062c35 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Thu, 1 Aug 2024 12:28:57 -0700 Subject: [PATCH 016/123] KAFKA-16521; kafka-metadata-quorum describe command changes for KIP-853 (#16759) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit describe --status now includes directory id and endpoint information for voter and observers. describe --replication now includes directory id. Reviewers: Colin P. McCabe , José Armando García Sancio --- .../common/message/DescribeQuorumRequest.json | 2 +- .../server/DescribeQuorumRequestTest.scala | 7 +++ .../kafka/tools/MetadataQuorumCommand.java | 60 ++++++++++++++++++- .../tools/MetadataQuorumCommandTest.java | 38 +++++++----- 4 files changed, 87 insertions(+), 20 deletions(-) diff --git a/clients/src/main/resources/common/message/DescribeQuorumRequest.json b/clients/src/main/resources/common/message/DescribeQuorumRequest.json index 2faaefec73..86d9975f74 100644 --- a/clients/src/main/resources/common/message/DescribeQuorumRequest.json +++ b/clients/src/main/resources/common/message/DescribeQuorumRequest.json @@ -22,7 +22,7 @@ // Version 2 adds additional fields in the response. The request is unchanged (KIP-853). "validVersions": "0-2", "flexibleVersions": "0+", - "latestVersionUnstable": true, // Version 2 is still under development. + "latestVersionUnstable": false, "fields": [ { "name": "Topics", "type": "[]TopicData", "versions": "0+", "fields": [ diff --git a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala index 292cfdab7e..7d4ccb957b 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala @@ -97,6 +97,13 @@ class DescribeQuorumRequestTest(cluster: ClusterInstance) { assertNotEquals(-1, state.lastCaughtUpTimestamp) } } + + if (version >= 2) { + val nodes = response.data.nodes().asScala + assertEquals(cluster.controllerIds().asScala, nodes.map(_.nodeId()).toSet) + val node = nodes.find(_.nodeId() == cluster.controllers().keySet().asScala.head) + assertEquals(cluster.controllerListenerName().get().value(), node.get.listeners().asScala.head.name()) + } } } diff --git a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java index 162663a9de..1fdc600cbc 100644 --- a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java @@ -18,7 +18,9 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.QuorumInfo; +import org.apache.kafka.clients.admin.RaftVoterEndpoint; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.util.CommandLineUtils; @@ -167,7 +169,7 @@ private static void handleDescribeReplication(Admin admin, boolean humanReadable rows.addAll(quorumInfoToRows(leader, quorumInfo.observers().stream(), "Observer", humanReadable)); ToolsUtils.prettyPrintTable( - asList("NodeId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"), + asList("NodeId", "DirectoryId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"), rows, System.out ); @@ -186,6 +188,7 @@ private static List> quorumInfoToRows(QuorumInfo.ReplicaState leade valueOf(info.lastCaughtUpTimestamp().getAsLong()); return Stream.of( info.replicaId(), + info.replicaDirectoryId(), info.logEndOffset(), leader.logEndOffset() - info.logEndOffset(), lastFetchTimestamp, @@ -232,9 +235,60 @@ else if (leader.lastCaughtUpTimestamp().isPresent() && maxLagFollower.lastCaught "\nHighWatermark: " + quorumInfo.highWatermark() + "\nMaxFollowerLag: " + maxFollowerLag + "\nMaxFollowerLagTimeMs: " + maxFollowerLagTimeMs + - "\nCurrentVoters: " + quorumInfo.voters().stream().map(QuorumInfo.ReplicaState::replicaId).map(Object::toString).collect(Collectors.joining(",", "[", "]")) + - "\nCurrentObservers: " + quorumInfo.observers().stream().map(QuorumInfo.ReplicaState::replicaId).map(Objects::toString).collect(Collectors.joining(",", "[", "]")) + "\nCurrentVoters: " + printVoterState(quorumInfo) + + "\nCurrentObservers: " + printObserverState(quorumInfo) ); } + // Constructs the CurrentVoters string + // CurrentVoters: [{"id": 0, "directoryId": "UUID1", "endpoints": [{"name": "C", "securityProtocol": "SSL", "host": "controller-0", "port": 1234}]}, {"id": 1, ... }]}] + private static String printVoterState(QuorumInfo quorumInfo) { + return printReplicaState(quorumInfo, quorumInfo.voters()); + } + + // Constructs the CurrentObservers string + private static String printObserverState(QuorumInfo quorumInfo) { + return printReplicaState(quorumInfo, quorumInfo.observers()); + } + + private static String printReplicaState(QuorumInfo quorumInfo, List replicas) { + List currentVoterList = replicas.stream().map(voter -> new Node( + voter.replicaId(), + voter.replicaDirectoryId(), + getEndpoints(quorumInfo.nodes().get(voter.replicaId())))).collect(Collectors.toList()); + return currentVoterList.stream().map(Objects::toString).collect(Collectors.joining(", ", "[", "]")); + } + + private static List getEndpoints(QuorumInfo.Node node) { + return node == null ? new ArrayList<>() : node.endpoints(); + } + + private static class Node { + private final int id; + private final Uuid directoryId; + private final List endpoints; + + private Node(int id, Uuid directoryId, List endpoints) { + this.id = id; + this.directoryId = Objects.requireNonNull(directoryId); + this.endpoints = Objects.requireNonNull(endpoints); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("{"); + sb.append("\"id\": ").append(id).append(", "); + sb.append("\"directoryId\": ").append("\"").append(directoryId.equals(Uuid.ZERO_UUID) ? "null" : directoryId).append("\""); + if (!endpoints.isEmpty()) { + sb.append(", \"endpoints\": "); + for (RaftVoterEndpoint endpoint : endpoints) { + sb.append(endpoint.toString()).append(", "); + } + sb.setLength(sb.length() - 2); // remove the last comma and space + } + sb.append("}"); + return sb.toString(); + } + } } diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java index 29d946ea47..809ce1ee16 100644 --- a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java @@ -62,25 +62,30 @@ public void testDescribeQuorumReplicationSuccessful(ClusterInstance cluster) thr MetadataQuorumCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication") ); - List outputs = Arrays.stream(describeOutput.split("\n")).skip(1).collect(Collectors.toList()); + List outputs = Arrays.stream(describeOutput.split("\n")).collect(Collectors.toList()); + String header = outputs.get(0); + List data = outputs.subList(1, outputs.size()); + + assertTrue(header.matches("NodeId\\s+DirectoryId\\s+LogEndOffset\\s+Lag\\s+LastFetchTimestamp\\s+LastCaughtUpTimestamp\\s+Status\\s+")); + if (cluster.type() == Type.CO_KRAFT) - assertEquals(Math.max(cluster.config().numControllers(), cluster.config().numBrokers()), outputs.size()); + assertEquals(Math.max(cluster.config().numControllers(), cluster.config().numBrokers()), data.size()); else - assertEquals(cluster.config().numBrokers() + cluster.config().numControllers(), outputs.size()); + assertEquals(cluster.config().numBrokers() + cluster.config().numControllers(), data.size()); - Pattern leaderPattern = Pattern.compile("\\d+\\s+\\d+\\s+\\d+\\s+[\\dmsago\\s]+-?[\\dmsago\\s]+Leader\\s*"); - assertTrue(leaderPattern.matcher(outputs.get(0)).find()); - assertTrue(outputs.stream().skip(1).noneMatch(o -> leaderPattern.matcher(o).find())); + Pattern leaderPattern = Pattern.compile("\\d+\\s+\\S+\\s+\\d+\\s+\\d+\\s+-?\\d+\\s+-?\\d+\\s+Leader\\s*"); + assertTrue(leaderPattern.matcher(data.get(0)).find()); + assertTrue(data.stream().skip(1).noneMatch(o -> leaderPattern.matcher(o).find())); - Pattern followerPattern = Pattern.compile("\\d+\\s+\\d+\\s+\\d+\\s+[\\dmsago\\s]+-?[\\dmsago\\s]+Follower\\s*"); - assertEquals(cluster.config().numControllers() - 1, outputs.stream().filter(o -> followerPattern.matcher(o).find()).count()); + Pattern followerPattern = Pattern.compile("\\d+\\s+\\S+\\s+\\d+\\s+\\d+\\s+-?\\d+\\s+-?\\d+\\s+Follower\\s*"); + assertEquals(cluster.config().numControllers() - 1, data.stream().filter(o -> followerPattern.matcher(o).find()).count()); - Pattern observerPattern = Pattern.compile("\\d+\\s+\\d+\\s+\\d+\\s+[\\dmsago\\s]+-?[\\dmsago\\s]+Observer\\s*"); + Pattern observerPattern = Pattern.compile("\\d+\\s+\\S+\\s+\\d+\\s+\\d+\\s+-?\\d+\\s+-?\\d+\\s+Observer\\s*"); if (cluster.type() == Type.CO_KRAFT) assertEquals(Math.max(0, cluster.config().numBrokers() - cluster.config().numControllers()), - outputs.stream().filter(o -> observerPattern.matcher(o).find()).count()); + data.stream().filter(o -> observerPattern.matcher(o).find()).count()); else - assertEquals(cluster.config().numBrokers(), outputs.stream().filter(o -> observerPattern.matcher(o).find()).count()); + assertEquals(cluster.config().numBrokers(), data.stream().filter(o -> observerPattern.matcher(o).find()).count()); } /** @@ -113,7 +118,7 @@ public void testDescribeQuorumStatusSuccessful(ClusterInstance cluster) throws I assertTrue(outputs[4].matches("MaxFollowerLag:\\s+\\d+"), describeOutput); assertTrue(outputs[5].matches("MaxFollowerLagTimeMs:\\s+-?\\d+"), describeOutput); assertTrue( - outputs[6].matches("CurrentVoters:\\s+\\[\\d+(,\\d+)*]"), + outputs[6].matches("CurrentVoters:\\s+\\[\\{\"id\":\\s+\\d+,\\s+\"directoryId\":\\s+\\S+,\\s+\"endpoints\":\\s+.*}]"), describeOutput ); @@ -122,7 +127,8 @@ public void testDescribeQuorumStatusSuccessful(ClusterInstance cluster) throws I assertTrue(outputs[7].matches("CurrentObservers:\\s+\\[]"), describeOutput); } else { assertTrue( - outputs[7].matches("CurrentObservers:\\s+\\[\\d+(,\\d+)*]"), + outputs[7].matches("CurrentObservers:\\s+\\[\\{\"id\":\\s+\\d+,\\s+\"directoryId\":\\s+\\S+}" + + "(,\\s+\\{\"id\":\\s+\\d+,\\s+\"directoryId\":\\s+\\S+})*]"), describeOutput ); } @@ -139,7 +145,7 @@ public void testOnlyOneBrokerAndOneController(ClusterInstance cluster) { String replicationOutput = ToolsTestUtils.captureStandardOut(() -> MetadataQuorumCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication") ); - assertEquals("0", replicationOutput.split("\n")[1].split("\\s+")[2]); + assertEquals("0", replicationOutput.split("\n")[1].split("\\s+")[3]); } @Test @@ -183,9 +189,9 @@ public void testHumanReadableOutput(ClusterInstance cluster) { private static void assertHumanReadable(String output) { String dataRow = output.split("\n")[1]; - String lastFetchTimestamp = dataRow.split("\t")[3]; + String lastFetchTimestamp = dataRow.split("\t")[4]; String lastFetchTimestampValue = lastFetchTimestamp.split(" ")[0]; - String lastCaughtUpTimestamp = dataRow.split("\t")[4]; + String lastCaughtUpTimestamp = dataRow.split("\t")[5]; String lastCaughtUpTimestampValue = lastCaughtUpTimestamp.split(" ")[0]; assertTrue(lastFetchTimestamp.contains("ms ago")); assertTrue(lastFetchTimestampValue.matches("\\d*")); From 60e1478fb9dc9d1d17d44d84c8a6076f8df6e687 Mon Sep 17 00:00:00 2001 From: Kondrat Bertalan Date: Wed, 31 Jul 2024 22:50:22 +0200 Subject: [PATCH 017/123] =?UTF-8?q?KAFKA-17192=20Fix=20MirrorMaker2=20work?= =?UTF-8?q?er=20config=20does=20not=20pass=20config.provi=E2=80=A6=20(#166?= =?UTF-8?q?78)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewers: Chris Egerton --- .../org/apache/kafka/common/utils/Utils.java | 17 ++++++++++++++++- .../kafka/connect/mirror/MirrorMakerConfig.java | 2 +- .../connect/mirror/MirrorMakerConfigTest.java | 1 + 3 files changed, 18 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index f2961a8f28..b23a9d72a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -1648,9 +1648,24 @@ public static Map entriesWithPrefix(Map map, String pr * @param the type of values stored in the map */ public static Map entriesWithPrefix(Map map, String prefix, boolean strip) { + return entriesWithPrefix(map, prefix, strip, false); + } + + /** + * Find all key/value pairs whose keys begin with the given prefix, optionally removing that prefix + * from all resulting keys. + * @param map the map to filter key/value pairs from + * @param prefix the prefix to search keys for + * @param strip whether the keys of the returned map should not include the prefix + * @param allowMatchingLength whether to include keys that are exactly the same length as the prefix + * @return a {@link Map} containing a key/value pair for every key/value pair in the {@code map} + * parameter whose key begins with the given {@code prefix}; may be empty, but never null + * @param the type of values stored in the map + */ + public static Map entriesWithPrefix(Map map, String prefix, boolean strip, boolean allowMatchingLength) { Map result = new HashMap<>(); for (Map.Entry entry : map.entrySet()) { - if (entry.getKey().startsWith(prefix) && entry.getKey().length() > prefix.length()) { + if (entry.getKey().startsWith(prefix) && (allowMatchingLength || entry.getKey().length() > prefix.length())) { if (strip) result.put(entry.getKey().substring(prefix.length()), entry.getValue()); else diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java index 8f9f06f058..014d976728 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java @@ -308,7 +308,7 @@ private Map stringsWithPrefixStripped(String prefix) { } private Map stringsWithPrefix(String prefix) { - return Utils.entriesWithPrefix(rawProperties, prefix, false); + return Utils.entriesWithPrefix(rawProperties, prefix, false, true); } static Map clusterConfigsWithPrefix(String prefix, Map props) { diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java index 163ebdd4b7..03f3bd6aaa 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java @@ -257,6 +257,7 @@ public void testWorkerConfigs() { assertEquals("b->a", aProps.get("client.id")); assertEquals("123", aProps.get("offset.storage.replication.factor")); assertEquals("__", aProps.get("replication.policy.separator")); + assertEquals("fake", aProps.get("config.providers")); Map bProps = mirrorConfig.workerConfig(b); assertEquals("a->b", bProps.get("client.id")); assertEquals("456", bProps.get("status.storage.replication.factor")); From fbb598ce829d923a297b896e4adbdb02b2c6891f Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Fri, 2 Aug 2024 11:53:57 +0800 Subject: [PATCH 018/123] KAFKA-16666 Migrate GroupMetadataMessageFormatter` to tools module (#16748) we need to migate GroupMetadataMessageFormatter from scala code to java code,and make the message format is json pattern Reviewers: Chia-Ping Tsai --- .../group/GroupMetadataManager.scala | 1 + .../tools/consumer/ApiMessageFormatter.java | 82 +++++++++ .../consumer/ConsoleConsumerOptions.java | 4 + .../GroupMetadataMessageFormatter.java | 81 +++++++++ .../consumer/OffsetsMessageFormatter.java | 68 ++------ .../TransactionLogMessageFormatter.java | 61 ++----- .../consumer/ConsoleConsumerOptionsTest.java | 14 ++ .../tools/consumer/ConsoleConsumerTest.java | 78 +++++++-- .../GroupMetadataMessageFormatterTest.java | 163 ++++++++++++++++++ 9 files changed, 436 insertions(+), 116 deletions(-) create mode 100644 tools/src/main/java/org/apache/kafka/tools/consumer/ApiMessageFormatter.java create mode 100644 tools/src/main/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatter.java create mode 100644 tools/src/test/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatterTest.java diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index 5bb7216ab8..252ff06320 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -1265,6 +1265,7 @@ object GroupMetadataManager { } // Formatter for use with tools to read group metadata history + @Deprecated class GroupMetadataMessageFormatter extends MessageFormatter { def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream): Unit = { Option(consumerRecord.key).map(key => GroupMetadataManager.readMessageKey(ByteBuffer.wrap(key))).foreach { diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ApiMessageFormatter.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ApiMessageFormatter.java new file mode 100644 index 0000000000..9bedbe265c --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ApiMessageFormatter.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.consumer; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.MessageFormatter; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.NullNode; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import java.io.IOException; +import java.io.PrintStream; +import java.nio.ByteBuffer; +import java.util.Objects; + +import static java.nio.charset.StandardCharsets.UTF_8; + +public abstract class ApiMessageFormatter implements MessageFormatter { + + private static final String VERSION = "version"; + private static final String DATA = "data"; + private static final String KEY = "key"; + private static final String VALUE = "value"; + static final String UNKNOWN = "unknown"; + + @Override + public void writeTo(ConsumerRecord consumerRecord, PrintStream output) { + ObjectNode json = new ObjectNode(JsonNodeFactory.instance); + + byte[] key = consumerRecord.key(); + if (Objects.nonNull(key)) { + short keyVersion = ByteBuffer.wrap(key).getShort(); + JsonNode dataNode = readToKeyJson(ByteBuffer.wrap(key), keyVersion); + + if (dataNode instanceof NullNode) { + return; + } + json.putObject(KEY) + .put(VERSION, keyVersion) + .set(DATA, dataNode); + } else { + json.set(KEY, NullNode.getInstance()); + } + + byte[] value = consumerRecord.value(); + if (Objects.nonNull(value)) { + short valueVersion = ByteBuffer.wrap(value).getShort(); + JsonNode dataNode = readToValueJson(ByteBuffer.wrap(value), valueVersion); + + json.putObject(VALUE) + .put(VERSION, valueVersion) + .set(DATA, dataNode); + } else { + json.set(VALUE, NullNode.getInstance()); + } + + try { + output.write(json.toString().getBytes(UTF_8)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + protected abstract JsonNode readToKeyJson(ByteBuffer byteBuffer, short version); + protected abstract JsonNode readToValueJson(ByteBuffer byteBuffer, short version); +} diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java index 455ca885a5..5c6f7a1027 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java @@ -372,6 +372,10 @@ private static String convertDeprecatedClass(String className) { System.err.println("WARNING: kafka.coordinator.group.GroupMetadataManager$OffsetsMessageFormatter is deprecated and will be removed in the next major release. " + "Please use org.apache.kafka.tools.consumer.OffsetsMessageFormatter instead"); return className; + case "kafka.coordinator.group.GroupMetadataManager$GroupMetadataMessageFormatter": + System.err.println("WARNING: kafka.coordinator.group.GroupMetadataManager$GroupMetadataMessageFormatter is deprecated and will be removed in the next major release. " + + "Please use org.apache.kafka.tools.consumer.GroupMetadataMessageFormatter instead"); + return className; default: return className; } diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatter.java b/tools/src/main/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatter.java new file mode 100644 index 0000000000..754c43193c --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatter.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.consumer; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKeyJsonConverter; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValueJsonConverter; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.NullNode; +import com.fasterxml.jackson.databind.node.TextNode; + +import java.nio.ByteBuffer; +import java.util.Optional; + +public class GroupMetadataMessageFormatter extends ApiMessageFormatter { + + @Override + protected JsonNode readToKeyJson(ByteBuffer byteBuffer, short version) { + return readToGroupMetadataKey(byteBuffer) + .map(logKey -> transferKeyMessageToJsonNode(logKey, version)) + .orElseGet(() -> new TextNode(UNKNOWN)); + } + + @Override + protected JsonNode readToValueJson(ByteBuffer byteBuffer, short version) { + return readToGroupMetadataValue(byteBuffer) + .map(logValue -> GroupMetadataValueJsonConverter.write(logValue, version)) + .orElseGet(() -> new TextNode(UNKNOWN)); + } + + private Optional readToGroupMetadataKey(ByteBuffer byteBuffer) { + short version = byteBuffer.getShort(); + if (version >= OffsetCommitKey.LOWEST_SUPPORTED_VERSION + && version <= OffsetCommitKey.HIGHEST_SUPPORTED_VERSION) { + return Optional.of(new OffsetCommitKey(new ByteBufferAccessor(byteBuffer), version)); + } else if (version >= GroupMetadataKey.LOWEST_SUPPORTED_VERSION && version <= GroupMetadataKey.HIGHEST_SUPPORTED_VERSION) { + return Optional.of(new GroupMetadataKey(new ByteBufferAccessor(byteBuffer), version)); + } else { + return Optional.empty(); + } + } + + private JsonNode transferKeyMessageToJsonNode(ApiMessage message, short version) { + if (message instanceof OffsetCommitKey) { + return NullNode.getInstance(); + } else if (message instanceof GroupMetadataKey) { + return GroupMetadataKeyJsonConverter.write((GroupMetadataKey) message, version); + } else { + return new TextNode(UNKNOWN); + } + } + + private Optional readToGroupMetadataValue(ByteBuffer byteBuffer) { + short version = byteBuffer.getShort(); + if (version >= GroupMetadataValue.LOWEST_SUPPORTED_VERSION + && version <= GroupMetadataValue.HIGHEST_SUPPORTED_VERSION) { + return Optional.of(new GroupMetadataValue(new ByteBufferAccessor(byteBuffer), version)); + } else { + return Optional.empty(); + } + } +} diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java b/tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java index 62dcb871c8..2927ed46c9 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.tools.consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.MessageFormatter; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; @@ -27,69 +25,29 @@ import org.apache.kafka.coordinator.group.generated.OffsetCommitValueJsonConverter; import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.NullNode; -import com.fasterxml.jackson.databind.node.ObjectNode; import com.fasterxml.jackson.databind.node.TextNode; -import java.io.IOException; -import java.io.PrintStream; import java.nio.ByteBuffer; -import java.util.Objects; import java.util.Optional; -import static java.nio.charset.StandardCharsets.UTF_8; - /** * Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false. */ -public class OffsetsMessageFormatter implements MessageFormatter { - - private static final String VERSION = "version"; - private static final String DATA = "data"; - private static final String KEY = "key"; - private static final String VALUE = "value"; - private static final String UNKNOWN = "unknown"; +public class OffsetsMessageFormatter extends ApiMessageFormatter { @Override - public void writeTo(ConsumerRecord consumerRecord, PrintStream output) { - ObjectNode json = new ObjectNode(JsonNodeFactory.instance); - - byte[] key = consumerRecord.key(); - if (Objects.nonNull(key)) { - short keyVersion = ByteBuffer.wrap(key).getShort(); - JsonNode dataNode = readToGroupMetadataKey(ByteBuffer.wrap(key)) - .map(logKey -> transferMetadataToJsonNode(logKey, keyVersion)) - .orElseGet(() -> new TextNode(UNKNOWN)); - // Only print if the message is an offset record. - if (dataNode instanceof NullNode) { - return; - } - json.putObject(KEY) - .put(VERSION, keyVersion) - .set(DATA, dataNode); - } else { - json.set(KEY, NullNode.getInstance()); - } - - byte[] value = consumerRecord.value(); - if (Objects.nonNull(value)) { - short valueVersion = ByteBuffer.wrap(value).getShort(); - JsonNode dataNode = readToOffsetCommitValue(ByteBuffer.wrap(value)) - .map(logValue -> OffsetCommitValueJsonConverter.write(logValue, valueVersion)) - .orElseGet(() -> new TextNode(UNKNOWN)); - json.putObject(VALUE) - .put(VERSION, valueVersion) - .set(DATA, dataNode); - } else { - json.set(VALUE, NullNode.getInstance()); - } + protected JsonNode readToKeyJson(ByteBuffer byteBuffer, short version) { + return readToGroupMetadataKey(byteBuffer) + .map(logKey -> transferKeyMessageToJsonNode(logKey, version)) + .orElseGet(() -> new TextNode(UNKNOWN)); + } - try { - output.write(json.toString().getBytes(UTF_8)); - } catch (IOException e) { - throw new RuntimeException(e); - } + @Override + protected JsonNode readToValueJson(ByteBuffer byteBuffer, short version) { + return readToOffsetMessageValue(byteBuffer) + .map(logValue -> OffsetCommitValueJsonConverter.write(logValue, version)) + .orElseGet(() -> new TextNode(UNKNOWN)); } private Optional readToGroupMetadataKey(ByteBuffer byteBuffer) { @@ -104,7 +62,7 @@ private Optional readToGroupMetadataKey(ByteBuffer byteBuffer) { } } - private static JsonNode transferMetadataToJsonNode(ApiMessage logKey, short keyVersion) { + private JsonNode transferKeyMessageToJsonNode(ApiMessage logKey, short keyVersion) { if (logKey instanceof OffsetCommitKey) { return OffsetCommitKeyJsonConverter.write((OffsetCommitKey) logKey, keyVersion); } else if (logKey instanceof GroupMetadataKey) { @@ -114,7 +72,7 @@ private static JsonNode transferMetadataToJsonNode(ApiMessage logKey, short keyV } } - private Optional readToOffsetCommitValue(ByteBuffer byteBuffer) { + private Optional readToOffsetMessageValue(ByteBuffer byteBuffer) { short version = byteBuffer.getShort(); if (version >= OffsetCommitValue.LOWEST_SUPPORTED_VERSION && version <= OffsetCommitValue.HIGHEST_SUPPORTED_VERSION) { diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatter.java b/tools/src/main/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatter.java index 5db761adad..d60231d320 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatter.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatter.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.tools.consumer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.MessageFormatter; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey; import org.apache.kafka.coordinator.transaction.generated.TransactionLogKeyJsonConverter; @@ -25,62 +23,25 @@ import org.apache.kafka.coordinator.transaction.generated.TransactionLogValueJsonConverter; import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.NullNode; -import com.fasterxml.jackson.databind.node.ObjectNode; import com.fasterxml.jackson.databind.node.TextNode; -import java.io.IOException; -import java.io.PrintStream; import java.nio.ByteBuffer; -import java.util.Objects; import java.util.Optional; -import static java.nio.charset.StandardCharsets.UTF_8; - -public class TransactionLogMessageFormatter implements MessageFormatter { - - private static final String VERSION = "version"; - private static final String DATA = "data"; - private static final String KEY = "key"; - private static final String VALUE = "value"; - private static final String UNKNOWN = "unknown"; +public class TransactionLogMessageFormatter extends ApiMessageFormatter { @Override - public void writeTo(ConsumerRecord consumerRecord, PrintStream output) { - ObjectNode json = new ObjectNode(JsonNodeFactory.instance); - - byte[] key = consumerRecord.key(); - if (Objects.nonNull(key)) { - short keyVersion = ByteBuffer.wrap(key).getShort(); - JsonNode dataNode = readToTransactionLogKey(ByteBuffer.wrap(key)) - .map(logKey -> TransactionLogKeyJsonConverter.write(logKey, keyVersion)) - .orElseGet(() -> new TextNode(UNKNOWN)); - json.putObject(KEY) - .put(VERSION, keyVersion) - .set(DATA, dataNode); - } else { - json.set(KEY, NullNode.getInstance()); - } - - byte[] value = consumerRecord.value(); - if (Objects.nonNull(value)) { - short valueVersion = ByteBuffer.wrap(value).getShort(); - JsonNode dataNode = readToTransactionLogValue(ByteBuffer.wrap(value)) - .map(logValue -> TransactionLogValueJsonConverter.write(logValue, valueVersion)) - .orElseGet(() -> new TextNode(UNKNOWN)); - json.putObject(VALUE) - .put(VERSION, valueVersion) - .set(DATA, dataNode); - } else { - json.set(VALUE, NullNode.getInstance()); - } + protected JsonNode readToKeyJson(ByteBuffer byteBuffer, short version) { + return readToTransactionLogKey(byteBuffer) + .map(logKey -> TransactionLogKeyJsonConverter.write(logKey, version)) + .orElseGet(() -> new TextNode(UNKNOWN)); + } - try { - output.write(json.toString().getBytes(UTF_8)); - } catch (IOException e) { - throw new RuntimeException(e); - } + @Override + protected JsonNode readToValueJson(ByteBuffer byteBuffer, short version) { + return readToTransactionLogValue(byteBuffer) + .map(logValue -> TransactionLogValueJsonConverter.write(logValue, version)) + .orElseGet(() -> new TextNode(UNKNOWN)); } private Optional readToTransactionLogKey(ByteBuffer byteBuffer) { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java index 7c84a10fd4..a4a24c6000 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java @@ -686,6 +686,20 @@ public void testNewAndDeprecateOffsetsMessageFormatter() throws Exception { assertInstanceOf(OffsetsMessageFormatter.class, new ConsoleConsumerOptions(offsetsMessageFormatter).formatter()); } + + @SuppressWarnings("deprecation") + @Test + public void testNewAndDeprecateGroupMetadataMessageFormatter() throws Exception { + String[] deprecatedGroupMetadataMessageFormatter = + generateArgsForFormatter("kafka.coordinator.group.GroupMetadataManager$GroupMetadataMessageFormatter"); + assertInstanceOf(kafka.coordinator.group.GroupMetadataManager.GroupMetadataMessageFormatter.class, + new ConsoleConsumerOptions(deprecatedGroupMetadataMessageFormatter).formatter()); + + String[] groupMetadataMessageFormatter = + generateArgsForFormatter("org.apache.kafka.tools.consumer.GroupMetadataMessageFormatter"); + assertInstanceOf(GroupMetadataMessageFormatter.class, + new ConsoleConsumerOptions(groupMetadataMessageFormatter).formatter()); + } private String[] generateArgsForFormatter(String formatter) { return new String[]{ diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java index 6e378b3a98..108e4c127d 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java @@ -39,6 +39,10 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKeyJsonConverter; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValueJsonConverter; import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; import org.apache.kafka.coordinator.group.generated.OffsetCommitKeyJsonConverter; import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; @@ -297,12 +301,9 @@ public void testTransactionLogMessageFormatter(ClusterInstance cluster) throws E admin.createTopics(singleton(newTopic)); produceMessagesWithTxn(cluster); - String[] transactionLogMessageFormatter = new String[]{ - "--bootstrap-server", cluster.bootstrapServers(), - "--topic", Topic.TRANSACTION_STATE_TOPIC_NAME, - "--formatter", "org.apache.kafka.tools.consumer.TransactionLogMessageFormatter", - "--from-beginning" - }; + String[] transactionLogMessageFormatter = createConsoleConsumerArgs(cluster, + Topic.TRANSACTION_STATE_TOPIC_NAME, + "org.apache.kafka.tools.consumer.TransactionLogMessageFormatter"); ConsoleConsumerOptions options = new ConsoleConsumerOptions(transactionLogMessageFormatter); ConsoleConsumer.ConsumerWrapper consumerWrapper = new ConsoleConsumer.ConsumerWrapper(options, createTxnConsumer(cluster)); @@ -339,11 +340,9 @@ public void testOffsetsMessageFormatter(ClusterInstance cluster) throws Exceptio admin.createTopics(singleton(newTopic)); produceMessages(cluster); - String[] offsetsMessageFormatter = new String[]{ - "--bootstrap-server", cluster.bootstrapServers(), - "--topic", Topic.GROUP_METADATA_TOPIC_NAME, - "--formatter", "org.apache.kafka.tools.consumer.OffsetsMessageFormatter" - }; + String[] offsetsMessageFormatter = createConsoleConsumerArgs(cluster, + Topic.GROUP_METADATA_TOPIC_NAME, + "org.apache.kafka.tools.consumer.OffsetsMessageFormatter"); ConsoleConsumerOptions options = new ConsoleConsumerOptions(offsetsMessageFormatter); ConsoleConsumer.ConsumerWrapper consumerWrapper = new ConsoleConsumer.ConsumerWrapper(options, createOffsetConsumer(cluster)); @@ -375,6 +374,49 @@ public void testOffsetsMessageFormatter(ClusterInstance cluster) throws Exceptio } } + @ClusterTest(brokers = 3) + public void testGroupMetadataMessageFormatter(ClusterInstance cluster) throws Exception { + try (Admin admin = cluster.createAdminClient()) { + + NewTopic newTopic = new NewTopic(topic, 1, (short) 1); + admin.createTopics(singleton(newTopic)); + produceMessages(cluster); + + String[] groupMetadataMessageFormatter = createConsoleConsumerArgs(cluster, + Topic.GROUP_METADATA_TOPIC_NAME, + "org.apache.kafka.tools.consumer.GroupMetadataMessageFormatter"); + + ConsoleConsumerOptions options = new ConsoleConsumerOptions(groupMetadataMessageFormatter); + ConsoleConsumer.ConsumerWrapper consumerWrapper = + new ConsoleConsumer.ConsumerWrapper(options, createGroupMetaDataConsumer(cluster)); + + try (ByteArrayOutputStream out = new ByteArrayOutputStream(); + PrintStream output = new PrintStream(out)) { + ConsoleConsumer.process(1, options.formatter(), consumerWrapper, output, true); + + JsonNode jsonNode = objectMapper.reader().readTree(out.toByteArray()); + JsonNode keyNode = jsonNode.get("key"); + + GroupMetadataKey groupMetadataKey = + GroupMetadataKeyJsonConverter.read(keyNode.get("data"), GroupMetadataKey.HIGHEST_SUPPORTED_VERSION); + assertNotNull(groupMetadataKey); + assertEquals(groupId, groupMetadataKey.group()); + + JsonNode valueNode = jsonNode.get("value"); + GroupMetadataValue groupMetadataValue = + GroupMetadataValueJsonConverter.read(valueNode.get("data"), GroupMetadataValue.HIGHEST_SUPPORTED_VERSION); + assertNotNull(groupMetadataValue); + assertEquals("consumer", groupMetadataValue.protocolType()); + assertEquals(1, groupMetadataValue.generation()); + assertEquals("range", groupMetadataValue.protocol()); + assertNotNull(groupMetadataValue.leader()); + assertEquals(1, groupMetadataValue.members().size()); + } finally { + consumerWrapper.cleanup(); + } + } + } + private void produceMessagesWithTxn(ClusterInstance cluster) { try (Producer producer = createTxnProducer(cluster)) { producer.initTransactions(); @@ -389,6 +431,14 @@ private void produceMessages(ClusterInstance cluster) { producer.send(new ProducerRecord<>(topic, new byte[1_000 * 100])); } } + + private String[] createConsoleConsumerArgs(ClusterInstance cluster, String topic, String formatter) { + return new String[]{ + "--bootstrap-server", cluster.bootstrapServers(), + "--topic", topic, + "--formatter", formatter + }; + } private Producer createTxnProducer(ClusterInstance cluster) { Properties props = producerProps(cluster); @@ -410,6 +460,12 @@ private Consumer createOffsetConsumer(ClusterInstance cluster) { props.put(EXCLUDE_INTERNAL_TOPICS_CONFIG, "false"); return new KafkaConsumer<>(props); } + + private Consumer createGroupMetaDataConsumer(ClusterInstance cluster) { + Properties props = consumerProps(cluster); + props.put(AUTO_OFFSET_RESET_CONFIG, "earliest"); + return new KafkaConsumer<>(props); + } private Properties producerProps(ClusterInstance cluster) { Properties props = new Properties(); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatterTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatterTest.java new file mode 100644 index 0000000000..bc06d2c645 --- /dev/null +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatterTest.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tools.consumer; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.MessageFormatter; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; +import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.Optional; +import java.util.stream.Stream; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonList; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class GroupMetadataMessageFormatterTest { + + private static final OffsetCommitKey OFFSET_COMMIT_KEY = new OffsetCommitKey() + .setGroup("group-id") + .setTopic("foo") + .setPartition(1); + private static final OffsetCommitValue OFFSET_COMMIT_VALUE = new OffsetCommitValue() + .setOffset(100L) + .setLeaderEpoch(10) + .setMetadata("metadata") + .setCommitTimestamp(1234L) + .setExpireTimestamp(-1L); + private static final GroupMetadataValue.MemberMetadata MEMBER_METADATA = new GroupMetadataValue.MemberMetadata() + .setMemberId("member-1") + .setClientId("client-1") + .setClientHost("host-1") + .setRebalanceTimeout(1000) + .setSessionTimeout(1500) + .setGroupInstanceId("group-instance-1") + .setSubscription(new byte[]{0, 1}) + .setAssignment(new byte[]{1, 2}); + private static final GroupMetadataKey GROUP_METADATA_KEY = new GroupMetadataKey() + .setGroup("group-id"); + private static final GroupMetadataValue GROUP_METADATA_VALUE = new GroupMetadataValue() + .setProtocolType("consumer") + .setGeneration(1) + .setProtocol("range") + .setLeader("leader") + .setMembers(singletonList(MEMBER_METADATA)) + .setCurrentStateTimestamp(1234L); + private static final String TOPIC = "TOPIC"; + + private static Stream parameters() { + return Stream.of( + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 10, GROUP_METADATA_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 10, GROUP_METADATA_VALUE).array(), + "{\"key\":{\"version\":10,\"data\":\"unknown\"},\"value\":{\"version\":10,\"data\":\"unknown\"}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 0, GROUP_METADATA_VALUE).array(), + "{\"key\":{\"version\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":0," + + "\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," + + "\"leader\":\"leader\",\"members\":[{\"memberId\":\"member-1\",\"clientId\":\"client-1\"," + + "\"clientHost\":\"host-1\",\"sessionTimeout\":1500,\"subscription\":\"AAE=\"," + + "\"assignment\":\"AQI=\"}]}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 1, GROUP_METADATA_VALUE).array(), + "{\"key\":{\"version\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":1," + + "\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," + + "\"leader\":\"leader\",\"members\":[{\"memberId\":\"member-1\",\"clientId\":\"client-1\"," + + "\"clientHost\":\"host-1\",\"rebalanceTimeout\":1000,\"sessionTimeout\":1500," + + "\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_VALUE).array(), + "{\"key\":{\"version\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":2," + + "\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," + + "\"leader\":\"leader\",\"currentStateTimestamp\":1234,\"members\":[{\"memberId\":\"member-1\"," + + "\"clientId\":\"client-1\",\"clientHost\":\"host-1\",\"rebalanceTimeout\":1000," + + "\"sessionTimeout\":1500,\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 3, GROUP_METADATA_VALUE).array(), + "{\"key\":{\"version\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":3," + + "\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," + + "\"leader\":\"leader\",\"currentStateTimestamp\":1234,\"members\":[{\"memberId\":\"member-1\"," + + "\"groupInstanceId\":\"group-instance-1\",\"clientId\":\"client-1\",\"clientHost\":\"host-1\"," + + "\"rebalanceTimeout\":1000,\"sessionTimeout\":1500,\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 4, GROUP_METADATA_VALUE).array(), + "{\"key\":{\"version\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":4," + + "\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," + + "\"leader\":\"leader\",\"currentStateTimestamp\":1234,\"members\":[{\"memberId\":\"member-1\"," + + "\"groupInstanceId\":\"group-instance-1\",\"clientId\":\"client-1\",\"clientHost\":\"host-1\"," + + "\"rebalanceTimeout\":1000,\"sessionTimeout\":1500,\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}" + ), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(), + null, + "{\"key\":{\"version\":2,\"data\":{\"group\":\"group-id\"}},\"value\":null}"), + Arguments.of( + null, + MessageUtil.toVersionPrefixedByteBuffer((short) 4, GROUP_METADATA_VALUE).array(), + "{\"key\":null,\"value\":{\"version\":4,\"data\":{\"protocolType\":\"consumer\",\"generation\":1," + + "\"protocol\":\"range\",\"leader\":\"leader\",\"currentStateTimestamp\":1234," + + "\"members\":[{\"memberId\":\"member-1\",\"groupInstanceId\":\"group-instance-1\"," + + "\"clientId\":\"client-1\",\"clientHost\":\"host-1\",\"rebalanceTimeout\":1000," + + "\"sessionTimeout\":1500,\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}"), + Arguments.of(null, null, "{\"key\":null,\"value\":null}"), + Arguments.of( + MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(), + MessageUtil.toVersionPrefixedByteBuffer((short) 4, OFFSET_COMMIT_VALUE).array(), + "" + ) + ); + } + + @ParameterizedTest + @MethodSource("parameters") + public void testTransactionLogMessageFormatter(byte[] keyBuffer, byte[] valueBuffer, String expectedOutput) { + ConsumerRecord record = new ConsumerRecord<>( + TOPIC, 0, 0, + 0L, TimestampType.CREATE_TIME, 0, + 0, keyBuffer, valueBuffer, + new RecordHeaders(), Optional.empty()); + + try (MessageFormatter formatter = new GroupMetadataMessageFormatter()) { + formatter.configure(emptyMap()); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + formatter.writeTo(record, new PrintStream(out)); + assertEquals(expectedOutput, out.toString()); + } + } +} From 4afe5f380a2cc0a3cc9c15198fb49f59b1672616 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Fri, 2 Aug 2024 18:52:24 +0200 Subject: [PATCH 019/123] KAFKA-16448: Update documentation (#16776) Updated docs for KIP-1033. Reviewers: Matthias J. Sax --- .../developer-guide/config-streams.html | 93 +++++++++++++++---- docs/streams/upgrade-guide.html | 9 ++ 2 files changed, 84 insertions(+), 18 deletions(-) diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html index f50945778a..68137dc4b4 100644 --- a/docs/streams/developer-guide/config-streams.html +++ b/docs/streams/developer-guide/config-streams.html @@ -82,6 +82,7 @@
  • num.standby.replicas
  • num.stream.threads
  • probing.rebalance.interval.ms
  • +
  • processing.exception.handler
  • processing.guarantee
  • rack.aware.assignment.non_overlap_cost
  • rack.aware.assignment.strategy
  • @@ -395,83 +396,88 @@

    num.standby.replicasThe maximum time in milliseconds to wait before triggering a rebalance to probe for warmup replicas that have sufficiently caught up. 600000 milliseconds (10 minutes) - processing.guarantee + processing.exception.handler + Medium + Exception handling class that implements the ProcessingExceptionHandler interface. + LogAndFailProcessingExceptionHandler + + processing.guarantee Medium The processing mode. Can be either "at_least_once" (default) or "exactly_once_v2" (for EOS version 2, requires broker version 2.5+). Deprecated config options are "exactly_once" (for EOS version 1) and "exactly_once_beta" (for EOS version 2, requires broker version 2.5+). See Processing Guarantee - poll.ms + poll.ms Low The amount of time in milliseconds to block waiting for input. 100 milliseconds - rack.aware.assignment.tags + rack.aware.assignment.tags Medium List of tag keys used to distribute standby replicas across Kafka Streams clients. When configured, Kafka Streams will make a best-effort to distribute the standby tasks over clients with different tag values. the empty list - replication.factor + replication.factor Medium The replication factor for changelog topics and repartition topics created by the application. The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer. -1 - retry.backoff.ms + retry.backoff.ms Medium The amount of time in milliseconds, before a request is retried. This applies if the retries parameter is configured to be greater than 0. 100 - rocksdb.config.setter + rocksdb.config.setter Medium The RocksDB configuration. - state.cleanup.delay.ms + state.cleanup.delay.ms Low The amount of time in milliseconds to wait before deleting state when a partition has migrated. 600000 milliseconds (10 minutes) - state.dir + state.dir High Directory location for state stores. /${java.io.tmpdir}/kafka-streams - task.assignor.class + task.assignor.class Medium A task assignor class or class name implementing the TaskAssignor interface. The high-availability task assignor. - task.timeout.ms + task.timeout.ms Medium The maximum amount of time in milliseconds a task might stall due to internal errors and retries until an error is raised. For a timeout of 0 ms, a task would raise an error for the first internal error. For any timeout larger than 0 ms, a task will retry at least once before an error is raised. 300000 milliseconds (5 minutes) - topology.optimization + topology.optimization Medium A configuration telling Kafka Streams if it should optimize the topology and what optimizations to apply. Acceptable values are: StreamsConfig.NO_OPTIMIZATION (none), StreamsConfig.OPTIMIZE (all) or a comma separated list of specific optimizations: StreamsConfig.REUSE_KTABLE_SOURCE_TOPICS (reuse.ktable.source.topics), StreamsConfig.MERGE_REPARTITION_TOPICS (merge.repartition.topics), - StreamsConfig.SINGLE_STORE_SELF_JOIN (single.store.self.join). + StreamsConfig.SINGLE_STORE_SELF_JOIN (single.store.self.join). NO_OPTIMIZATION - upgrade.from + upgrade.from Medium The version you are upgrading from during a rolling upgrade. See Upgrade From - windowstore.changelog.additional.retention.ms + windowstore.changelog.additional.retention.ms Low Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. 86400000 milliseconds (1 day) - window.size.ms + window.size.ms Low Sets window size for the deserializer in order to calculate window end times. null - log.summary.interval.ms + log.summary.interval.ms Low Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. 120000milliseconds (2 minutes) @@ -523,7 +529,7 @@

    acceptable.recovery.lagacceptable.recovery.lagprobing.rebalance.interval.ms +
    +

    processing.exception.handler

    +
    +

    The processing exception handler allows you to manage exceptions triggered during the processing of a record. The implemented exception + handler needs to return a FAIL or CONTINUE depending on the record and the exception thrown. Returning + FAIL will signal that Streams should shut down and CONTINUE will signal that Streams should ignore the issue + and continue processing. The following library built-in exception handlers are available:

    +
      +
    • LogAndContinueProcessingExceptionHandler: + This handler logs the processing exception and then signals the processing pipeline to continue processing more records. + This log-and-skip strategy allows Kafka Streams to make progress instead of failing if there are records that fail to be processed.
    • +
    • LogAndFailProcessingExceptionHandler. + This handler logs the processing exception and then signals the processing pipeline to stop processing more records.
    • +
    + +

    You can also provide your own customized exception handler besides the library provided ones to meet your needs. For example, you can choose to forward corrupt + records into a quarantine topic (think: a "dead letter queue") for further processing. To do this, use the Producer API to write a corrupted record directly to + the quarantine topic. To be more concrete, you can create a separate KafkaProducer object outside the Streams client, and pass in this object + as well as the dead letter queue topic name into the Properties map, which then can be retrieved from the configure function call. + The drawback of this approach is that "manual" writes are side effects that are invisible to the Kafka Streams runtime library, + so they do not benefit from the end-to-end processing guarantees of the Streams API:

    + +
    public class SendToDeadLetterQueueExceptionHandler implements ProcessingExceptionHandler {
    +    KafkaProducer<byte[], byte[]> dlqProducer;
    +    String dlqTopic;
    +
    +    @Override
    +    public ProcessingHandlerResponse handle(final ErrorHandlerContext context,
    +                                            final Record record,
    +                                            final Exception exception) {
    +
    +        log.warn("Exception caught during message processing, sending to the dead queue topic; " +
    +            "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}",
    +            context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(),
    +            exception);
    +
    +        dlqProducer.send(new ProducerRecord<>(dlqTopic, null, record.timestamp(), (byte[]) record.key(), (byte[]) record.value(), record.headers()));
    +
    +        return ProcessingHandlerResponse.CONTINUE;
    +    }
    +
    +    @Override
    +    public void configure(final Map<String, ?> configs) {
    +        dlqProducer = .. // get a producer from the configs map
    +        dlqTopic = .. // get the topic name from the configs map
    +    }
    +}
    + +
    +

    processing.guarantee

    diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index f24edfbd42..6102219bf1 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -133,6 +133,15 @@

    < More details about the new config StreamsConfig#TOPOLOGY_OPTIMIZATION can be found in KIP-295.

    +

    Streams API changes in 3.9.0

    + +

    + The introduction of KIP-1033 + enables you to provide a processing exception handler to manage exceptions during the processing of a record rather than throwing the exception all the way out of your streams application. + You can provide the configs via the StreamsConfig as StreamsConfig#PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG. + The specified handler must implement the org.apache.kafka.streams.errors.ProcessingExceptionHandler interface. +

    +

    Streams API changes in 3.8.0

    From 6b039ce75b8a21542944ace44f1873435ab9820f Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Sat, 3 Aug 2024 07:07:45 +0800 Subject: [PATCH 020/123] KAFKA-16390: add `group.coordinator.rebalance.protocols=classic,consumer` to broker configs when system tests need the new coordinator (#16715) Fix an issue that cause system test failing when using AsyncKafkaConsumer. A configuration option, group.coordinator.rebalance.protocols, was introduced to specify the rebalance protocols used by the group coordinator. By default, the rebalance protocol is set to classic. When the new group coordinator is enabled, the rebalance protocols are set to classic,consumer. Reviewers: Chia-Ping Tsai , David Jacot , Lianet Magrans , Kirk True , Justine Olshan --- tests/kafkatest/services/kafka/config_property.py | 1 + tests/kafkatest/services/kafka/kafka.py | 12 ++++++++++-- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index 335cb02bcc..bc4708d0d3 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -70,6 +70,7 @@ SASL_ENABLED_MECHANISMS="sasl.enabled.mechanisms" NEW_GROUP_COORDINATOR_ENABLE="group.coordinator.new.enable" +GROUP_COORDINATOR_REBALANCE_PROTOCOLS="group.coordinator.rebalance.protocols" """ From KafkaConfig.scala diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 5f245b1005..bd1c9b1e33 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -409,6 +409,12 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.interbroker_sasl_mechanism = interbroker_sasl_mechanism self._security_config = None + # When the new group coordinator is enabled, the new consumer rebalance + # protocol is enabled too. + rebalance_protocols = "classic" + if self.use_new_coordinator: + rebalance_protocols = "classic,consumer" + for node in self.nodes: node_quorum_info = quorum.NodeQuorumInfo(self.quorum_info, node) @@ -422,7 +428,8 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI kraft_broker_configs = { config_property.PORT: config_property.FIRST_BROKER_PORT, config_property.NODE_ID: self.idx(node), - config_property.NEW_GROUP_COORDINATOR_ENABLE: use_new_coordinator + config_property.NEW_GROUP_COORDINATOR_ENABLE: use_new_coordinator, + config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS: rebalance_protocols } kraft_broker_plus_zk_configs = kraft_broker_configs.copy() kraft_broker_plus_zk_configs.update(zk_broker_configs) @@ -782,7 +789,8 @@ def prop_file(self, node): if self.use_new_coordinator: override_configs[config_property.NEW_GROUP_COORDINATOR_ENABLE] = 'true' - + override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = 'classic,consumer' + for prop in self.server_prop_overrides: override_configs[prop[0]] = prop[1] From 4e75c57bbb79014a0efd3d396630c810506c5921 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Sat, 3 Aug 2024 20:15:51 +0800 Subject: [PATCH 021/123] KAFKA-17245: Revert TopicRecord changes. (#16780) Revert KAFKA-16257 changes because KIP-950 doesn't need it anymore. Reviewers: Luke Chen --- .../kafka/log/remote/RemoteLogManager.java | 3 +-- .../log/remote/RemoteLogManagerTest.java | 12 +++++----- .../kafka/admin/RemoteTopicCrudTest.scala | 2 +- .../log/remote/RemoteIndexCacheTest.scala | 8 +++---- .../common/metadata/TopicRecord.json | 10 ++------- .../metadata/util/RecordRedactorTest.java | 2 +- .../storage/RemoteLogSegmentMetadata.java | 22 ++++--------------- .../storage/RemoteLogSegmentMetadataTest.java | 8 +++---- .../RemoteLogSegmentMetadataTransform.java | 2 +- .../metadata/storage/ConsumerTaskTest.java | 2 +- .../storage/RemoteLogMetadataCacheTest.java | 4 ++-- .../RemoteLogMetadataFormatterTest.java | 2 +- .../storage/RemoteLogMetadataSerdeTest.java | 2 +- .../RemoteLogMetadataTransformTest.java | 2 +- .../RemoteLogSegmentLifecycleTest.java | 6 ++--- ...adataManagerMultipleSubscriptionsTest.java | 4 ++-- ...edRemoteLogMetadataManagerRestartTest.java | 6 ++--- ...opicBasedRemoteLogMetadataManagerTest.java | 20 ++++++++--------- ...RemoteLogSegmentMetadataTransformTest.java | 3 ++- .../storage/LocalTieredStorageTest.java | 4 ++-- .../storage/RemoteLogMetadataManagerTest.java | 4 ++-- 21 files changed, 53 insertions(+), 75 deletions(-) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index f3f0ccc11a..7a087497bb 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -913,7 +913,6 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment logger.info("Copying {} to remote storage.", logFileName); long endOffset = nextSegmentBaseOffset - 1; - int tieredEpoch = 0; File producerStateSnapshotFile = log.producerStateManager().fetchSnapshot(nextSegmentBaseOffset).orElse(null); List epochEntries = getLeaderEpochEntries(log, segment.baseOffset(), nextSegmentBaseOffset); @@ -922,7 +921,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment RemoteLogSegmentMetadata copySegmentStartedRlsm = new RemoteLogSegmentMetadata(segmentId, segment.baseOffset(), endOffset, segment.largestTimestamp(), brokerId, time.milliseconds(), segment.log().sizeInBytes(), - segmentLeaderEpochs, tieredEpoch); + segmentLeaderEpochs); remoteLogMetadataManager.addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 4e5f70f933..ce0939737b 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -1566,7 +1566,7 @@ private static RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(long star 100000L, 1000, Optional.empty(), - RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs, 0); + RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs); } @Test @@ -1906,9 +1906,9 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException int segmentSize = 1024; List segmentMetadataList = Arrays.asList( new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), - 500, 539, timestamp, brokerId, timestamp, segmentSize, truncateAndGetLeaderEpochs(epochEntries, 500L, 539L), 0), + 500, 539, timestamp, brokerId, timestamp, segmentSize, truncateAndGetLeaderEpochs(epochEntries, 500L, 539L)), new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), - 540, 700, timestamp, brokerId, timestamp, segmentSize, truncateAndGetLeaderEpochs(epochEntries, 540L, 700L), 0) + 540, 700, timestamp, brokerId, timestamp, segmentSize, truncateAndGetLeaderEpochs(epochEntries, 540L, 700L)) ); when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition), anyInt())) .thenAnswer(invocation -> { @@ -2231,7 +2231,7 @@ public void testDeletionOnOverlappingRetentionBreachedSegments(long retentionSiz RemoteLogSegmentMetadata metadata2 = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), metadata1.startOffset(), metadata1.endOffset() + 5, metadata1.maxTimestampMs(), metadata1.brokerId() + 1, metadata1.eventTimestampMs(), metadata1.segmentSizeInBytes() + 128, - metadata1.customMetadata(), metadata1.state(), metadata1.segmentLeaderEpochs(), 0); + metadata1.customMetadata(), metadata1.state(), metadata1.segmentLeaderEpochs()); // When there are overlapping/duplicate segments, the RemoteLogMetadataManager#listRemoteLogSegments // returns the segments in order of (valid ++ unreferenced) segments: @@ -2657,8 +2657,8 @@ private List listRemoteLogSegmentMetadataByTime(TopicI segmentSize, Optional.empty(), state, - segmentLeaderEpochs, - 0); + segmentLeaderEpochs + ); segmentMetadataList.add(metadata); } return segmentMetadataList; diff --git a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala index e362832d63..f995b86b70 100644 --- a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala +++ b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala @@ -436,7 +436,7 @@ class MyRemoteLogMetadataManager extends NoOpRemoteLogMetadataManager { val startOffset = idx * recordsPerSegment val endOffset = startOffset + recordsPerSegment - 1 val segmentLeaderEpochs: util.Map[Integer, java.lang.Long] = Collections.singletonMap(0, 0L) - segmentMetadataList.add(new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), startOffset, endOffset, timestamp, 0, timestamp, segmentSize, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentLeaderEpochs, 0)) + segmentMetadataList.add(new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), startOffset, endOffset, timestamp, 0, timestamp, segmentSize, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentLeaderEpochs)) } segmentMetadataList.iterator() } diff --git a/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala b/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala index 95e28282f9..6ebcbeb4fc 100644 --- a/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala +++ b/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala @@ -67,7 +67,7 @@ class RemoteIndexCacheTest { Files.createDirectory(tpDir.toPath) val remoteLogSegmentId = RemoteLogSegmentId.generateNew(idPartition) - rlsMetadata = new RemoteLogSegmentMetadata(remoteLogSegmentId, baseOffset, lastOffset, time.milliseconds(), brokerId, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L), 0) + rlsMetadata = new RemoteLogSegmentMetadata(remoteLogSegmentId, baseOffset, lastOffset, time.milliseconds(), brokerId, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L)) cache = new RemoteIndexCache(defaultRemoteIndexCacheSizeBytes, rsm, tpDir.toString) @@ -678,7 +678,7 @@ class RemoteIndexCacheTest { @Test def testConcurrentRemoveReadForCache(): Unit = { // Create a spy Cache Entry - val rlsMetadata = new RemoteLogSegmentMetadata(RemoteLogSegmentId.generateNew(idPartition), baseOffset, lastOffset, time.milliseconds(), brokerId, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L), 0) + val rlsMetadata = new RemoteLogSegmentMetadata(RemoteLogSegmentId.generateNew(idPartition), baseOffset, lastOffset, time.milliseconds(), brokerId, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L)) val timeIndex = spy(createTimeIndexForSegmentMetadata(rlsMetadata, new File(tpDir, DIR_NAME))) val txIndex = spy(createTxIndexForSegmentMetadata(rlsMetadata, new File(tpDir, DIR_NAME))) @@ -926,7 +926,7 @@ class RemoteIndexCacheTest { private def generateSpyCacheEntry(remoteLogSegmentId: RemoteLogSegmentId = RemoteLogSegmentId.generateNew(idPartition)): RemoteIndexCache.Entry = { - val rlsMetadata = new RemoteLogSegmentMetadata(remoteLogSegmentId, baseOffset, lastOffset, time.milliseconds(), brokerId, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L), 0) + val rlsMetadata = new RemoteLogSegmentMetadata(remoteLogSegmentId, baseOffset, lastOffset, time.milliseconds(), brokerId, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L)) val timeIndex = spy(createTimeIndexForSegmentMetadata(rlsMetadata, tpDir)) val txIndex = spy(createTxIndexForSegmentMetadata(rlsMetadata, tpDir)) val offsetIndex = spy(createOffsetIndexForSegmentMetadata(rlsMetadata, tpDir)) @@ -993,7 +993,7 @@ class RemoteIndexCacheTest { tpId: TopicIdPartition): List[RemoteLogSegmentMetadata] = { val metadataList = mutable.Buffer.empty[RemoteLogSegmentMetadata] for (i <- 0 until size) { - metadataList.append(new RemoteLogSegmentMetadata(new RemoteLogSegmentId(tpId, Uuid.randomUuid()), baseOffset * i, baseOffset * i + 10, time.milliseconds(), brokerId, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L), 0)) + metadataList.append(new RemoteLogSegmentMetadata(new RemoteLogSegmentId(tpId, Uuid.randomUuid()), baseOffset * i, baseOffset * i + 10, time.milliseconds(), brokerId, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L))) } metadataList.toList } diff --git a/metadata/src/main/resources/common/metadata/TopicRecord.json b/metadata/src/main/resources/common/metadata/TopicRecord.json index 5a316e6dbb..6fa5a05096 100644 --- a/metadata/src/main/resources/common/metadata/TopicRecord.json +++ b/metadata/src/main/resources/common/metadata/TopicRecord.json @@ -17,18 +17,12 @@ "apiKey": 2, "type": "metadata", "name": "TopicRecord", - // Version 0 first version of TopicRecord with Name and TopicId - // Version 1 adds TieredEpoch and TieredState for KIP-950 - "validVersions": "0-1", + "validVersions": "0", "flexibleVersions": "0+", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "about": "The topic name." }, { "name": "TopicId", "type": "uuid", "versions": "0+", - "about": "The unique ID of this topic." }, - { "name": "TieredEpoch", "type": "int32", "versions": "1+", - "about": "The epoch denoting how many times the tiered state has changed" }, - { "name": "TieredState", "type": "bool", "versions": "1+", - "about": "Denotes whether the topic is currently tiered or not" } + "about": "The unique ID of this topic." } ] } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/util/RecordRedactorTest.java b/metadata/src/test/java/org/apache/kafka/metadata/util/RecordRedactorTest.java index a5a18e582a..0fbd4ef00a 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/util/RecordRedactorTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/util/RecordRedactorTest.java @@ -50,7 +50,7 @@ public final class RecordRedactorTest { @Test public void testTopicRecordToString() { - assertEquals("TopicRecord(name='foo', topicId=UOovKkohSU6AGdYW33ZUNg, tieredEpoch=0, tieredState=false)", + assertEquals("TopicRecord(name='foo', topicId=UOovKkohSU6AGdYW33ZUNg)", REDACTOR.toLoggableString(new TopicRecord(). setTopicId(Uuid.fromString("UOovKkohSU6AGdYW33ZUNg")). setName("foo"))); diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java index 4389a90ab8..9b589322bb 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java @@ -78,11 +78,6 @@ public class RemoteLogSegmentMetadata extends RemoteLogMetadata { */ private final RemoteLogSegmentState state; - /** - * The epoch denoting how many times the tiered state has changed - */ - private final int tieredEpoch; - /** * Creates an instance with the given metadata of remote log segment. *

    @@ -99,7 +94,6 @@ public class RemoteLogSegmentMetadata extends RemoteLogMetadata { * @param customMetadata Custom metadata. * @param state State of the respective segment of remoteLogSegmentId. * @param segmentLeaderEpochs leader epochs occurred within this segment. - * @param tieredEpoch The epoch denoting how many times the tiered state has changed */ public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId, long startOffset, @@ -110,11 +104,10 @@ public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId, int segmentSizeInBytes, Optional customMetadata, RemoteLogSegmentState state, - Map segmentLeaderEpochs, int tieredEpoch) { + Map segmentLeaderEpochs) { super(brokerId, eventTimestampMs); this.remoteLogSegmentId = Objects.requireNonNull(remoteLogSegmentId, "remoteLogSegmentId can not be null"); this.state = Objects.requireNonNull(state, "state can not be null"); - this.tieredEpoch = tieredEpoch; if (startOffset < 0) { throw new IllegalArgumentException("Unexpected start offset = " + startOffset + ". StartOffset for a remote segment cannot be negative"); @@ -151,7 +144,6 @@ public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId, * @param eventTimestampMs Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage. * @param segmentSizeInBytes Size of this segment in bytes. * @param segmentLeaderEpochs leader epochs occurred within this segment - * @param tieredEpoch */ public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId, long startOffset, @@ -160,8 +152,7 @@ public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId, int brokerId, long eventTimestampMs, int segmentSizeInBytes, - Map segmentLeaderEpochs, - int tieredEpoch) { + Map segmentLeaderEpochs) { this(remoteLogSegmentId, startOffset, endOffset, @@ -170,8 +161,7 @@ public RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId, eventTimestampMs, segmentSizeInBytes, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_STARTED, - segmentLeaderEpochs, - tieredEpoch); + segmentLeaderEpochs); } @@ -237,10 +227,6 @@ public RemoteLogSegmentState state() { return state; } - public int tieredEpoch() { - return tieredEpoch; - } - /** * Creates a new RemoteLogSegmentMetadata applying the given {@code rlsmUpdate} on this instance. This method will * not update this instance. @@ -255,7 +241,7 @@ public RemoteLogSegmentMetadata createWithUpdates(RemoteLogSegmentMetadataUpdate return new RemoteLogSegmentMetadata(remoteLogSegmentId, startOffset, endOffset, maxTimestampMs, rlsmUpdate.brokerId(), rlsmUpdate.eventTimestampMs(), - segmentSizeInBytes, rlsmUpdate.customMetadata(), rlsmUpdate.state(), segmentLeaderEpochs, tieredEpoch); + segmentSizeInBytes, rlsmUpdate.customMetadata(), rlsmUpdate.state(), segmentLeaderEpochs); } @Override diff --git a/storage/api/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataTest.java b/storage/api/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataTest.java index 4e11f57e39..0aaaa4d99e 100644 --- a/storage/api/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataTest.java +++ b/storage/api/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataTest.java @@ -43,14 +43,13 @@ void createWithUpdates() { long endOffset = 100L; int segmentSize = 123; long maxTimestamp = -1L; - int tieredEpoch = 0; Map segmentLeaderEpochs = new HashMap<>(); segmentLeaderEpochs.put(0, 0L); RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, startOffset, endOffset, maxTimestamp, brokerId, eventTimestamp, segmentSize, - segmentLeaderEpochs, tieredEpoch); + segmentLeaderEpochs); CustomMetadata customMetadata = new CustomMetadata(new byte[]{0, 1, 2, 3}); RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate( @@ -62,8 +61,8 @@ void createWithUpdates() { segmentId, startOffset, endOffset, maxTimestamp, brokerIdFinished, timestampFinished, segmentSize, Optional.of(customMetadata), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, - segmentLeaderEpochs, - tieredEpoch); + segmentLeaderEpochs + ); assertEquals(expectedUpdatedMetadata, updatedMetadata); // Check that the original metadata have not changed. @@ -75,6 +74,5 @@ void createWithUpdates() { assertEquals(eventTimestamp, segmentMetadata.eventTimestampMs()); assertEquals(segmentSize, segmentMetadata.segmentSizeInBytes()); assertEquals(segmentLeaderEpochs, segmentMetadata.segmentLeaderEpochs()); - assertEquals(tieredEpoch, segmentMetadata.tieredEpoch()); } } diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java index f97ca86b9f..9e893d2cbc 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java @@ -83,7 +83,7 @@ public RemoteLogSegmentMetadata fromApiMessageAndVersion(ApiMessageAndVersion ap new RemoteLogSegmentMetadata(remoteLogSegmentId, record.startOffset(), record.endOffset(), record.maxTimestampMs(), record.brokerId(), record.eventTimestampMs(), record.segmentSizeInBytes(), - segmentLeaderEpochs, 0); + segmentLeaderEpochs); RemoteLogSegmentMetadataUpdate rlsmUpdate = new RemoteLogSegmentMetadataUpdate(remoteLogSegmentId, record.eventTimestampMs(), customMetadata, diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java index c92efce68a..66176c6847 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTaskTest.java @@ -425,7 +425,7 @@ private void addRecord(final MockConsumer consumer, final TopicIdPartition idPartition, final long recordOffset) { final RemoteLogSegmentId segmentId = new RemoteLogSegmentId(idPartition, Uuid.randomUuid()); - final RemoteLogMetadata metadata = new RemoteLogSegmentMetadata(segmentId, 0L, 1L, 0L, 0, 0L, 1, Collections.singletonMap(0, 0L), 0); + final RemoteLogMetadata metadata = new RemoteLogSegmentMetadata(segmentId, 0L, 1L, 0L, 0, 0L, 1, Collections.singletonMap(0, 0L)); final ConsumerRecord record = new ConsumerRecord<>(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME, metadataPartition, recordOffset, null, serde.serialize(metadata)); consumer.addRecord(record); } diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java index 20918e0ca0..6b93f61dc7 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java @@ -60,7 +60,7 @@ public void testCacheAddMetadataOnInvalidArgs() { if (state != RemoteLogSegmentState.COPY_SEGMENT_STARTED) { RemoteLogSegmentId segmentId = new RemoteLogSegmentId(tpId0, Uuid.randomUuid()); RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 0, 100L, - -1L, brokerId0, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L), 0); + -1L, brokerId0, time.milliseconds(), segmentSize, Collections.singletonMap(0, 0L)); RemoteLogSegmentMetadata updatedMetadata = segmentMetadata.createWithUpdates( new RemoteLogSegmentMetadataUpdate(segmentId, time.milliseconds(), Optional.empty(), state, brokerId1)); @@ -102,7 +102,7 @@ public void testDropEventOnInvalidStateTransition() throws RemoteResourceNotFoun long offset = 10L; RemoteLogSegmentId segmentId = new RemoteLogSegmentId(tpId0, Uuid.randomUuid()); RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, offset, 100L, - -1L, brokerId0, time.milliseconds(), segmentSize, Collections.singletonMap(leaderEpoch, offset), 0); + -1L, brokerId0, time.milliseconds(), segmentSize, Collections.singletonMap(leaderEpoch, offset)); cache.addCopyInProgressSegment(segmentMetadata); // invalid-transition-1. COPY_SEGMENT_STARTED -> DELETE_SEGMENT_FINISHED diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataFormatterTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataFormatterTest.java index a4ac17cef2..d6a03441e8 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataFormatterTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataFormatterTest.java @@ -53,7 +53,7 @@ public void testFormat() throws IOException { Optional customMetadata = Optional.of(new CustomMetadata(new byte[10])); RemoteLogSegmentMetadata remoteLogMetadata = new RemoteLogSegmentMetadata( remoteLogSegmentId, 0L, 100L, -1L, 1, 123L, 1024, customMetadata, COPY_SEGMENT_STARTED, - segLeaderEpochs, 0); + segLeaderEpochs); byte[] metadataBytes = new RemoteLogMetadataSerde().serialize(remoteLogMetadata); ConsumerRecord metadataRecord = new ConsumerRecord<>( diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSerdeTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSerdeTest.java index 6e2b8960da..ba2c3d1f26 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSerdeTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSerdeTest.java @@ -72,7 +72,7 @@ private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() { RemoteLogSegmentId remoteLogSegmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); return new RemoteLogSegmentMetadata(remoteLogSegmentId, 0L, 100L, -1L, 1, time.milliseconds(), 1024, Optional.of(new CustomMetadata(new byte[] {0, 1, 2, 3})), - COPY_SEGMENT_STARTED, segLeaderEpochs, 0); + COPY_SEGMENT_STARTED, segLeaderEpochs); } private RemoteLogSegmentMetadataUpdate createRemoteLogSegmentMetadataUpdate() { diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java index 41d8289703..e7131a8130 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java @@ -68,7 +68,7 @@ public void testRemoteLogSegmentMetadataUpdateTransform() { private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() { RemoteLogSegmentId remoteLogSegmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); return new RemoteLogSegmentMetadata(remoteLogSegmentId, 0L, 100L, -1L, 1, - time.milliseconds(), 1024, Collections.singletonMap(0, 0L), 0); + time.milliseconds(), 1024, Collections.singletonMap(0, 0L)); } @Test diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java index 0a36ef1b55..0d91f69d4b 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java @@ -94,7 +94,7 @@ public void testRemoteLogSegmentLifeCycle() throws Exception { leaderEpochSegment0.put(2, 80L); RemoteLogSegmentId segmentId0 = new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()); RemoteLogSegmentMetadata metadataSegment0 = new RemoteLogSegmentMetadata(segmentId0, 0L, - 100L, -1L, brokerId0, time.milliseconds(), segSize, leaderEpochSegment0, 0); + 100L, -1L, brokerId0, time.milliseconds(), segSize, leaderEpochSegment0); metadataManager.addRemoteLogSegmentMetadata(metadataSegment0).get(); verify(spyRemotePartitionMetadataStore).handleRemoteLogSegmentMetadata(metadataSegment0); @@ -223,7 +223,7 @@ private RemoteLogSegmentMetadata upsertSegmentState(RemoteLogMetadataManager met throws RemoteStorageException, ExecutionException, InterruptedException { RemoteLogSegmentId segmentId = new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()); RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, startOffset, endOffset, - -1L, brokerId0, time.milliseconds(), segSize, segmentLeaderEpochs, 0); + -1L, brokerId0, time.milliseconds(), segSize, segmentLeaderEpochs); metadataManager.addRemoteLogSegmentMetadata(segmentMetadata).get(); verify(spyRemotePartitionMetadataStore).handleRemoteLogSegmentMetadata(segmentMetadata); @@ -258,7 +258,7 @@ public void testCacheSegmentWithCopySegmentStartedState() throws Exception { // segments. RemoteLogSegmentId segmentId = new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()); RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 0L, 50L, - -1L, brokerId0, time.milliseconds(), segSize, Collections.singletonMap(0, 0L), 0); + -1L, brokerId0, time.milliseconds(), segSize, Collections.singletonMap(0, 0L)); metadataManager.addRemoteLogSegmentMetadata(segmentMetadata).get(); verify(spyRemotePartitionMetadataStore).handleRemoteLogSegmentMetadata(segmentMetadata); diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java index 55541d18a4..f64996ae46 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java @@ -127,7 +127,7 @@ public int metadataPartition(TopicIdPartition topicIdPartition) { int segSize = 1048576; RemoteLogSegmentMetadata leaderSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), 0, 100, -1L, 0, - time.milliseconds(), segSize, Collections.singletonMap(0, 0L), 0); + time.milliseconds(), segSize, Collections.singletonMap(0, 0L)); ExecutionException exception = assertThrows(ExecutionException.class, () -> remoteLogMetadataManager.addRemoteLogSegmentMetadata(leaderSegmentMetadata).get()); assertEquals("org.apache.kafka.common.KafkaException: This consumer is not assigned to the target partition 0. Currently assigned partitions: []", @@ -135,7 +135,7 @@ public int metadataPartition(TopicIdPartition topicIdPartition) { RemoteLogSegmentMetadata followerSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(followerTopicIdPartition, Uuid.randomUuid()), 0, 100, -1L, 0, - time.milliseconds(), segSize, Collections.singletonMap(0, 0L), 0); + time.milliseconds(), segSize, Collections.singletonMap(0, 0L)); exception = assertThrows(ExecutionException.class, () -> remoteLogMetadataManager.addRemoteLogSegmentMetadata(followerSegmentMetadata).get()); assertEquals("org.apache.kafka.common.KafkaException: This consumer is not assigned to the target partition 0. Currently assigned partitions: []", exception.getMessage()); diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java index 4f95fac24d..783aa022df 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java @@ -79,11 +79,11 @@ public void testRLMMAPIsAfterRestart() throws Exception { RemoteLogSegmentMetadata leaderSegmentMetadata = new RemoteLogSegmentMetadata( new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), 0, 100, -1L, 0, - time.milliseconds(), segSize, Collections.singletonMap(0, 0L), 0); + time.milliseconds(), segSize, Collections.singletonMap(0, 0L)); RemoteLogSegmentMetadata followerSegmentMetadata = new RemoteLogSegmentMetadata( new RemoteLogSegmentId(followerTopicIdPartition, Uuid.randomUuid()), 0, 100, -1L, 0, - time.milliseconds(), segSize, Collections.singletonMap(0, 0L), 0); + time.milliseconds(), segSize, Collections.singletonMap(0, 0L)); try (TopicBasedRemoteLogMetadataManager topicBasedRemoteLogMetadataManager = createTopicBasedRemoteLogMetadataManager()) { // Register these partitions to RemoteLogMetadataManager. @@ -113,7 +113,7 @@ public void testRLMMAPIsAfterRestart() throws Exception { RemoteLogSegmentMetadata leaderSegmentMetadata2 = new RemoteLogSegmentMetadata( new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), 101, 200, -1L, 0, - time.milliseconds(), segSize, Collections.singletonMap(0, 101L), 0); + time.milliseconds(), segSize, Collections.singletonMap(0, 101L)); topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(leaderSegmentMetadata2).get(); // Check that both the stored segment and recently added segment are available. diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java index b0a65bd9c0..d784218759 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java @@ -141,12 +141,12 @@ public void testNewPartitionUpdates() throws Exception { // has not yet been subscribing as they are not yet registered. RemoteLogSegmentMetadata leaderSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(newLeaderTopicIdPartition, Uuid.randomUuid()), 0, 100, -1L, 0, - time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L), 0); + time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); assertThrows(Exception.class, () -> topicBasedRlmm().addRemoteLogSegmentMetadata(leaderSegmentMetadata).get()); RemoteLogSegmentMetadata followerSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(newFollowerTopicIdPartition, Uuid.randomUuid()), 0, 100, -1L, 0, - time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L), 0); + time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); assertThrows(Exception.class, () -> topicBasedRlmm().addRemoteLogSegmentMetadata(followerSegmentMetadata).get()); // `listRemoteLogSegments` will receive an exception as these topic partitions are not yet registered. @@ -195,11 +195,11 @@ public void testRemoteLogSizeCalculationWithSegmentsOfTheSameEpoch() throws Remo }).when(spyRemotePartitionMetadataEventHandler).handleRemoteLogSegmentMetadata(any()); RemoteLogSegmentMetadata firstSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), - 0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L), 0); + 0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); RemoteLogSegmentMetadata secondSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), - 100, 200, -1L, 0, time.milliseconds(), SEG_SIZE * 2, Collections.singletonMap(0, 0L), 0); + 100, 200, -1L, 0, time.milliseconds(), SEG_SIZE * 2, Collections.singletonMap(0, 0L)); RemoteLogSegmentMetadata thirdSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), - 200, 300, -1L, 0, time.milliseconds(), SEG_SIZE * 3, Collections.singletonMap(0, 0L), 0); + 200, 300, -1L, 0, time.milliseconds(), SEG_SIZE * 3, Collections.singletonMap(0, 0L)); topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(firstSegmentMetadata); topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(secondSegmentMetadata); @@ -240,11 +240,11 @@ public void testRemoteLogSizeCalculationWithSegmentsOfDifferentEpochs() throws R }).when(spyRemotePartitionMetadataEventHandler).handleRemoteLogSegmentMetadata(any()); RemoteLogSegmentMetadata firstSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), - 0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L), 0); + 0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); RemoteLogSegmentMetadata secondSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), - 100, 200, -1L, 0, time.milliseconds(), SEG_SIZE * 2, Collections.singletonMap(1, 100L), 0); + 100, 200, -1L, 0, time.milliseconds(), SEG_SIZE * 2, Collections.singletonMap(1, 100L)); RemoteLogSegmentMetadata thirdSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), - 200, 300, -1L, 0, time.milliseconds(), SEG_SIZE * 3, Collections.singletonMap(2, 200L), 0); + 200, 300, -1L, 0, time.milliseconds(), SEG_SIZE * 3, Collections.singletonMap(2, 200L)); topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(firstSegmentMetadata); topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(secondSegmentMetadata); @@ -285,9 +285,9 @@ public void testRemoteLogSizeCalculationWithSegmentsHavingNonExistentEpochs() th }).when(spyRemotePartitionMetadataEventHandler).handleRemoteLogSegmentMetadata(any()); RemoteLogSegmentMetadata firstSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), - 0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L), 0); + 0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); RemoteLogSegmentMetadata secondSegmentMetadata = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, Uuid.randomUuid()), - 100, 200, -1L, 0, time.milliseconds(), SEG_SIZE * 2, Collections.singletonMap(1, 100L), 0); + 100, 200, -1L, 0, time.milliseconds(), SEG_SIZE * 2, Collections.singletonMap(1, 100L)); topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(firstSegmentMetadata); topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(secondSegmentMetadata); diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransformTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransformTest.java index bf370d55c1..0745bbaeb8 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransformTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransformTest.java @@ -45,7 +45,8 @@ void testToAndFromMessage(Optional customMetadata) { 0L, 100L, -1L, 0, 0, 1234, customMetadata, RemoteLogSegmentState.COPY_SEGMENT_FINISHED, - segmentLeaderEpochs, 0); + segmentLeaderEpochs + ); RemoteLogSegmentMetadataTransform transform = new RemoteLogSegmentMetadataTransform(); ApiMessageAndVersion message = transform.toApiMessageAndVersion(metadata); diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTest.java index d3391a4538..983fb0a46f 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTest.java @@ -358,7 +358,7 @@ public void assertStartAndEndPositionConsistency() { private RemoteLogSegmentMetadata newRemoteLogSegmentMetadata(final RemoteLogSegmentId id) { return new RemoteLogSegmentMetadata(id, 0, 0, -1L, -1, 1000L, - 1024, Collections.singletonMap(0, 0L), 0); + 1024, Collections.singletonMap(0, 0L)); } private RemoteLogSegmentId newRemoteLogSegmentId() { @@ -539,7 +539,7 @@ private void verifyFileContents(final Function segmentLeaderEpochs = Collections.singletonMap(0, 101L); RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata( - segmentId, 101L, 200L, -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs, 0); + segmentId, 101L, 200L, -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs); // Wait until the segment is added successfully. assertDoesNotThrow(() -> remoteLogMetadataManager.addRemoteLogSegmentMetadata(segmentMetadata).get()); @@ -115,7 +115,7 @@ public void testRemotePartitionDeletion() throws Exception { segmentLeaderEpochs.put(3, 80L); RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata( - segmentId, 0L, 100L, -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs, 0); + segmentId, 0L, 100L, -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs); // Wait until the segment is added successfully. assertDoesNotThrow(() -> remoteLogMetadataManager.addRemoteLogSegmentMetadata(segmentMetadata).get()); From 66485b04c69c12c8c6152053554d078840f34809 Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Sat, 3 Aug 2024 14:27:27 +0800 Subject: [PATCH 022/123] KAFKA-16480: ListOffsets change should have an associated API/IBP version update (#16781) 1. Use oldestAllowedVersion as 9 if using ListOffsetsRequest#EARLIEST_LOCAL_TIMESTAMP or ListOffsetsRequest#LATEST_TIERED_TIMESTAMP. 2. Add test cases to ListOffsetsRequestTest#testListOffsetsRequestOldestVersion to make sure requireTieredStorageTimestamp return 9 as minVersion. 3. Add EarliestLocalSpec and LatestTierSpec to OffsetSpec. 4. Add more cases to KafkaAdminClient#getOffsetFromSpec. 5. Add testListOffsetsEarliestLocalSpecMinVersion and testListOffsetsLatestTierSpecSpecMinVersion to KafkaAdminClientTest to make sure request builder has oldestAllowedVersion as 9. Signed-off-by: PoAn Yang Reviewers: Luke Chen --- .../kafka/clients/admin/KafkaAdminClient.java | 4 ++ .../kafka/clients/admin/OffsetSpec.java | 19 +++++++ .../admin/internals/ListOffsetsHandler.java | 6 +- .../consumer/internals/OffsetFetcher.java | 2 +- .../internals/OffsetsRequestManager.java | 2 +- .../common/requests/ListOffsetsRequest.java | 9 ++- .../clients/admin/KafkaAdminClientTest.java | 56 +++++++++++++++++++ .../requests/ListOffsetsRequestTest.java | 4 +- .../common/requests/RequestResponseTest.java | 6 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- .../unit/kafka/server/KafkaApisTest.scala | 4 +- .../kafka/server/ListOffsetsRequestTest.scala | 24 ++++++-- .../unit/kafka/server/LogOffsetTest.scala | 2 +- .../unit/kafka/server/RequestQuotaTest.scala | 2 +- .../common/ListOffsetRequestBenchmark.java | 2 +- 15 files changed, 122 insertions(+), 22 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 640a08a378..8eb7fb4e8c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -4860,6 +4860,10 @@ private static long getOffsetFromSpec(OffsetSpec offsetSpec) { return ListOffsetsRequest.EARLIEST_TIMESTAMP; } else if (offsetSpec instanceof OffsetSpec.MaxTimestampSpec) { return ListOffsetsRequest.MAX_TIMESTAMP; + } else if (offsetSpec instanceof OffsetSpec.EarliestLocalSpec) { + return ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP; + } else if (offsetSpec instanceof OffsetSpec.LatestTierSpec) { + return ListOffsetsRequest.LATEST_TIERED_TIMESTAMP; } return ListOffsetsRequest.LATEST_TIMESTAMP; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java b/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java index dcf90452c5..5b2fbb3e2e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java @@ -26,6 +26,8 @@ public class OffsetSpec { public static class EarliestSpec extends OffsetSpec { } public static class LatestSpec extends OffsetSpec { } public static class MaxTimestampSpec extends OffsetSpec { } + public static class EarliestLocalSpec extends OffsetSpec { } + public static class LatestTierSpec extends OffsetSpec { } public static class TimestampSpec extends OffsetSpec { private final long timestamp; @@ -70,4 +72,21 @@ public static OffsetSpec maxTimestamp() { return new MaxTimestampSpec(); } + /** + * Used to retrieve the offset with the local log start offset, + * log start offset is the offset of a log above which reads are guaranteed to be served + * from the disk of the leader broker, when Tiered Storage is not enabled, it behaves the same + * as the earliest timestamp + */ + public static OffsetSpec earliestLocalSpec() { + return new EarliestLocalSpec(); + } + + /** + * Used to retrieve the offset with the highest offset of data stored in remote storage, + * and when Tiered Storage is not enabled, we won't return any offset (i.e. Unknown offset) + */ + public static OffsetSpec latestTierSpec() { + return new LatestTierSpec(); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java index 0bb42ed769..7dfcb22afb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java @@ -93,8 +93,12 @@ ListOffsetsRequest.Builder buildBatchedRequest(int brokerId, Set .stream() .anyMatch(key -> offsetTimestampsByPartition.get(key) == ListOffsetsRequest.MAX_TIMESTAMP); + boolean requireTieredStorageTimestamp = keys + .stream() + .anyMatch(key -> offsetTimestampsByPartition.get(key) == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP || offsetTimestampsByPartition.get(key) == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP); + return ListOffsetsRequest.Builder - .forConsumer(true, options.isolationLevel(), supportsMaxTimestamp) + .forConsumer(true, options.isolationLevel(), supportsMaxTimestamp, requireTieredStorageTimestamp) .setTargetTimes(new ArrayList<>(topicsByName.values())); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java index f624941c52..ec0bfe2fc1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java @@ -391,7 +391,7 @@ private RequestFuture sendListOffsetRequest(final Node node, final Map timestampsToSearch, boolean requireTimestamp) { ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder - .forConsumer(requireTimestamp, isolationLevel, false) + .forConsumer(requireTimestamp, isolationLevel, false, false) .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(timestampsToSearch)); log.debug("Sending ListOffsetRequest {} to broker {}", builder, node); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index b4f6371412..71da2f5bf6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -337,7 +337,7 @@ private CompletableFuture buildListOffsetRequestToNode( boolean requireTimestamps, List unsentRequests) { ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder - .forConsumer(requireTimestamps, isolationLevel, false) + .forConsumer(requireTimestamps, isolationLevel, false, false) .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(targetTimes)); log.debug("Creating ListOffset request {} for broker {} to reset positions", builder, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java index 56dea5262f..8ebf0886be 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java @@ -62,9 +62,14 @@ public static Builder forReplica(short allowedVersion, int replicaId) { return new Builder((short) 0, allowedVersion, replicaId, IsolationLevel.READ_UNCOMMITTED); } - public static Builder forConsumer(boolean requireTimestamp, IsolationLevel isolationLevel, boolean requireMaxTimestamp) { + public static Builder forConsumer(boolean requireTimestamp, + IsolationLevel isolationLevel, + boolean requireMaxTimestamp, + boolean requireTieredStorageTimestamp) { short minVersion = 0; - if (requireMaxTimestamp) + if (requireTieredStorageTimestamp) + minVersion = 9; + else if (requireMaxTimestamp) minVersion = 7; else if (isolationLevel == IsolationLevel.READ_COMMITTED) minVersion = 2; diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 34c2722d46..8d70e60fc0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -5844,6 +5844,62 @@ public void testListOffsetsNonMaxTimestampDowngradedImmediately() throws Excepti } } + @Test + public void testListOffsetsEarliestLocalSpecMinVersion() throws Exception { + Node node = new Node(0, "localhost", 8120); + List nodes = Collections.singletonList(node); + List pInfos = new ArrayList<>(); + pInfos.add(new PartitionInfo("foo", 0, node, new Node[]{node}, new Node[]{node})); + final Cluster cluster = new Cluster( + "mockClusterId", + nodes, + pInfos, + Collections.emptySet(), + Collections.emptySet(), + node); + final TopicPartition tp0 = new TopicPartition("foo", 0); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster, + AdminClientConfig.RETRIES_CONFIG, "2")) { + + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.adminClient().listOffsets(Collections.singletonMap(tp0, OffsetSpec.earliestLocalSpec())); + + TestUtils.waitForCondition(() -> env.kafkaClient().requests().stream().anyMatch(request -> + request.requestBuilder().apiKey().messageType == ApiMessageType.LIST_OFFSETS && request.requestBuilder().oldestAllowedVersion() == 9 + ), "no listOffsets request has the expected oldestAllowedVersion"); + } + } + + @Test + public void testListOffsetsLatestTierSpecSpecMinVersion() throws Exception { + Node node = new Node(0, "localhost", 8120); + List nodes = Collections.singletonList(node); + List pInfos = new ArrayList<>(); + pInfos.add(new PartitionInfo("foo", 0, node, new Node[]{node}, new Node[]{node})); + final Cluster cluster = new Cluster( + "mockClusterId", + nodes, + pInfos, + Collections.emptySet(), + Collections.emptySet(), + node); + final TopicPartition tp0 = new TopicPartition("foo", 0); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster, + AdminClientConfig.RETRIES_CONFIG, "2")) { + + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.adminClient().listOffsets(Collections.singletonMap(tp0, OffsetSpec.latestTierSpec())); + + TestUtils.waitForCondition(() -> env.kafkaClient().requests().stream().anyMatch(request -> + request.requestBuilder().apiKey().messageType == ApiMessageType.LIST_OFFSETS && request.requestBuilder().oldestAllowedVersion() == 9 + ), "no listOffsets request has the expected oldestAllowedVersion"); + } + } + private Map makeTestFeatureUpdates() { return Utils.mkMap( Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, FeatureUpdate.UpgradeType.UPGRADE)), diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java index d748203cce..e9a2bec2ff 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java @@ -68,7 +68,7 @@ public void testGetErrorResponse() { new ListOffsetsPartition() .setPartitionIndex(0)))); ListOffsetsRequest request = ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_COMMITTED, false) + .forConsumer(true, IsolationLevel.READ_COMMITTED, false, false) .setTargetTimes(topics) .build(version); ListOffsetsResponse response = (ListOffsetsResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception()); @@ -101,7 +101,7 @@ public void testGetErrorResponseV0() { new ListOffsetsPartition() .setPartitionIndex(0)))); ListOffsetsRequest request = ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_UNCOMMITTED, false) + .forConsumer(true, IsolationLevel.READ_UNCOMMITTED, false, false) .setTargetTimes(topics) .build((short) 0); ListOffsetsResponse response = (ListOffsetsResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception()); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index ac044a5a53..54e681b468 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -2347,7 +2347,7 @@ private ListOffsetsRequest createListOffsetRequest(short version) { .setMaxNumOffsets(10) .setCurrentLeaderEpoch(5))); return ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) .setTargetTimes(singletonList(topic)) .build(version); } else if (version == 1) { @@ -2358,7 +2358,7 @@ private ListOffsetsRequest createListOffsetRequest(short version) { .setTimestamp(1000000L) .setCurrentLeaderEpoch(5))); return ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_UNCOMMITTED, false) + .forConsumer(true, IsolationLevel.READ_UNCOMMITTED, false, false) .setTargetTimes(singletonList(topic)) .build(version); } else if (version >= 2 && version <= LIST_OFFSETS.latestVersion()) { @@ -2371,7 +2371,7 @@ private ListOffsetsRequest createListOffsetRequest(short version) { .setName("test") .setPartitions(singletonList(partition)); return ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_COMMITTED, false) + .forConsumer(true, IsolationLevel.READ_COMMITTED, false, false) .setTargetTimes(singletonList(topic)) .build(version); } else { diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index c00abf9427..c58caaae8b 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -287,7 +287,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } private def createListOffsetsRequest = { - requests.ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false).setTargetTimes( + requests.ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false).setTargetTimes( List(new ListOffsetsTopic() .setName(tp.topic) .setPartitions(List(new ListOffsetsPartition() diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index e66e345a45..91b45dcc03 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -4000,7 +4000,7 @@ class KafkaApisTest extends Logging { .setPartitionIndex(tp.partition) .setTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP) .setCurrentLeaderEpoch(currentLeaderEpoch.get)).asJava)).asJava - val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel, false) + val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel, false, false) .setTargetTimes(targetTimes).build() val request = buildRequest(listOffsetRequest) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), @@ -6151,7 +6151,7 @@ class KafkaApisTest extends Logging { .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(tp.partition) .setTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)).asJava)).asJava - val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel, false) + val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel, false, false) .setTargetTimes(targetTimes).build() val request = buildRequest(listOffsetRequest) kafkaApis = createKafkaApis() diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala index 60b8789e42..f48e3546d8 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala @@ -54,7 +54,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { .setCurrentLeaderEpoch(0)).asJava)).asJava val consumerRequest = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) .setTargetTimes(targetTimes) .build() @@ -94,15 +94,27 @@ class ListOffsetsRequestTest extends BaseRequestTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) - def testListOffsetsMaxTimeStampOldestVersion(quorum: String): Unit = { + def testListOffsetsRequestOldestVersion(): Unit = { val consumerRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + + val requireTimestampRequestBuilder = ListOffsetsRequest.Builder + .forConsumer(true, IsolationLevel.READ_UNCOMMITTED, false, false) + + val requestCommittedRequestBuilder = ListOffsetsRequest.Builder + .forConsumer(false, IsolationLevel.READ_COMMITTED, false, false) val maxTimestampRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, true) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, true, false) + + val requireTieredStorageTimestampRequestBuilder = ListOffsetsRequest.Builder + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, true) assertEquals(0.toShort, consumerRequestBuilder.oldestAllowedVersion()) + assertEquals(1.toShort, requireTimestampRequestBuilder.oldestAllowedVersion()) + assertEquals(2.toShort, requestCommittedRequestBuilder.oldestAllowedVersion()) assertEquals(7.toShort, maxTimestampRequestBuilder.oldestAllowedVersion()) + assertEquals(9.toShort, requireTieredStorageTimestampRequestBuilder.oldestAllowedVersion()) } def assertResponseErrorForEpoch(error: Errors, brokerId: Int, currentLeaderEpoch: Optional[Integer]): Unit = { @@ -115,7 +127,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { .setName(topic) .setPartitions(List(listOffsetPartition).asJava)).asJava val request = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) .setTargetTimes(targetTimes) .build() assertResponseError(error, brokerId, request) @@ -159,7 +171,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { .setTimestamp(timestamp)).asJava)).asJava val builder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) .setTargetTimes(targetTimes) val request = if (version == -1) builder.build() else builder.build(version) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index dffecb4d9c..61fcd6fcf2 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -60,7 +60,7 @@ class LogOffsetTest extends BaseRequestTest { @ValueSource(strings = Array("zk", "kraft")) def testGetOffsetsForUnknownTopic(quorum: String): Unit = { val topicPartition = new TopicPartition("foo", 0) - val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false) + val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.LATEST_TIMESTAMP, 10).asJava).build(0) val response = sendListOffsetsRequest(request) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, findPartition(response.topics.asScala, topicPartition).errorCode) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 33c7df2fc5..fbffa4ce37 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -288,7 +288,7 @@ class RequestQuotaTest extends BaseRequestTest { .setPartitionIndex(tp.partition) .setTimestamp(0L) .setCurrentLeaderEpoch(15)).asJava) - ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false) + ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) .setTargetTimes(List(topic).asJava) case ApiKeys.LEADER_AND_ISR => diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java index fe791350a5..786648f904 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java @@ -71,7 +71,7 @@ public void setup() { } } - this.offsetRequest = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false) + this.offsetRequest = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) .build(ApiKeys.LIST_OFFSETS.latestVersion()); } From 38db4c46ffd3a600ebfdeea07c0ba24a67d3405e Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Wed, 31 Jul 2024 01:07:09 +0900 Subject: [PATCH 023/123] KAFKA-17205: Allow topic config validation in controller level in KRaft mode (#16693) Reviewers: Kamal Chandraprakash , Christo Lolov --- .../ControllerConfigurationValidator.scala | 9 ++-- .../main/scala/kafka/zk/AdminZkClient.scala | 4 +- .../kafka/admin/RemoteTopicCrudTest.scala | 20 ++++++++ .../kafka/api/BaseAdminIntegrationTest.scala | 2 +- .../scala/unit/kafka/log/LogConfigTest.scala | 43 +++++++++++----- ...ControllerConfigurationValidatorTest.scala | 49 +++++++++++++------ .../ConfigurationControlManager.java | 10 ++-- .../controller/ConfigurationValidator.java | 9 ++-- .../storage/internals/log/LogConfig.java | 38 +++++++++----- 9 files changed, 132 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala index b99065b573..06a60e3007 100644 --- a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala +++ b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala @@ -89,14 +89,15 @@ class ControllerConfigurationValidator(kafkaConfig: KafkaConfig) extends Configu override def validate( resource: ConfigResource, - config: util.Map[String, String] + newConfigs: util.Map[String, String], + oldConfigs: util.Map[String, String] ): Unit = { resource.`type`() match { case TOPIC => validateTopicName(resource.name()) val properties = new Properties() val nullTopicConfigs = new mutable.ArrayBuffer[String]() - config.forEach((key, value) => { + newConfigs.forEach((key, value) => { if (value == null) { nullTopicConfigs += key } else { @@ -107,12 +108,12 @@ class ControllerConfigurationValidator(kafkaConfig: KafkaConfig) extends Configu throw new InvalidConfigurationException("Null value not supported for topic configs: " + nullTopicConfigs.mkString(",")) } - LogConfig.validate(properties, kafkaConfig.extractLogConfigMap, + LogConfig.validate(oldConfigs, properties, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) case BROKER => validateBrokerName(resource.name()) case CLIENT_METRICS => val properties = new Properties() - config.forEach((key, value) => properties.setProperty(key, value)) + newConfigs.forEach((key, value) => properties.setProperty(key, value)) ClientMetricsConfigs.validate(resource.name(), properties) case _ => throwExceptionForUnknownResourceType(resource) } diff --git a/core/src/main/scala/kafka/zk/AdminZkClient.scala b/core/src/main/scala/kafka/zk/AdminZkClient.scala index cd9153c07d..8db20583e2 100644 --- a/core/src/main/scala/kafka/zk/AdminZkClient.scala +++ b/core/src/main/scala/kafka/zk/AdminZkClient.scala @@ -161,7 +161,7 @@ class AdminZkClient(zkClient: KafkaZkClient, partitionReplicaAssignment.keys.filter(_ >= 0).sum != sequenceSum) throw new InvalidReplicaAssignmentException("partitions should be a consecutive 0-based integer sequence") - LogConfig.validate(config, + LogConfig.validate(Collections.emptyMap(), config, kafkaConfig.map(_.extractLogConfigMap).getOrElse(Collections.emptyMap()), kafkaConfig.exists(_.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) } @@ -479,7 +479,7 @@ class AdminZkClient(zkClient: KafkaZkClient, if (!zkClient.topicExists(topic)) throw new UnknownTopicOrPartitionException(s"Topic '$topic' does not exist.") // remove the topic overrides - LogConfig.validate(configs, + LogConfig.validate(Collections.emptyMap(), configs, kafkaConfig.map(_.extractLogConfigMap).getOrElse(Collections.emptyMap()), kafkaConfig.exists(_.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) } diff --git a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala index f995b86b70..c0e976bf1d 100644 --- a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala +++ b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala @@ -291,6 +291,26 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { () => admin.incrementalAlterConfigs(configs).all().get(), "Invalid local retention size") } + // The remote storage config validation on controller level only works in KRaft + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testUpdateTopicConfigWithDisablingRemoteStorage(quorum: String): Unit = { + val admin = createAdminClient() + val topicConfig = new Properties + topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, controllerServers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + + val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName), + util.Arrays.asList( + new AlterConfigOp(new ConfigEntry(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false"), + AlterConfigOp.OpType.SET), + )) + assertThrowsException(classOf[InvalidConfigurationException], + () => admin.incrementalAlterConfigs(configs).all().get(), "Disabling remote storage feature on the topic level is not supported.") + } + @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testTopicDeletion(quorum: String): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala index 1a51633674..436673806a 100644 --- a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach , TestInfo, Timeout} +import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 2793da5151..4e7e4e23b3 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -22,6 +22,7 @@ import kafka.utils.TestUtils import org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM import org.apache.kafka.common.config.ConfigDef.Type.INT import org.apache.kafka.common.config.{ConfigException, SslConfigs, TopicConfig} +import org.apache.kafka.common.errors.InvalidConfigurationException import org.apache.kafka.server.common.MetadataVersion import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -299,7 +300,7 @@ class LogConfigTest { props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs.toString) props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, localRetentionBytes.toString) assertThrows(classOf[ConfigException], - () => LogConfig.validate(props, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), props, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) } @Test @@ -311,17 +312,17 @@ class LogConfigTest { val logProps = new Properties() logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete,compact") assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact,delete") assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) } @ParameterizedTest(name = "testEnableRemoteLogStorage with sysRemoteStorageEnabled: {0}") @@ -334,14 +335,34 @@ class LogConfigTest { val logProps = new Properties() logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") if (sysRemoteStorageEnabled) { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) } else { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) assertTrue(message.getMessage.contains("Tiered Storage functionality is disabled in the broker")) } } + @ParameterizedTest(name = "testDisableRemoteLogStorage with wasRemoteStorageEnabled: {0}") + @ValueSource(booleans = Array(true, false)) + def testDisableRemoteLogStorage(wasRemoteStorageEnabled: Boolean): Unit = { + val kafkaProps = TestUtils.createDummyBrokerConfig() + kafkaProps.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true") + val kafkaConfig = KafkaConfig.fromProps(kafkaProps) + + val logProps = new Properties() + logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false") + if (wasRemoteStorageEnabled) { + val message = assertThrows(classOf[InvalidConfigurationException], + () => LogConfig.validate(Collections.singletonMap(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"), + logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + assertTrue(message.getMessage.contains("Disabling remote storage feature on the topic level is not supported.")) + } else { + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.singletonMap(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false"), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + } + } + @ParameterizedTest(name = "testTopicCreationWithInvalidRetentionTime with sysRemoteStorageEnabled: {0}") @ValueSource(booleans = Array(true, false)) def testTopicCreationWithInvalidRetentionTime(sysRemoteStorageEnabled: Boolean): Unit = { @@ -357,10 +378,10 @@ class LogConfigTest { logProps.put(TopicConfig.RETENTION_MS_CONFIG, "500") if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG)) } else { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) } } @@ -379,10 +400,10 @@ class LogConfigTest { logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, "128") if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG)) } else { - LogConfig.validate(logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) } } diff --git a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala index 00bb93811b..250f07ca23 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala @@ -20,11 +20,13 @@ package kafka.server import kafka.utils.TestUtils import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER, CLIENT_METRICS, TOPIC} -import org.apache.kafka.common.config.TopicConfig.{SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} +import org.apache.kafka.common.config.TopicConfig.{REMOTE_LOG_STORAGE_ENABLE_CONFIG, SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, InvalidTopicException} import org.apache.kafka.server.metrics.ClientMetricsConfigs import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import java.util import java.util.Collections.emptyMap @@ -37,7 +39,7 @@ class ControllerConfigurationValidatorTest { def testDefaultTopicResourceIsRejected(): Unit = { assertEquals("Default topic resources are not allowed.", assertThrows(classOf[InvalidRequestException], () => validator.validate( - new ConfigResource(TOPIC, ""), emptyMap())). getMessage) + new ConfigResource(TOPIC, ""), emptyMap(), emptyMap())). getMessage) } @Test @@ -45,14 +47,14 @@ class ControllerConfigurationValidatorTest { assertEquals("Topic name is invalid: '(<-invalid->)' contains " + "one or more characters other than ASCII alphanumerics, '.', '_' and '-'", assertThrows(classOf[InvalidTopicException], () => validator.validate( - new ConfigResource(TOPIC, "(<-invalid->)"), emptyMap())). getMessage) + new ConfigResource(TOPIC, "(<-invalid->)"), emptyMap(), emptyMap())). getMessage) } @Test def testUnknownResourceType(): Unit = { assertEquals("Unknown resource type BROKER_LOGGER", assertThrows(classOf[InvalidRequestException], () => validator.validate( - new ConfigResource(BROKER_LOGGER, "foo"), emptyMap())). getMessage) + new ConfigResource(BROKER_LOGGER, "foo"), emptyMap(), emptyMap())). getMessage) } @Test @@ -63,7 +65,7 @@ class ControllerConfigurationValidatorTest { config.put(SEGMENT_MS_CONFIG, null) assertEquals("Null value not supported for topic configs: segment.bytes,segment.ms", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( - new ConfigResource(TOPIC, "foo"), config)). getMessage) + new ConfigResource(TOPIC, "foo"), config, emptyMap())). getMessage) } @Test @@ -71,7 +73,7 @@ class ControllerConfigurationValidatorTest { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") config.put(SEGMENT_BYTES_CONFIG, "67108864") - validator.validate(new ConfigResource(TOPIC, "foo"), config) + validator.validate(new ConfigResource(TOPIC, "foo"), config, emptyMap()) } @Test @@ -82,7 +84,24 @@ class ControllerConfigurationValidatorTest { config.put("foobar", "abc") assertEquals("Unknown topic config name: foobar", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( - new ConfigResource(TOPIC, "foo"), config)). getMessage) + new ConfigResource(TOPIC, "foo"), config, emptyMap())). getMessage) + } + + @ParameterizedTest(name = "testDisablingRemoteStorageTopicConfig with wasRemoteStorageEnabled: {0}") + @ValueSource(booleans = Array(true, false)) + def testDisablingRemoteStorageTopicConfig(wasRemoteStorageEnabled: Boolean): Unit = { + val config = new util.TreeMap[String, String]() + config.put(REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false") + if (wasRemoteStorageEnabled) { + assertEquals("Disabling remote storage feature on the topic level is not supported.", + assertThrows(classOf[InvalidConfigurationException], () => validator.validate( + new ConfigResource(TOPIC, "foo"), config, util.Collections.singletonMap(REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"))).getMessage) + } else { + validator.validate( + new ConfigResource(TOPIC, "foo"), config, util.Collections.emptyMap()) + validator.validate( + new ConfigResource(TOPIC, "foo"), config, util.Collections.singletonMap(REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false")) + } } @Test @@ -91,7 +110,7 @@ class ControllerConfigurationValidatorTest { config.put(SEGMENT_JITTER_MS_CONFIG, "1000") assertEquals("Unable to parse broker name as a base 10 number.", assertThrows(classOf[InvalidRequestException], () => validator.validate( - new ConfigResource(BROKER, "blah"), config)). getMessage) + new ConfigResource(BROKER, "blah"), config, emptyMap())). getMessage) } @Test @@ -100,7 +119,7 @@ class ControllerConfigurationValidatorTest { config.put(SEGMENT_JITTER_MS_CONFIG, "1000") assertEquals("Invalid negative broker ID.", assertThrows(classOf[InvalidRequestException], () => validator.validate( - new ConfigResource(BROKER, "-1"), config)). getMessage) + new ConfigResource(BROKER, "-1"), config, emptyMap())). getMessage) } @Test @@ -111,7 +130,7 @@ class ControllerConfigurationValidatorTest { config.put(ClientMetricsConfigs.CLIENT_MATCH_PATTERN, "client_instance_id=b69cc35a-7a54-4790-aa69-cc2bd4ee4538,client_id=1" + ",client_software_name=apache-kafka-java,client_software_version=2.8.0-SNAPSHOT,client_source_address=127.0.0.1," + "client_source_port=1234") - validator.validate(new ConfigResource(CLIENT_METRICS, "subscription-1"), config) + validator.validate(new ConfigResource(CLIENT_METRICS, "subscription-1"), config, emptyMap()) } @Test @@ -119,7 +138,7 @@ class ControllerConfigurationValidatorTest { val config = new util.TreeMap[String, String]() assertEquals("Subscription name can't be empty", assertThrows(classOf[InvalidRequestException], () => validator.validate( - new ConfigResource(CLIENT_METRICS, ""), config)). getMessage) + new ConfigResource(CLIENT_METRICS, ""), config, emptyMap())). getMessage) } @Test @@ -128,12 +147,12 @@ class ControllerConfigurationValidatorTest { config.put(ClientMetricsConfigs.PUSH_INTERVAL_MS, "10") assertEquals("Invalid value 10 for interval.ms, interval must be between 100 and 3600000 (1 hour)", assertThrows(classOf[InvalidRequestException], () => validator.validate( - new ConfigResource(CLIENT_METRICS, "subscription-1"), config)). getMessage) + new ConfigResource(CLIENT_METRICS, "subscription-1"), config, emptyMap())). getMessage) config.put(ClientMetricsConfigs.PUSH_INTERVAL_MS, "3600001") assertEquals("Invalid value 3600001 for interval.ms, interval must be between 100 and 3600000 (1 hour)", assertThrows(classOf[InvalidRequestException], () => validator.validate( - new ConfigResource(CLIENT_METRICS, "subscription-1"), config)). getMessage) + new ConfigResource(CLIENT_METRICS, "subscription-1"), config, emptyMap())). getMessage) } @Test @@ -142,7 +161,7 @@ class ControllerConfigurationValidatorTest { config.put("random", "10") assertEquals("Unknown client metrics configuration: random", assertThrows(classOf[InvalidRequestException], () => validator.validate( - new ConfigResource(CLIENT_METRICS, "subscription-1"), config)). getMessage) + new ConfigResource(CLIENT_METRICS, "subscription-1"), config, emptyMap())). getMessage) } @Test @@ -151,6 +170,6 @@ class ControllerConfigurationValidatorTest { config.put(ClientMetricsConfigs.CLIENT_MATCH_PATTERN, "10") assertEquals("Illegal client matching pattern: 10", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( - new ConfigResource(CLIENT_METRICS, "subscription-1"), config)). getMessage) + new ConfigResource(CLIENT_METRICS, "subscription-1"), config, emptyMap())). getMessage) } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index 4ea7d9757d..b10614cc2e 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -269,9 +269,13 @@ private ApiError validateAlterConfig(ConfigResource configResource, List recordsImplicitlyDeleted, boolean newlyCreatedResource) { Map allConfigs = new HashMap<>(); + Map existingConfigsMap = new HashMap<>(); Map alteredConfigsForAlterConfigPolicyCheck = new HashMap<>(); - TimelineHashMap existingConfigs = configData.get(configResource); - if (existingConfigs != null) allConfigs.putAll(existingConfigs); + TimelineHashMap existingConfigsSnapshot = configData.get(configResource); + if (existingConfigsSnapshot != null) { + allConfigs.putAll(existingConfigsSnapshot); + existingConfigsMap.putAll(existingConfigsSnapshot); + } for (ApiMessageAndVersion newRecord : recordsExplicitlyAltered) { ConfigRecord configRecord = (ConfigRecord) newRecord.message(); if (configRecord.value() == null) { @@ -288,7 +292,7 @@ private ApiError validateAlterConfig(ConfigResource configResource, // in the list passed to the policy in order to maintain backwards compatibility } try { - validator.validate(configResource, allConfigs); + validator.validate(configResource, allConfigs, existingConfigsMap); if (!newlyCreatedResource) { existenceChecker.accept(configResource); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationValidator.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationValidator.java index 7e8f505f40..c23c64d8c4 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationValidator.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationValidator.java @@ -28,7 +28,7 @@ public interface ConfigurationValidator { public void validate(ConfigResource resource) { } @Override - public void validate(ConfigResource resource, Map config) { } + public void validate(ConfigResource resource, Map newConfigs, Map existingConfigs) { } }; /** @@ -41,8 +41,9 @@ public void validate(ConfigResource resource, Map config) { } /** * Throws an ApiException if a configuration is invalid for the given resource. * - * @param resource The configuration resource. - * @param config The new configuration. + * @param resource The configuration resource. + * @param newConfigs The new configuration. + * @param existingConfigs The existing configuration. */ - void validate(ConfigResource resource, Map config); + void validate(ConfigResource resource, Map newConfigs, Map existingConfigs); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 430486bdca..9cf4a71e38 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -608,19 +608,32 @@ public static void validateBrokerLogConfigValues(Map props, /** * Validates the values of the given properties. Should be called only by the broker. - * The `props` supplied contains the topic-level configs, + * The `newConfigs` supplied contains the topic-level configs, * The default values should be extracted from the KafkaConfig. - * @param props The properties to be validated + * @param existingConfigs The existing properties + * @param newConfigs The new properties to be validated + * @param isRemoteLogStorageSystemEnabled true if system wise remote log storage is enabled */ - private static void validateTopicLogConfigValues(Map props, + private static void validateTopicLogConfigValues(Map existingConfigs, + Map newConfigs, boolean isRemoteLogStorageSystemEnabled) { - validateValues(props); - boolean isRemoteLogStorageEnabled = (Boolean) props.get(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); + validateValues(newConfigs); + boolean isRemoteLogStorageEnabled = (Boolean) newConfigs.get(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); if (isRemoteLogStorageEnabled) { - validateRemoteStorageOnlyIfSystemEnabled(props, isRemoteLogStorageSystemEnabled, false); - validateNoRemoteStorageForCompactedTopic(props); - validateRemoteStorageRetentionSize(props); - validateRemoteStorageRetentionTime(props); + validateRemoteStorageOnlyIfSystemEnabled(newConfigs, isRemoteLogStorageSystemEnabled, false); + validateNoRemoteStorageForCompactedTopic(newConfigs); + validateRemoteStorageRetentionSize(newConfigs); + validateRemoteStorageRetentionTime(newConfigs); + } else { + // The new config "remote.storage.enable" is false, validate if it's turning from true to false + validateNotTurningOffRemoteStorage(existingConfigs); + } + } + + public static void validateNotTurningOffRemoteStorage(Map existingConfigs) { + boolean wasRemoteLogEnabledBeforeUpdate = Boolean.parseBoolean(existingConfigs.getOrDefault(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false")); + if (wasRemoteLogEnabledBeforeUpdate) { + throw new InvalidConfigurationException("Disabling remote storage feature on the topic level is not supported."); } } @@ -681,10 +694,11 @@ private static void validateRemoteStorageRetentionTime(Map props) { * Check that the given properties contain only valid log config names and that all values can be parsed and are valid */ public static void validate(Properties props) { - validate(props, Collections.emptyMap(), false); + validate(Collections.emptyMap(), props, Collections.emptyMap(), false); } - public static void validate(Properties props, + public static void validate(Map existingConfigs, + Properties props, Map configuredProps, boolean isRemoteLogStorageSystemEnabled) { validateNames(props); @@ -695,7 +709,7 @@ public static void validate(Properties props, Map combinedConfigs = new HashMap<>(configuredProps); combinedConfigs.putAll(props); Map valueMaps = CONFIG.parse(combinedConfigs); - validateTopicLogConfigValues(valueMaps, isRemoteLogStorageSystemEnabled); + validateTopicLogConfigValues(existingConfigs, valueMaps, isRemoteLogStorageSystemEnabled); } } From b622121c0acd296754114148fe360890599a676d Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Sat, 3 Aug 2024 17:38:41 +0900 Subject: [PATCH 024/123] KAFKA-16855: remote log disable policy in KRaft (#16653) Reviewers: Kamal Chandraprakash , Christo Lolov --- .../kafka/common/config/TopicConfig.java | 19 ++- .../kafka/log/remote/RemoteLogManager.java | 80 ++++++++---- .../src/main/scala/kafka/log/LogManager.scala | 10 +- .../scala/kafka/server/ConfigHandler.scala | 62 +++++++-- .../ControllerConfigurationValidator.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 5 +- .../main/scala/kafka/zk/AdminZkClient.scala | 5 +- .../log/remote/RemoteLogManagerTest.java | 15 ++- .../kafka/admin/RemoteTopicCrudTest.scala | 49 ++++++- .../scala/unit/kafka/log/LogConfigTest.scala | 75 +++++++---- .../scala/unit/kafka/log/LogManagerTest.scala | 2 +- ...ControllerConfigurationValidatorTest.scala | 4 +- .../server/DynamicConfigChangeTest.scala | 13 +- .../storage/internals/log/LogConfig.java | 62 ++++++--- .../DisableRemoteLogOnTopicTest.java | 120 ++++++++++++++++++ 15 files changed, 410 insertions(+), 113 deletions(-) create mode 100644 storage/src/test/java/org/apache/kafka/tiered/storage/integration/DisableRemoteLogOnTopicTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 3689227d1f..04c6c487cd 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -93,17 +93,14 @@ public class TopicConfig { "deletes the old segments. Default value is -2, it represents `retention.bytes` value to be used. The effective value should always be " + "less than or equal to `retention.bytes` value."; - public static final String REMOTE_LOG_DISABLE_POLICY_RETAIN = "retain"; - public static final String REMOTE_LOG_DISABLE_POLICY_DELETE = "delete"; - - public static final String REMOTE_LOG_DISABLE_POLICY_CONFIG = "remote.log.disable.policy"; - - public static final String REMOTE_LOG_DISABLE_POLICY_DOC = String.format("Determines whether tiered data for a topic should be retained or " + - "deleted after tiered storage disablement on a topic. The two valid options are \"%s\" and \"%s\". If %s is " + - "selected then all data in remote will be kept post-disablement and will only be deleted when it breaches expiration " + - "thresholds. If %s is selected then the data will be made inaccessible immediately by advancing the log start offset and will be " + - "deleted asynchronously.", REMOTE_LOG_DISABLE_POLICY_RETAIN, REMOTE_LOG_DISABLE_POLICY_DELETE, - REMOTE_LOG_DISABLE_POLICY_RETAIN, REMOTE_LOG_DISABLE_POLICY_DELETE); + public static final String REMOTE_LOG_COPY_DISABLE_CONFIG = "remote.log.copy.disable"; + public static final String REMOTE_LOG_COPY_DISABLE_DOC = "Determines whether tiered data for a topic should become read only," + + " and no more data uploading on a topic."; + + public static final String REMOTE_LOG_DELETE_ON_DISABLE_CONFIG = "remote.log.delete.on.disable"; + public static final String REMOTE_LOG_DELETE_ON_DISABLE_DOC = "Determines whether tiered data for a topic should be " + + "deleted after tiered storage is disabled on a topic. This configuration should be enabled when trying to " + + "set `remote.storage.enable` from true to false"; public static final String MAX_MESSAGE_BYTES_CONFIG = "max.message.bytes"; public static final String MAX_MESSAGE_BYTES_DOC = diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 7a087497bb..a1714211af 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -437,30 +437,48 @@ public void onLeadershipChange(Set partitionsBecomeLeader, throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled"); } - Set leaderPartitions = filterPartitions(partitionsBecomeLeader) - .map(p -> new TopicIdPartition(topicIds.get(p.topic()), p.topicPartition())).collect(Collectors.toSet()); + Map leaderPartitions = filterPartitions(partitionsBecomeLeader) + .collect(Collectors.toMap(p -> new TopicIdPartition(topicIds.get(p.topic()), p.topicPartition()), + p -> p.log().exists(log -> log.config().remoteLogCopyDisable()))); - Set followerPartitions = filterPartitions(partitionsBecomeFollower) - .map(p -> new TopicIdPartition(topicIds.get(p.topic()), p.topicPartition())).collect(Collectors.toSet()); + Map followerPartitions = filterPartitions(partitionsBecomeFollower) + .collect(Collectors.toMap(p -> new TopicIdPartition(topicIds.get(p.topic()), p.topicPartition()), + p -> p.log().exists(log -> log.config().remoteLogCopyDisable()))); if (!leaderPartitions.isEmpty() || !followerPartitions.isEmpty()) { LOGGER.debug("Effective topic partitions after filtering compact and internal topics, leaders: {} and followers: {}", leaderPartitions, followerPartitions); - leaderPartitions.forEach(this::cacheTopicPartitionIds); - followerPartitions.forEach(this::cacheTopicPartitionIds); + leaderPartitions.forEach((tp, __) -> cacheTopicPartitionIds(tp)); + followerPartitions.forEach((tp, __) -> cacheTopicPartitionIds(tp)); - remoteLogMetadataManager.onPartitionLeadershipChanges(leaderPartitions, followerPartitions); - followerPartitions.forEach(this::doHandleFollowerPartition); + remoteLogMetadataManager.onPartitionLeadershipChanges(leaderPartitions.keySet(), followerPartitions.keySet()); + followerPartitions.forEach((tp, __) -> doHandleFollowerPartition(tp)); // If this node was the previous leader for the partition, then the RLMTask might be running in the // background thread and might emit metrics. So, removing the metrics after marking this node as follower. - followerPartitions.forEach(this::removeRemoteTopicPartitionMetrics); + followerPartitions.forEach((tp, __) -> removeRemoteTopicPartitionMetrics(tp)); leaderPartitions.forEach(this::doHandleLeaderPartition); } } + public void stopLeaderCopyRLMTasks(Set partitions) { + for (Partition partition : partitions) { + TopicPartition tp = partition.topicPartition(); + if (topicIdByPartitionMap.containsKey(tp)) { + TopicIdPartition tpId = new TopicIdPartition(topicIdByPartitionMap.get(tp), tp); + leaderCopyRLMTasks.computeIfPresent(tpId, (topicIdPartition, task) -> { + LOGGER.info("Cancelling the copy RLM task for tpId: {}", tpId); + task.cancel(); + LOGGER.info("Resetting remote copy lag metrics for tpId: {}", tpId); + ((RLMCopyTask) task.rlmTask).resetLagStats(); + return null; + }); + } + } + } + /** * Stop the remote-log-manager task for the given partitions. And, calls the * {@link RemoteLogMetadataManager#onStopPartitions(Set)} when {@link StopPartition#deleteLocalLog()} is true. @@ -507,16 +525,18 @@ public void stopPartitions(Set stopPartitions, LOGGER.error("Error while stopping the partition: {}", stopPartition, ex); } } - // Note `deleteLocalLog` will always be true when `deleteRemoteLog` is true but not the other way around. - Set deleteLocalPartitions = stopPartitions.stream() - .filter(sp -> sp.deleteLocalLog() && topicIdByPartitionMap.containsKey(sp.topicPartition())) + + // We want to remote topicId map and stopPartition on RLMM for deleteLocalLog or stopRLMM partitions because + // in both case, they all mean the topic will not be held in this broker anymore. + // NOTE: In ZK mode, this#stopPartitions method is called when Replica state changes to Offline and ReplicaDeletionStarted + Set pendingActionsPartitions = stopPartitions.stream() + .filter(sp -> (sp.stopRemoteLogMetadataManager() || sp.deleteLocalLog()) && topicIdByPartitionMap.containsKey(sp.topicPartition())) .map(sp -> new TopicIdPartition(topicIdByPartitionMap.get(sp.topicPartition()), sp.topicPartition())) .collect(Collectors.toSet()); - if (!deleteLocalPartitions.isEmpty()) { - // NOTE: In ZK mode, this#stopPartitions method is called when Replica state changes to Offline and - // ReplicaDeletionStarted - remoteLogMetadataManager.onStopPartitions(deleteLocalPartitions); - deleteLocalPartitions.forEach(tpId -> topicIdByPartitionMap.remove(tpId.topicPartition())); + + if (!pendingActionsPartitions.isEmpty()) { + pendingActionsPartitions.forEach(tpId -> topicIdByPartitionMap.remove(tpId.topicPartition())); + remoteLogMetadataManager.onStopPartitions(pendingActionsPartitions); } } @@ -985,6 +1005,13 @@ void recordLagStats(long bytesLag, long segmentsLag) { } } + void resetLagStats() { + String topic = topicIdPartition.topic(); + int partition = topicIdPartition.partition(); + brokerTopicStats.recordRemoteCopyLagBytes(topic, partition, 0); + brokerTopicStats.recordRemoteCopyLagSegments(topic, partition, 0); + } + private Path toPathIfExists(File file) { return file.exists() ? file.toPath() : null; } @@ -1793,20 +1820,23 @@ public Future asyncRead(RemoteStorageFetchInfo fetchInfo, Consumer { - RLMCopyTask task = new RLMCopyTask(topicIdPartition, this.rlmConfig.remoteLogMetadataCustomMetadataMaxBytes()); - // set this upfront when it is getting initialized instead of doing it after scheduling. - LOGGER.info("Created a new copy task: {} and getting scheduled", task); - ScheduledFuture future = rlmCopyThreadPool.scheduleWithFixedDelay(task, 0, delayInMs, TimeUnit.MILLISECONDS); - return new RLMTaskWithFuture(task, future); - }); + // Only create copy task when remoteLogCopyDisable is disabled + if (!remoteLogCopyDisable) { + leaderCopyRLMTasks.computeIfAbsent(topicPartition, topicIdPartition -> { + RLMCopyTask task = new RLMCopyTask(topicIdPartition, this.rlmConfig.remoteLogMetadataCustomMetadataMaxBytes()); + // set this upfront when it is getting initialized instead of doing it after scheduling. + LOGGER.info("Created a new copy task: {} and getting scheduled", task); + ScheduledFuture future = rlmCopyThreadPool.scheduleWithFixedDelay(task, 0, delayInMs, TimeUnit.MILLISECONDS); + return new RLMTaskWithFuture(task, future); + }); + } leaderExpirationRLMTasks.computeIfAbsent(topicPartition, topicIdPartition -> { RLMExpirationTask task = new RLMExpirationTask(topicIdPartition); diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 95d2062bf8..66fedbfae6 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -964,15 +964,23 @@ class LogManager(logDirs: Seq[File], */ def updateTopicConfig(topic: String, newTopicConfig: Properties, - isRemoteLogStorageSystemEnabled: Boolean): Unit = { + isRemoteLogStorageSystemEnabled: Boolean, + wasRemoteLogEnabled: Boolean, + fromZK: Boolean): Unit = { topicConfigUpdated(topic) val logs = logsByTopic(topic) // Combine the default properties with the overrides in zk to create the new LogConfig val newLogConfig = LogConfig.fromProps(currentDefaultConfig.originals, newTopicConfig) + val isRemoteLogStorageEnabled = newLogConfig.remoteStorageEnable() // We would like to validate the configuration no matter whether the logs have materialised on disk or not. // Otherwise we risk someone creating a tiered-topic, disabling Tiered Storage cluster-wide and the check // failing since the logs for the topic are non-existent. LogConfig.validateRemoteStorageOnlyIfSystemEnabled(newLogConfig.values(), isRemoteLogStorageSystemEnabled, true) + // `remote.log.delete.on.disable` and `remote.log.copy.disable` are unsupported in ZK mode + if (fromZK) { + LogConfig.validateNoInvalidRemoteStorageConfigsInZK(newLogConfig.values()) + } + LogConfig.validateTurningOffRemoteStorageWithDelete(newLogConfig.values(), wasRemoteLogEnabled, isRemoteLogStorageEnabled) if (logs.nonEmpty) { logs.foreach { log => val oldLogConfig = log.updateConfig(newLogConfig) diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 2d478cfc30..9f183b63ea 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -26,14 +26,14 @@ import kafka.server.Constants._ import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.Implicits._ import kafka.utils.Logging -import org.apache.kafka.server.config.{ReplicationConfigs, QuotaConfigs, ZooKeeperInternals} +import org.apache.kafka.server.config.{QuotaConfigs, ReplicationConfigs, ZooKeeperInternals} import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.metrics.Quota._ import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.ClientMetricsManager -import org.apache.kafka.storage.internals.log.ThrottledReplicaListValidator +import org.apache.kafka.storage.internals.log.{LogStartOffsetIncrementReason, ThrottledReplicaListValidator} import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion import scala.annotation.nowarn @@ -68,25 +68,61 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager, } val logs = logManager.logsByTopic(topic) - val wasRemoteLogEnabledBeforeUpdate = logs.exists(_.remoteLogEnabled()) + val wasRemoteLogEnabled = logs.exists(_.remoteLogEnabled()) + val wasCopyDisabled = logs.exists(_.config.remoteLogCopyDisable()) - logManager.updateTopicConfig(topic, props, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) - maybeBootstrapRemoteLogComponents(topic, logs, wasRemoteLogEnabledBeforeUpdate) + // kafkaController is only defined in Zookeeper's mode + logManager.updateTopicConfig(topic, props, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), + wasRemoteLogEnabled, kafkaController.isDefined) + maybeUpdateRemoteLogComponents(topic, logs, wasRemoteLogEnabled, wasCopyDisabled) } - private[server] def maybeBootstrapRemoteLogComponents(topic: String, - logs: Seq[UnifiedLog], - wasRemoteLogEnabledBeforeUpdate: Boolean): Unit = { + private[server] def maybeUpdateRemoteLogComponents(topic: String, + logs: Seq[UnifiedLog], + wasRemoteLogEnabled: Boolean, + wasCopyDisabled: Boolean): Unit = { val isRemoteLogEnabled = logs.exists(_.remoteLogEnabled()) + val isCopyDisabled = logs.exists(_.config.remoteLogCopyDisable()) + val isDeleteOnDisable = logs.exists(_.config.remoteLogDeleteOnDisable()) + + val (leaderPartitions, followerPartitions) = + logs.flatMap(log => replicaManager.onlinePartition(log.topicPartition)).partition(_.isLeader) + // Topic configs gets updated incrementally. This check is added to prevent redundant updates. - if (!wasRemoteLogEnabledBeforeUpdate && isRemoteLogEnabled) { - val (leaderPartitions, followerPartitions) = - logs.flatMap(log => replicaManager.onlinePartition(log.topicPartition)).partition(_.isLeader) + // When remote log is enabled, or remote copy is enabled, we should create RLM tasks accordingly via `onLeadershipChange`. + if (isRemoteLogEnabled && (!wasRemoteLogEnabled || (wasCopyDisabled && !isCopyDisabled))) { val topicIds = Collections.singletonMap(topic, replicaManager.metadataCache.getTopicId(topic)) replicaManager.remoteLogManager.foreach(rlm => rlm.onLeadershipChange(leaderPartitions.toSet.asJava, followerPartitions.toSet.asJava, topicIds)) - } else if (wasRemoteLogEnabledBeforeUpdate && !isRemoteLogEnabled) { - warn(s"Disabling remote log on the topic: $topic is not supported.") + } + + // When copy disabled, we should stop leaderCopyRLMTask, but keep expirationTask + if (isRemoteLogEnabled && !wasCopyDisabled && isCopyDisabled) { + replicaManager.remoteLogManager.foreach(rlm => { + rlm.stopLeaderCopyRLMTasks(leaderPartitions.toSet.asJava); + }) + } + + // Disabling remote log storage on this topic + if (wasRemoteLogEnabled && !isRemoteLogEnabled && isDeleteOnDisable) { + val stopPartitions: java.util.HashSet[StopPartition] = new java.util.HashSet[StopPartition]() + leaderPartitions.foreach(partition => { + // delete remote logs and stop RemoteLogMetadataManager + stopPartitions.add(StopPartition(partition.topicPartition, deleteLocalLog = false, + deleteRemoteLog = true, stopRemoteLogMetadataManager = true)) + }) + + followerPartitions.foreach(partition => { + // we need to cancel follower tasks and stop RemoteLogMetadataManager + stopPartitions.add(StopPartition(partition.topicPartition, deleteLocalLog = false, + deleteRemoteLog = false, stopRemoteLogMetadataManager = true)) + }) + + // update the log start offset to local log start offset for the leader replicas + logs.filter(log => leaderPartitions.exists(p => p.topicPartition.equals(log.topicPartition))) + .foreach(log => log.maybeIncrementLogStartOffset(log.localLogStartOffset(), LogStartOffsetIncrementReason.SegmentDeletion)) + + replicaManager.remoteLogManager.foreach(rlm => rlm.stopPartitions(stopPartitions, (_, _) => {})) } } diff --git a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala index 06a60e3007..49083e3a0c 100644 --- a/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala +++ b/core/src/main/scala/kafka/server/ControllerConfigurationValidator.scala @@ -109,7 +109,7 @@ class ControllerConfigurationValidator(kafkaConfig: KafkaConfig) extends Configu nullTopicConfigs.mkString(",")) } LogConfig.validate(oldConfigs, properties, kafkaConfig.extractLogConfigMap, - kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), false) case BROKER => validateBrokerName(resource.name()) case CLIENT_METRICS => val properties = new Properties() diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 6ebd151a03..11f2e4ff03 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -99,7 +99,10 @@ case class LogDeleteRecordsResult(requestedOffset: Long, lowWatermark: Long, exc } } -case class StopPartition(topicPartition: TopicPartition, deleteLocalLog: Boolean, deleteRemoteLog: Boolean = false) +case class StopPartition(topicPartition: TopicPartition, + deleteLocalLog: Boolean, + deleteRemoteLog: Boolean = false, + stopRemoteLogMetadataManager: Boolean = false) /** * Result metadata of a log read operation on the log diff --git a/core/src/main/scala/kafka/zk/AdminZkClient.scala b/core/src/main/scala/kafka/zk/AdminZkClient.scala index 8db20583e2..15c95b998b 100644 --- a/core/src/main/scala/kafka/zk/AdminZkClient.scala +++ b/core/src/main/scala/kafka/zk/AdminZkClient.scala @@ -163,7 +163,8 @@ class AdminZkClient(zkClient: KafkaZkClient, LogConfig.validate(Collections.emptyMap(), config, kafkaConfig.map(_.extractLogConfigMap).getOrElse(Collections.emptyMap()), - kafkaConfig.exists(_.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + kafkaConfig.exists(_.remoteLogManagerConfig.isRemoteStorageSystemEnabled()), + true) } private def writeTopicPartitionAssignment(topic: String, replicaAssignment: Map[Int, ReplicaAssignment], @@ -481,7 +482,7 @@ class AdminZkClient(zkClient: KafkaZkClient, // remove the topic overrides LogConfig.validate(Collections.emptyMap(), configs, kafkaConfig.map(_.extractLogConfigMap).getOrElse(Collections.emptyMap()), - kafkaConfig.exists(_.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + kafkaConfig.exists(_.remoteLogManagerConfig.isRemoteStorageSystemEnabled()), true) } /** diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index ce0939737b..302b6adfe4 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -1309,12 +1309,12 @@ void testTopicIdCacheUpdates() throws RemoteStorageException { verifyInCache(followerTopicIdPartition, leaderTopicIdPartition); // Evicts from topicId cache - remoteLogManager.stopPartitions(Collections.singleton(new StopPartition(leaderTopicIdPartition.topicPartition(), true, true)), (tp, ex) -> { }); + remoteLogManager.stopPartitions(Collections.singleton(new StopPartition(leaderTopicIdPartition.topicPartition(), true, true, true)), (tp, ex) -> { }); verifyNotInCache(leaderTopicIdPartition); verifyInCache(followerTopicIdPartition); // Evicts from topicId cache - remoteLogManager.stopPartitions(Collections.singleton(new StopPartition(followerTopicIdPartition.topicPartition(), true, true)), (tp, ex) -> { }); + remoteLogManager.stopPartitions(Collections.singleton(new StopPartition(followerTopicIdPartition.topicPartition(), true, true, true)), (tp, ex) -> { }); verifyNotInCache(leaderTopicIdPartition, followerTopicIdPartition); } @@ -1344,7 +1344,7 @@ void testOnLeadershipChangeWillInvokeHandleLeaderOrFollowerPartitions() { spyRemoteLogManager.onLeadershipChange( Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.emptySet(), topicIds); - verify(spyRemoteLogManager).doHandleLeaderPartition(eq(leaderTopicIdPartition)); + verify(spyRemoteLogManager).doHandleLeaderPartition(eq(leaderTopicIdPartition), eq(false)); } private MemoryRecords records(long timestamp, @@ -1837,8 +1837,8 @@ public void testStopPartitionsWithoutDeletion() throws RemoteStorageException { remoteLogManager.startup(); BiConsumer errorHandler = (topicPartition, throwable) -> fail("shouldn't be called"); Set partitions = new HashSet<>(); - partitions.add(new StopPartition(leaderTopicIdPartition.topicPartition(), true, false)); - partitions.add(new StopPartition(followerTopicIdPartition.topicPartition(), true, false)); + partitions.add(new StopPartition(leaderTopicIdPartition.topicPartition(), true, false, false)); + partitions.add(new StopPartition(followerTopicIdPartition.topicPartition(), true, false, false)); remoteLogManager.onLeadershipChange(Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); assertNotNull(remoteLogManager.leaderCopyTask(leaderTopicIdPartition)); @@ -1860,8 +1860,8 @@ public void testStopPartitionsWithDeletion() throws RemoteStorageException { BiConsumer errorHandler = (topicPartition, ex) -> fail("shouldn't be called: " + ex); Set partitions = new HashSet<>(); - partitions.add(new StopPartition(leaderTopicIdPartition.topicPartition(), true, true)); - partitions.add(new StopPartition(followerTopicIdPartition.topicPartition(), true, true)); + partitions.add(new StopPartition(leaderTopicIdPartition.topicPartition(), true, true, true)); + partitions.add(new StopPartition(followerTopicIdPartition.topicPartition(), true, true, true)); remoteLogManager.onLeadershipChange(Collections.singleton(mockPartition(leaderTopicIdPartition)), Collections.singleton(mockPartition(followerTopicIdPartition)), topicIds); assertNotNull(remoteLogManager.leaderCopyTask(leaderTopicIdPartition)); @@ -3208,6 +3208,7 @@ private Partition mockPartition(TopicIdPartition topicIdPartition) { when(partition.topic()).thenReturn(tp.topic()); when(log.remoteLogEnabled()).thenReturn(true); when(partition.log()).thenReturn(Option.apply(log)); + when(log.config()).thenReturn(new LogConfig(new Properties())); return partition; } diff --git a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala index c0e976bf1d..c88962cb84 100644 --- a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala +++ b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala @@ -156,12 +156,13 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { topicConfig = topicConfig)) } + // `remote.log.delete.on.disable` and `remote.log.copy.disable` only works in KRaft mode. @ParameterizedTest - @CsvSource(Array("zk,retain", "zk,delete", "kraft,retain", "kraft,delete")) - def testCreateRemoteTopicWithDisablePolicyRetain(quorum: String, policy: String): Unit = { + @CsvSource(Array("kraft,true,true", "kraft,true,false", "kraft,false,true", "kraft,false,false")) + def testCreateRemoteTopicWithCopyDisabledAndDeleteOnDisable(quorum: String, copyDisabled: Boolean, deleteOnDisable: Boolean): Unit = { val topicConfig = new Properties() - topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - topicConfig.put(TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG, policy) + topicConfig.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, copyDisabled.toString) + topicConfig.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, deleteOnDisable.toString) TestUtils.createTopicWithAdmin(createAdminClient(), testTopicName, brokers, controllerServers, numPartitions, numReplicationFactor, topicConfig = topicConfig) verifyRemoteLogTopicConfigs(topicConfig) @@ -311,6 +312,35 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { () => admin.incrementalAlterConfigs(configs).all().get(), "Disabling remote storage feature on the topic level is not supported.") } + @ParameterizedTest + @ValueSource(strings = Array("zk")) + def testUpdateInvalidRemoteStorageConfigUnderZK(quorum: String): Unit = { + val admin = createAdminClient() + val errorMsg = "It is invalid to set `remote.log.delete.on.disable` or `remote.log.copy.disable` under Zookeeper's mode." + val topicConfig = new Properties + topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") + TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, controllerServers, numPartitions, numReplicationFactor, + topicConfig = topicConfig) + + val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName), + util.Arrays.asList( + new AlterConfigOp(new ConfigEntry(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, "true"), + AlterConfigOp.OpType.SET), + )) + assertThrowsException(classOf[InvalidConfigurationException], + () => admin.incrementalAlterConfigs(configs).all().get(), errorMsg) + + configs.clear() + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, testTopicName), + util.Arrays.asList( + new AlterConfigOp(new ConfigEntry(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, "true"), + AlterConfigOp.OpType.SET), + )) + assertThrowsException(classOf[InvalidConfigurationException], + () => admin.incrementalAlterConfigs(configs).all().get(), errorMsg) + } + @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testTopicDeletion(quorum: String): Unit = { @@ -409,10 +439,15 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { topicConfig.getProperty(TopicConfig.RETENTION_BYTES_CONFIG).toLong == logBuffer.head.config.retentionSize } - if (topicConfig.contains(TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG)) { + if (topicConfig.contains(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG)) { + result = result && + topicConfig.getProperty(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG).toBoolean == + logBuffer.head.config.remoteLogCopyDisable() + } + if (topicConfig.contains(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG)) { result = result && - topicConfig.getProperty(TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG).equals( - logBuffer.head.config.remoteLogDisablePolicy()) + topicConfig.getProperty(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG).toBoolean == + logBuffer.head.config.remoteLogDeleteOnDisable() } } result diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 4e7e4e23b3..8126bb08b0 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -100,7 +100,8 @@ class LogConfigTest { case TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-2") case TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-1") case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") - case TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0", "true") + case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") + case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case _ => assertPropertyInvalid(name, "not_a_number", "-1") }) @@ -300,7 +301,7 @@ class LogConfigTest { props.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs.toString) props.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, localRetentionBytes.toString) assertThrows(classOf[ConfigException], - () => LogConfig.validate(Collections.emptyMap(), props, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), props, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true)) } @Test @@ -312,17 +313,17 @@ class LogConfigTest { val logProps = new Properties() logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") - LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) assertThrows(classOf[ConfigException], - () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true)) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete,compact") assertThrows(classOf[ConfigException], - () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true)) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact,delete") assertThrows(classOf[ConfigException], - () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true)) } @ParameterizedTest(name = "testEnableRemoteLogStorage with sysRemoteStorageEnabled: {0}") @@ -335,10 +336,10 @@ class LogConfigTest { val logProps = new Properties() logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") if (sysRemoteStorageEnabled) { - LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true) } else { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true)) assertTrue(message.getMessage.contains("Tiered Storage functionality is disabled in the broker")) } } @@ -355,11 +356,20 @@ class LogConfigTest { if (wasRemoteStorageEnabled) { val message = assertThrows(classOf[InvalidConfigurationException], () => LogConfig.validate(Collections.singletonMap(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"), - logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) - assertTrue(message.getMessage.contains("Disabling remote storage feature on the topic level is not supported.")) + logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), false)) + assertTrue(message.getMessage.contains("It is invalid to disable remote storage without deleting remote data. " + + "If you want to keep the remote data and turn to read only, please set `remote.storage.enable=true,remote.log.copy.disable=true`. " + + "If you want to disable remote storage and delete all remote data, please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.")) + + + // It should be able to disable the remote log storage when delete on disable is set to true + logProps.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, "true") + LogConfig.validate(Collections.singletonMap(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"), + logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), false) } else { - LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) - LogConfig.validate(Collections.singletonMap(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false"), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true) + LogConfig.validate(Collections.singletonMap(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false"), logProps, + kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), false) } } @@ -378,10 +388,12 @@ class LogConfigTest { logProps.put(TopicConfig.RETENTION_MS_CONFIG, "500") if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, + kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true)) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG)) } else { - LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, + kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true) } } @@ -400,10 +412,12 @@ class LogConfigTest { logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, "128") if (sysRemoteStorageEnabled) { val message = assertThrows(classOf[ConfigException], - () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled())) + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, + kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true)) assertTrue(message.getMessage.contains(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG)) } else { - LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) + LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, + kafkaConfig.remoteLogManagerConfig.isRemoteStorageSystemEnabled(), true) } } @@ -426,19 +440,34 @@ class LogConfigTest { } @ParameterizedTest - @ValueSource(strings = Array(TopicConfig.REMOTE_LOG_DISABLE_POLICY_RETAIN, TopicConfig.REMOTE_LOG_DISABLE_POLICY_DELETE)) - def testValidRemoteLogDisablePolicy(policy: String): Unit = { + @ValueSource(booleans = Array(true, false)) + def testValidRemoteLogCopyDisabled(copyDisabled: Boolean): Unit = { val logProps = new Properties - logProps.put(TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG, policy) + logProps.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, copyDisabled.toString) LogConfig.validate(logProps) } @ParameterizedTest - @ValueSource(strings = Array("keep", "remove")) - def testInvalidRemoteLogDisablePolicy(policy: String): Unit = { + @ValueSource(booleans = Array(true, false)) + def testValidRemoteLogDeleteOnDisable(deleteOnDisable: Boolean): Unit = { val logProps = new Properties - logProps.put(TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG, policy) - assertThrows(classOf[ConfigException], () => LogConfig.validate(logProps)) + logProps.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, deleteOnDisable.toString) + LogConfig.validate(logProps) + } + + @ParameterizedTest + @ValueSource(strings = Array(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG)) + def testInValidRemoteConfigsInZK(configKey: String): Unit = { + val kafkaProps = TestUtils.createDummyBrokerConfig() + kafkaProps.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true") + val kafkaConfig = KafkaConfig.fromProps(kafkaProps) + val logProps = new Properties + logProps.put(configKey, "true") + + val message = assertThrows(classOf[InvalidConfigurationException], + () => LogConfig.validate(Collections.emptyMap(), logProps, kafkaConfig.extractLogConfigMap, true, true)) + assertTrue(message.getMessage.contains("It is invalid to set `remote.log.delete.on.disable` or " + + "`remote.log.copy.disable` under Zookeeper's mode.")) } /* Verify that when the deprecated config LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG has non default value the new configs diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index b5a029a741..ceb2342446 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -797,7 +797,7 @@ class LogManagerTest { val newProperties = new Properties() newProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) - spyLogManager.updateTopicConfig(topic, newProperties, isRemoteLogStorageSystemEnabled = false) + spyLogManager.updateTopicConfig(topic, newProperties, isRemoteLogStorageSystemEnabled = false, wasRemoteLogEnabled = false, fromZK = false) assertTrue(log0.config.delete) assertTrue(log1.config.delete) diff --git a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala index 250f07ca23..4cf5ad70ce 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala @@ -93,7 +93,9 @@ class ControllerConfigurationValidatorTest { val config = new util.TreeMap[String, String]() config.put(REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false") if (wasRemoteStorageEnabled) { - assertEquals("Disabling remote storage feature on the topic level is not supported.", + assertEquals("It is invalid to disable remote storage without deleting remote data. " + + "If you want to keep the remote data and turn to read only, please set `remote.storage.enable=true,remote.log.copy.disable=true`. " + + "If you want to disable remote storage and delete all remote data, please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( new ConfigResource(TOPIC, "foo"), config, util.Collections.singletonMap(REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"))).getMessage) } else { diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index cc4623b127..dcec4a75d9 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -38,6 +38,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1 import org.apache.kafka.server.config.{ConfigType, QuotaConfigs, ServerLogConfigs, ZooKeeperInternals} +import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.params.ParameterizedTest @@ -659,6 +660,7 @@ class DynamicConfigChangeUnitTest { when(log0.remoteLogEnabled()).thenReturn(true) when(partition0.isLeader).thenReturn(true) when(replicaManager.onlinePartition(tp0)).thenReturn(Some(partition0)) + when(log0.config).thenReturn(new LogConfig(Collections.emptyMap())) val tp1 = new TopicPartition(topic, 1) val log1: UnifiedLog = mock(classOf[UnifiedLog]) @@ -667,6 +669,7 @@ class DynamicConfigChangeUnitTest { when(log1.remoteLogEnabled()).thenReturn(true) when(partition1.isLeader).thenReturn(false) when(replicaManager.onlinePartition(tp1)).thenReturn(Some(partition1)) + when(log1.config).thenReturn(new LogConfig(Collections.emptyMap())) val leaderPartitionsArg: ArgumentCaptor[util.Set[Partition]] = ArgumentCaptor.forClass(classOf[util.Set[Partition]]) val followerPartitionsArg: ArgumentCaptor[util.Set[Partition]] = ArgumentCaptor.forClass(classOf[util.Set[Partition]]) @@ -674,7 +677,7 @@ class DynamicConfigChangeUnitTest { val isRemoteLogEnabledBeforeUpdate = false val configHandler: TopicConfigHandler = new TopicConfigHandler(replicaManager, null, null, None) - configHandler.maybeBootstrapRemoteLogComponents(topic, Seq(log0, log1), isRemoteLogEnabledBeforeUpdate) + configHandler.maybeUpdateRemoteLogComponents(topic, Seq(log0, log1), isRemoteLogEnabledBeforeUpdate, false) assertEquals(Collections.singleton(partition0), leaderPartitionsArg.getValue) assertEquals(Collections.singleton(partition1), followerPartitionsArg.getValue) } @@ -682,17 +685,23 @@ class DynamicConfigChangeUnitTest { @Test def testEnableRemoteLogStorageOnTopicOnAlreadyEnabledTopic(): Unit = { val topic = "test-topic" + val tp0 = new TopicPartition(topic, 0) val rlm: RemoteLogManager = mock(classOf[RemoteLogManager]) val replicaManager: ReplicaManager = mock(classOf[ReplicaManager]) + val partition: Partition = mock(classOf[Partition]) when(replicaManager.remoteLogManager).thenReturn(Some(rlm)) + when(replicaManager.onlinePartition(tp0)).thenReturn(Some(partition)) val log0: UnifiedLog = mock(classOf[UnifiedLog]) when(log0.remoteLogEnabled()).thenReturn(true) doNothing().when(rlm).onLeadershipChange(any(), any(), any()) + when(log0.config).thenReturn(new LogConfig(Collections.emptyMap())) + when(log0.topicPartition).thenReturn(tp0) + when(partition.isLeader).thenReturn(true) val isRemoteLogEnabledBeforeUpdate = true val configHandler: TopicConfigHandler = new TopicConfigHandler(replicaManager, null, null, None) - configHandler.maybeBootstrapRemoteLogComponents(topic, Seq(log0), isRemoteLogEnabledBeforeUpdate) + configHandler.maybeUpdateRemoteLogComponents(topic, Seq(log0), isRemoteLogEnabledBeforeUpdate, false) verify(rlm, never()).onLeadershipChange(any(), any(), any()) } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 9cf4a71e38..7f8fe4a982 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -113,13 +113,15 @@ public String topicWarningMessage(String topicName) { public static class RemoteLogConfig { public final boolean remoteStorageEnable; - public final String remoteLogDisablePolicy; + public final boolean remoteLogDeleteOnDisable; + public final boolean remoteLogCopyDisable; public final long localRetentionMs; public final long localRetentionBytes; private RemoteLogConfig(LogConfig config) { this.remoteStorageEnable = config.getBoolean(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); - this.remoteLogDisablePolicy = config.getString(TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG); + this.remoteLogCopyDisable = config.getBoolean(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG); + this.remoteLogDeleteOnDisable = config.getBoolean(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG); this.localRetentionMs = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG); this.localRetentionBytes = config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG); } @@ -128,7 +130,8 @@ private RemoteLogConfig(LogConfig config) { public String toString() { return "RemoteLogConfig{" + "remoteStorageEnable=" + remoteStorageEnable + - ", remoteLogDisablePolicy=" + remoteLogDisablePolicy + + ", remoteLogCopyDisable=" + remoteLogCopyDisable + + ", remoteLogDeleteOnDisable=" + remoteLogDeleteOnDisable + ", localRetentionMs=" + localRetentionMs + ", localRetentionBytes=" + localRetentionBytes + '}'; @@ -204,7 +207,8 @@ public Optional serverConfigName(String configName) { // Visible for testing public static final Set CONFIGS_WITH_NO_SERVER_DEFAULTS = Collections.unmodifiableSet(Utils.mkSet( TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, - TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG, + TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, + TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, QuotaConfigs.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, QuotaConfigs.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG )); @@ -325,9 +329,8 @@ public Optional serverConfigName(String configName) { TopicConfig.LOCAL_LOG_RETENTION_MS_DOC) .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) - .define(TopicConfig.REMOTE_LOG_DISABLE_POLICY_CONFIG, STRING, TopicConfig.REMOTE_LOG_DISABLE_POLICY_RETAIN, - in(TopicConfig.REMOTE_LOG_DISABLE_POLICY_RETAIN, TopicConfig.REMOTE_LOG_DISABLE_POLICY_DELETE), - MEDIUM, TopicConfig.REMOTE_LOG_DISABLE_POLICY_DOC); + .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) + .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC); } public final Set overriddenConfigs; @@ -508,8 +511,12 @@ public boolean remoteStorageEnable() { return remoteLogConfig.remoteStorageEnable; } - public String remoteLogDisablePolicy() { - return remoteLogConfig.remoteLogDisablePolicy; + public Boolean remoteLogDeleteOnDisable() { + return remoteLogConfig.remoteLogDeleteOnDisable; + } + + public Boolean remoteLogCopyDisable() { + return remoteLogConfig.remoteLogCopyDisable; } public long localRetentionMs() { @@ -613,11 +620,17 @@ public static void validateBrokerLogConfigValues(Map props, * @param existingConfigs The existing properties * @param newConfigs The new properties to be validated * @param isRemoteLogStorageSystemEnabled true if system wise remote log storage is enabled + * @param fromZK true if this is a ZK cluster */ private static void validateTopicLogConfigValues(Map existingConfigs, Map newConfigs, - boolean isRemoteLogStorageSystemEnabled) { + boolean isRemoteLogStorageSystemEnabled, + boolean fromZK) { validateValues(newConfigs); + + if (fromZK) { + validateNoInvalidRemoteStorageConfigsInZK(newConfigs); + } boolean isRemoteLogStorageEnabled = (Boolean) newConfigs.get(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); if (isRemoteLogStorageEnabled) { validateRemoteStorageOnlyIfSystemEnabled(newConfigs, isRemoteLogStorageSystemEnabled, false); @@ -626,14 +639,26 @@ private static void validateTopicLogConfigValues(Map existingCon validateRemoteStorageRetentionTime(newConfigs); } else { // The new config "remote.storage.enable" is false, validate if it's turning from true to false - validateNotTurningOffRemoteStorage(existingConfigs); + boolean wasRemoteLogEnabled = Boolean.parseBoolean(existingConfigs.getOrDefault(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false")); + validateTurningOffRemoteStorageWithDelete(newConfigs, wasRemoteLogEnabled, isRemoteLogStorageEnabled); + } + } + + public static void validateTurningOffRemoteStorageWithDelete(Map newConfigs, boolean wasRemoteLogEnabled, boolean isRemoteLogStorageEnabled) { + boolean isRemoteLogDeleteOnDisable = (Boolean) Utils.castToStringObjectMap(newConfigs).getOrDefault(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, false); + if (wasRemoteLogEnabled && !isRemoteLogStorageEnabled && !isRemoteLogDeleteOnDisable) { + throw new InvalidConfigurationException("It is invalid to disable remote storage without deleting remote data. " + + "If you want to keep the remote data and turn to read only, please set `remote.storage.enable=true,remote.log.copy.disable=true`. " + + "If you want to disable remote storage and delete all remote data, please set `remote.storage.enable=false,remote.log.delete.on.disable=true`."); } } - public static void validateNotTurningOffRemoteStorage(Map existingConfigs) { - boolean wasRemoteLogEnabledBeforeUpdate = Boolean.parseBoolean(existingConfigs.getOrDefault(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false")); - if (wasRemoteLogEnabledBeforeUpdate) { - throw new InvalidConfigurationException("Disabling remote storage feature on the topic level is not supported."); + public static void validateNoInvalidRemoteStorageConfigsInZK(Map newConfigs) { + boolean isRemoteLogDeleteOnDisable = (Boolean) Utils.castToStringObjectMap(newConfigs).getOrDefault(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, false); + boolean isRemoteLogCopyDisabled = (Boolean) Utils.castToStringObjectMap(newConfigs).getOrDefault(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, false); + if (isRemoteLogDeleteOnDisable || isRemoteLogCopyDisabled) { + throw new InvalidConfigurationException("It is invalid to set `remote.log.delete.on.disable` or " + + "`remote.log.copy.disable` under Zookeeper's mode."); } } @@ -694,13 +719,14 @@ private static void validateRemoteStorageRetentionTime(Map props) { * Check that the given properties contain only valid log config names and that all values can be parsed and are valid */ public static void validate(Properties props) { - validate(Collections.emptyMap(), props, Collections.emptyMap(), false); + validate(Collections.emptyMap(), props, Collections.emptyMap(), false, false); } public static void validate(Map existingConfigs, Properties props, Map configuredProps, - boolean isRemoteLogStorageSystemEnabled) { + boolean isRemoteLogStorageSystemEnabled, + boolean fromZK) { validateNames(props); if (configuredProps == null || configuredProps.isEmpty()) { Map valueMaps = CONFIG.parse(props); @@ -709,7 +735,7 @@ public static void validate(Map existingConfigs, Map combinedConfigs = new HashMap<>(configuredProps); combinedConfigs.putAll(props); Map valueMaps = CONFIG.parse(combinedConfigs); - validateTopicLogConfigValues(existingConfigs, valueMaps, isRemoteLogStorageSystemEnabled); + validateTopicLogConfigValues(existingConfigs, valueMaps, isRemoteLogStorageSystemEnabled, fromZK); } } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DisableRemoteLogOnTopicTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DisableRemoteLogOnTopicTest.java new file mode 100644 index 0000000000..6bcaea6a36 --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/DisableRemoteLogOnTopicTest.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.tiered.storage.integration; + +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.tiered.storage.TieredStorageTestBuilder; +import org.apache.kafka.tiered.storage.TieredStorageTestHarness; +import org.apache.kafka.tiered.storage.specs.KeyValueSpec; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent.EventType.DELETE_SEGMENT; + +public final class DisableRemoteLogOnTopicTest extends TieredStorageTestHarness { + + @Override + public int brokerCount() { + return 2; + } + + @ParameterizedTest(name = "{displayName}.quorum={0}") + @ValueSource(strings = {"kraft"}) + public void executeTieredStorageTest(String quorum) { + super.executeTieredStorageTest(quorum); + } + + @Override + protected void writeTestSpecifications(TieredStorageTestBuilder builder) { + final Integer broker0 = 0; + final Integer broker1 = 1; + final String topicA = "topicA"; + final Integer p0 = 0; + final Integer partitionCount = 1; + final Integer replicationFactor = 2; + final Integer maxBatchCountPerSegment = 1; + final boolean enableRemoteLogStorage = true; + final Map> assignment = mkMap( + mkEntry(p0, Arrays.asList(broker0, broker1)) + ); + final Map disableCopy = new HashMap<>(); + disableCopy.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, "true"); + + final Map deleteOnDisable = new HashMap<>(); + deleteOnDisable.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "false"); + deleteOnDisable.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, "true"); + + builder + .createTopic(topicA, partitionCount, replicationFactor, maxBatchCountPerSegment, assignment, + enableRemoteLogStorage) + // send records to partition 0 + .expectSegmentToBeOffloaded(broker0, topicA, p0, 0, new KeyValueSpec("k0", "v0")) + .expectSegmentToBeOffloaded(broker0, topicA, p0, 1, new KeyValueSpec("k1", "v1")) + .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 2L) + .produce(topicA, p0, new KeyValueSpec("k0", "v0"), new KeyValueSpec("k1", "v1"), + new KeyValueSpec("k2", "v2")) + // disable remote log copy + .updateTopicConfig(topicA, + Collections.singletonMap(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, "true"), + Collections.emptyList()) + + // make sure we can still consume from the beginning of the topic to read data from local and remote storage + .expectFetchFromTieredStorage(broker0, topicA, p0, 2) + .consume(topicA, p0, 0L, 3, 2) + + // re-enable remote log copy + .updateTopicConfig(topicA, + Collections.singletonMap(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, "false"), + Collections.emptyList()) + + // make sure the logs can be offloaded + .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 3L) + .produce(topicA, p0, new KeyValueSpec("k3", "v3")) + + // explicitly disable remote log copy + .updateTopicConfig(topicA, + disableCopy, + Collections.emptyList()) + // make sure we can still consume from the beginning of the topic to read data from local and remote storage + .expectFetchFromTieredStorage(broker0, topicA, p0, 3) + .consume(topicA, p0, 0L, 4, 3) + + // verify the remote retention policy is working. + // Use DELETE_RECORDS API to delete the records upto offset 1 and expect 1 remote segment to be deleted + .expectDeletionInRemoteStorage(broker0, topicA, p0, DELETE_SEGMENT, 1) + .deleteRecords(topicA, p0, 1L) + .waitForRemoteLogSegmentDeletion(topicA) + + // disabling remote log on topicA and enabling deleteOnDisable + .updateTopicConfig(topicA, + deleteOnDisable, + Collections.emptyList()) + // make sure all remote data is deleted + .expectEmptyRemoteStorage(topicA, p0) + // verify the local log is still consumable + .consume(topicA, p0, 3L, 1, 0); + } +} From 2ddbfebecbdd082ae18adc88b49b9a3f1811c7ab Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Sat, 3 Aug 2024 12:40:51 -0700 Subject: [PATCH 025/123] KAFKA-16448: Unify error-callback exception handling (#16745) Follow up code cleanup for KIP-1033. This PR unifies the handling of both error cases for exception handlers: - handler throws an exception - handler returns null The unification happens for all 5 handler cases: - deserialzation - production / serialization - production / send - processing - punctuation Reviewers: Sebastien Viale , Loic Greffier , Bill Bejeck --- .../DeserializationExceptionHandler.java | 2 +- .../errors/ProductionExceptionHandler.java | 2 +- .../internals/DefaultErrorHandlerContext.java | 12 + .../internals/FailedProcessingException.java | 7 +- .../internals/GlobalStateManagerImpl.java | 5 +- .../processor/internals/ProcessorNode.java | 21 +- .../internals/RecordCollectorImpl.java | 106 +++++---- .../internals/RecordDeserializer.java | 50 ++--- .../processor/internals/StreamTask.java | 49 ++-- .../integration/EosIntegrationTest.java | 5 +- .../EosV2UpgradeIntegrationTest.java | 6 +- ...essingExceptionHandlerIntegrationTest.java | 30 +-- .../internals/RecordCollectorTest.java | 152 +++++++++++-- .../internals/RecordDeserializerTest.java | 210 ++++++++++++------ .../processor/internals/StreamTaskTest.java | 140 ++++++++---- 15 files changed, 568 insertions(+), 229 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java index 0d64611de6..198a97cce4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java @@ -37,7 +37,7 @@ public interface DeserializationExceptionHandler extends Configurable { * @param context processor context * @param record record that failed deserialization * @param exception the actual exception - * @deprecated Since 3.9. Use Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} + * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} instead. */ @Deprecated default DeserializationHandlerResponse handle(final ProcessorContext context, diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java index 25aa00f7a9..939b1ecbcd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java @@ -59,7 +59,7 @@ default ProductionExceptionHandlerResponse handle(final ErrorHandlerContext cont * * @param record the record that failed to serialize * @param exception the exception that occurred during serialization - * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead. + * @deprecated Since 3.9. Use {@link #handleSerializationException(ErrorHandlerContext, ProducerRecord, Exception, SerializationExceptionOrigin)} instead. */ @Deprecated default ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java index c907ff3eb8..77500ce3c3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java @@ -81,6 +81,18 @@ public TaskId taskId() { return taskId; } + @Override + public String toString() { + // we do exclude headers on purpose, to not accidentally log user data + return "ErrorHandlerContext{" + + "topic='" + topic + '\'' + + ", partition=" + partition + + ", offset=" + offset + + ", processorNodeId='" + processorNodeId + '\'' + + ", taskId=" + taskId + + '}'; + } + public Optional processorContext() { return Optional.ofNullable(processorContext); } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/internals/FailedProcessingException.java b/streams/src/main/java/org/apache/kafka/streams/errors/internals/FailedProcessingException.java index 25f2ae9f6c..03d687d268 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/internals/FailedProcessingException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/FailedProcessingException.java @@ -25,7 +25,12 @@ public class FailedProcessingException extends StreamsException { private static final long serialVersionUID = 1L; + public FailedProcessingException(final String errorMessage, final Exception exception) { + super(errorMessage, exception); + } + public FailedProcessingException(final Exception exception) { - super(exception); + // we need to explicitly set `message` to `null` here + super(null, exception); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index 6b7214a9ed..12d4c6c603 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -319,7 +319,7 @@ private void reprocessState(final List topicPartitions, record.headers())); restoreCount++; } - } catch (final Exception deserializationException) { + } catch (final RuntimeException deserializationException) { handleDeserializationFailure( deserializationExceptionHandler, globalProcessorContext, @@ -330,7 +330,8 @@ private void reprocessState(final List topicPartitions, Thread.currentThread().getName(), globalProcessorContext.taskId().toString(), globalProcessorContext.metrics() - ) + ), + null ); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index eaed7c6b8d..5df1a1bd00 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -38,6 +38,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; @@ -202,7 +203,7 @@ public void process(final Record record) { } catch (final FailedProcessingException | TaskCorruptedException | TaskMigratedException e) { // Rethrow exceptions that should not be handled here throw e; - } catch (final RuntimeException e) { + } catch (final RuntimeException processingException) { final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( null, // only required to pass for DeserializationExceptionHandler internalProcessorContext.topic(), @@ -213,18 +214,26 @@ public void process(final Record record) { internalProcessorContext.taskId()); final ProcessingExceptionHandler.ProcessingHandlerResponse response; - try { - response = processingExceptionHandler.handle(errorHandlerContext, record, e); - } catch (final Exception fatalUserException) { - throw new FailedProcessingException(fatalUserException); + response = Objects.requireNonNull( + processingExceptionHandler.handle(errorHandlerContext, record, processingException), + "Invalid ProductionExceptionHandler response." + ); + } catch (final RuntimeException fatalUserException) { + log.error( + "Processing error callback failed after processing error for record: {}", + errorHandlerContext, + processingException + ); + throw new FailedProcessingException("Fatal user code error in processing error callback", fatalUserException); } + if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { log.error("Processing exception handler is set to fail upon" + " a processing error. If you would rather have the streaming pipeline" + " continue after a processing error, please set the " + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately."); - throw new FailedProcessingException(e); + throw new FailedProcessingException(processingException); } else { droppedRecordsSensor.record(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 7a8b77b8a5..ced6fdef77 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -44,6 +44,7 @@ import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; +import org.apache.kafka.streams.errors.internals.FailedProcessingException; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -58,6 +59,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; @@ -207,7 +209,7 @@ public void send(final String topic, key, keySerializer, exception); - } catch (final Exception exception) { + } catch (final RuntimeException serializationException) { handleException( ProductionExceptionHandler.SerializationExceptionOrigin.KEY, topic, @@ -218,7 +220,7 @@ public void send(final String topic, timestamp, processorNodeId, context, - exception); + serializationException); return; } @@ -231,7 +233,7 @@ public void send(final String topic, value, valueSerializer, exception); - } catch (final Exception exception) { + } catch (final RuntimeException serializationException) { handleException( ProductionExceptionHandler.SerializationExceptionOrigin.VALUE, topic, @@ -242,7 +244,7 @@ public void send(final String topic, timestamp, processorNodeId, context, - exception); + serializationException); return; } @@ -296,42 +298,51 @@ private void handleException(final ProductionExceptionHandler.Serializati final Long timestamp, final String processorNodeId, final InternalProcessorContext context, - final Exception exception) { + final RuntimeException serializationException) { + log.debug(String.format("Error serializing record for topic %s", topic), serializationException); + + final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( + null, // only required to pass for DeserializationExceptionHandler + context.recordContext().topic(), + context.recordContext().partition(), + context.recordContext().offset(), + context.recordContext().headers(), + processorNodeId, + taskId + ); final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); - final ProductionExceptionHandlerResponse response; - - log.debug(String.format("Error serializing record to topic %s", topic), exception); + final ProductionExceptionHandlerResponse response; try { - final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( - null, // only required to pass for DeserializationExceptionHandler - context.recordContext().topic(), - context.recordContext().partition(), - context.recordContext().offset(), - context.recordContext().headers(), - processorNodeId, - taskId + response = Objects.requireNonNull( + productionExceptionHandler.handleSerializationException(errorHandlerContext, record, serializationException, origin), + "Invalid ProductionExceptionHandler response." ); - response = productionExceptionHandler.handleSerializationException(errorHandlerContext, record, exception, origin); - } catch (final Exception e) { - log.error("Fatal when handling serialization exception", e); - recordSendError(topic, e, null, context, processorNodeId); - return; + } catch (final RuntimeException fatalUserException) { + log.error( + String.format( + "Production error callback failed after serialization error for record %s: %s", + origin.toString().toLowerCase(Locale.ROOT), + errorHandlerContext + ), + serializationException + ); + throw new FailedProcessingException("Fatal user code error in production error callback", fatalUserException); } if (response == ProductionExceptionHandlerResponse.FAIL) { throw new StreamsException( String.format( "Unable to serialize record. ProducerRecord(topic=[%s], partition=[%d], timestamp=[%d]", - topic, - partition, - timestamp), - exception + topic, + partition, + timestamp), + serializationException ); } log.warn("Unable to serialize record, continue processing. " + - "ProducerRecord(topic=[{}], partition=[{}], timestamp=[{}])", + "ProducerRecord(topic=[{}], partition=[{}], timestamp=[{}])", topic, partition, timestamp); @@ -363,24 +374,24 @@ private StreamsException createStreamsExceptionForClassCastException(final } private void recordSendError(final String topic, - final Exception exception, + final Exception productionException, final ProducerRecord serializedRecord, final InternalProcessorContext context, final String processorNodeId) { - String errorMessage = String.format(SEND_EXCEPTION_MESSAGE, topic, taskId, exception.toString()); + String errorMessage = String.format(SEND_EXCEPTION_MESSAGE, topic, taskId, productionException.toString()); - if (isFatalException(exception)) { + if (isFatalException(productionException)) { errorMessage += "\nWritten offsets would not be recorded and no more records would be sent since this is a fatal error."; - sendException.set(new StreamsException(errorMessage, exception)); - } else if (exception instanceof ProducerFencedException || - exception instanceof InvalidPidMappingException || - exception instanceof InvalidProducerEpochException || - exception instanceof OutOfOrderSequenceException) { + sendException.set(new StreamsException(errorMessage, productionException)); + } else if (productionException instanceof ProducerFencedException || + productionException instanceof InvalidPidMappingException || + productionException instanceof InvalidProducerEpochException || + productionException instanceof OutOfOrderSequenceException) { errorMessage += "\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced, " + "indicating the task may be migrated out"; - sendException.set(new TaskMigratedException(errorMessage, exception)); + sendException.set(new TaskMigratedException(errorMessage, productionException)); } else { - if (exception instanceof RetriableException) { + if (productionException instanceof RetriableException) { errorMessage += "\nThe broker is either slow or in bad state (like not having enough replicas) in responding the request, " + "or the connection to broker was interrupted sending the request or receiving the response. " + "\nConsider overwriting `max.block.ms` and /or " + @@ -397,17 +408,34 @@ private void recordSendError(final String topic, taskId ); - if (productionExceptionHandler.handle(errorHandlerContext, serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { + final ProductionExceptionHandlerResponse response; + try { + response = Objects.requireNonNull( + productionExceptionHandler.handle(errorHandlerContext, serializedRecord, productionException), + "Invalid ProductionExceptionHandler response." + ); + } catch (final RuntimeException fatalUserException) { + log.error( + "Production error callback failed after production error for record {}", + serializedRecord, + productionException + ); + sendException.set(new FailedProcessingException("Fatal user code error in production error callback", fatalUserException)); + return; + } + + if (response == ProductionExceptionHandlerResponse.FAIL) { errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent."; - sendException.set(new StreamsException(errorMessage, exception)); + sendException.set(new StreamsException(errorMessage, productionException)); } else { errorMessage += "\nException handler choose to CONTINUE processing in spite of this error but written offsets would not be recorded."; droppedRecordsSensor.record(); } + } } - log.error(errorMessage, exception); + log.error(errorMessage, productionException); } private boolean isFatalException(final Exception exception) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 8ee2dc014e..5fc03352ec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -21,12 +21,14 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.DeserializationExceptionHandler.DeserializationHandlerResponse; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.slf4j.Logger; +import java.util.Objects; import java.util.Optional; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; @@ -49,7 +51,7 @@ public class RecordDeserializer { /** * @throws StreamsException if a deserialization error occurs and the deserialization callback returns - * {@link DeserializationExceptionHandler.DeserializationHandlerResponse#FAIL FAIL} + * {@link DeserializationHandlerResponse#FAIL FAIL} * or throws an exception itself */ ConsumerRecord deserialize(final ProcessorContext processorContext, @@ -69,7 +71,7 @@ ConsumerRecord deserialize(final ProcessorContext processo rawRecord.headers(), Optional.empty() ); - } catch (final Exception deserializationException) { + } catch (final RuntimeException deserializationException) { handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, sourceNode().name()); return null; // 'handleDeserializationFailure' would either throw or swallow -- if we swallow we need to skip the record by returning 'null' } @@ -77,39 +79,37 @@ ConsumerRecord deserialize(final ProcessorContext processo public static void handleDeserializationFailure(final DeserializationExceptionHandler deserializationExceptionHandler, final ProcessorContext processorContext, - final Exception deserializationException, - final ConsumerRecord rawRecord, - final Logger log, - final Sensor droppedRecordsSensor) { - handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, null); - } - - public static void handleDeserializationFailure(final DeserializationExceptionHandler deserializationExceptionHandler, - final ProcessorContext processorContext, - final Exception deserializationException, + final RuntimeException deserializationException, final ConsumerRecord rawRecord, final Logger log, final Sensor droppedRecordsSensor, final String sourceNodeName) { - final DeserializationExceptionHandler.DeserializationHandlerResponse response; + + final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( + (InternalProcessorContext) processorContext, + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + rawRecord.headers(), + sourceNodeName, + processorContext.taskId()); + + final DeserializationHandlerResponse response; try { - final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( - (InternalProcessorContext) processorContext, - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - rawRecord.headers(), - sourceNodeName, - processorContext.taskId()); - response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException); - } catch (final Exception fatalUserException) { + response = Objects.requireNonNull( + deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException), + "Invalid DeserializationExceptionHandler response." + ); + } catch (final RuntimeException fatalUserException) { log.error( "Deserialization error callback failed after deserialization error for record {}", rawRecord, - deserializationException); + deserializationException + ); throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException); } - if (response == DeserializationExceptionHandler.DeserializationHandlerResponse.FAIL) { + + if (response == DeserializationHandlerResponse.FAIL) { throw new StreamsException("Deserialization exception handler is set to fail upon" + " a deserialization error. If you would rather have the streaming pipeline" + " continue after a deserialization error, please set the " + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 6f2edd442b..f08cfa7fd6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -59,6 +59,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.function.Function; @@ -807,7 +808,7 @@ record = null; } } catch (final FailedProcessingException failedProcessingException) { // Do not keep the failed processing exception in the stack trace - handleException(failedProcessingException.getCause()); + handleException(failedProcessingException.getMessage(), failedProcessingException.getCause()); } catch (final StreamsException exception) { record = null; throw exception; @@ -820,19 +821,25 @@ record = null; return true; } - private void handleException(final Throwable e) { - final StreamsException error = new StreamsException( + private void handleException(final Throwable originalException) { + handleException( String.format( - "Exception caught in process. taskId=%s, processor=%s, topic=%s, partition=%d, offset=%d, stacktrace=%s", + "Exception caught in process. taskId=%s, processor=%s, topic=%s, partition=%d, offset=%d", id(), processorContext.currentNode().name(), record.topic(), record.partition(), - record.offset(), - getStacktraceString(e) + record.offset() ), - e - ); + originalException); + } + + private void handleException(final String errorMessage, final Throwable originalException) { + if (errorMessage == null) { + handleException(originalException); + } + + final StreamsException error = new StreamsException(errorMessage, originalException); record = null; throw error; @@ -920,11 +927,18 @@ public void punctuate(final ProcessorNode node, try { maybeMeasureLatency(() -> punctuator.punctuate(timestamp), time, punctuateLatencySensor); + } catch (final TimeoutException timeoutException) { + if (!eosEnabled) { + throw timeoutException; + } else { + record = null; + throw new TaskCorruptedException(Collections.singleton(id)); + } } catch (final FailedProcessingException e) { throw createStreamsException(node.name(), e.getCause()); } catch (final TaskCorruptedException | TaskMigratedException e) { throw e; - } catch (final Exception e) { + } catch (final RuntimeException processingException) { final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( null, recordContext.topic(), @@ -936,11 +950,18 @@ public void punctuate(final ProcessorNode node, ); final ProcessingExceptionHandler.ProcessingHandlerResponse response; - try { - response = processingExceptionHandler.handle(errorHandlerContext, null, e); - } catch (final Exception fatalUserException) { - throw new FailedProcessingException(fatalUserException); + response = Objects.requireNonNull( + processingExceptionHandler.handle(errorHandlerContext, null, processingException), + "Invalid ProcessingExceptionHandler response." + ); + } catch (final RuntimeException fatalUserException) { + log.error( + "Processing error callback failed after processing error for record: {}", + errorHandlerContext, + processingException + ); + throw new FailedProcessingException("Fatal user code error in processing error callback", fatalUserException); } if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { @@ -949,7 +970,7 @@ public void punctuate(final ProcessorNode node, " continue after a processing error, please set the " + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately."); - throw createStreamsException(node.name(), e); + throw createStreamsException(node.name(), processingException); } else { droppedRecordsSensor.record(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java index 38e7e5cc0c..5052781195 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java @@ -39,6 +39,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig.InternalConfig; import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; @@ -1177,7 +1178,9 @@ public void close() { } final KafkaStreams streams = new KafkaStreams(builder.build(), config); streams.setUncaughtExceptionHandler((t, e) -> { - if (uncaughtException != null || !e.getMessage().contains("Injected test exception")) { + if (uncaughtException != null || + !(e instanceof StreamsException) || + !e.getCause().getMessage().equals("Injected test exception.")) { e.printStackTrace(System.err); hasUnexpectedError = true; } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java index 087a439d08..36f716a7d0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java @@ -35,6 +35,7 @@ import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig.InternalConfig; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; @@ -952,8 +953,9 @@ public void close() {} } else { int exceptionCount = exceptionCounts.get(appDir); // should only have our injected exception or commit exception, and 2 exceptions for each stream - if (++exceptionCount > 2 || !(e instanceof RuntimeException) || - !(e.getMessage().contains("test exception"))) { + if (++exceptionCount > 2 || + !(e instanceof StreamsException) || + !(e.getCause().getMessage().endsWith(" test exception."))) { // The exception won't cause the test fail since we actually "expected" exception thrown and failed the stream. // So, log to stderr for debugging when the exception is not what we expected, and fail in the main thread e.printStackTrace(System.err); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index 61b5ed16bb..d0c3231055 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -51,6 +51,7 @@ import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertIterableEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -61,7 +62,7 @@ public class ProcessingExceptionHandlerIntegrationTest { private final String threadId = Thread.currentThread().getName(); @Test - public void shouldFailWhenProcessingExceptionOccurs() { + public void shouldFailWhenProcessingExceptionOccursIfExceptionHandlerReturnsFail() { final List> events = Arrays.asList( new KeyValue<>("ID123-1", "ID123-A1"), new KeyValue<>("ID123-2-ERR", "ID123-A2"), @@ -93,8 +94,7 @@ public void shouldFailWhenProcessingExceptionOccurs() { assertTrue(exception.getMessage().contains("Exception caught in process. " + "taskId=0_0, processor=KSTREAM-SOURCE-0000000000, topic=TOPIC_NAME, " - + "partition=0, offset=1, stacktrace=java.lang.RuntimeException: " - + "Exception should be handled by processing exception handler")); + + "partition=0, offset=1")); assertEquals(1, processor.theCapturedProcessor().processed().size()); assertIterableEquals(expectedProcessedRecords, processor.theCapturedProcessor().processed()); @@ -107,7 +107,7 @@ public void shouldFailWhenProcessingExceptionOccurs() { } @Test - public void shouldContinueWhenProcessingExceptionOccurs() { + public void shouldContinueWhenProcessingExceptionOccursIfExceptionHandlerReturnsContinue() { final List> events = Arrays.asList( new KeyValue<>("ID123-1", "ID123-A1"), new KeyValue<>("ID123-2-ERR", "ID123-A2"), @@ -182,8 +182,7 @@ public void shouldStopOnFailedProcessorWhenProcessingExceptionOccursInFailProces final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); assertTrue(e.getMessage().contains("Exception caught in process. " + "taskId=0_0, processor=KSTREAM-SOURCE-0000000000, topic=TOPIC_NAME, " - + "partition=0, offset=1, stacktrace=java.lang.RuntimeException: " - + "Exception should be handled by processing exception handler")); + + "partition=0, offset=1")); assertFalse(isExecuted.get()); } } @@ -222,9 +221,9 @@ public void shouldStopOnFailedProcessorWhenProcessingExceptionOccursInContinuePr } @Test - public void shouldStopProcessingWhenFatalUserExceptionInFailProcessingExceptionHandler() { + public void shouldStopProcessingWhenProcessingExceptionHandlerReturnsNull() { final KeyValue event = new KeyValue<>("ID123-1", "ID123-A1"); - final KeyValue eventError = new KeyValue<>("ID123-ERR-FATAL", "ID123-A2"); + final KeyValue eventError = new KeyValue<>("ID123-ERR-NULL", "ID123-A2"); final MockProcessorSupplier processor = new MockProcessorSupplier<>(); final StreamsBuilder builder = new StreamsBuilder(); @@ -241,7 +240,7 @@ public void shouldStopProcessingWhenFatalUserExceptionInFailProcessingExceptionH .process(processor); final Properties properties = new Properties(); - properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -250,13 +249,15 @@ public void shouldStopProcessingWhenFatalUserExceptionInFailProcessingExceptionH assertTrue(isExecuted.get()); isExecuted.set(false); final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); - assertEquals("KABOOM!", e.getCause().getMessage()); + assertEquals("Fatal user code error in processing error callback", e.getMessage()); + assertInstanceOf(NullPointerException.class, e.getCause()); + assertEquals("Invalid ProductionExceptionHandler response.", e.getCause().getMessage()); assertFalse(isExecuted.get()); } } @Test - public void shouldStopProcessingWhenFatalUserExceptionInContinueProcessingExceptionHandler() { + public void shouldStopProcessingWhenFatalUserExceptionProcessingExceptionHandler() { final KeyValue event = new KeyValue<>("ID123-1", "ID123-A1"); final KeyValue eventError = new KeyValue<>("ID123-ERR-FATAL", "ID123-A2"); @@ -284,6 +285,7 @@ public void shouldStopProcessingWhenFatalUserExceptionInContinueProcessingExcept assertTrue(isExecuted.get()); isExecuted.set(false); final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); + assertEquals("Fatal user code error in processing error callback", e.getMessage()); assertEquals("KABOOM!", e.getCause().getMessage()); assertFalse(isExecuted.get()); } @@ -295,6 +297,9 @@ public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHa if (((String) record.key()).contains("FATAL")) { throw new RuntimeException("KABOOM!"); } + if (((String) record.key()).contains("NULL")) { + return null; + } assertProcessingExceptionHandlerInputs(context, record, exception); return ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE; } @@ -308,9 +313,6 @@ public void configure(final Map configs) { public static class FailProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { @Override public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - if (((String) record.key()).contains("FATAL")) { - throw new RuntimeException("KABOOM!"); - } assertProcessingExceptionHandlerInputs(context, record, exception); return ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index 735a7b7910..c4479bbcc5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -50,6 +50,8 @@ import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.ProductionExceptionHandler; +import org.apache.kafka.streams.errors.ProductionExceptionHandler.ProductionExceptionHandlerResponse; +import org.apache.kafka.streams.errors.ProductionExceptionHandler.SerializationExceptionOrigin; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.processor.StreamPartitioner; @@ -1240,7 +1242,7 @@ public void shouldThrowStreamsExceptionOnSubsequentSendIfFatalEvenWithContinueEx logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandlerResponse.CONTINUE)), streamsMetrics, topology ); @@ -1267,7 +1269,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfFatalEvenWithContinueE logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandlerResponse.CONTINUE)), streamsMetrics, topology ); @@ -1291,7 +1293,7 @@ public void shouldThrowStreamsExceptionOnSubsequentCloseIfFatalEvenWithContinueE logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandlerResponse.CONTINUE)), streamsMetrics, topology ); @@ -1315,7 +1317,7 @@ public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContin taskId, getExceptionalStreamsProducerOnSend(new RuntimeException("KABOOM!")), new ProductionExceptionHandlerMock( - ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE, + Optional.of(ProductionExceptionHandlerResponse.CONTINUE), context, sinkNodeName, taskId @@ -1489,11 +1491,11 @@ public void shouldThrowStreamsExceptionUsingDefaultExceptionHandler() { public void shouldDropRecordExceptionUsingAlwaysContinueExceptionHandler() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE, + Optional.of(ProductionExceptionHandlerResponse.CONTINUE), context, sinkNodeName, taskId, - ProductionExceptionHandler.SerializationExceptionOrigin.KEY + SerializationExceptionOrigin.KEY )); collector.initialize(); @@ -1518,11 +1520,11 @@ public void shouldDropRecordExceptionUsingAlwaysContinueExceptionHandler() { public void shouldThrowStreamsExceptionWhenValueSerializationFailedAndProductionExceptionHandlerRepliesWithFail() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL, + Optional.of(ProductionExceptionHandlerResponse.FAIL), context, sinkNodeName, taskId, - ProductionExceptionHandler.SerializationExceptionOrigin.VALUE + SerializationExceptionOrigin.VALUE )); collector.initialize(); @@ -1539,11 +1541,11 @@ public void shouldThrowStreamsExceptionWhenValueSerializationFailedAndProduction public void shouldThrowStreamsExceptionWhenKeySerializationFailedAndProductionExceptionHandlerRepliesWithFail() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL, + Optional.of(ProductionExceptionHandlerResponse.FAIL), context, sinkNodeName, taskId, - ProductionExceptionHandler.SerializationExceptionOrigin.KEY + SerializationExceptionOrigin.KEY )); collector.initialize(); @@ -1556,11 +1558,109 @@ public void shouldThrowStreamsExceptionWhenKeySerializationFailedAndProductionEx } } + @Test + public void shouldThrowStreamsExceptionWhenSerializationFailedAndProductionExceptionHandlerReturnsNull() { + try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { + final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( + Optional.empty(), + context, + sinkNodeName, + taskId, + SerializationExceptionOrigin.KEY + )); + collector.initialize(); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> collector.send(topic, "key", "val", null, 0, null, errorSerializer, stringSerializer, sinkNodeName, context) + ); + + assertEquals("Fatal user code error in production error callback", exception.getMessage()); + assertInstanceOf(NullPointerException.class, exception.getCause()); + assertEquals("Invalid ProductionExceptionHandler response.", exception.getCause().getMessage()); + } + } + + @Test + public void shouldThrowStreamsExceptionWhenSerializationFailedAndProductionExceptionHandlerThrows() { + try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { + final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( + true, + context, + sinkNodeName, + taskId, + SerializationExceptionOrigin.KEY + )); + collector.initialize(); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> collector.send(topic, "key", "val", null, 0, null, errorSerializer, stringSerializer, sinkNodeName, context) + ); + + assertEquals("Fatal user code error in production error callback", exception.getMessage()); + assertEquals("CRASH", exception.getCause().getMessage()); + } + } + + @Test + public void shouldThrowStreamsExceptionOnSubsequentFlushIfASendFailsAndProductionExceptionHandlerReturnsNull() { + final KafkaException exception = new KafkaException("KABOOM!"); + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + getExceptionalStreamsProducerOnSend(exception), + new ProductionExceptionHandlerMock( + Optional.empty(), + context, + sinkNodeName, + taskId, + SerializationExceptionOrigin.KEY + ), + streamsMetrics, + topology + ); + + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); + + final StreamsException thrown = assertThrows(StreamsException.class, collector::flush); + assertEquals("Fatal user code error in production error callback", thrown.getMessage()); + assertInstanceOf(NullPointerException.class, thrown.getCause()); + assertEquals("Invalid ProductionExceptionHandler response.", thrown.getCause().getMessage()); + } + + @Test + public void shouldThrowStreamsExceptionOnSubsequentFlushIfASendFailsAndProductionExceptionHandlerThrows() { + final KafkaException exception = new KafkaException("KABOOM!"); + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + getExceptionalStreamsProducerOnSend(exception), + new ProductionExceptionHandlerMock( + true, + context, + sinkNodeName, + taskId, + SerializationExceptionOrigin.KEY + ), + streamsMetrics, + topology + ); + + collector.send(topic, "3", "0", null, null, stringSerializer, stringSerializer, sinkNodeName, context, streamPartitioner); + + final StreamsException thrown = assertThrows(StreamsException.class, collector::flush); + assertEquals("Fatal user code error in production error callback", thrown.getMessage()); + assertEquals("CRASH", thrown.getCause().getMessage()); + } + @SuppressWarnings({"unchecked", "rawtypes"}) @Test public void shouldNotCallProductionExceptionHandlerOnClassCastException() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { - final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock(ProductionExceptionHandler.ProductionExceptionHandlerResponse.CONTINUE)); + final RecordCollector collector = newRecordCollector( + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandlerResponse.CONTINUE)) + ); collector.initialize(); assertThat(mockProducer.history().isEmpty(), equalTo(true)); @@ -1716,17 +1816,18 @@ public byte[] serialize(final String topic, final Headers headers, final String } public static class ProductionExceptionHandlerMock implements ProductionExceptionHandler { - private final ProductionExceptionHandlerResponse response; + private final Optional response; + private boolean shouldThrowException; private InternalProcessorContext expectedContext; private String expectedProcessorNodeId; private TaskId expectedTaskId; private SerializationExceptionOrigin expectedSerializationExceptionOrigin; - public ProductionExceptionHandlerMock(final ProductionExceptionHandlerResponse response) { + public ProductionExceptionHandlerMock(final Optional response) { this.response = response; } - public ProductionExceptionHandlerMock(final ProductionExceptionHandlerResponse response, + public ProductionExceptionHandlerMock(final Optional response, final InternalProcessorContext context, final String processorNodeId, final TaskId taskId) { @@ -1736,13 +1837,24 @@ public ProductionExceptionHandlerMock(final ProductionExceptionHandlerResponse r this.expectedTaskId = taskId; } - public ProductionExceptionHandlerMock(final ProductionExceptionHandlerResponse response, + public ProductionExceptionHandlerMock(final boolean shouldThrowException, + final InternalProcessorContext context, + final String processorNodeId, + final TaskId taskId, + final SerializationExceptionOrigin origin) { + this(Optional.empty(), context, processorNodeId, taskId); + this.expectedSerializationExceptionOrigin = origin; + this.shouldThrowException = shouldThrowException; + } + + public ProductionExceptionHandlerMock(final Optional response, final InternalProcessorContext context, final String processorNodeId, final TaskId taskId, final SerializationExceptionOrigin origin) { this(response, context, processorNodeId, taskId); this.expectedSerializationExceptionOrigin = origin; + this.shouldThrowException = false; } @Override @@ -1750,7 +1862,10 @@ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext conte final ProducerRecord record, final Exception exception) { assertInputs(context, exception); - return response; + if (shouldThrowException) { + throw new RuntimeException("CRASH"); + } + return response.orElse(null); } @Override @@ -1760,7 +1875,10 @@ public ProductionExceptionHandlerResponse handleSerializationException(final Err final SerializationExceptionOrigin origin) { assertInputs(context, exception); assertEquals(expectedSerializationExceptionOrigin, origin); - return response; + if (shouldThrowException) { + throw new RuntimeException("CRASH"); + } + return response.orElse(null); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index 23f364fc6a..1bca1c9e37 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.DeserializationExceptionHandler.DeserializationHandlerResponse; import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.TaskId; @@ -62,27 +63,29 @@ public class RecordDeserializerTest { @Test public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { - final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - sourceNodeName, - false, - false, - "key", - "value" - ), - null, - new LogContext(), - new Metrics().sensor("dropped-records") - ); - final ConsumerRecord record = recordDeserializer.deserialize(null, rawRecord); - assertEquals(rawRecord.topic(), record.topic()); - assertEquals(rawRecord.partition(), record.partition()); - assertEquals(rawRecord.offset(), record.offset()); - assertEquals("key", record.key()); - assertEquals("value", record.value()); - assertEquals(rawRecord.timestamp(), record.timestamp()); - assertEquals(TimestampType.CREATE_TIME, record.timestampType()); - assertEquals(rawRecord.headers(), record.headers()); + try (final Metrics metrics = new Metrics()) { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + sourceNodeName, + false, + false, + "key", + "value" + ), + null, + new LogContext(), + metrics.sensor("dropped-records") + ); + final ConsumerRecord record = recordDeserializer.deserialize(null, rawRecord); + assertEquals(rawRecord.topic(), record.topic()); + assertEquals(rawRecord.partition(), record.partition()); + assertEquals(rawRecord.offset(), record.offset()); + assertEquals("key", record.key()); + assertEquals("value", record.value()); + assertEquals(rawRecord.timestamp(), record.timestamp()); + assertEquals(TimestampType.CREATE_TIME, record.timestampType()); + assertEquals(rawRecord.headers(), record.headers()); + } } @ParameterizedTest @@ -93,30 +96,35 @@ public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { }) public void shouldThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandlerRepliesWithFail(final boolean keyThrowsException, final boolean valueThrowsException) { - final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - sourceNodeName, - keyThrowsException, - valueThrowsException, - "key", - "value" - ), - new DeserializationExceptionHandlerMock( - DeserializationExceptionHandler.DeserializationHandlerResponse.FAIL, - rawRecord, - sourceNodeName, - taskId - ), - new LogContext(), - new Metrics().sensor("dropped-records") - ); - - final StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); - assertEquals(e.getMessage(), "Deserialization exception handler is set " - + "to fail upon a deserialization error. " - + "If you would rather have the streaming pipeline " - + "continue after a deserialization error, please set the " - + "default.deserialization.exception.handler appropriately."); + try (final Metrics metrics = new Metrics()) { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + sourceNodeName, + keyThrowsException, + valueThrowsException, + "key", + "value" + ), + new DeserializationExceptionHandlerMock( + Optional.of(DeserializationHandlerResponse.FAIL), + rawRecord, + sourceNodeName, + taskId + ), + new LogContext(), + metrics.sensor("dropped-records") + ); + + final StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); + assertEquals( + e.getMessage(), + "Deserialization exception handler is set " + + "to fail upon a deserialization error. " + + "If you would rather have the streaming pipeline " + + "continue after a deserialization error, please set the " + + "default.deserialization.exception.handler appropriately." + ); + } } @ParameterizedTest @@ -127,26 +135,89 @@ public void shouldThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandl }) public void shouldNotThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandlerRepliesWithContinue(final boolean keyThrowsException, final boolean valueThrowsException) { - final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - sourceNodeName, - keyThrowsException, - valueThrowsException, - "key", - "value" - ), - new DeserializationExceptionHandlerMock( - DeserializationExceptionHandler.DeserializationHandlerResponse.CONTINUE, - rawRecord, - sourceNodeName, - taskId - ), - new LogContext(), - new Metrics().sensor("dropped-records") - ); - - final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); - assertNull(record); + try (final Metrics metrics = new Metrics()) { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + sourceNodeName, + keyThrowsException, + valueThrowsException, + "key", + "value" + ), + new DeserializationExceptionHandlerMock( + Optional.of(DeserializationHandlerResponse.CONTINUE), + rawRecord, + sourceNodeName, + taskId + ), + new LogContext(), + metrics.sensor("dropped-records") + ); + + final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); + assertNull(record); + } + } + + @Test + public void shouldFailWhenDeserializationFailsAndExceptionHandlerReturnsNull() { + try (final Metrics metrics = new Metrics()) { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + sourceNodeName, + true, + false, + "key", + "value" + ), + new DeserializationExceptionHandlerMock( + Optional.empty(), + rawRecord, + sourceNodeName, + taskId + ), + new LogContext(), + metrics.sensor("dropped-records") + ); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> recordDeserializer.deserialize(context, rawRecord) + ); + assertEquals("Fatal user code error in deserialization error callback", exception.getMessage()); + assertInstanceOf(NullPointerException.class, exception.getCause()); + assertEquals("Invalid DeserializationExceptionHandler response.", exception.getCause().getMessage()); + } + } + + @Test + public void shouldFailWhenDeserializationFailsAndExceptionHandlerThrows() { + try (final Metrics metrics = new Metrics()) { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + sourceNodeName, + true, + false, + "key", + "value" + ), + new DeserializationExceptionHandlerMock( + null, // indicate to throw an exception + rawRecord, + sourceNodeName, + taskId + ), + new LogContext(), + metrics.sensor("dropped-records") + ); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> recordDeserializer.deserialize(context, rawRecord) + ); + assertEquals("Fatal user code error in deserialization error callback", exception.getMessage()); + assertEquals("CRASH", exception.getCause().getMessage()); + } } static class TheSourceNode extends SourceNode { @@ -185,12 +256,12 @@ public Object deserializeValue(final String topic, final Headers headers, final } public static class DeserializationExceptionHandlerMock implements DeserializationExceptionHandler { - private final DeserializationHandlerResponse response; + private final Optional response; private final ConsumerRecord expectedRecord; private final String expectedProcessorNodeId; private final TaskId expectedTaskId; - public DeserializationExceptionHandlerMock(final DeserializationHandlerResponse response, + public DeserializationExceptionHandlerMock(final Optional response, final ConsumerRecord record, final String processorNodeId, final TaskId taskId) { @@ -212,7 +283,10 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, assertEquals(expectedRecord, record); assertInstanceOf(RuntimeException.class, exception); assertEquals("KABOOM!", exception.getMessage()); - return response; + if (response == null) { + throw new RuntimeException("CRASH"); + } + return response.orElse(null); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index a8771c2153..3fa33ef895 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -53,6 +53,7 @@ import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; import org.apache.kafka.streams.errors.LogAndFailProcessingExceptionHandler; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler.ProcessingHandlerResponse; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; @@ -2661,17 +2662,20 @@ public void shouldNotCheckpointAfterRestorationWhenExactlyOnceEnabled() { } @Test - public void shouldPunctuateNotHandleFailProcessingExceptionAndThrowStreamsException() { + public void punctuateShouldNotHandleFailProcessingExceptionAndThrowStreamsException() { when(stateManager.taskId()).thenReturn(taskId); when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", - LogAndFailExceptionHandler.class.getName(), LogAndContinueProcessingExceptionHandler.class.getName())); + task = createStatelessTask(createConfig( + AT_LEAST_ONCE, + "100", + LogAndFailExceptionHandler.class.getName(), + LogAndContinueProcessingExceptionHandler.class.getName() + )); - final StreamsException streamsException = assertThrows(StreamsException.class, () -> - task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { - throw new FailedProcessingException( - new RuntimeException("KABOOM!") - ); + final StreamsException streamsException = assertThrows( + StreamsException.class, + () -> task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + throw new FailedProcessingException(new RuntimeException("KABOOM!")); }) ); @@ -2680,11 +2684,15 @@ public void shouldPunctuateNotHandleFailProcessingExceptionAndThrowStreamsExcept } @Test - public void shouldPunctuateNotHandleTaskCorruptedExceptionAndThrowItAsIs() { + public void punctuateShouldNotHandleTaskCorruptedExceptionAndThrowItAsIs() { when(stateManager.taskId()).thenReturn(taskId); when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", - LogAndFailExceptionHandler.class.getName(), LogAndContinueProcessingExceptionHandler.class.getName())); + task = createStatelessTask(createConfig( + AT_LEAST_ONCE, + "100", + LogAndFailExceptionHandler.class.getName(), + LogAndContinueProcessingExceptionHandler.class.getName() + )); final Set tasksIds = new HashSet<>(); tasksIds.add(new TaskId(0, 0)); @@ -2695,8 +2703,9 @@ public Set partitions() { } }); - final TaskCorruptedException taskCorruptedException = assertThrows(TaskCorruptedException.class, () -> - task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + final TaskCorruptedException taskCorruptedException = assertThrows( + TaskCorruptedException.class, + () -> task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { throw expectedException; }) ); @@ -2705,16 +2714,21 @@ public Set partitions() { } @Test - public void shouldPunctuateNotHandleTaskMigratedExceptionAndThrowItAsIs() { + public void punctuateShouldNotHandleTaskMigratedExceptionAndThrowItAsIs() { when(stateManager.taskId()).thenReturn(taskId); when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", - LogAndFailExceptionHandler.class.getName(), LogAndContinueProcessingExceptionHandler.class.getName())); + task = createStatelessTask(createConfig( + AT_LEAST_ONCE, + "100", + LogAndFailExceptionHandler.class.getName(), + LogAndContinueProcessingExceptionHandler.class.getName() + )); final TaskMigratedException expectedException = new TaskMigratedException("TaskMigratedException", new RuntimeException("Task migrated cause")); - final TaskMigratedException taskCorruptedException = assertThrows(TaskMigratedException.class, () -> - task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + final TaskMigratedException taskCorruptedException = assertThrows( + TaskMigratedException.class, + () -> task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { throw expectedException; }) ); @@ -2723,56 +2737,106 @@ public void shouldPunctuateNotHandleTaskMigratedExceptionAndThrowItAsIs() { } @Test - public void shouldPunctuateNotThrowStreamsExceptionWhenProcessingExceptionHandlerRepliesWithContinue() { + public void punctuateShouldNotThrowStreamsExceptionWhenProcessingExceptionHandlerRepliesWithContinue() { when(stateManager.taskId()).thenReturn(taskId); when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", - LogAndFailExceptionHandler.class.getName(), LogAndContinueProcessingExceptionHandler.class.getName())); + task = createStatelessTask(createConfig( + AT_LEAST_ONCE, + "100", + LogAndFailExceptionHandler.class.getName(), + LogAndContinueProcessingExceptionHandler.class.getName() + )); - assertDoesNotThrow(() -> - task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { + throw new KafkaException("KABOOM!"); + }); + } + + @Test + public void punctuateShouldThrowStreamsExceptionWhenProcessingExceptionHandlerRepliesWithFail() { + when(stateManager.taskId()).thenReturn(taskId); + when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); + task = createStatelessTask(createConfig( + AT_LEAST_ONCE, + "100", + LogAndFailExceptionHandler.class.getName(), + LogAndFailProcessingExceptionHandler.class.getName() + )); + + final StreamsException streamsException = assertThrows( + StreamsException.class, + () -> task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { throw new KafkaException("KABOOM!"); }) ); + + assertInstanceOf(KafkaException.class, streamsException.getCause()); + assertEquals("KABOOM!", streamsException.getCause().getMessage()); } @Test - public void shouldPunctuateThrowStreamsExceptionWhenProcessingExceptionHandlerRepliesWithFail() { + public void punctuateShouldThrowStreamsExceptionWhenProcessingExceptionHandlerReturnsNull() { when(stateManager.taskId()).thenReturn(taskId); when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", - LogAndFailExceptionHandler.class.getName(), LogAndFailProcessingExceptionHandler.class.getName())); + task = createStatelessTask(createConfig( + AT_LEAST_ONCE, + "100", + LogAndFailExceptionHandler.class.getName(), + NullProcessingExceptionHandler.class.getName() + )); - final StreamsException streamsException = assertThrows(StreamsException.class, + final StreamsException streamsException = assertThrows( + StreamsException.class, () -> task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { throw new KafkaException("KABOOM!"); - })); + }) + ); - assertInstanceOf(KafkaException.class, streamsException.getCause()); - assertEquals("KABOOM!", streamsException.getCause().getMessage()); + assertEquals("Fatal user code error in processing error callback", streamsException.getMessage()); + assertInstanceOf(NullPointerException.class, streamsException.getCause()); + assertEquals("Invalid ProcessingExceptionHandler response.", streamsException.getCause().getMessage()); } @Test - public void shouldPunctuateThrowFailedProcessingExceptionWhenProcessingExceptionHandlerThrowsAnException() { + public void punctuateShouldThrowFailedProcessingExceptionWhenProcessingExceptionHandlerThrowsAnException() { when(stateManager.taskId()).thenReturn(taskId); when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - task = createStatelessTask(createConfig(AT_LEAST_ONCE, "100", - LogAndFailExceptionHandler.class.getName(), ProcessingExceptionHandlerMock.class.getName())); + task = createStatelessTask(createConfig( + AT_LEAST_ONCE, + "100", + LogAndFailExceptionHandler.class.getName(), + CrashingProcessingExceptionHandler.class.getName() + )); - final FailedProcessingException streamsException = assertThrows(FailedProcessingException.class, + final FailedProcessingException streamsException = assertThrows( + FailedProcessingException.class, () -> task.punctuate(processorStreamTime, 1, PunctuationType.STREAM_TIME, timestamp -> { throw new KafkaException("KABOOM!"); - })); + }) + ); - assertInstanceOf(RuntimeException.class, streamsException.getCause()); + assertEquals("Fatal user code error in processing error callback", streamsException.getMessage()); assertEquals("KABOOM from ProcessingExceptionHandlerMock!", streamsException.getCause().getMessage()); } - public static class ProcessingExceptionHandlerMock implements ProcessingExceptionHandler { + public static class CrashingProcessingExceptionHandler implements ProcessingExceptionHandler { @Override - public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { throw new RuntimeException("KABOOM from ProcessingExceptionHandlerMock!"); } + + @Override + public void configure(final Map configs) { + // No-op + } + } + + public static class NullProcessingExceptionHandler implements ProcessingExceptionHandler { + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + return null; + } + @Override public void configure(final Map configs) { // No-op From b65644c3e37c22417416647c3791cb2a2d718485 Mon Sep 17 00:00:00 2001 From: Kuan-Po Tseng Date: Mon, 5 Aug 2024 10:41:14 +0800 Subject: [PATCH 026/123] KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP (#16783) This pr support EarliestLocalSpec LatestTierSpec in GetOffsetShell, and add integration tests. Reviewers: Luke Chen , Chia-Ping Tsai , PoAn Yang --- build.gradle | 1 + checkstyle/import-control.xml | 2 + .../kafka/clients/admin/KafkaAdminClient.java | 2 +- .../kafka/clients/admin/OffsetSpec.java | 22 +-- .../clients/admin/KafkaAdminClientTest.java | 4 +- .../apache/kafka/tools/GetOffsetShell.java | 15 ++- .../tools/GetOffsetShellParsingTest.java | 8 ++ .../kafka/tools/GetOffsetShellTest.java | 127 +++++++++++++++++- 8 files changed, 158 insertions(+), 23 deletions(-) diff --git a/build.gradle b/build.gradle index 51f9659e58..becac56bf8 100644 --- a/build.gradle +++ b/build.gradle @@ -2141,6 +2141,7 @@ project(':tools') { testImplementation project(':connect:runtime') testImplementation project(':connect:runtime').sourceSets.test.output testImplementation project(':storage:storage-api').sourceSets.main.output + testImplementation project(':storage').sourceSets.test.output testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation libs.mockitoJunitJupiter // supports MockitoExtension diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index a5784ef935..3f8212f997 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -284,6 +284,8 @@ + + diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 8eb7fb4e8c..2f195489ad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -4862,7 +4862,7 @@ private static long getOffsetFromSpec(OffsetSpec offsetSpec) { return ListOffsetsRequest.MAX_TIMESTAMP; } else if (offsetSpec instanceof OffsetSpec.EarliestLocalSpec) { return ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP; - } else if (offsetSpec instanceof OffsetSpec.LatestTierSpec) { + } else if (offsetSpec instanceof OffsetSpec.LatestTieredSpec) { return ListOffsetsRequest.LATEST_TIERED_TIMESTAMP; } return ListOffsetsRequest.LATEST_TIMESTAMP; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java b/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java index 5b2fbb3e2e..68f94cc493 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/OffsetSpec.java @@ -27,7 +27,7 @@ public static class EarliestSpec extends OffsetSpec { } public static class LatestSpec extends OffsetSpec { } public static class MaxTimestampSpec extends OffsetSpec { } public static class EarliestLocalSpec extends OffsetSpec { } - public static class LatestTierSpec extends OffsetSpec { } + public static class LatestTieredSpec extends OffsetSpec { } public static class TimestampSpec extends OffsetSpec { private final long timestamp; @@ -73,20 +73,22 @@ public static OffsetSpec maxTimestamp() { } /** - * Used to retrieve the offset with the local log start offset, - * log start offset is the offset of a log above which reads are guaranteed to be served - * from the disk of the leader broker, when Tiered Storage is not enabled, it behaves the same - * as the earliest timestamp + * Used to retrieve the local log start offset. + * Local log start offset is the offset of a log above which reads + * are guaranteed to be served from the disk of the leader broker. + *
    + * Note: When tiered Storage is not enabled, it behaves the same as retrieving the earliest timestamp offset. */ - public static OffsetSpec earliestLocalSpec() { + public static OffsetSpec earliestLocal() { return new EarliestLocalSpec(); } /** - * Used to retrieve the offset with the highest offset of data stored in remote storage, - * and when Tiered Storage is not enabled, we won't return any offset (i.e. Unknown offset) + * Used to retrieve the highest offset of data stored in remote storage. + *
    + * Note: When tiered storage is not enabled, we will return unknown offset. */ - public static OffsetSpec latestTierSpec() { - return new LatestTierSpec(); + public static OffsetSpec latestTiered() { + return new LatestTieredSpec(); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 8d70e60fc0..dc74a17844 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -5864,7 +5864,7 @@ public void testListOffsetsEarliestLocalSpecMinVersion() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); - env.adminClient().listOffsets(Collections.singletonMap(tp0, OffsetSpec.earliestLocalSpec())); + env.adminClient().listOffsets(Collections.singletonMap(tp0, OffsetSpec.earliestLocal())); TestUtils.waitForCondition(() -> env.kafkaClient().requests().stream().anyMatch(request -> request.requestBuilder().apiKey().messageType == ApiMessageType.LIST_OFFSETS && request.requestBuilder().oldestAllowedVersion() == 9 @@ -5892,7 +5892,7 @@ public void testListOffsetsLatestTierSpecSpecMinVersion() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); - env.adminClient().listOffsets(Collections.singletonMap(tp0, OffsetSpec.latestTierSpec())); + env.adminClient().listOffsets(Collections.singletonMap(tp0, OffsetSpec.latestTiered())); TestUtils.waitForCondition(() -> env.kafkaClient().requests().stream().anyMatch(request -> request.requestBuilder().apiKey().messageType == ApiMessageType.LIST_OFFSETS && request.requestBuilder().oldestAllowedVersion() == 9 diff --git a/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java b/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java index 4ba0f6c3e3..60b78acd22 100644 --- a/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java +++ b/tools/src/main/java/org/apache/kafka/tools/GetOffsetShell.java @@ -132,7 +132,7 @@ public GetOffsetShellOptions(String[] args) throws TerseException { .ofType(String.class); timeOpt = parser.accepts("time", "timestamp of the offsets before that. [Note: No offset is returned, if the timestamp greater than recently committed record timestamp is given.]") .withRequiredArg() - .describedAs(" / -1 or latest / -2 or earliest / -3 or max-timestamp") + .describedAs(" / -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered") .ofType(String.class) .defaultsTo("latest"); commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client.") @@ -275,7 +275,8 @@ public Map fetchOffsets(GetOffsetShellOptions options) thr } } - private OffsetSpec parseOffsetSpec(String listOffsetsTimestamp) throws TerseException { + // visible for tseting + static OffsetSpec parseOffsetSpec(String listOffsetsTimestamp) throws TerseException { switch (listOffsetsTimestamp) { case "earliest": return OffsetSpec.earliest(); @@ -283,6 +284,10 @@ private OffsetSpec parseOffsetSpec(String listOffsetsTimestamp) throws TerseExce return OffsetSpec.latest(); case "max-timestamp": return OffsetSpec.maxTimestamp(); + case "earliest-local": + return OffsetSpec.earliestLocal(); + case "latest-tiered": + return OffsetSpec.latestTiered(); default: long timestamp; @@ -290,7 +295,7 @@ private OffsetSpec parseOffsetSpec(String listOffsetsTimestamp) throws TerseExce timestamp = Long.parseLong(listOffsetsTimestamp); } catch (NumberFormatException e) { throw new TerseException("Malformed time argument " + listOffsetsTimestamp + ". " + - "Please use -1 or latest / -2 or earliest / -3 or max-timestamp, or a specified long format timestamp"); + "Please use -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered, or a specified long format timestamp"); } if (timestamp == ListOffsetsRequest.EARLIEST_TIMESTAMP) { @@ -299,6 +304,10 @@ private OffsetSpec parseOffsetSpec(String listOffsetsTimestamp) throws TerseExce return OffsetSpec.latest(); } else if (timestamp == ListOffsetsRequest.MAX_TIMESTAMP) { return OffsetSpec.maxTimestamp(); + } else if (timestamp == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP) { + return OffsetSpec.earliestLocal(); + } else if (timestamp == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP) { + return OffsetSpec.latestTiered(); } else { return OffsetSpec.forTimestamp(timestamp); } diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellParsingTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellParsingTest.java index 3c4ef0894f..9e81c23f30 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellParsingTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellParsingTest.java @@ -22,6 +22,7 @@ import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -243,6 +244,13 @@ public void testInvalidTimeValue() { assertThrows(TerseException.class, () -> GetOffsetShell.execute("--bootstrap-server", "localhost:9092", "--time", "invalid")); } + @Test + public void testInvalidOffset() { + assertEquals("Malformed time argument foo. " + + "Please use -1 or latest / -2 or earliest / -3 or max-timestamp / -4 or earliest-local / -5 or latest-tiered, or a specified long format timestamp", + assertThrows(TerseException.class, () -> GetOffsetShell.parseOffsetSpec("foo")).getMessage()); + } + private TopicPartition getTopicPartition(String topic, Integer partition) { return new TopicPartition(topic, partition); } diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 2d588c6025..95007d7bf8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -17,8 +17,10 @@ package org.apache.kafka.tools; +import kafka.test.ClusterConfig; import kafka.test.ClusterInstance; import kafka.test.annotation.ClusterConfigProperty; +import kafka.test.annotation.ClusterTemplate; import kafka.test.annotation.ClusterTest; import kafka.test.annotation.ClusterTestDefaults; import kafka.test.junit.ClusterTestExtensions; @@ -31,23 +33,36 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.server.config.ServerLogConfigs; +import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig; +import org.apache.kafka.server.log.remote.storage.LocalTieredStorage; +import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; +import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Properties; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; +import static kafka.test.annotation.Type.CO_KRAFT; +import static kafka.test.annotation.Type.KRAFT; +import static kafka.test.annotation.Type.ZK; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -70,15 +85,41 @@ private String getTopicName(int i) { return "topic" + i; } + private String getRemoteLogStorageEnabledTopicName(int i) { + return "topicRLS" + i; + } + private void setUp() { + setupTopics(this::getTopicName, Collections.emptyMap()); + sendProducerRecords(this::getTopicName); + } + + private void setUpRemoteLogTopics() { + // In this method, we'll create 4 topics and produce records to the log like this: + // topicRLS1 -> 1 segment + // topicRLS2 -> 2 segments (1 local log segment + 1 segment in the remote storage) + // topicRLS3 -> 3 segments (1 local log segment + 2 segments in the remote storage) + // topicRLS4 -> 4 segments (1 local log segment + 3 segments in the remote storage) + Map rlsConfigs = new HashMap<>(); + rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); + rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); + rlsConfigs.put(TopicConfig.SEGMENT_BYTES_CONFIG, "100"); + setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); + sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); + } + + private void setupTopics(Function topicName, Map configs) { try (Admin admin = cluster.createAdminClient()) { List topics = new ArrayList<>(); - IntStream.range(0, topicCount + 1).forEach(i -> topics.add(new NewTopic(getTopicName(i), i, (short) 1))); + IntStream.range(0, topicCount + 1).forEach(i -> + topics.add(new NewTopic(topicName.apply(i), i, (short) 1).configs(configs))); admin.createTopics(topics); } + } + private void sendProducerRecords(Function topicName) { Properties props = new Properties(); props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); @@ -86,15 +127,34 @@ private void setUp() { try (KafkaProducer producer = new KafkaProducer<>(props)) { IntStream.range(0, topicCount + 1) - .forEach(i -> IntStream.range(0, i * i) - .forEach(msgCount -> { - assertDoesNotThrow(() -> producer.send( - new ProducerRecord<>(getTopicName(i), msgCount % i, null, "val" + msgCount)).get()); - }) - ); + .forEach(i -> IntStream.range(0, i * i) + .forEach(msgCount -> assertDoesNotThrow(() -> producer.send( + new ProducerRecord<>(topicName.apply(i), msgCount % i, null, "val" + msgCount)).get()))); } } + private static List withRemoteStorage() { + Map serverProperties = new HashMap<>(); + serverProperties.put(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX + TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP, "1"); + serverProperties.put(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX + TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP, "1"); + serverProperties.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true"); + serverProperties.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, LocalTieredStorage.class.getName()); + serverProperties.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, "1000"); + serverProperties.put(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG, "1000"); + serverProperties.put(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, "100"); + serverProperties.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP, "EXTERNAL"); + + return Collections.singletonList( + // we set REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP to EXTERNAL, so we need to + // align listener name here as KafkaClusterTestKit (KRAFT/CO_KRAFT) the default + // broker listener name is EXTERNAL while in ZK it is PLAINTEXT + ClusterConfig.defaultBuilder() + .setTypes(Stream.of(ZK, KRAFT, CO_KRAFT).collect(Collectors.toSet())) + .setServerProperties(serverProperties) + .setListenerName("EXTERNAL") + .build()); + } + private void createConsumerAndPoll() { Properties props = new Properties(); props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); @@ -274,6 +334,59 @@ public void testGetOffsetsByMaxTimestamp() { } } + @ClusterTemplate("withRemoteStorage") + public void testGetOffsetsByEarliestLocalSpec() throws InterruptedException { + setUp(); + setUpRemoteLogTopics(); + + for (String time : new String[] {"-4", "earliest-local"}) { + // test topics disable remote log storage + // as remote log disabled, broker return the same result as earliest offset + TestUtils.waitForCondition(() -> + Arrays.asList( + new Row("topic1", 0, 0L), + new Row("topic2", 0, 0L), + new Row("topic3", 0, 0L), + new Row("topic4", 0, 0L)) + .equals(executeAndParse("--topic-partitions", "topic\\d+.*:0", "--time", time)), + "testGetOffsetsByEarliestLocalSpec get topics with remote log disabled result not match"); + + // test topics enable remote log storage + TestUtils.waitForCondition(() -> + Arrays.asList( + new Row("topicRLS1", 0, 0L), + new Row("topicRLS2", 0, 1L), + new Row("topicRLS3", 0, 2L), + new Row("topicRLS4", 0, 3L)) + .equals(executeAndParse("--topic-partitions", "topicRLS.*:0", "--time", time)), + "testGetOffsetsByEarliestLocalSpec get topics with remote log enabled result not match"); + } + } + + @ClusterTemplate("withRemoteStorage") + public void testGetOffsetsByLatestTieredSpec() throws InterruptedException { + setUp(); + setUpRemoteLogTopics(); + + for (String time : new String[] {"-5", "latest-tiered"}) { + // test topics disable remote log storage + // as remote log not enabled, broker return unknown offset for each topic partition and these + // unknown offsets are ignored by GetOffsetShell hence we have empty result here. + assertEquals(Collections.emptyList(), + executeAndParse("--topic-partitions", "topic\\d+:0", "--time", time)); + + // test topics enable remote log storage + // topicRLS1 has no result because there's no log segments being uploaded to the remote storage + TestUtils.waitForCondition(() -> + Arrays.asList( + new Row("topicRLS2", 0, 0L), + new Row("topicRLS3", 0, 1L), + new Row("topicRLS4", 0, 2L)) + .equals(executeAndParse("--topic-partitions", "topicRLS.*:0", "--time", time)), + "testGetOffsetsByLatestTieredSpec result not match"); + } + } + @ClusterTest public void testGetOffsetsByTimestamp() { setUp(); From c7d02127b19612b606f1b85f2ac88e82725121fb Mon Sep 17 00:00:00 2001 From: Josep Prat Date: Mon, 5 Aug 2024 09:51:46 +0200 Subject: [PATCH 027/123] KAFKA-17227: Update zstd-jni lib (#16763) * KAFKA-17227: Update zstd-jni lib * Add note in upgrade docs * Change zstd-jni version in docker native file and add warning in dependencies.gradle file * Add reference to snappy in upgrade Reviewers: Chia-Ping Tsai , Mickael Maison --- LICENSE-binary | 2 +- .../native/native-image-configs/resource-config.json | 4 ++-- docs/upgrade.html | 11 +++++++++++ gradle/dependencies.gradle | 3 ++- 4 files changed, 16 insertions(+), 4 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 0ca280efed..817bc3d6e5 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -327,7 +327,7 @@ pcollections-4.0.1, see: licenses/pcollections-MIT --------------------------------------- BSD 2-Clause -zstd-jni-1.5.6-3 see: licenses/zstd-jni-BSD-2-clause +zstd-jni-1.5.6-4 see: licenses/zstd-jni-BSD-2-clause --------------------------------------- BSD 3-Clause diff --git a/docker/native/native-image-configs/resource-config.json b/docker/native/native-image-configs/resource-config.json index 121f24b637..324d979e8d 100644 --- a/docker/native/native-image-configs/resource-config.json +++ b/docker/native/native-image-configs/resource-config.json @@ -25,9 +25,9 @@ }, { "pattern":"\\Qkafka/kafka-version.properties\\E" }, { - "pattern":"\\Qlinux/amd64/libzstd-jni-1.5.6-3.so\\E" + "pattern":"\\Qlinux/amd64/libzstd-jni-1.5.6-4.so\\E" }, { - "pattern":"\\Qlinux/aarch64/libzstd-jni-1.5.6-3.so\\E" + "pattern":"\\Qlinux/aarch64/libzstd-jni-1.5.6-4.so\\E" }, { "pattern":"\\Qnet/jpountz/util/linux/amd64/liblz4-java.so\\E" }, { diff --git a/docs/upgrade.html b/docs/upgrade.html index be12f0441d..5c0387460b 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -19,6 +19,17 @@

    diff --git a/docs/upgrade.html b/docs/upgrade.html index 5c0387460b..95e94515a2 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -28,6 +28,10 @@
    Notable changes in 3 For example, this could be done via the KAFKA_OPTS environment variable like follows: export KAFKA_OPTS="-DZstdTempFolder=/opt/kafka/tmp -Dorg.xerial.snappy.tempdir=/opt/kafka/tmp". This is a known issue for version 3.8.0 as well. +
  • In KRaft mode, the tiered storage feature can be dynamically disabled and then re-enabled on topic level. + See KIP-950 for more details.
  • +
  • Tiered storage quota is implemented. Users can set an "upper bound" on the rate at which logs are copied/read to/from the remote storage. + See KIP-956 for more details.
  • Upgrading to 3.8.0 from any version 0.8.x through 3.7.x

    From 32346c646bd90908e13329806b85e43c32d6bf9e Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Tue, 13 Aug 2024 20:44:20 +0800 Subject: [PATCH 047/123] KAFKA-17319 change ListOffsetsRequest latestVersionUnstable to false (#16865) Reviewers: Luke Chen , PoAn Yang , TengYao Chi , Chia-Ping Tsai --- .../src/main/resources/common/message/ListOffsetsRequest.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/resources/common/message/ListOffsetsRequest.json b/clients/src/main/resources/common/message/ListOffsetsRequest.json index 2b30c974c9..e228476d7d 100644 --- a/clients/src/main/resources/common/message/ListOffsetsRequest.json +++ b/clients/src/main/resources/common/message/ListOffsetsRequest.json @@ -39,7 +39,7 @@ "validVersions": "0-9", "deprecatedVersions": "0", "flexibleVersions": "6+", - "latestVersionUnstable": true, + "latestVersionUnstable": false, "fields": [ { "name": "ReplicaId", "type": "int32", "versions": "0+", "entityType": "brokerId", "about": "The broker ID of the requester, or -1 if this request is being made by a normal consumer." }, From 41107041f3216cd476bb73d8524b4ef7963f49b3 Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Tue, 13 Aug 2024 21:06:20 +0800 Subject: [PATCH 048/123] KAFKA-17309 Fix flaky testCallFailWithUnsupportedVersionExceptionDoesNotHaveConcurrentModificationException (#16854) Reviewers: TengYao Chi , Chia-Ping Tsai --- .../clients/admin/KafkaAdminClientTest.java | 35 ++++++++++++++++--- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 96b243ca9c..fbae396830 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -287,6 +287,7 @@ import static org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.ReassignableTopicResponse; import static org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingPartitionReassignment; import static org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingTopicReassignment; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -7828,7 +7829,23 @@ public void testListClientMetricsResourcesNotSupported() { @Test public void testCallFailWithUnsupportedVersionExceptionDoesNotHaveConcurrentModificationException() throws InterruptedException { - try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + Cluster cluster = mockCluster(1, 0); + try (MockClient mockClient = new MockClient(Time.SYSTEM, new MockClient.MockMetadataUpdater() { + @Override + public List fetchNodes() { + return cluster.nodes(); + } + + @Override + public boolean isUpdateNeeded() { + return false; + } + + @Override + public void update(Time time, MockClient.MetadataUpdate update) { + throw new UnsupportedOperationException(); + } + })) { AdminMetadataManager metadataManager = mock(AdminMetadataManager.class); // first false result make sure LeastLoadedBrokerOrActiveKController#provide can go to requestUpdate @@ -7843,16 +7860,24 @@ public void testCallFailWithUnsupportedVersionExceptionDoesNotHaveConcurrentModi // avoid sending fetchMetadata request doReturn(1L).when(metadataManager).metadataFetchDelayMs(anyLong()); - env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + mockClient.setNodeApiVersions(NodeApiVersions.create()); try (KafkaAdminClient admin = KafkaAdminClient.createInternal( - new AdminClientConfig(Collections.emptyMap()), metadataManager, env.kafkaClient(), env.time())) { - admin.describeCluster(new DescribeClusterOptions().timeoutMs(1000)); + new AdminClientConfig(Collections.emptyMap()), metadataManager, mockClient, Time.SYSTEM)) { + DescribeClusterResult result = admin.describeCluster(new DescribeClusterOptions()); // make sure maybeDrainPendingCalls doesn't remove duplicate pending calls // the listNodes call will be added again in call.fail and remove one in maybeDrainPendingCalls - TestUtils.waitForCondition(() -> env.kafkaClient().inFlightRequestCount() != 0, + TestUtils.waitForCondition(() -> mockClient.inFlightRequestCount() != 0, "Timed out waiting for listNodes request"); + + // after handleUnsupportedVersionException, describe cluster use MetadataRequest + ClientRequest request = mockClient.requests().peek(); + assertEquals(ApiKeys.METADATA, request.apiKey()); + + // clear active external request + mockClient.respondToRequest(request, prepareMetadataResponse(cluster, Errors.NONE)); + assertEquals(cluster.clusterResource().clusterId(), assertDoesNotThrow(() -> result.clusterId().get())); } } } From 4ea3b6181a527d8cdc85d5216f5b73b34243566f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 14 Aug 2024 15:42:04 -0400 Subject: [PATCH 049/123] KAFKA-17304; Make RaftClient API for writing to log explicit (#16862) RaftClient API is changed to separate the batch accumulation (RaftClient#prepareAppend) from scheduling the append of accumulated batches (RaftClient#schedulePrepatedAppend) to the KRaft log. This change is needed to better match the controller's flow of replaying the generated records before replicating them. When the controller replay records it needs to know the offset associated with the record. To compute a table offset the KafkaClient needs to be aware of the records and their log position. The controller uses this new API by generated the cluster metadata records, compute their offset using RaftClient#prepareAppend, replay the records in the state machine, and finally allowing KRaft to append the records with RaftClient#schedulePreparedAppend. To implement this API the BatchAccumulator is changed to also support this access pattern. This is done by adding a drainOffset to the implementation. The batch accumulator is allowed to return any record and batch that is less than the drain offset. Lastly, this change also removes some functionality that is no longer needed like non-atomic appends and validation of the base offset. Reviewers: Colin Patrick McCabe , David Arthur --- .../scala/kafka/tools/TestRaftServer.scala | 3 +- .../controller/OffsetControlManager.java | 10 +- .../kafka/controller/QuorumController.java | 23 +- .../errors/EventHandlerExceptionInfo.java | 7 - .../kafka/metadata/util/BatchFileWriter.java | 6 +- .../controller/OffsetControlManagerTest.java | 2 +- ...uorumControllerMetricsIntegrationTest.java | 2 +- .../controller/QuorumControllerTest.java | 112 ----- .../errors/EventHandlerExceptionInfoTest.java | 23 - .../image/publisher/SnapshotEmitterTest.java | 10 +- .../apache/kafka/metalog/LocalLogManager.java | 77 +--- .../kafka/metalog/LocalLogManagerTest.java | 3 +- .../kafka/metalog/LocalLogManagerTestEnv.java | 19 +- .../apache/kafka/raft/KafkaRaftClient.java | 45 +- .../org/apache/kafka/raft/LeaderState.java | 1 - .../org/apache/kafka/raft/RaftClient.java | 68 +-- .../apache/kafka/raft/ReplicatedCounter.java | 3 +- .../errors/UnexpectedBaseOffsetException.java | 29 -- .../raft/internals/BatchAccumulator.java | 148 +++--- .../kafka/snapshot/RecordsSnapshotWriter.java | 4 +- .../raft/KafkaRaftClientSnapshotTest.java | 12 +- .../kafka/raft/KafkaRaftClientTest.java | 93 ++-- .../raft/internals/BatchAccumulatorTest.java | 424 +++++++++--------- 23 files changed, 455 insertions(+), 669 deletions(-) delete mode 100644 raft/src/main/java/org/apache/kafka/raft/errors/UnexpectedBaseOffsetException.java diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 06812f2795..549c4e130d 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -211,7 +211,8 @@ class TestRaftServer( ): Unit = { recordCount.incrementAndGet() try { - val offset = raftManager.client.scheduleAppend(leaderEpoch, List(payload).asJava) + val offset = raftManager.client.prepareAppend(leaderEpoch, List(payload).asJava) + raftManager.client.schedulePreparedAppend() pendingAppends.offer(PendingAppend(offset, currentTimeMs)) } catch { case e: NotLeaderException => diff --git a/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java index 4094c34e58..9a04301456 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java @@ -293,14 +293,14 @@ void handleCommitBatch(Batch batch) { * Called by the active controller after it has invoked scheduleAtomicAppend to schedule some * records to be written. * - * @param endOffset The offset of the last record that was written. + * @param lastOffset The offset of the last record that was written. */ - void handleScheduleAtomicAppend(long endOffset) { - this.nextWriteOffset = endOffset + 1; + void handleScheduleAppend(long lastOffset) { + this.nextWriteOffset = lastOffset + 1; - snapshotRegistry.getOrCreateSnapshot(endOffset); + snapshotRegistry.getOrCreateSnapshot(lastOffset); - metrics.setLastAppliedRecordOffset(endOffset); + metrics.setLastAppliedRecordOffset(lastOffset); // This is not truly the append timestamp. The KRaft client doesn't expose the append // time when scheduling a write. This is good enough because this is called right after diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 5632720d83..56cbb80d54 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -817,9 +817,10 @@ public void run() throws Exception { // succeed; if it does not, that's a fatal error. It is important to do this before // scheduling the record for Raft replication. int recordIndex = 0; - long nextWriteOffset = offsetControl.nextWriteOffset(); + long lastOffset = raftClient.prepareAppend(controllerEpoch, records); + long baseOffset = lastOffset - records.size() + 1; for (ApiMessageAndVersion message : records) { - long recordOffset = nextWriteOffset + recordIndex; + long recordOffset = baseOffset + recordIndex; try { replay(message.message(), Optional.empty(), recordOffset); } catch (Throwable e) { @@ -827,17 +828,14 @@ public void run() throws Exception { "record at offset %d on active controller, from the " + "batch with baseOffset %d", message.message().getClass().getSimpleName(), - recordOffset, nextWriteOffset); + recordOffset, baseOffset); throw fatalFaultHandler.handleFault(failureMessage, e); } recordIndex++; } - long nextEndOffset = nextWriteOffset - 1 + recordIndex; - raftClient.scheduleAtomicAppend(controllerEpoch, - OptionalLong.of(nextWriteOffset), - records); - offsetControl.handleScheduleAtomicAppend(nextEndOffset); - return nextEndOffset; + raftClient.schedulePreparedAppend(); + offsetControl.handleScheduleAppend(lastOffset); + return lastOffset; } ); op.processBatchEndOffset(offset); @@ -2345,13 +2343,6 @@ QuorumControllerMetrics controllerMetrics() { return controllerMetrics; } - // VisibleForTesting - void setNewNextWriteOffset(long newNextWriteOffset) { - appendControlEvent("setNewNextWriteOffset", () -> { - offsetControl.setNextWriteOffset(newNextWriteOffset); - }); - } - void handleUncleanBrokerShutdown(int brokerId, List records) { replicationControl.handleBrokerUncleanShutdown(brokerId, records); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/errors/EventHandlerExceptionInfo.java b/metadata/src/main/java/org/apache/kafka/controller/errors/EventHandlerExceptionInfo.java index 09848d0c2e..3a49e412ea 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/errors/EventHandlerExceptionInfo.java +++ b/metadata/src/main/java/org/apache/kafka/controller/errors/EventHandlerExceptionInfo.java @@ -18,12 +18,10 @@ package org.apache.kafka.controller.errors; import org.apache.kafka.common.errors.ApiException; -import org.apache.kafka.common.errors.NotControllerException; import org.apache.kafka.common.errors.PolicyViolationException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.raft.errors.UnexpectedBaseOffsetException; import org.apache.kafka.server.mutable.BoundedListTooLongException; import java.util.Objects; @@ -88,11 +86,6 @@ public static EventHandlerExceptionInfo fromInternal( return new EventHandlerExceptionInfo(false, false, internal, new PolicyViolationException("Unable to perform excessively large batch " + "operation.")); - } else if (internal instanceof UnexpectedBaseOffsetException) { - // The active controller picked the wrong end offset for its next batch. It must now - // fail over. This should be pretty rare. - return new EventHandlerExceptionInfo(false, true, internal, - new NotControllerException("Unexpected end offset. Controller will resign.")); } else if (internal instanceof InterruptedException) { // The controller event queue has been interrupted. This normally only happens during // a JUnit test that has hung. The test framework sometimes sends an InterruptException diff --git a/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java b/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java index 9466a4f3f7..9792efee72 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java @@ -34,7 +34,6 @@ import java.nio.file.StandardOpenOption; import java.util.Collections; import java.util.List; -import java.util.OptionalLong; import static org.apache.kafka.raft.KafkaRaftClient.MAX_BATCH_SIZE_BYTES; @@ -62,11 +61,11 @@ private BatchFileWriter( } public void append(ApiMessageAndVersion apiMessageAndVersion) { - batchAccumulator.append(0, Collections.singletonList(apiMessageAndVersion), OptionalLong.empty(), false); + batchAccumulator.append(0, Collections.singletonList(apiMessageAndVersion), false); } public void append(List messageBatch) { - batchAccumulator.append(0, messageBatch, OptionalLong.empty(), false); + batchAccumulator.append(0, messageBatch, false); } public void close() throws IOException { @@ -95,6 +94,7 @@ public static BatchFileWriter open(Path snapshotPath) throws IOException { 0, Integer.MAX_VALUE, MAX_BATCH_SIZE_BYTES, + Integer.MAX_VALUE, new BatchMemoryPool(5, MAX_BATCH_SIZE_BYTES), time, Compression.NONE, diff --git a/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java index 584060c346..8b646f4eac 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java @@ -138,7 +138,7 @@ public void testHandleCommitBatch() { public void testHandleScheduleAtomicAppend() { OffsetControlManager offsetControl = new OffsetControlManager.Builder().build(); - offsetControl.handleScheduleAtomicAppend(2000L); + offsetControl.handleScheduleAppend(2000L); assertEquals(2001L, offsetControl.nextWriteOffset()); assertEquals(2000L, offsetControl.metrics().lastAppliedRecordOffset()); assertEquals(-1L, offsetControl.lastStableOffset()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsIntegrationTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsIntegrationTest.java index 6f312f8512..08ac06d720 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsIntegrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerMetricsIntegrationTest.java @@ -109,7 +109,7 @@ public void testFailingOverIncrementsNewActiveControllerCount( } }); if (forceFailoverUsingLogLayer) { - controlEnv.activeController().setNewNextWriteOffset(123L); + logEnv.activeLogManager().get().throwOnNextAppend(); TestUtils.retryOnExceptionWithTimeout(30_000, () -> createTopics(controlEnv.activeController(), "test_", 1, 1) diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 4a2d0d7016..23f8bdbd66 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -114,7 +114,6 @@ import org.apache.kafka.test.TestUtils; import org.apache.kafka.timeline.SnapshotRegistry; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; @@ -168,8 +167,6 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -1087,115 +1084,6 @@ public void testEarlyControllerResults() throws Throwable { } } - @Disabled // TODO: need to fix leader election in LocalLog. - @Test - public void testMissingInMemorySnapshot() throws Exception { - int numBrokers = 3; - int numPartitions = 3; - String topicName = "topic-name"; - - try ( - LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). - build(); - QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). - build() - ) { - QuorumController controller = controlEnv.activeController(); - - Map brokerEpochs = registerBrokersAndUnfence(controller, numBrokers); - - // Create a lot of partitions - List partitions = IntStream - .range(0, numPartitions) - .mapToObj(partitionIndex -> new CreatableReplicaAssignment() - .setPartitionIndex(partitionIndex) - .setBrokerIds(Arrays.asList(0, 1, 2)) - ) - .collect(Collectors.toList()); - - Uuid topicId = controller.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData() - .setTopics(new CreatableTopicCollection(Collections.singleton(new CreatableTopic() - .setName(topicName) - .setNumPartitions(-1) - .setReplicationFactor((short) -1) - .setAssignments(new CreatableReplicaAssignmentCollection(partitions.iterator())) - ).iterator())), - Collections.singleton("foo")).get().topics().find(topicName).topicId(); - - // Create a lot of alter isr - List alterPartitions = IntStream - .range(0, numPartitions) - .mapToObj(partitionIndex -> { - PartitionRegistration partitionRegistration = controller.replicationControl().getPartition( - topicId, - partitionIndex - ); - - return new AlterPartitionRequestData.PartitionData() - .setPartitionIndex(partitionIndex) - .setLeaderEpoch(partitionRegistration.leaderEpoch) - .setPartitionEpoch(partitionRegistration.partitionEpoch) - .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(0, 1))); - }) - .collect(Collectors.toList()); - - AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() - .setTopicName(topicName); - topicData.partitions().addAll(alterPartitions); - - int leaderId = 0; - AlterPartitionRequestData alterPartitionRequest = new AlterPartitionRequestData() - .setBrokerId(leaderId) - .setBrokerEpoch(brokerEpochs.get(leaderId)); - alterPartitionRequest.topics().add(topicData); - - logEnv.logManagers().get(0).resignAfterNonAtomicCommit(); - - int oldClaimEpoch = controller.curClaimEpoch(); - assertThrows(ExecutionException.class, - () -> controller.alterPartition(ANONYMOUS_CONTEXT, new AlterPartitionRequest - .Builder(alterPartitionRequest, false).build((short) 0).data()).get()); - - // Wait for the controller to become active again - assertSame(controller, controlEnv.activeController()); - assertTrue( - oldClaimEpoch < controller.curClaimEpoch(), - String.format("oldClaimEpoch = %s, newClaimEpoch = %s", oldClaimEpoch, controller.curClaimEpoch()) - ); - - // Since the alterPartition partially failed we expect to see - // some partitions to still have 2 in the ISR. - int partitionsWithReplica2 = Utils.toList( - controller - .replicationControl() - .brokersToIsrs() - .partitionsWithBrokerInIsr(2) - ).size(); - int partitionsWithReplica0 = Utils.toList( - controller - .replicationControl() - .brokersToIsrs() - .partitionsWithBrokerInIsr(0) - ).size(); - - assertEquals(numPartitions, partitionsWithReplica0); - assertNotEquals(0, partitionsWithReplica2); - assertTrue( - partitionsWithReplica0 > partitionsWithReplica2, - String.format( - "partitionsWithReplica0 = %s, partitionsWithReplica2 = %s", - partitionsWithReplica0, - partitionsWithReplica2 - ) - ); - - testToImages(logEnv.allRecords()); - } - } - @Test public void testConfigResourceExistenceChecker() throws Throwable { try ( diff --git a/metadata/src/test/java/org/apache/kafka/controller/errors/EventHandlerExceptionInfoTest.java b/metadata/src/test/java/org/apache/kafka/controller/errors/EventHandlerExceptionInfoTest.java index d6bbdfbda4..f854456cfa 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/errors/EventHandlerExceptionInfoTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/errors/EventHandlerExceptionInfoTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.raft.errors.UnexpectedBaseOffsetException; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -60,11 +59,6 @@ public class EventHandlerExceptionInfoTest { new NotLeaderException("Append failed"), () -> OptionalInt.of(2)); - private static final EventHandlerExceptionInfo UNEXPECTED_END_OFFSET = - EventHandlerExceptionInfo.fromInternal( - new UnexpectedBaseOffsetException("Wanted base offset 3, but the next offset was 4"), - () -> OptionalInt.of(1)); - @Test public void testTopicExistsExceptionInfo() { assertEquals(new EventHandlerExceptionInfo(false, false, @@ -155,22 +149,6 @@ public void testNotLeaderExceptionFailureMessage() { NOT_LEADER.failureMessage(123, OptionalLong.empty(), true, 456L)); } - @Test - public void testUnexpectedBaseOffsetExceptionInfo() { - assertEquals(new EventHandlerExceptionInfo(false, true, - new UnexpectedBaseOffsetException("Wanted base offset 3, but the next offset was 4"), - new NotControllerException("Unexpected end offset. Controller will resign.")), - UNEXPECTED_END_OFFSET); - } - - @Test - public void testUnexpectedBaseOffsetFailureMessage() { - assertEquals("event failed with UnexpectedBaseOffsetException (treated as " + - "NotControllerException) at epoch 123 in 90 microseconds. Renouncing leadership " + - "and reverting to the last committed offset 456. Exception message: Wanted base offset 3, but the next offset was 4", - UNEXPECTED_END_OFFSET.failureMessage(123, OptionalLong.of(90L), true, 456L)); - } - @Test public void testFaultExceptionFailureMessage() { EventHandlerExceptionInfo faultExceptionInfo = EventHandlerExceptionInfo.fromInternal( @@ -189,7 +167,6 @@ public void testIsNotTimeoutException() { assertFalse(INTERRUPTED.isTimeoutException()); assertFalse(NULL_POINTER.isTimeoutException()); assertFalse(NOT_LEADER.isTimeoutException()); - assertFalse(UNEXPECTED_END_OFFSET.isTimeoutException()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java index 3c07b1447e..b92a1876e6 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java @@ -74,17 +74,13 @@ public OptionalInt nodeId() { } @Override - public long scheduleAppend(int epoch, List records) { + public long prepareAppend(int epoch, List records) { return 0; } @Override - public long scheduleAtomicAppend( - int epoch, - OptionalLong requiredEndOffset, - List records - ) { - return 0; + public void schedulePreparedAppend() { + // nothing to do } @Override diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index ec733b3ca1..2f24d3f966 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -29,8 +29,8 @@ import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.raft.errors.UnexpectedBaseOffsetException; import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.KRaftVersion; @@ -68,9 +68,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; - /** * The LocalLogManager is a test implementation that relies on the contents of memory. */ @@ -224,21 +221,18 @@ synchronized void unregisterLogManager(LocalLogManager logManager) { synchronized long tryAppend( int nodeId, int epoch, - OptionalLong requiredBaseOffset, List batch ) { // No easy access to the concept of time. Use the base offset as the append timestamp long appendTimestamp = (prevOffset + 1) * 10; return tryAppend(nodeId, epoch, - requiredBaseOffset, new LocalRecordBatch(epoch, appendTimestamp, batch)); } synchronized long tryAppend( int nodeId, int epoch, - OptionalLong requiredBaseOffset, LocalBatch batch ) { if (!leader.isLeader(nodeId)) { @@ -256,22 +250,15 @@ synchronized long tryAppend( } log.trace("tryAppend(nodeId={}): appending {}.", nodeId, batch); - long offset = append(requiredBaseOffset, batch); + long offset = append(batch); electLeaderIfNeeded(); return offset; } public synchronized long append( - OptionalLong requiredBaseOffset, LocalBatch batch ) { long nextEndOffset = prevOffset + batch.size(); - requiredBaseOffset.ifPresent(r -> { - if (r != prevOffset + 1) { - throw new UnexpectedBaseOffsetException("Wanted base offset " + r + - ", but the next offset was " + nextEndOffset); - } - }); log.debug("append(batch={}, nextEndOffset={})", batch, nextEndOffset); batches.put(nextEndOffset, batch); if (batch instanceof LeaderChangeBatch) { @@ -297,7 +284,7 @@ synchronized void electLeaderIfNeeded() { } LeaderAndEpoch newLeader = new LeaderAndEpoch(OptionalInt.of(nextLeaderNode), leader.epoch() + 1); log.info("Elected new leader: {}.", newLeader); - append(OptionalLong.empty(), new LeaderChangeBatch(newLeader)); + append(new LeaderChangeBatch(newLeader)); } synchronized LeaderAndEpoch leaderAndEpoch() { @@ -505,11 +492,9 @@ void beginShutdown() { private volatile LeaderAndEpoch leader = new LeaderAndEpoch(OptionalInt.empty(), 0); /* - * If this variable is true the next non-atomic append with more than 1 record will - * result is half the records getting appended with leader election following that. - * This is done to emulate having some of the records not getting committed. + * If this variable is true the next scheduleAppend will fail */ - private final AtomicBoolean resignAfterNonAtomicCommit = new AtomicBoolean(false); + private final AtomicBoolean throwOnNextAppend = new AtomicBoolean(false); public LocalLogManager(LogContext logContext, int nodeId, @@ -729,52 +714,23 @@ public synchronized OptionalLong highWatermark() { } @Override - public long scheduleAppend(int epoch, List batch) { + public long prepareAppend( + int epoch, + List batch + ) { if (batch.isEmpty()) { throw new IllegalArgumentException("Batch cannot be empty"); } - List first = batch.subList(0, batch.size() / 2); - List second = batch.subList(batch.size() / 2, batch.size()); - - assertEquals(batch.size(), first.size() + second.size()); - assertFalse(second.isEmpty()); - - OptionalLong firstOffset = first - .stream() - .mapToLong(record -> scheduleAtomicAppend(epoch, - OptionalLong.empty(), - Collections.singletonList(record))) - .max(); - - if (firstOffset.isPresent() && resignAfterNonAtomicCommit.getAndSet(false)) { - // Emulate losing leadership in the middle of a non-atomic append by not writing - // the rest of the batch and instead writing a leader change message - resign(leader.epoch()); - - return firstOffset.getAsLong() + second.size(); - } else { - return second - .stream() - .mapToLong(record -> scheduleAtomicAppend(epoch, - OptionalLong.empty(), - Collections.singletonList(record))) - .max() - .getAsLong(); + if (throwOnNextAppend.getAndSet(false)) { + throw new BufferAllocationException("Test asked to fail the next prepareAppend"); } + + return shared.tryAppend(nodeId, leader.epoch(), batch); } @Override - public long scheduleAtomicAppend( - int epoch, - OptionalLong requiredEndOffset, - List batch - ) { - if (batch.isEmpty()) { - throw new IllegalArgumentException("Batch cannot be empty"); - } - return shared.tryAppend(nodeId, leader.epoch(), requiredEndOffset, batch); - } + public void schedulePreparedAppend() { } @Override public void resign(int epoch) { @@ -802,7 +758,6 @@ public void resign(int epoch) { try { shared.tryAppend(nodeId, currentEpoch, - OptionalLong.empty(), new LeaderChangeBatch(nextLeader)); } catch (NotLeaderException exp) { // the leader epoch has already advanced. resign is a no op. @@ -879,8 +834,8 @@ public void setMaxReadOffset(long maxReadOffset) { } } - public void resignAfterNonAtomicCommit() { - resignAfterNonAtomicCommit.set(true); + public void throwOnNextAppend() { + throwOnNextAppend.set(true); } @Override diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java index a37c453073..3e1c77181d 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java @@ -145,7 +145,8 @@ public void testCommits() throws Exception { new ApiMessageAndVersion(new RegisterBrokerRecord().setBrokerId(0), (short) 0), new ApiMessageAndVersion(new RegisterBrokerRecord().setBrokerId(1), (short) 0), new ApiMessageAndVersion(new RegisterBrokerRecord().setBrokerId(2), (short) 0)); - assertEquals(3, activeLogManager.scheduleAppend(epoch, messages)); + assertEquals(3, activeLogManager.prepareAppend(epoch, messages)); + activeLogManager.schedulePreparedAppend(); for (LocalLogManager logManager : env.logManagers()) { waitForLastCommittedOffset(3, logManager); } diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java index 287a7b0251..dd6ba56a3b 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java @@ -37,7 +37,6 @@ import java.util.List; import java.util.Optional; import java.util.OptionalInt; -import java.util.OptionalLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -154,12 +153,9 @@ public List allRecords() { */ public void appendInitialRecords(List records) { int initialLeaderEpoch = 1; - shared.append(OptionalLong.empty(), new LeaderChangeBatch( - new LeaderAndEpoch(OptionalInt.empty(), initialLeaderEpoch + 1))); - shared.append(OptionalLong.empty(), - new LocalRecordBatch(initialLeaderEpoch + 1, 0, records)); - shared.append(OptionalLong.empty(), - new LeaderChangeBatch(new LeaderAndEpoch(OptionalInt.of(0), initialLeaderEpoch + 2))); + shared.append(new LeaderChangeBatch(new LeaderAndEpoch(OptionalInt.empty(), initialLeaderEpoch + 1))); + shared.append(new LocalRecordBatch(initialLeaderEpoch + 1, 0, records)); + shared.append(new LeaderChangeBatch(new LeaderAndEpoch(OptionalInt.of(0), initialLeaderEpoch + 2))); } public String clusterId() { @@ -201,6 +197,15 @@ public List logManagers() { return logManagers; } + public Optional activeLogManager() { + OptionalInt leader = shared.leaderAndEpoch().leaderId(); + if (leader.isPresent()) { + return Optional.of(logManagers.get(leader.getAsInt())); + } else { + return Optional.empty(); + } + } + public RawSnapshotReader waitForSnapshot(long committedOffset) throws InterruptedException { return shared.waitForSnapshot(committedOffset); } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index ad650fadb5..b28686fc31 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -162,6 +162,7 @@ */ public final class KafkaRaftClient implements RaftClient { private static final int RETRY_BACKOFF_BASE_MS = 100; + private static final int MAX_NUMBER_OF_BATCHES = 10; public static final int MAX_FETCH_WAIT_MS = 500; public static final int MAX_BATCH_SIZE_BYTES = 8 * 1024 * 1024; public static final int MAX_FETCH_SIZE_BYTES = MAX_BATCH_SIZE_BYTES; @@ -608,6 +609,7 @@ private void onBecomeLeader(long currentTimeMs) { endOffset, quorumConfig.appendLingerMs(), MAX_BATCH_SIZE_BYTES, + MAX_NUMBER_OF_BATCHES, memoryPool, time, Compression.NONE, @@ -2828,7 +2830,8 @@ private long maybeAppendBatches( } } } - return timeUntilDrain; + + return state.accumulator().timeUntilDrain(currentTimeMs); } private long maybeSendBeginQuorumEpochRequests( @@ -3282,16 +3285,11 @@ public void poll() { } @Override - public long scheduleAppend(int epoch, List records) { - return append(epoch, records, OptionalLong.empty(), false); - } - - @Override - public long scheduleAtomicAppend(int epoch, OptionalLong requiredBaseOffset, List records) { - return append(epoch, records, requiredBaseOffset, true); + public long prepareAppend(int epoch, List records) { + return append(epoch, records); } - private long append(int epoch, List records, OptionalLong requiredBaseOffset, boolean isAtomic) { + private long append(int epoch, List records) { if (!isInitialized()) { throw new NotLeaderException("Append failed because the replica is not the current leader"); } @@ -3302,7 +3300,7 @@ private long append(int epoch, List records, OptionalLong requiredBaseOffset, BatchAccumulator accumulator = leaderState.accumulator(); boolean isFirstAppend = accumulator.isEmpty(); - final long offset = accumulator.append(epoch, records, requiredBaseOffset, isAtomic); + final long offset = accumulator.append(epoch, records, true); // Wakeup the network channel if either this is the first append // or the accumulator is ready to drain now. Checking for the first @@ -3315,6 +3313,24 @@ private long append(int epoch, List records, OptionalLong requiredBaseOffset, return offset; } + @Override + public void schedulePreparedAppend() { + if (!isInitialized()) { + throw new NotLeaderException("Flush failed because the replica is not the current leader"); + } + + LeaderState leaderState = quorum.maybeLeaderState().orElseThrow( + () -> new NotLeaderException("Flush failed because the replica is not the current leader") + ); + + leaderState.accumulator().allowDrain(); + + // Wakeup the network channel if the accumulator is ready to drain now. + if (leaderState.accumulator().needsDrain(time.milliseconds())) { + wakeup(); + } + } + @Override public CompletableFuture shutdown(int timeoutMs) { logger.info("Beginning graceful shutdown"); @@ -3608,11 +3624,10 @@ private void fireHandleCommit(long baseOffset, Records records) { } /** - * This API is used for committed records originating from {@link #scheduleAppend(int, List)} - * or {@link #scheduleAtomicAppend(int, OptionalLong, List)} on this instance. In this case, - * we are able to save the original record objects, which saves the need to read them back - * from disk. This is a nice optimization for the leader which is typically doing more work - * than all of the * followers. + * This API is used for committed records originating from {@link #prepareAppend(int, List)} + * on this instance. In this case, we are able to save the original record objects, which + * saves the need to read them back from disk. This is a nice optimization for the leader + * which is typically doing more work than all of the * followers. */ private void fireHandleCommit( long baseOffset, diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index df1cc6e142..5c468fc995 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -372,7 +372,6 @@ public void appendStartOfEpochControlRecords(VoterSet.VoterNode localVoterNode, return builder.build(); } }); - accumulator.forceDrain(); } public long appendVotersRecord(VoterSet voters, long currentTimeMs) { diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index c09297a704..a7205b746c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -18,7 +18,6 @@ import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.raft.errors.UnexpectedBaseOffsetException; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; @@ -38,13 +37,10 @@ interface Listener { * after consuming the reader. * * Note that there is not a one-to-one correspondence between writes through - * {@link #scheduleAppend(int, List)} or {@link #scheduleAtomicAppend(int, OptionalLong, List)} - * and this callback. The Raft implementation is free to batch together the records - * from multiple append calls provided that batch boundaries are respected. Records - * specified through {@link #scheduleAtomicAppend(int, OptionalLong, List)} are guaranteed to be a - * subset of a batch provided by the {@link BatchReader}. Records specified through - * {@link #scheduleAppend(int, List)} are guaranteed to be in the same order but - * they can map to any number of batches provided by the {@link BatchReader}. + * {@link #prepareAppend(int, List)} and this callback. The Raft implementation is free to + * batch together the records from multiple append calls provided that batch boundaries are + * respected. Records specified through {@link #prepareAppend(int, List)} are guaranteed + * to be a subset of a batch provided by the {@link BatchReader}. * * @param reader reader instance which must be iterated and closed */ @@ -105,7 +101,7 @@ default void beginShutdown() {} * To distinguish from events that happened before the call to {@code unregister} and a future * call to {@code register}, different {@code Listener} instances must be used. * - * If the {@code Listener} provided was never registered then the unregistration is ignored. + * If the {@code Listener} provided was never registered then the unregistration is ignored. * * @param listener the listener to unregister */ @@ -132,12 +128,12 @@ default void beginShutdown() {} OptionalInt nodeId(); /** - * Append a list of records to the log. The write will be scheduled for some time - * in the future. There is no guarantee that appended records will be written to - * the log and eventually committed. While the order of the records is preserve, they can - * be appended to the log using one or more batches. Each record may be committed independently. - * If a record is committed, then all records scheduled for append during this epoch - * and prior to this record are also committed. + * Prepare a list of records to be appended to the log. + * + * This method will not write any records to the log. To have the KRaft implementation write + * records to the log, the {@code schedulePreparedAppend} method must be called. There is no + * guarantee that appended records will be written to the log and eventually committed. However, + * it is guaranteed that if any of the records become committed, then all of them will be. * * If the provided current leader epoch does not match the current epoch, which * is possible when the state machine has yet to observe the epoch change, then @@ -147,42 +143,26 @@ default void beginShutdown() {} * * @param epoch the current leader epoch * @param records the list of records to append - * @return the expected offset of the last record if append succeed - * @throws org.apache.kafka.common.errors.RecordBatchTooLargeException if the size of the records is greater than the maximum - * batch size; if this exception is throw none of the elements in records were - * committed + * @return the expected offset of the last record + * @throws org.apache.kafka.common.errors.RecordBatchTooLargeException if the size of the + * records is greater than the maximum batch size; if this exception is throw none of + * the elements in records were committed * @throws NotLeaderException if we are not the current leader or the epoch doesn't match the leader epoch - * @throws BufferAllocationException if we failed to allocate memory for the records + * @throws BufferAllocationException we failed to allocate memory for the records + * @throws IllegalStateException if the number of accumulated batches reaches the maximum + * number of batches */ - long scheduleAppend(int epoch, List records); + long prepareAppend(int epoch, List records); /** - * Append a list of records to the log. The write will be scheduled for some time - * in the future. There is no guarantee that appended records will be written to - * the log and eventually committed. However, it is guaranteed that if any of the - * records become committed, then all of them will be. - * - * If the provided current leader epoch does not match the current epoch, which - * is possible when the state machine has yet to observe the epoch change, then - * this method will throw an {@link NotLeaderException} to indicate the leader - * to resign its leadership. The state machine is expected to discard all - * uncommitted entries after observing an epoch change. + * Schedule for all of prepared batches to get appended to the log. * - * If the current base offset does not match the supplied required base offset, - * then this method will throw {@link UnexpectedBaseOffsetException}. + * Any batches previously prepared for append with {@code prepareAppend(int List)} will be + * scheduled to get appended to the log. * - * @param epoch the current leader epoch - * @param requiredBaseOffset if this is set, it is the offset we must use as the base offset. - * @param records the list of records to append - * @return the expected offset of the last record if append succeed - * @throws org.apache.kafka.common.errors.RecordBatchTooLargeException if the size of the records is greater than the maximum - * batch size; if this exception is throw none of the elements in records were - * committed - * @throws NotLeaderException if we are not the current leader or the epoch doesn't match the leader epoch - * @throws BufferAllocationException we failed to allocate memory for the records - * @throws UnexpectedBaseOffsetException the requested base offset could not be obtained. + * @throws NotLeaderException if we are not the current leader */ - long scheduleAtomicAppend(int epoch, OptionalLong requiredBaseOffset, List records); + void schedulePreparedAppend(); /** * Attempt a graceful shutdown of the client. This allows the leader to proactively diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index 2758481ef6..4b35d9e755 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -65,7 +65,8 @@ public synchronized void increment() { int epoch = claimedEpoch.getAsInt(); uncommitted += 1; try { - long offset = client.scheduleAppend(epoch, singletonList(uncommitted)); + long offset = client.prepareAppend(epoch, singletonList(uncommitted)); + client.schedulePreparedAppend(); log.debug("Scheduled append of record {} with epoch {} at offset {}", uncommitted, epoch, offset); } catch (NotLeaderException e) { diff --git a/raft/src/main/java/org/apache/kafka/raft/errors/UnexpectedBaseOffsetException.java b/raft/src/main/java/org/apache/kafka/raft/errors/UnexpectedBaseOffsetException.java deleted file mode 100644 index ccf34de404..0000000000 --- a/raft/src/main/java/org/apache/kafka/raft/errors/UnexpectedBaseOffsetException.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.raft.errors; - -/** - * Indicates that an append operation cannot be completed because it would have resulted in an - * unexpected base offset. - */ -public class UnexpectedBaseOffsetException extends RaftException { - private static final long serialVersionUID = 1L; - - public UnexpectedBaseOffsetException(String s) { - super(s); - } -} diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index ff4de9b4a5..89ea44d9e5 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -29,7 +29,6 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.raft.errors.UnexpectedBaseOffsetException; import org.apache.kafka.server.common.serialization.RecordSerde; import java.io.Closeable; @@ -60,18 +59,20 @@ MemoryRecords create( private final int epoch; private final Time time; - private final SimpleTimer lingerTimer; private final int lingerMs; private final int maxBatchSize; + private final int maxNumberOfBatches; private final Compression compression; private final MemoryPool memoryPool; - private final ReentrantLock appendLock; private final RecordSerde serde; - private final ConcurrentLinkedQueue> completed; - private volatile DrainStatus drainStatus; + private final SimpleTimer lingerTimer = new SimpleTimer(); + private final AtomicLong drainOffset = new AtomicLong(Long.MAX_VALUE); + private final ConcurrentLinkedQueue> completed = new ConcurrentLinkedQueue<>(); + private volatile DrainStatus drainStatus = DrainStatus.NONE; // These fields are protected by the append lock + private final ReentrantLock appendLock = new ReentrantLock(); private long nextOffset; private BatchBuilder currentBatch; @@ -84,6 +85,7 @@ public BatchAccumulator( long baseOffset, int lingerMs, int maxBatchSize, + int maxNumberOfBatches, MemoryPool memoryPool, Time time, Compression compression, @@ -92,46 +94,45 @@ public BatchAccumulator( this.epoch = epoch; this.lingerMs = lingerMs; this.maxBatchSize = maxBatchSize; + this.maxNumberOfBatches = maxNumberOfBatches; this.memoryPool = memoryPool; this.time = time; - this.lingerTimer = new SimpleTimer(); this.compression = compression; this.serde = serde; this.nextOffset = baseOffset; - this.drainStatus = DrainStatus.NONE; - this.completed = new ConcurrentLinkedQueue<>(); - this.appendLock = new ReentrantLock(); } /** * Append to the accumulator. * - * @param epoch The leader epoch to append at. - * @param records The records to append. - * @param requiredBaseOffset If this is non-empty, the base offset which we must use. - * @param isAtomic True if we should append the records as a single batch. - * @return The end offset. + * @param epoch the leader epoch to append at + * @param records the records to append + * @param delayDrain whether the records could be drained + * @return the offset of the last record * - * @throws NotLeaderException Indicates that an append operation cannot be completed - * because the provided leader epoch was too old. - * @throws IllegalArgumentException Indicates that an append operation cannot be completed - * because the provided leader epoch was too new. - * @throws UnexpectedBaseOffsetException Indicates that an append operation cannot - * be completed because it would have resulted - * in an unexpected base offset. + * @throws NotLeaderException indicates that an append operation cannot be completed because the + * provided leader epoch was too old + * @throws IllegalArgumentException indicates that an append operation cannot be completed + * because the provided leader epoch was too new + * @throws IllegalStateException if the number of accumulated batches reaches the maximum + * number of batches */ - public long append( - int epoch, - List records, - OptionalLong requiredBaseOffset, - boolean isAtomic - ) { + public long append(int epoch, List records, boolean delayDrain) { + int numberOfCompletedBatches = completed.size(); if (epoch < this.epoch) { throw new NotLeaderException("Append failed because the given epoch " + epoch + " is stale. " + "Current leader epoch = " + this.epoch()); } else if (epoch > this.epoch) { throw new IllegalArgumentException("Attempt to append from epoch " + epoch + " which is larger than the current epoch " + this.epoch); + } else if (numberOfCompletedBatches >= maxNumberOfBatches) { + throw new IllegalStateException( + String.format( + "Attempting to append records when the number of batches %s reached %s", + numberOfCompletedBatches, + maxNumberOfBatches + ) + ); } ObjectSerializationCache serializationCache = new ObjectSerializationCache(); @@ -139,28 +140,22 @@ public long append( appendLock.lock(); try { long lastOffset = nextOffset + records.size() - 1; - requiredBaseOffset.ifPresent(r -> { - if (r != nextOffset) { - throw new UnexpectedBaseOffsetException("Wanted base offset " + r + - ", but the next offset was " + nextOffset); - } - }); maybeCompleteDrain(); BatchBuilder batch = null; - if (isAtomic) { - batch = maybeAllocateBatch(records, serializationCache); + batch = maybeAllocateBatch(records, serializationCache); + if (batch == null) { + throw new BufferAllocationException("Append failed because we failed to allocate memory to write the batch"); } - for (T record : records) { - if (!isAtomic) { - batch = maybeAllocateBatch(Collections.singleton(record), serializationCache); - } - - if (batch == null) { - throw new BufferAllocationException("Append failed because we failed to allocate memory to write the batch"); - } + if (delayDrain) { + // The user asked to not drain these records. If the drainOffset is not already set, + // then set the record at the current end offset (nextOffset) as maximum offset + // that can be drained. + drainOffset.compareAndSet(Long.MAX_VALUE, nextOffset); + } + for (T record : records) { batch.appendRecord(record, serializationCache); } @@ -218,6 +213,13 @@ private void completeCurrentBatch() { currentBatch = null; } + /** + * Allows draining of all batches. + */ + public void allowDrain() { + drainOffset.set(Long.MAX_VALUE); + } + /** * Append a control batch from a supplied memory record. * @@ -453,7 +455,10 @@ public boolean needsDrain(long currentTimeMs) { * @return the delay in milliseconds before the next expected drain */ public long timeUntilDrain(long currentTimeMs) { - if (drainStatus == DrainStatus.FINISHED) { + boolean drainableBatches = Optional.ofNullable(completed.peek()) + .map(batch -> batch.drainable(drainOffset.get())) + .orElse(false); + if (drainableBatches) { return 0; } else { return lingerTimer.remainingMs(currentTimeMs); @@ -488,6 +493,10 @@ public int epoch() { * @return the list of completed batches */ public List> drain() { + return drain(drainOffset.get()); + } + + private List> drain(long drainOffset) { // Start the drain if it has not been started already if (drainStatus == DrainStatus.NONE) { drainStatus = DrainStatus.STARTED; @@ -505,19 +514,21 @@ public List> drain() { // If the drain has finished, then all of the batches will be completed if (drainStatus == DrainStatus.FINISHED) { drainStatus = DrainStatus.NONE; - return drainCompleted(); + return drainCompleted(drainOffset); } else { return Collections.emptyList(); } } - private List> drainCompleted() { - List> res = new ArrayList<>(completed.size()); + private List> drainCompleted(long drainOffset) { + List> res = new ArrayList<>(); while (true) { - CompletedBatch batch = completed.poll(); - if (batch == null) { + CompletedBatch batch = completed.peek(); + if (batch == null || !batch.drainable(drainOffset)) { return res; } else { + // The batch can be drained so remove the batch and add it to the result. + completed.poll(); res.add(batch); } } @@ -530,17 +541,16 @@ public boolean isEmpty() { return !lingerTimer.isRunning(); } - /** - * Get the number of completed batches which are ready to be drained. - * This does not include the batch that is currently being filled. - */ - public int numCompletedBatches() { - return completed.size(); - } - @Override public void close() { - List> unwritten = drain(); + // Acquire the lock so that drain is guaranteed to complete the current batch + appendLock.lock(); + List> unwritten; + try { + unwritten = drain(Long.MAX_VALUE); + } finally { + appendLock.unlock(); + } unwritten.forEach(CompletedBatch::release); } @@ -561,14 +571,14 @@ private CompletedBatch( MemoryPool pool, ByteBuffer initialBuffer ) { - Objects.requireNonNull(data.firstBatch(), "Expected memory records to contain one batch"); - this.baseOffset = baseOffset; this.records = Optional.of(records); this.numRecords = records.size(); this.data = data; this.pool = pool; this.initialBuffer = initialBuffer; + + validateContruction(); } private CompletedBatch( @@ -578,14 +588,24 @@ private CompletedBatch( MemoryPool pool, ByteBuffer initialBuffer ) { - Objects.requireNonNull(data.firstBatch(), "Expected memory records to contain one batch"); - this.baseOffset = baseOffset; this.records = Optional.empty(); this.numRecords = numRecords; this.data = data; this.pool = pool; this.initialBuffer = initialBuffer; + + validateContruction(); + } + + private void validateContruction() { + Objects.requireNonNull(data.firstBatch(), "Expected memory records to contain one batch"); + + if (numRecords <= 0) { + throw new IllegalArgumentException( + String.format("Completed batch must contain at least one record: %s", numRecords) + ); + } } public int sizeInBytes() { @@ -602,6 +622,10 @@ public long appendTimestamp() { // to return the LastContainedLogTimestamp of the SnapshotHeaderRecord return data.firstBatch().maxTimestamp(); } + + public boolean drainable(long drainOffset) { + return baseOffset + numRecords - 1 < drainOffset; + } } private static class SimpleTimer { diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index 375fd6b494..dd4cc3b2d0 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -36,7 +36,6 @@ import java.util.List; import java.util.Optional; -import java.util.OptionalLong; public final class RecordsSnapshotWriter implements SnapshotWriter { private final RawSnapshotWriter snapshot; @@ -59,6 +58,7 @@ private RecordsSnapshotWriter( 0, Integer.MAX_VALUE, maxBatchSize, + 10, // maxNumberOfBatches memoryPool, time, compression, @@ -109,7 +109,7 @@ public void append(List records) { throw new IllegalStateException(message); } - accumulator.append(snapshot.snapshotId().epoch(), records, OptionalLong.empty(), false); + accumulator.append(snapshot.snapshotId().epoch(), records, false); if (accumulator.needsDrain(time.milliseconds())) { appendBatches(accumulator.drain()); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index dac88b1b9c..0b74ee69d3 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -329,7 +329,8 @@ public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws int epoch = context.currentEpoch(); List appendRecords = Arrays.asList("a", "b", "c"); - context.client.scheduleAppend(epoch, appendRecords); + context.client.prepareAppend(epoch, appendRecords); + context.client.schedulePreparedAppend(); context.time.sleep(context.appendLingerMs()); context.client.poll(); @@ -378,7 +379,8 @@ public void testFetchRequestOffsetAtZero(boolean withKip853Rpc) throws Exception int epoch = context.currentEpoch(); List appendRecords = Arrays.asList("a", "b", "c"); - context.client.scheduleAppend(epoch, appendRecords); + context.client.prepareAppend(epoch, appendRecords); + context.client.schedulePreparedAppend(); context.time.sleep(context.appendLingerMs()); context.client.poll(); @@ -439,7 +441,8 @@ public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) th } context.client.poll(); - context.client.scheduleAppend(epoch, Arrays.asList("g", "h", "i")); + context.client.prepareAppend(epoch, Arrays.asList("g", "h", "i")); + context.client.schedulePreparedAppend(); context.time.sleep(context.appendLingerMs()); context.client.poll(); @@ -1922,7 +1925,8 @@ public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rp context.advanceLocalLeaderHighWatermarkToLogEndOffset(); // append some more records to make the LEO > high watermark List newRecords = Arrays.asList("d", "e", "f"); - context.client.scheduleAppend(currentEpoch, newRecords); + context.client.prepareAppend(currentEpoch, newRecords); + context.client.schedulePreparedAppend(); context.time.sleep(context.appendLingerMs()); context.client.poll(); assertEquals(context.log.endOffset().offset(), context.client.highWatermark().getAsLong() + newRecords.size()); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 6ac11baf57..a783ccdc7c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -40,7 +40,6 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.raft.errors.UnexpectedBaseOffsetException; import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.raft.internals.VoterSetTest; @@ -341,7 +340,7 @@ public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) assertEquals(0L, context.log.endOffset().offset()); context.assertElectedLeader(epoch, localId); context.client.poll(); - assertThrows(NotLeaderException.class, () -> context.client.scheduleAppend(epoch, Arrays.asList("a", "b"))); + assertThrows(NotLeaderException.class, () -> context.client.prepareAppend(epoch, Arrays.asList("a", "b"))); context.pollUntilRequest(); RaftRequest.Outbound request = context.assertSentEndQuorumEpochRequest(epoch, remoteId); @@ -371,7 +370,7 @@ public void testAppendFailedWithNotLeaderException(boolean withKip853Rpc) throws .withKip853Rpc(withKip853Rpc) .build(); - assertThrows(NotLeaderException.class, () -> context.client.scheduleAppend(epoch, Arrays.asList("a", "b"))); + assertThrows(NotLeaderException.class, () -> context.client.prepareAppend(epoch, Arrays.asList("a", "b"))); } @ParameterizedTest @@ -386,7 +385,7 @@ public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) // Return null when allocation error Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer) // Buffer for the leader message control record - .thenReturn(null); // Buffer for the scheduleAppend call + .thenReturn(null); // Buffer for the prepareAppend call RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withMemoryPool(memoryPool) @@ -397,7 +396,7 @@ public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); - assertThrows(BufferAllocationException.class, () -> context.client.scheduleAppend(epoch, singletonList("a"))); + assertThrows(BufferAllocationException.class, () -> context.client.prepareAppend(epoch, singletonList("a"))); Mockito.verify(memoryPool).release(buffer); } @@ -417,9 +416,9 @@ public void testAppendFailedWithFencedEpoch(boolean withKip853Rpc) throws Except int epoch = context.currentEpoch(); // Throws IllegalArgumentException on higher epoch - assertThrows(IllegalArgumentException.class, () -> context.client.scheduleAppend(epoch + 1, singletonList("a"))); + assertThrows(IllegalArgumentException.class, () -> context.client.prepareAppend(epoch + 1, singletonList("a"))); // Throws NotLeaderException on smaller epoch - assertThrows(NotLeaderException.class, () -> context.client.scheduleAppend(epoch - 1, singletonList("a"))); + assertThrows(NotLeaderException.class, () -> context.client.prepareAppend(epoch - 1, singletonList("a"))); } @ParameterizedTest @@ -442,8 +441,10 @@ public void testAppendFailedWithRecordBatchTooLargeException(boolean withKip853R for (int i = 0; i < size; i++) batchToLarge.add("a"); - assertThrows(RecordBatchTooLargeException.class, - () -> context.client.scheduleAtomicAppend(epoch, OptionalLong.empty(), batchToLarge)); + assertThrows( + RecordBatchTooLargeException.class, + () -> context.client.prepareAppend(epoch, batchToLarge) + ); } @ParameterizedTest @@ -1101,7 +1102,8 @@ public void testAccumulatorClearedAfterBecomingFollower(boolean withKip853Rpc) t assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); - assertEquals(1L, context.client.scheduleAppend(epoch, singletonList("a"))); + assertEquals(1L, context.client.prepareAppend(epoch, singletonList("a"))); + context.client.schedulePreparedAppend(); context.deliverRequest(context.beginEpochRequest(epoch + 1, otherNodeId)); context.pollUntilResponse(); @@ -1133,7 +1135,8 @@ public void testAccumulatorClearedAfterBecomingVoted(boolean withKip853Rpc) thro assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); - assertEquals(1L, context.client.scheduleAppend(epoch, singletonList("a"))); + assertEquals(1L, context.client.prepareAppend(epoch, singletonList("a"))); + context.client.schedulePreparedAppend(); context.deliverRequest( context.voteRequest(epoch + 1, otherNodeKey, epoch, context.log.endOffset().offset()) ); @@ -1166,7 +1169,8 @@ public void testAccumulatorClearedAfterBecomingUnattached(boolean withKip853Rpc) assertEquals(OptionalInt.of(localId), context.currentLeader()); int epoch = context.currentEpoch(); - assertEquals(1L, context.client.scheduleAppend(epoch, singletonList("a"))); + assertEquals(1L, context.client.prepareAppend(epoch, singletonList("a"))); + context.client.schedulePreparedAppend(); context.deliverRequest(context.voteRequest(epoch + 1, otherNodeKey, epoch, 0L)); context.pollUntilResponse(); @@ -1195,7 +1199,8 @@ public void testChannelWokenUpIfLingerTimeoutReachedWithoutAppend(boolean withKi assertEquals(1L, context.log.endOffset().offset()); int epoch = context.currentEpoch(); - assertEquals(1L, context.client.scheduleAppend(epoch, singletonList("a"))); + assertEquals(1L, context.client.prepareAppend(epoch, singletonList("a"))); + context.client.schedulePreparedAppend(); assertTrue(context.messageQueue.wakeupRequested()); context.client.poll(); @@ -1230,7 +1235,8 @@ public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip assertEquals(1L, context.log.endOffset().offset()); int epoch = context.currentEpoch(); - assertEquals(1L, context.client.scheduleAppend(epoch, singletonList("a"))); + assertEquals(1L, context.client.prepareAppend(epoch, singletonList("a"))); + context.client.schedulePreparedAppend(); assertTrue(context.messageQueue.wakeupRequested()); context.client.poll(); @@ -1238,7 +1244,8 @@ public void testChannelWokenUpIfLingerTimeoutReachedDuringAppend(boolean withKip assertEquals(OptionalLong.of(lingerMs), context.messageQueue.lastPollTimeoutMs()); context.time.sleep(lingerMs); - assertEquals(2L, context.client.scheduleAppend(epoch, singletonList("b"))); + assertEquals(2L, context.client.prepareAppend(epoch, singletonList("b"))); + context.client.schedulePreparedAppend(); assertTrue(context.messageQueue.wakeupRequested()); context.client.poll(); @@ -1514,7 +1521,8 @@ public void testListenerCommitCallbackAfterLeaderWrite(boolean withKip853Rpc) th assertEquals(OptionalLong.of(1L), context.client.highWatermark()); List records = Arrays.asList("a", "b", "c"); - long offset = context.client.scheduleAppend(epoch, records); + long offset = context.client.prepareAppend(epoch, records); + context.client.schedulePreparedAppend(); context.client.poll(); assertEquals(OptionalLong.of(0L), context.listener.lastCommitOffset()); @@ -2442,7 +2450,8 @@ public void testPurgatoryFetchSatisfiedByWrite(boolean withKip853Rpc) throws Exc // Append some records that can fulfill the Fetch request String[] appendRecords = new String[]{"a", "b", "c"}; - context.client.scheduleAppend(epoch, Arrays.asList(appendRecords)); + context.client.prepareAppend(epoch, Arrays.asList(appendRecords)); + context.client.schedulePreparedAppend(); context.client.poll(); MemoryRecords fetchedRecords = context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); @@ -2775,7 +2784,8 @@ public void testEndQuorumEpochSentBasedOnFetchOffset(boolean withKip853Rpc) thro context.assertSentFetchPartitionResponse(1L, epoch); // Append some records, so that the close follower will be able to advance further. - context.client.scheduleAppend(epoch, Arrays.asList("foo", "bar")); + context.client.prepareAppend(epoch, Arrays.asList("foo", "bar")); + context.client.schedulePreparedAppend(); context.client.poll(); context.deliverRequest(context.fetchRequest(epoch, closeFollower, 3L, epoch, 0)); @@ -2936,7 +2946,8 @@ public void testDescribeQuorumWithFollowers(boolean withKip853Rpc, boolean withB List records = Arrays.asList("foo", "bar"); long nextFetchOffset = fetchOffset + records.size(); - context.client.scheduleAppend(epoch, records); + context.client.prepareAppend(epoch, records); + context.client.schedulePreparedAppend(); context.client.poll(); context.time.sleep(100); @@ -3076,7 +3087,8 @@ public void testDescribeQuorumWithObserver(boolean withKip853Rpc, boolean withBo // Observer falls behind context.time.sleep(100); List records = Arrays.asList("foo", "bar"); - context.client.scheduleAppend(epoch, records); + context.client.prepareAppend(epoch, records); + context.client.schedulePreparedAppend(); context.client.poll(); context.deliverRequest(context.describeQuorumRequest()); @@ -3151,7 +3163,8 @@ public void testDescribeQuorumNonMonotonicFollowerFetch(boolean withKip853Rpc, b // Update HW to non-initial value context.time.sleep(100); List batch = Arrays.asList("foo", "bar"); - context.client.scheduleAppend(epoch, batch); + context.client.prepareAppend(epoch, batch); + context.client.schedulePreparedAppend(); context.client.poll(); long fetchOffset = withBootstrapSnapshot ? 5L : 3L; long followerFetchTime = context.time.milliseconds(); @@ -3509,7 +3522,8 @@ public void testLeaderAppendSingleMemberQuorum(boolean withKip853Rpc) throws Exc context.client.poll(); assertEquals(OptionalLong.of(1L), context.client.highWatermark()); - context.client.scheduleAppend(context.currentEpoch(), Arrays.asList(appendRecords)); + context.client.prepareAppend(context.currentEpoch(), Arrays.asList(appendRecords)); + context.client.schedulePreparedAppend(); // Then poll the appended data with leader change record context.client.poll(); @@ -3633,7 +3647,8 @@ public void testMetrics(boolean withKip853Rpc) throws Exception { assertEquals((double) 1L, getMetric(context.metrics, "log-end-offset").metricValue()); assertEquals((double) epoch, getMetric(context.metrics, "log-end-epoch").metricValue()); - context.client.scheduleAppend(epoch, Arrays.asList("a", "b", "c")); + context.client.prepareAppend(epoch, Arrays.asList("a", "b", "c")); + context.client.schedulePreparedAppend(); context.client.poll(); assertEquals((double) 4L, getMetric(context.metrics, "high-watermark").metricValue()); @@ -3949,7 +3964,8 @@ public void testReregistrationChangesListenerContext(boolean withKip853Rpc) thro context.client.unregister(secondListener); // Write to the log and show that the default listener gets updated... - assertEquals(10L, context.client.scheduleAppend(epoch, singletonList("a"))); + assertEquals(10L, context.client.prepareAppend(epoch, singletonList("a"))); + context.client.schedulePreparedAppend(); context.client.poll(); context.advanceLocalLeaderHighWatermarkToLogEndOffset(); context.pollUntil(() -> OptionalLong.of(10).equals(context.listener.lastCommitOffset())); @@ -4239,35 +4255,6 @@ private static KafkaMetric getMetric(final Metrics metrics, final String name) { return metrics.metrics().get(metrics.metricName(name, "raft-metrics")); } - @ParameterizedTest - @CsvSource({ "false,false", "false,true", "true,false", "true,true" }) - public void testAppendWithRequiredBaseOffset(boolean correctOffset, boolean withKip853Rpc) throws Exception { - int localId = randomReplicaId(); - int otherNodeId = localId + 1; - Set voters = Utils.mkSet(localId, otherNodeId); - - RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withKip853Rpc(withKip853Rpc) - .build(); - context.becomeLeader(); - assertEquals(OptionalInt.of(localId), context.currentLeader()); - int epoch = context.currentEpoch(); - - if (correctOffset) { - assertEquals(1L, context.client.scheduleAtomicAppend(epoch, - OptionalLong.of(1), - singletonList("a"))); - context.deliverRequest(context.beginEpochRequest(epoch + 1, otherNodeId)); - context.pollUntilResponse(); - } else { - assertThrows(UnexpectedBaseOffsetException.class, () -> { - context.client.scheduleAtomicAppend(epoch, - OptionalLong.of(2), - singletonList("a")); - }); - } - } - static ReplicaKey replicaKey(int id, boolean withDirectoryId) { Uuid directoryId = withDirectoryId ? Uuid.randomUuid() : ReplicaKey.NO_DIRECTORY_ID; return ReplicaKey.of(id, directoryId); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java index af2d5822ed..de1c83abdd 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java @@ -23,8 +23,6 @@ import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.Writable; -import org.apache.kafka.common.record.AbstractRecords; -import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.ControlRecordUtils; import org.apache.kafka.common.record.DefaultRecord; import org.apache.kafka.common.record.MemoryRecordsBuilder; @@ -33,25 +31,20 @@ import org.apache.kafka.common.utils.ByteUtils; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.errors.UnexpectedBaseOffsetException; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; -import java.util.OptionalLong; import java.util.concurrent.CountDownLatch; -import java.util.stream.Collectors; -import java.util.stream.Stream; import static java.util.Arrays.asList; import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -59,6 +52,7 @@ class BatchAccumulatorTest { private final MemoryPool memoryPool = Mockito.mock(MemoryPool.class); private final MockTime time = new MockTime(); private final StringSerde serde = new StringSerde(); + private final int maxNumberOfBatches = 10; private BatchAccumulator buildAccumulator( int leaderEpoch, @@ -71,6 +65,7 @@ private BatchAccumulator buildAccumulator( baseOffset, lingerMs, maxBatchSize, + maxNumberOfBatches, memoryPool, time, Compression.NONE, @@ -109,102 +104,98 @@ public void testLeaderChangeMessageWritten() { @Test public void testForceDrain() { - asList(APPEND, APPEND_ATOMIC).forEach(appender -> { - int leaderEpoch = 17; - long baseOffset = 157; - int lingerMs = 50; - int maxBatchSize = 512; + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 512; - Mockito.when(memoryPool.tryAllocate(maxBatchSize)) - .thenReturn(ByteBuffer.allocate(maxBatchSize)); + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(ByteBuffer.allocate(maxBatchSize)); - BatchAccumulator acc = buildAccumulator( - leaderEpoch, - baseOffset, - lingerMs, - maxBatchSize - ); + BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ); - List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); + List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); - // Append records - assertEquals(baseOffset, appender.call(acc, leaderEpoch, records.subList(0, 1))); - assertEquals(baseOffset + 2, appender.call(acc, leaderEpoch, records.subList(1, 3))); - assertEquals(baseOffset + 5, appender.call(acc, leaderEpoch, records.subList(3, 6))); - assertEquals(baseOffset + 7, appender.call(acc, leaderEpoch, records.subList(6, 8))); - assertEquals(baseOffset + 8, appender.call(acc, leaderEpoch, records.subList(8, 9))); + // Append records + assertEquals(baseOffset, acc.append(leaderEpoch, records.subList(0, 1), false)); + assertEquals(baseOffset + 2, acc.append(leaderEpoch, records.subList(1, 3), false)); + assertEquals(baseOffset + 5, acc.append(leaderEpoch, records.subList(3, 6), false)); + assertEquals(baseOffset + 7, acc.append(leaderEpoch, records.subList(6, 8), false)); + assertEquals(baseOffset + 8, acc.append(leaderEpoch, records.subList(8, 9), false)); - assertFalse(acc.needsDrain(time.milliseconds())); - acc.forceDrain(); - assertTrue(acc.needsDrain(time.milliseconds())); - assertEquals(0, acc.timeUntilDrain(time.milliseconds())); + assertFalse(acc.needsDrain(time.milliseconds())); + acc.forceDrain(); + assertTrue(acc.needsDrain(time.milliseconds())); + assertEquals(0, acc.timeUntilDrain(time.milliseconds())); - // Drain completed batches - List> batches = acc.drain(); + // Drain completed batches + List> batches = acc.drain(); - assertEquals(1, batches.size()); - assertFalse(acc.needsDrain(time.milliseconds())); - assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); + assertEquals(1, batches.size()); + assertFalse(acc.needsDrain(time.milliseconds())); + assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); - BatchAccumulator.CompletedBatch batch = batches.get(0); - assertEquals(records, batch.records.get()); - assertEquals(baseOffset, batch.baseOffset); - assertEquals(time.milliseconds(), batch.appendTimestamp()); - }); + BatchAccumulator.CompletedBatch batch = batches.get(0); + assertEquals(records, batch.records.get()); + assertEquals(baseOffset, batch.baseOffset); + assertEquals(time.milliseconds(), batch.appendTimestamp()); } @Test public void testForceDrainBeforeAppendLeaderChangeMessage() { - asList(APPEND, APPEND_ATOMIC).forEach(appender -> { - int leaderEpoch = 17; - long baseOffset = 157; - int lingerMs = 50; - int maxBatchSize = 512; - - Mockito.when(memoryPool.tryAllocate(maxBatchSize)) - .thenReturn(ByteBuffer.allocate(maxBatchSize)); - Mockito.when(memoryPool.tryAllocate(256)) - .thenReturn(ByteBuffer.allocate(256)); - - BatchAccumulator acc = buildAccumulator( - leaderEpoch, - baseOffset, - lingerMs, - maxBatchSize - ); + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 512; + + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(ByteBuffer.allocate(maxBatchSize)); + Mockito.when(memoryPool.tryAllocate(256)) + .thenReturn(ByteBuffer.allocate(256)); - List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); + BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ); - // Append records - assertEquals(baseOffset, appender.call(acc, leaderEpoch, records.subList(0, 1))); - assertEquals(baseOffset + 2, appender.call(acc, leaderEpoch, records.subList(1, 3))); - assertEquals(baseOffset + 5, appender.call(acc, leaderEpoch, records.subList(3, 6))); - assertEquals(baseOffset + 7, appender.call(acc, leaderEpoch, records.subList(6, 8))); - assertEquals(baseOffset + 8, appender.call(acc, leaderEpoch, records.subList(8, 9))); + List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); - assertFalse(acc.needsDrain(time.milliseconds())); + // Append records + assertEquals(baseOffset, acc.append(leaderEpoch, records.subList(0, 1), false)); + assertEquals(baseOffset + 2, acc.append(leaderEpoch, records.subList(1, 3), false)); + assertEquals(baseOffset + 5, acc.append(leaderEpoch, records.subList(3, 6), false)); + assertEquals(baseOffset + 7, acc.append(leaderEpoch, records.subList(6, 8), false)); + assertEquals(baseOffset + 8, acc.append(leaderEpoch, records.subList(8, 9), false)); - // Append a leader change message - acc.appendLeaderChangeMessage(new LeaderChangeMessage(), time.milliseconds()); + assertFalse(acc.needsDrain(time.milliseconds())); - assertTrue(acc.needsDrain(time.milliseconds())); + // Append a leader change message + acc.appendLeaderChangeMessage(new LeaderChangeMessage(), time.milliseconds()); - // Test that drain status is FINISHED - assertEquals(0, acc.timeUntilDrain(time.milliseconds())); + assertTrue(acc.needsDrain(time.milliseconds())); - // Drain completed batches - List> batches = acc.drain(); + // Test that drain status is FINISHED + assertEquals(0, acc.timeUntilDrain(time.milliseconds())); - // Should have 2 batches, one consisting of `records` and one `leaderChangeMessage` - assertEquals(2, batches.size()); - assertFalse(acc.needsDrain(time.milliseconds())); - assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); + // Drain completed batches + List> batches = acc.drain(); - BatchAccumulator.CompletedBatch batch = batches.get(0); - assertEquals(records, batch.records.get()); - assertEquals(baseOffset, batch.baseOffset); - assertEquals(time.milliseconds(), batch.appendTimestamp()); - }); + // Should have 2 batches, one consisting of `records` and one `leaderChangeMessage` + assertEquals(2, batches.size()); + assertFalse(acc.needsDrain(time.milliseconds())); + assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); + + BatchAccumulator.CompletedBatch batch = batches.get(0); + assertEquals(records, batch.records.get()); + assertEquals(baseOffset, batch.baseOffset); + assertEquals(time.milliseconds(), batch.appendTimestamp()); } @Test @@ -244,7 +235,7 @@ public void testLingerBeginsOnFirstWrite() { ); time.sleep(15); - assertEquals(baseOffset, acc.append(leaderEpoch, singletonList("a"), OptionalLong.empty(), false)); + assertEquals(baseOffset, acc.append(leaderEpoch, singletonList("a"), false)); assertEquals(lingerMs, acc.timeUntilDrain(time.milliseconds())); assertFalse(acc.isEmpty()); @@ -276,7 +267,7 @@ public void testCompletedBatchReleaseBuffer() { maxBatchSize ); - assertEquals(baseOffset, acc.append(leaderEpoch, singletonList("a"), OptionalLong.empty(), false)); + assertEquals(baseOffset, acc.append(leaderEpoch, singletonList("a"), false)); time.sleep(lingerMs); List> batches = acc.drain(); @@ -305,94 +296,57 @@ public void testUnflushedBuffersReleasedByClose() { maxBatchSize ); - assertEquals(baseOffset, acc.append(leaderEpoch, singletonList("a"), OptionalLong.empty(), false)); + assertEquals(baseOffset, acc.append(leaderEpoch, singletonList("a"), false)); acc.close(); Mockito.verify(memoryPool).release(buffer); } @Test public void testSingleBatchAccumulation() { - asList(APPEND, APPEND_ATOMIC).forEach(appender -> { - int leaderEpoch = 17; - long baseOffset = 157; - int lingerMs = 50; - int maxBatchSize = 512; + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 512; - Mockito.when(memoryPool.tryAllocate(maxBatchSize)) - .thenReturn(ByteBuffer.allocate(maxBatchSize)); + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(ByteBuffer.allocate(maxBatchSize)); - BatchAccumulator acc = buildAccumulator( - leaderEpoch, - baseOffset, - lingerMs, - maxBatchSize - ); + BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ); - List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); - assertEquals(baseOffset, appender.call(acc, leaderEpoch, records.subList(0, 1))); - assertEquals(baseOffset + 2, appender.call(acc, leaderEpoch, records.subList(1, 3))); - assertEquals(baseOffset + 5, appender.call(acc, leaderEpoch, records.subList(3, 6))); - assertEquals(baseOffset + 7, appender.call(acc, leaderEpoch, records.subList(6, 8))); - assertEquals(baseOffset + 8, appender.call(acc, leaderEpoch, records.subList(8, 9))); + List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); + assertEquals(baseOffset, acc.append(leaderEpoch, records.subList(0, 1), false)); + assertEquals(baseOffset + 2, acc.append(leaderEpoch, records.subList(1, 3), false)); + assertEquals(baseOffset + 5, acc.append(leaderEpoch, records.subList(3, 6), false)); + assertEquals(baseOffset + 7, acc.append(leaderEpoch, records.subList(6, 8), false)); + assertEquals(baseOffset + 8, acc.append(leaderEpoch, records.subList(8, 9), false)); - long expectedAppendTimestamp = time.milliseconds(); - time.sleep(lingerMs); - assertTrue(acc.needsDrain(time.milliseconds())); + long expectedAppendTimestamp = time.milliseconds(); + time.sleep(lingerMs); + assertTrue(acc.needsDrain(time.milliseconds())); - List> batches = acc.drain(); - assertEquals(1, batches.size()); - assertFalse(acc.needsDrain(time.milliseconds())); - assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); + List> batches = acc.drain(); + assertEquals(1, batches.size()); + assertFalse(acc.needsDrain(time.milliseconds())); + assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); - BatchAccumulator.CompletedBatch batch = batches.get(0); - assertEquals(records, batch.records.get()); - assertEquals(baseOffset, batch.baseOffset); - assertEquals(expectedAppendTimestamp, batch.appendTimestamp()); - }); + BatchAccumulator.CompletedBatch batch = batches.get(0); + assertEquals(records, batch.records.get()); + assertEquals(baseOffset, batch.baseOffset); + assertEquals(expectedAppendTimestamp, batch.appendTimestamp()); } @Test public void testMultipleBatchAccumulation() { - asList(APPEND, APPEND_ATOMIC).forEach(appender -> { - int leaderEpoch = 17; - long baseOffset = 157; - int lingerMs = 50; - int maxBatchSize = 256; - - Mockito.when(memoryPool.tryAllocate(maxBatchSize)) - .thenReturn(ByteBuffer.allocate(maxBatchSize)); - - BatchAccumulator acc = buildAccumulator( - leaderEpoch, - baseOffset, - lingerMs, - maxBatchSize - ); - - // Append entries until we have 4 batches to drain (3 completed, 1 building) - while (acc.numCompletedBatches() < 3) { - appender.call(acc, leaderEpoch, singletonList("foo")); - } - - List> batches = acc.drain(); - assertEquals(4, batches.size()); - assertTrue(batches.stream().allMatch(batch -> batch.data.sizeInBytes() <= maxBatchSize)); - }); - } - - @Test - public void testRecordsAreSplit() { int leaderEpoch = 17; long baseOffset = 157; int lingerMs = 50; - String record = "a"; - int numberOfRecords = 9; - int recordsPerBatch = 2; - int batchHeaderSize = AbstractRecords.recordBatchHeaderSizeInBytes( - RecordBatch.MAGIC_VALUE_V2, - CompressionType.NONE - ); - int maxBatchSize = batchHeaderSize + recordsPerBatch * recordSizeInBytes(record, recordsPerBatch); + int maxBatchSize = 256; + int numberOfRecords = 100; Mockito.when(memoryPool.tryAllocate(maxBatchSize)) .thenReturn(ByteBuffer.allocate(maxBatchSize)); @@ -404,19 +358,13 @@ public void testRecordsAreSplit() { maxBatchSize ); - List records = Stream - .generate(() -> record) - .limit(numberOfRecords) - .collect(Collectors.toList()); - assertEquals(baseOffset + numberOfRecords - 1, acc.append(leaderEpoch, records, OptionalLong.empty(), false)); - - time.sleep(lingerMs); - assertTrue(acc.needsDrain(time.milliseconds())); + // Append enough records so that multiple batches get created + for (int records = 0; records < numberOfRecords; records++) { + acc.append(leaderEpoch, singletonList("foo"), false); + } List> batches = acc.drain(); - // ceilingDiv(records.size(), recordsPerBatch) - int expectedBatches = (records.size() + recordsPerBatch - 1) / recordsPerBatch; - assertEquals(expectedBatches, batches.size()); + assertTrue(batches.size() > 1, () -> String.format("number of batches is %s", batches.size())); assertTrue(batches.stream().allMatch(batch -> batch.data.sizeInBytes() <= maxBatchSize)); } @@ -444,6 +392,7 @@ public void testDrainDoesNotBlockWithConcurrentAppend() throws Exception { long baseOffset = 157; int lingerMs = 50; int maxBatchSize = 256; + int maxNumberOfBatches = 10; StringSerde serde = Mockito.spy(new StringSerde()); BatchAccumulator acc = new BatchAccumulator<>( @@ -451,6 +400,7 @@ public void testDrainDoesNotBlockWithConcurrentAppend() throws Exception { baseOffset, lingerMs, maxBatchSize, + maxNumberOfBatches, memoryPool, time, Compression.NONE, @@ -463,7 +413,7 @@ public void testDrainDoesNotBlockWithConcurrentAppend() throws Exception { // Do the first append outside the thread to start the linger timer Mockito.when(memoryPool.tryAllocate(maxBatchSize)) .thenReturn(ByteBuffer.allocate(maxBatchSize)); - acc.append(leaderEpoch, singletonList("a"), OptionalLong.empty(), false); + acc.append(leaderEpoch, singletonList("a"), false); // Let the serde block to simulate a slow append Mockito.doAnswer(invocation -> { @@ -478,7 +428,7 @@ public void testDrainDoesNotBlockWithConcurrentAppend() throws Exception { Mockito.any(Writable.class) ); - Thread appendThread = new Thread(() -> acc.append(leaderEpoch, singletonList("b"), OptionalLong.empty(), false)); + Thread appendThread = new Thread(() -> acc.append(leaderEpoch, singletonList("b"), false)); appendThread.start(); // Attempt to drain while the append thread is holding the lock @@ -516,52 +466,6 @@ int recordSizeInBytes(String record, int numberOfRecords) { return ByteUtils.sizeOfVarint(recordSizeInBytes) + recordSizeInBytes; } - interface Appender { - Long call(BatchAccumulator acc, int epoch, List records); - } - - static final Appender APPEND_ATOMIC = (acc, epoch, records) -> - acc.append(epoch, records, OptionalLong.empty(), true); - - static final Appender APPEND = (acc, epoch, records) -> - acc.append(epoch, records, OptionalLong.empty(), false); - - @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testAppendWithRequiredBaseOffset(boolean correctOffset) { - int leaderEpoch = 17; - long baseOffset = 157; - int lingerMs = 50; - int maxBatchSize = 512; - - ByteBuffer buffer = ByteBuffer.allocate(maxBatchSize); - Mockito.when(memoryPool.tryAllocate(maxBatchSize)) - .thenReturn(buffer); - - BatchAccumulator acc = buildAccumulator( - leaderEpoch, - baseOffset, - lingerMs, - maxBatchSize - ); - - if (correctOffset) { - assertEquals(baseOffset, acc.append(leaderEpoch, - singletonList("a"), - OptionalLong.of(baseOffset), - true)); - } else { - assertEquals("Wanted base offset 156, but the next offset was 157", - assertThrows(UnexpectedBaseOffsetException.class, () -> - acc.append(leaderEpoch, - singletonList("a"), - OptionalLong.of(baseOffset - 1), - true) - ).getMessage()); - } - acc.close(); - } - @Test public void testMultipleControlRecords() { int leaderEpoch = 17; @@ -743,6 +647,100 @@ public void testEmptyControlBatch() { } } + @Test + void testDelayedDrain() { + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 256; + + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(ByteBuffer.allocate(maxBatchSize)); + + BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ); + + List records = asList("a", "b", "c", "d", "e", "f", "g"); + assertEquals(baseOffset, acc.append(leaderEpoch, records.subList(0, 1), true)); + acc.forceDrain(); + assertEquals(baseOffset + 2, acc.append(leaderEpoch, records.subList(1, 3), true)); + acc.forceDrain(); + assertEquals(baseOffset + 5, acc.append(leaderEpoch, records.subList(3, 6), true)); + acc.forceDrain(); + + assertFalse(acc.needsDrain(time.milliseconds())); + assertNotEquals(0, acc.timeUntilDrain(time.milliseconds())); + assertEquals(0, acc.drain().size()); + + acc.allowDrain(); + assertTrue(acc.needsDrain(time.milliseconds())); + assertEquals(0, acc.timeUntilDrain(time.milliseconds())); + assertEquals(3, acc.drain().size()); + } + + @Test + void testDelayedDrainAreReleased() { + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 256; + + ByteBuffer buffer = ByteBuffer.allocate(maxBatchSize); + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(buffer); + + BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ); + + List records = asList("a", "b", "c", "d", "e", "f", "g"); + assertEquals(baseOffset, acc.append(leaderEpoch, records.subList(0, 1), true)); + acc.forceDrain(); + + acc.close(); + Mockito.verify(memoryPool).release(buffer); + } + + @Test + public void testMaxNumberOfBatches() { + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 256; + int maxNumberOfBatches = 2; + + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(ByteBuffer.allocate(maxBatchSize)); + + StringSerde serde = Mockito.spy(new StringSerde()); + BatchAccumulator acc = new BatchAccumulator<>( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize, + maxNumberOfBatches, + memoryPool, + time, + Compression.NONE, + serde + ); + + List records = asList("a", "b", "c", "d", "e", "f", "g"); + assertEquals(baseOffset, acc.append(leaderEpoch, records.subList(0, 1), true)); + acc.forceDrain(); + assertEquals(baseOffset + 2, acc.append(leaderEpoch, records.subList(1, 3), true)); + acc.forceDrain(); + + assertThrows(IllegalStateException.class, () -> acc.append(leaderEpoch, records.subList(3, 6), true)); + } + private static MemoryRecordsBuilder controlRecordsBuilder( long baseOffset, int epoch, From ba1995704a197760d7ee33bc82e879eb42376a1a Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Thu, 15 Aug 2024 18:59:38 +0800 Subject: [PATCH 050/123] KAFKA-17326 The LIST_OFFSET request is removed from the "Api Keys" page (#16870) Reviewers: Chia-Ping Tsai --- .../main/java/org/apache/kafka/common/protocol/ApiKeys.java | 2 +- .../java/org/apache/kafka/common/protocol/ApiKeysTest.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index e8d35f93f5..e95882be69 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -286,7 +286,7 @@ static String toHtml() { b.append("Key\n"); b.append(""); clientApis().stream() - .filter(apiKey -> !apiKey.messageType.latestVersionUnstable()) + .filter(apiKey -> apiKey.toApiVersion(false).isPresent()) .forEach(apiKey -> { b.append("\n"); b.append(""); diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java index f3e725a6e5..a4b785a908 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/ApiKeysTest.java @@ -92,10 +92,10 @@ public void testApiScope() { public void testHtmlOnlyHaveStableApi() { String html = ApiKeys.toHtml(); for (ApiKeys apiKeys : ApiKeys.clientApis()) { - if (apiKeys.messageType.latestVersionUnstable()) { - assertFalse(html.contains("The_Messages_" + apiKeys.name), "Html should not contain unstable api: " + apiKeys.name); - } else { + if (apiKeys.toApiVersion(false).isPresent()) { assertTrue(html.contains("The_Messages_" + apiKeys.name), "Html should contain stable api: " + apiKeys.name); + } else { + assertFalse(html.contains("The_Messages_" + apiKeys.name), "Html should not contain unstable api: " + apiKeys.name); } } } From 3b90bbaf6fb3d1519f7647c926126ceebb68d7b5 Mon Sep 17 00:00:00 2001 From: Josep Prat Date: Thu, 15 Aug 2024 18:10:10 +0200 Subject: [PATCH 051/123] MINOR: Fix visibility for classes exposed outside of their scope (#16886) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit These 2 classes are package protected but they are part of the public API of public methods. To have clean APIs we should make this consistent. Static class ReplicaState is exposed in RaftUtil#singletonDescribeQuorumResponse method which is public. RequestSender is implemented by a public class and it's exposed in the public constructor of AddVoterHandler. Reviewers: José Armando García Sancio --- raft/src/main/java/org/apache/kafka/raft/LeaderState.java | 2 +- .../java/org/apache/kafka/raft/internals/RequestSender.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index 5c468fc995..05f6eaf5eb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -691,7 +691,7 @@ private void updateVoterAndObserverStates(VoterSet lastVoterSet) { } } - static class ReplicaState implements Comparable { + public static class ReplicaState implements Comparable { private ReplicaKey replicaKey; private Endpoints listeners; private Optional endOffset; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RequestSender.java b/raft/src/main/java/org/apache/kafka/raft/internals/RequestSender.java index d30b259c7d..8d00bb9a27 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RequestSender.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RequestSender.java @@ -29,7 +29,7 @@ * Responsible for managing the connection state and sending request when the connection is * available. */ -interface RequestSender { +public interface RequestSender { /** * The name of the listener used for sending request. * From bcf4c73bae59d0425417c24fd526a3a232f6f640 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 16 Aug 2024 00:24:51 +0800 Subject: [PATCH 052/123] KAFKA-17238 Move VoterSet and ReplicaKey from raft.internals to raft (#16775) Reviewers: Chia-Ping Tsai --- .../test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala | 3 +-- .../java/org/apache/kafka/metadata/storage/Formatter.java | 2 +- raft/src/main/java/org/apache/kafka/raft/CandidateState.java | 2 -- raft/src/main/java/org/apache/kafka/raft/DynamicVoter.java | 2 -- raft/src/main/java/org/apache/kafka/raft/DynamicVoters.java | 1 - raft/src/main/java/org/apache/kafka/raft/ElectionState.java | 1 - raft/src/main/java/org/apache/kafka/raft/EpochState.java | 2 -- raft/src/main/java/org/apache/kafka/raft/FollowerState.java | 1 - .../src/main/java/org/apache/kafka/raft/KafkaRaftClient.java | 2 -- raft/src/main/java/org/apache/kafka/raft/LeaderState.java | 2 -- raft/src/main/java/org/apache/kafka/raft/QuorumState.java | 2 -- raft/src/main/java/org/apache/kafka/raft/RaftUtil.java | 1 - .../org/apache/kafka/raft/{internals => }/ReplicaKey.java | 2 +- raft/src/main/java/org/apache/kafka/raft/ResignedState.java | 1 - .../src/main/java/org/apache/kafka/raft/UnattachedState.java | 1 - .../java/org/apache/kafka/raft/{internals => }/VoterSet.java | 5 ++--- .../org/apache/kafka/raft/internals/AddVoterHandler.java | 2 ++ .../apache/kafka/raft/internals/AddVoterHandlerState.java | 1 + .../kafka/raft/internals/KRaftControlRecordStateMachine.java | 1 + .../org/apache/kafka/raft/internals/KafkaRaftMetrics.java | 1 + .../org/apache/kafka/raft/internals/RemoveVoterHandler.java | 2 ++ .../org/apache/kafka/raft/internals/UpdateVoterHandler.java | 2 ++ .../org/apache/kafka/raft/internals/VoterSetHistory.java | 2 ++ .../org/apache/kafka/snapshot/RecordsSnapshotWriter.java | 2 +- .../test/java/org/apache/kafka/raft/CandidateStateTest.java | 3 --- .../test/java/org/apache/kafka/raft/DynamicVoterTest.java | 2 -- .../test/java/org/apache/kafka/raft/DynamicVotersTest.java | 1 - .../test/java/org/apache/kafka/raft/ElectionStateTest.java | 1 - .../java/org/apache/kafka/raft/FileQuorumStateStoreTest.java | 1 - .../test/java/org/apache/kafka/raft/FollowerStateTest.java | 1 - .../java/org/apache/kafka/raft/KafkaNetworkChannelTest.java | 1 - .../org/apache/kafka/raft/KafkaRaftClientReconfigTest.java | 3 --- .../org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java | 3 --- .../test/java/org/apache/kafka/raft/KafkaRaftClientTest.java | 3 --- .../src/test/java/org/apache/kafka/raft/LeaderStateTest.java | 3 --- .../test/java/org/apache/kafka/raft/MockNetworkChannel.java | 1 - .../src/test/java/org/apache/kafka/raft/QuorumStateTest.java | 3 --- .../java/org/apache/kafka/raft/RaftClientTestContext.java | 2 -- .../test/java/org/apache/kafka/raft/ResignedStateTest.java | 2 -- .../test/java/org/apache/kafka/raft/UnattachedStateTest.java | 1 - .../org/apache/kafka/raft/UnattachedStateWithVoteTest.java | 1 - .../org/apache/kafka/raft/{internals => }/VoterSetTest.java | 3 +-- .../raft/internals/KRaftControlRecordStateMachineTest.java | 2 ++ .../apache/kafka/raft/internals/KafkaRaftMetricsTest.java | 3 +++ .../org/apache/kafka/raft/internals/RecordsIteratorTest.java | 2 ++ .../org/apache/kafka/raft/internals/VoterSetHistoryTest.java | 3 +++ .../org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java | 4 ++-- 47 files changed, 30 insertions(+), 62 deletions(-) rename raft/src/main/java/org/apache/kafka/raft/{internals => }/ReplicaKey.java (98%) rename raft/src/main/java/org/apache/kafka/raft/{internals => }/VoterSet.java (99%) rename raft/src/test/java/org/apache/kafka/raft/{internals => }/VoterSetTest.java (99%) diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 977f9dcb3c..82c2bfd03a 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -42,8 +42,7 @@ import org.apache.kafka.coordinator.group.{CoordinatorRecord, CoordinatorRecordS import org.apache.kafka.coordinator.group.generated.{ConsumerGroupMemberMetadataValue, ConsumerGroupMetadataKey, ConsumerGroupMetadataValue, GroupMetadataKey, GroupMetadataValue} import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.metadata.MetadataRecordSerde -import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch} -import org.apache.kafka.raft.internals.VoterSetTest +import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch, VoterSetTest} import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion} import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde diff --git a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java index df639eef5b..72995fb753 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java @@ -27,7 +27,7 @@ import org.apache.kafka.metadata.properties.MetaPropertiesVersion; import org.apache.kafka.raft.DynamicVoters; import org.apache.kafka.raft.KafkaRaftClient; -import org.apache.kafka.raft.internals.VoterSet; +import org.apache.kafka.raft.VoterSet; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.FeatureVersion; import org.apache.kafka.server.common.Features; diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 1fd71e5e56..175df7760d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -20,8 +20,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/DynamicVoter.java b/raft/src/main/java/org/apache/kafka/raft/DynamicVoter.java index f0546c910f..6ae03c34fb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/DynamicVoter.java +++ b/raft/src/main/java/org/apache/kafka/raft/DynamicVoter.java @@ -20,8 +20,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.network.ListenerName; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; import java.net.InetSocketAddress; import java.util.Collections; diff --git a/raft/src/main/java/org/apache/kafka/raft/DynamicVoters.java b/raft/src/main/java/org/apache/kafka/raft/DynamicVoters.java index 0dfb54be58..8f746ff2c4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/DynamicVoters.java +++ b/raft/src/main/java/org/apache/kafka/raft/DynamicVoters.java @@ -17,7 +17,6 @@ package org.apache.kafka.raft; -import org.apache.kafka.raft.internals.VoterSet; import java.util.ArrayList; import java.util.Collection; diff --git a/raft/src/main/java/org/apache/kafka/raft/ElectionState.java b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java index 320ae47dbe..e65e72890f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ElectionState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ElectionState.java @@ -17,7 +17,6 @@ package org.apache.kafka.raft; import org.apache.kafka.raft.generated.QuorumStateData; -import org.apache.kafka.raft.internals.ReplicaKey; import java.util.List; import java.util.Objects; diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochState.java b/raft/src/main/java/org/apache/kafka/raft/EpochState.java index a0d643999b..4f6baec79d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/EpochState.java +++ b/raft/src/main/java/org/apache/kafka/raft/EpochState.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.raft; -import org.apache.kafka.raft.internals.ReplicaKey; - import java.io.Closeable; import java.util.Optional; diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 2e26e7c5a7..49eecab5d6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.snapshot.RawSnapshotWriter; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index b28686fc31..d9afa80382 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -77,10 +77,8 @@ import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.raft.internals.RecordsBatchReader; import org.apache.kafka.raft.internals.RemoveVoterHandler; -import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.ThresholdPurgatory; import org.apache.kafka.raft.internals.UpdateVoterHandler; -import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.NotifyingRawSnapshotWriter; diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index 05f6eaf5eb..803804858a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -30,8 +30,6 @@ import org.apache.kafka.raft.internals.AddVoterHandlerState; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.RemoveVoterHandlerState; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.server.common.KRaftVersion; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index f61bdab0db..da2b35609e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -22,8 +22,6 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.KRaftControlRecordStateMachine; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java index 7ec629bc9b..018bec0d63 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java @@ -42,7 +42,6 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.raft.internals.ReplicaKey; import java.net.InetSocketAddress; import java.util.Collection; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/ReplicaKey.java b/raft/src/main/java/org/apache/kafka/raft/ReplicaKey.java similarity index 98% rename from raft/src/main/java/org/apache/kafka/raft/internals/ReplicaKey.java rename to raft/src/main/java/org/apache/kafka/raft/ReplicaKey.java index 4e7cea17c6..a1acc39d57 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/ReplicaKey.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicaKey.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.raft.internals; +package org.apache.kafka.raft; import org.apache.kafka.common.Uuid; diff --git a/raft/src/main/java/org/apache/kafka/raft/ResignedState.java b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java index 267a88ad78..eaee0496b8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ResignedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.raft.internals.ReplicaKey; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java index 999a32f866..4b21849f81 100644 --- a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; -import org.apache.kafka.raft.internals.ReplicaKey; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/VoterSet.java similarity index 99% rename from raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java rename to raft/src/main/java/org/apache/kafka/raft/VoterSet.java index 41f9b93b7d..a74d2a49e8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/VoterSet.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.raft.internals; +package org.apache.kafka.raft; import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; @@ -22,7 +22,6 @@ import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.Endpoints; import java.net.InetSocketAddress; import java.util.Collections; @@ -49,7 +48,7 @@ public final class VoterSet { private final Map voters; - VoterSet(Map voters) { + public VoterSet(Map voters) { if (voters.isEmpty()) { throw new IllegalArgumentException("Voters cannot be empty"); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/AddVoterHandler.java b/raft/src/main/java/org/apache/kafka/raft/internals/AddVoterHandler.java index 444e747b1e..44b9eb2a39 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/AddVoterHandler.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/AddVoterHandler.java @@ -29,6 +29,8 @@ import org.apache.kafka.raft.LeaderState; import org.apache.kafka.raft.LogOffsetMetadata; import org.apache.kafka.raft.RaftUtil; +import org.apache.kafka.raft.ReplicaKey; +import org.apache.kafka.raft.VoterSet; import org.apache.kafka.server.common.KRaftVersion; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/AddVoterHandlerState.java b/raft/src/main/java/org/apache/kafka/raft/internals/AddVoterHandlerState.java index 5bef0afa70..c403d0e0cd 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/AddVoterHandlerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/AddVoterHandlerState.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.message.AddRaftVoterResponseData; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.raft.Endpoints; +import org.apache.kafka.raft.ReplicaKey; import java.util.OptionalLong; import java.util.concurrent.CompletableFuture; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java index 82f1435a30..02bc2c823a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java @@ -25,6 +25,7 @@ import org.apache.kafka.raft.Isolation; import org.apache.kafka.raft.LogFetchInfo; import org.apache.kafka.raft.ReplicatedLog; +import org.apache.kafka.raft.VoterSet; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.RawSnapshotReader; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java index 49abcd0005..64230b45f7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java @@ -28,6 +28,7 @@ import org.apache.kafka.raft.LogOffsetMetadata; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.QuorumState; +import org.apache.kafka.raft.ReplicaKey; import java.util.Arrays; import java.util.OptionalLong; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RemoveVoterHandler.java b/raft/src/main/java/org/apache/kafka/raft/internals/RemoveVoterHandler.java index 3a62d383dd..29093cc30b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RemoveVoterHandler.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RemoveVoterHandler.java @@ -24,6 +24,8 @@ import org.apache.kafka.raft.LeaderState; import org.apache.kafka.raft.LogOffsetMetadata; import org.apache.kafka.raft.RaftUtil; +import org.apache.kafka.raft.ReplicaKey; +import org.apache.kafka.raft.VoterSet; import org.apache.kafka.server.common.KRaftVersion; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java b/raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java index 985ec18919..417c1decad 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java @@ -27,6 +27,8 @@ import org.apache.kafka.raft.LeaderState; import org.apache.kafka.raft.LogOffsetMetadata; import org.apache.kafka.raft.RaftUtil; +import org.apache.kafka.raft.ReplicaKey; +import org.apache.kafka.raft.VoterSet; import org.apache.kafka.server.common.KRaftVersion; import java.util.Optional; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index 4732d6e799..7f671e6660 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.raft.internals; +import org.apache.kafka.raft.VoterSet; + import java.util.Optional; import java.util.OptionalLong; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index dd4cc3b2d0..ef26ce3bc0 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -28,9 +28,9 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.raft.VoterSet; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.BatchAccumulator.CompletedBatch; -import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.serialization.RecordSerde; diff --git a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index be9a083684..c71f758d14 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -20,9 +20,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; -import org.apache.kafka.raft.internals.VoterSetTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/raft/src/test/java/org/apache/kafka/raft/DynamicVoterTest.java b/raft/src/test/java/org/apache/kafka/raft/DynamicVoterTest.java index fe9f295c5b..f53d006f63 100644 --- a/raft/src/test/java/org/apache/kafka/raft/DynamicVoterTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/DynamicVoterTest.java @@ -19,8 +19,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.network.ListenerName; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; import org.junit.jupiter.api.Test; diff --git a/raft/src/test/java/org/apache/kafka/raft/DynamicVotersTest.java b/raft/src/test/java/org/apache/kafka/raft/DynamicVotersTest.java index 75861213cc..94f2d54eae 100644 --- a/raft/src/test/java/org/apache/kafka/raft/DynamicVotersTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/DynamicVotersTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Uuid; -import org.apache.kafka.raft.internals.VoterSet; import org.junit.jupiter.api.Test; diff --git a/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java index 18b748061c..715f5d7b48 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ElectionStateTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.generated.QuorumStateData; -import org.apache.kafka.raft.internals.ReplicaKey; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java b/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java index 1cf480e5eb..4759c5c4e6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FileQuorumStateStoreTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.protocol.types.TaggedFields; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.generated.QuorumStateData; -import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.test.TestUtils; diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index 01b20a0570..4ab40379e8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.internals.ReplicaKey; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java index 579508a256..96a5df1845 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java @@ -47,7 +47,6 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; -import org.apache.kafka.raft.internals.ReplicaKey; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index bb1cbbb7bb..0d46af767a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -36,9 +36,6 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; -import org.apache.kafka.raft.internals.VoterSetTest; import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.snapshot.RecordsSnapshotReader; diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 0b74ee69d3..3298bd8dca 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -26,10 +26,7 @@ import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.requests.FetchSnapshotRequest; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.StringSerde; -import org.apache.kafka.raft.internals.VoterSet; -import org.apache.kafka.raft.internals.VoterSetTest; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; import org.apache.kafka.snapshot.RecordsSnapshotWriter; diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index a783ccdc7c..9106115d6a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -40,9 +40,6 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; -import org.apache.kafka.raft.internals.VoterSetTest; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; diff --git a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java index 8ce18ec9d4..b5e135275c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java @@ -20,9 +20,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.raft.internals.BatchAccumulator; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; -import org.apache.kafka.raft.internals.VoterSetTest; import org.apache.kafka.server.common.KRaftVersion; import org.junit.jupiter.api.Test; diff --git a/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java b/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java index b5698545bd..c8e732e880 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockNetworkChannel.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.raft.internals.VoterSetTest; import java.util.ArrayList; import java.util.HashMap; diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index a22cf16123..fc14f4d9bc 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -22,9 +22,6 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.KRaftControlRecordStateMachine; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSet; -import org.apache.kafka.raft.internals.VoterSetTest; import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.KRaftVersion; diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index bbe9ddf8bf..d12140a6c9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -63,9 +63,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.internals.BatchBuilder; -import org.apache.kafka.raft.internals.ReplicaKey; import org.apache.kafka.raft.internals.StringSerde; -import org.apache.kafka.raft.internals.VoterSet; import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.serialization.RecordSerde; diff --git a/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java index 2df8293397..38987eac26 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java @@ -19,8 +19,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.internals.ReplicaKey; -import org.apache.kafka.raft.internals.VoterSetTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java index 113b24c8a6..926c852556 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.internals.ReplicaKey; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java index fdeded219f..b0cd57bbc0 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateWithVoteTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.raft.internals.ReplicaKey; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java similarity index 99% rename from raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java rename to raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java index a4f2354d69..1f41f4fd22 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java @@ -14,14 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.raft.internals; +package org.apache.kafka.raft; import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.Endpoints; import org.apache.kafka.server.common.Features; import org.junit.jupiter.api.Test; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java index 93cd37fb25..e3673bef52 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java @@ -24,6 +24,8 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.MockLog; import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.raft.VoterSet; +import org.apache.kafka.raft.VoterSetTest; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.RecordsSnapshotWriter; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index 3e399de3c2..86f18a1aef 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -26,6 +26,9 @@ import org.apache.kafka.raft.MockQuorumStateStore; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.QuorumState; +import org.apache.kafka.raft.ReplicaKey; +import org.apache.kafka.raft.VoterSet; +import org.apache.kafka.raft.VoterSetTest; import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.KRaftVersion; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index a8d34ec5b3..84904032a5 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -40,6 +40,8 @@ import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.ControlRecord; import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.raft.VoterSet; +import org.apache.kafka.raft.VoterSetTest; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.MockRawSnapshotWriter; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index 4cefd44877..f40311d8b7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.raft.internals; +import org.apache.kafka.raft.VoterSet; +import org.apache.kafka.raft.VoterSetTest; + import org.junit.jupiter.api.Test; import java.util.HashMap; diff --git a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java index 8a5d384462..3934b9f96f 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java @@ -25,9 +25,9 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.raft.VoterSet; +import org.apache.kafka.raft.VoterSetTest; import org.apache.kafka.raft.internals.StringSerde; -import org.apache.kafka.raft.internals.VoterSet; -import org.apache.kafka.raft.internals.VoterSetTest; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.serialization.RecordSerde; From ed7cadd4c040e8aa4c7e2bdb0d0f42d00ab12159 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 16 Aug 2024 15:53:13 -0400 Subject: [PATCH 053/123] KAFKA-16842; Fix config validation and support unknown voters (#16892) This change fixes the Kafka configuration validation to take into account the reconfiguration changes to configuration and allows KRaft observers to start with an unknown set of voters. For the Kafka configuration validation the high-level change is that now the user only needs to specify either the controller.quorum.bootstrap.servers property or the controller.quorum.voters property. The other notable change in the configuration is that controller listeners can now be (and should be) specified in advertise.listeners property. Because Kafka can now be configured without any voters and just the bootstrap servers. The KRaft client needs to allow for an unknown set of voters during the initial startup. This is done by adding the VoterSet#empty set of voters to the KRaftControlRecordStateMachine. Lastly the RaftClientTestContext type is updated to support this new configuration for KRaft and a test is added to verify that observers can start and send Fetch requests when the voters are unknown. Reviewers: David Arthur --- build.gradle | 15 ++--- .../main/scala/kafka/server/KafkaConfig.scala | 53 +++++++++-------- .../scala/unit/kafka/KafkaConfigTest.scala | 8 ++- .../unit/kafka/server/KafkaConfigTest.scala | 12 +++- .../apache/kafka/raft/KafkaRaftClient.java | 17 +++++- .../java/org/apache/kafka/raft/VoterSet.java | 11 ++-- .../KRaftControlRecordStateMachine.java | 22 ++++--- .../kafka/raft/internals/VoterSetHistory.java | 14 ++--- .../raft/KafkaRaftClientReconfigTest.java | 33 ++++++----- .../kafka/raft/KafkaRaftClientTest.java | 21 ++++--- .../kafka/raft/RaftClientTestContext.java | 45 +++++++------- .../org/apache/kafka/raft/VoterSetTest.java | 36 ++++++------ .../KRaftControlRecordStateMachineTest.java | 28 ++++++--- .../raft/internals/RecordsIteratorTest.java | 2 +- .../raft/internals/VoterSetHistoryTest.java | 58 +++++++++---------- 15 files changed, 210 insertions(+), 165 deletions(-) diff --git a/build.gradle b/build.gradle index 41229843ee..28ff4495e2 100644 --- a/build.gradle +++ b/build.gradle @@ -162,8 +162,8 @@ allprojects { // ZooKeeper (potentially older and containing CVEs) libs.nettyHandler, libs.nettyTransportNativeEpoll, - // be explicit about the reload4j version instead of relying on the transitive versions - libs.reload4j + // be explicit about the reload4j version instead of relying on the transitive versions + libs.reload4j ) } } @@ -734,7 +734,7 @@ subprojects { jacoco { toolVersion = versions.jacoco } - + jacocoTestReport { dependsOn tasks.test sourceSets sourceSets.main @@ -758,8 +758,8 @@ subprojects { skipProjects = [ ":jmh-benchmarks", ":trogdor" ] skipConfigurations = [ "zinc" ] } - // the task `removeUnusedImports` is implemented by google-java-format, - // and unfortunately the google-java-format version used by spotless 6.14.0 can't work with JDK 21. + // the task `removeUnusedImports` is implemented by google-java-format, + // and unfortunately the google-java-format version used by spotless 6.14.0 can't work with JDK 21. // Hence, we apply spotless tasks only if the env is either JDK11 or JDK17 if ((JavaVersion.current().isJava11() || (JavaVersion.current() == JavaVersion.VERSION_17))) { apply plugin: 'com.diffplug.spotless' @@ -1192,7 +1192,7 @@ project(':core') { //By default gradle does not handle test dependencies between the sub-projects //This line is to include clients project test jar to dependant-testlibs from (project(':clients').testJar ) { "$buildDir/dependant-testlibs" } - // log4j-appender is not in core dependencies, + // log4j-appender is not in core dependencies, // so we add it to dependant-testlibs to avoid ClassNotFoundException in running kafka_log4j_appender.py from (project(':log4j-appender').jar ) { "$buildDir/dependant-testlibs" } duplicatesStrategy 'exclude' @@ -1442,7 +1442,7 @@ project(':transaction-coordinator') { implementation project(':clients') generator project(':generator') } - + sourceSets { main { java { @@ -1717,6 +1717,7 @@ project(':raft') { testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation libs.jqwik + testImplementation libs.hamcrest testRuntimeOnly libs.slf4jReload4j testRuntimeOnly libs.junitPlatformLanucher diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 22790933cd..fb71ad84c0 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -881,30 +881,34 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) // validate KRaft-related configs val voterIds = QuorumConfig.parseVoterIds(quorumVoters) - def validateNonEmptyQuorumVotersForKRaft(): Unit = { - if (voterIds.isEmpty) { - throw new ConfigException(s"If using ${KRaftConfigs.PROCESS_ROLES_CONFIG}, ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable set of voters.") + def validateQuorumVotersAndQuorumBootstrapServerForKRaft(): Unit = { + if (voterIds.isEmpty && quorumBootstrapServers.isEmpty) { + throw new ConfigException( + s"""If using ${KRaftConfigs.PROCESS_ROLES_CONFIG}, either ${QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG} must + |contain the set of bootstrap controllers or ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable + |set of controllers.""".stripMargin.replace("\n", " ") + ) } } - def validateNonEmptyQuorumVotersForMigration(): Unit = { - if (voterIds.isEmpty) { - throw new ConfigException(s"If using ${KRaftConfigs.MIGRATION_ENABLED_CONFIG}, ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable set of voters.") + def validateQuorumVotersAndQuorumBootstrapServerForMigration(): Unit = { + if (voterIds.isEmpty && quorumBootstrapServers.isEmpty) { + throw new ConfigException( + s"""If using ${KRaftConfigs.MIGRATION_ENABLED_CONFIG}, either ${QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG} must + |contain the set of bootstrap controllers or ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable + |set of controllers.""".stripMargin.replace("\n", " ") + ) } } def validateControlPlaneListenerEmptyForKRaft(): Unit = { require(controlPlaneListenerName.isEmpty, s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG} is not supported in KRaft mode.") } - def validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker(): Unit = { - require(advertisedBrokerListenerNames.forall(aln => !controllerListenerNames.contains(aln.value())), - s"The advertised.listeners config must not contain KRaft controller listeners from ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} when ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the broker role because Kafka clients that send requests via advertised listeners do not send requests to KRaft controllers -- they only send requests to KRaft brokers.") - } def validateControllerQuorumVotersMustContainNodeIdForKRaftController(): Unit = { - require(voterIds.contains(nodeId), + require(voterIds.isEmpty || voterIds.contains(nodeId), s"If ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the 'controller' role, the node id $nodeId must be included in the set of voters ${QuorumConfig.QUORUM_VOTERS_CONFIG}=${voterIds.asScala.toSet}") } - def validateControllerListenerExistsForKRaftController(): Unit = { - require(controllerListeners.nonEmpty, + def validateAdvertisedControllerListenersNonEmptyForKRaftController(): Unit = { + require(effectiveAdvertisedControllerListeners.nonEmpty, s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must contain at least one value appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running the KRaft controller role") } def validateControllerListenerNamesMustAppearInListenersForKRaftController(): Unit = { @@ -912,16 +916,15 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) require(controllerListenerNames.forall(cln => listenerNameValues.contains(cln)), s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must only contain values appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running the KRaft controller role") } - def validateAdvertisedListenersNonEmptyForBroker(): Unit = { + def validateAdvertisedBrokerListenersNonEmptyForBroker(): Unit = { require(advertisedBrokerListenerNames.nonEmpty, - "There must be at least one advertised listener." + ( + "There must be at least one broker advertised listener." + ( if (processRoles.contains(ProcessRole.BrokerRole)) s" Perhaps all listeners appear in ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}?" else "")) } if (processRoles == Set(ProcessRole.BrokerRole)) { // KRaft broker-only - validateNonEmptyQuorumVotersForKRaft() + validateQuorumVotersAndQuorumBootstrapServerForKRaft() validateControlPlaneListenerEmptyForKRaft() - validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker() // nodeId must not appear in controller.quorum.voters require(!voterIds.contains(nodeId), s"If ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains just the 'broker' role, the node id $nodeId must not be included in the set of voters ${QuorumConfig.QUORUM_VOTERS_CONFIG}=${voterIds.asScala.toSet}") @@ -943,10 +946,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) if (controllerListenerNames.size > 1) { warn(s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} has multiple entries; only the first will be used since ${KRaftConfigs.PROCESS_ROLES_CONFIG}=broker: ${controllerListenerNames.asJava}") } - validateAdvertisedListenersNonEmptyForBroker() } else if (processRoles == Set(ProcessRole.ControllerRole)) { // KRaft controller-only - validateNonEmptyQuorumVotersForKRaft() + validateQuorumVotersAndQuorumBootstrapServerForKRaft() validateControlPlaneListenerEmptyForKRaft() // listeners should only contain listeners also enumerated in the controller listener require( @@ -954,21 +956,19 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) s"The ${SocketServerConfigs.LISTENERS_CONFIG} config must only contain KRaft controller listeners from ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} when ${KRaftConfigs.PROCESS_ROLES_CONFIG}=controller" ) validateControllerQuorumVotersMustContainNodeIdForKRaftController() - validateControllerListenerExistsForKRaftController() + validateAdvertisedControllerListenersNonEmptyForKRaftController() validateControllerListenerNamesMustAppearInListenersForKRaftController() } else if (isKRaftCombinedMode) { // KRaft combined broker and controller - validateNonEmptyQuorumVotersForKRaft() + validateQuorumVotersAndQuorumBootstrapServerForKRaft() validateControlPlaneListenerEmptyForKRaft() - validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker() validateControllerQuorumVotersMustContainNodeIdForKRaftController() - validateControllerListenerExistsForKRaftController() + validateAdvertisedControllerListenersNonEmptyForKRaftController() validateControllerListenerNamesMustAppearInListenersForKRaftController() - validateAdvertisedListenersNonEmptyForBroker() } else { // ZK-based if (migrationEnabled) { - validateNonEmptyQuorumVotersForMigration() + validateQuorumVotersAndQuorumBootstrapServerForMigration() require(controllerListenerNames.nonEmpty, s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must not be empty when running in ZooKeeper migration mode: ${controllerListenerNames.asJava}") require(interBrokerProtocolVersion.isMigrationSupported, s"Cannot enable ZooKeeper migration without setting " + @@ -983,13 +983,12 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) require(controllerListenerNames.isEmpty, s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must be empty when not running in KRaft mode: ${controllerListenerNames.asJava}") } - validateAdvertisedListenersNonEmptyForBroker() } val listenerNames = listeners.map(_.listenerName).toSet if (processRoles.isEmpty || processRoles.contains(ProcessRole.BrokerRole)) { // validations for all broker setups (i.e. ZooKeeper and KRaft broker-only and KRaft co-located) - validateAdvertisedListenersNonEmptyForBroker() + validateAdvertisedBrokerListenersNonEmptyForBroker() require(advertisedBrokerListenerNames.contains(interBrokerListenerName), s"${ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG} must be a listener name defined in ${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG}. " + s"The valid options based on currently configured listeners are ${advertisedBrokerListenerNames.map(_.value).mkString(",")}") diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 3a1fc2e4bd..fec4eed488 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -151,8 +151,12 @@ class KafkaConfigTest { propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "") setListenerProps(propertiesFile) - assertBadConfigContainingMessage(propertiesFile, - "If using process.roles, controller.quorum.voters must contain a parseable set of voters.") + assertBadConfigContainingMessage( + propertiesFile, + """If using process.roles, either controller.quorum.bootstrap.servers + |must contain the set of bootstrap controllers or controller.quorum.voters must contain a + |parseable set of controllers.""".stripMargin.replace("\n", " ") + ) // Ensure that if neither process.roles nor controller.quorum.voters is populated, then an exception is thrown if zookeeper.connect is not defined propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "") diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index af7444e4fe..3d37cfccd6 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -446,7 +446,10 @@ class KafkaConfigTest { props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) - assertBadConfigContainingMessage(props, "There must be at least one advertised listener. Perhaps all listeners appear in controller.listener.names?") + assertBadConfigContainingMessage( + props, + "There must be at least one broker advertised listener. Perhaps all listeners appear in controller.listener.names?" + ) props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093") KafkaConfig.fromProps(props) @@ -1832,8 +1835,11 @@ class KafkaConfigTest { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true") assertEquals( - "If using zookeeper.metadata.migration.enable, controller.quorum.voters must contain a parseable set of voters.", - assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage) + """If using zookeeper.metadata.migration.enable, either controller.quorum.bootstrap.servers + |must contain the set of bootstrap controllers or controller.quorum.voters must contain a + |parseable set of controllers.""".stripMargin.replace("\n", " "), + assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage + ) props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093") assertEquals( diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index d9afa80382..4d151b3128 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.feature.SupportedVersionRange; @@ -455,9 +456,9 @@ public void initialize( QuorumStateStore quorumStateStore, Metrics metrics ) { - Optional staticVoters = voterAddresses.isEmpty() ? - Optional.empty() : - Optional.of(VoterSet.fromInetSocketAddresses(channel.listenerName(), voterAddresses)); + VoterSet staticVoters = voterAddresses.isEmpty() ? + VoterSet.empty() : + VoterSet.fromInetSocketAddresses(channel.listenerName(), voterAddresses); partitionState = new KRaftControlRecordStateMachine( staticVoters, @@ -470,8 +471,18 @@ public void initialize( // Read the entire log logger.info("Reading KRaft snapshot and log as part of the initialization"); partitionState.updateState(); + logger.info("Starting voters are {}", partitionState.lastVoterSet()); if (requestManager == null) { + if (voterAddresses.isEmpty()) { + throw new ConfigException( + String.format( + "Missing kraft bootstrap servers. Must specify a value for %s.", + QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG + ) + ); + } + // The request manager wasn't created using the bootstrap servers // create it using the voters static configuration List bootstrapNodes = voterAddresses diff --git a/raft/src/main/java/org/apache/kafka/raft/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/VoterSet.java index a74d2a49e8..90813d660d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/VoterSet.java @@ -48,11 +48,7 @@ public final class VoterSet { private final Map voters; - public VoterSet(Map voters) { - if (voters.isEmpty()) { - throw new IllegalArgumentException("Voters cannot be empty"); - } - + private VoterSet(Map voters) { this.voters = voters; } @@ -409,6 +405,11 @@ public static VoterNode of( } } + private static final VoterSet EMPTY = new VoterSet(Collections.emptyMap()); + public static VoterSet empty() { + return EMPTY; + } + /** * Converts a {@code VotersRecord} to a {@code VoterSet}. * diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java index 02bc2c823a..c1d4a0b2f2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java @@ -82,7 +82,7 @@ public final class KRaftControlRecordStateMachine { * @param logContext the log context */ public KRaftControlRecordStateMachine( - Optional staticVoterSet, + VoterSet staticVoterSet, ReplicatedLog log, RecordSerde serde, BufferSupplier bufferSupplier, @@ -280,19 +280,25 @@ private void handleBatch(Batch batch, OptionalLong overrideOffset) { long currentOffset = overrideOffset.orElse(batch.baseOffset() + offsetDelta); switch (record.type()) { case KRAFT_VOTERS: + VoterSet voters = VoterSet.fromVotersRecord((VotersRecord) record.message()); + logger.info("Latest set of voters is {} at offset {}", voters, currentOffset); synchronized (voterSetHistory) { - voterSetHistory.addAt(currentOffset, VoterSet.fromVotersRecord((VotersRecord) record.message())); + voterSetHistory.addAt(currentOffset, voters); } break; case KRAFT_VERSION: + KRaftVersion kraftVersion = KRaftVersion.fromFeatureLevel( + ((KRaftVersionRecord) record.message()).kRaftVersion() + ); + logger.info( + "Latest {} is {} at offset {}", + KRaftVersion.FEATURE_NAME, + kraftVersion, + currentOffset + ); synchronized (kraftVersionHistory) { - kraftVersionHistory.addAt( - currentOffset, - KRaftVersion.fromFeatureLevel( - ((KRaftVersionRecord) record.message()).kRaftVersion() - ) - ); + kraftVersionHistory.addAt(currentOffset, kraftVersion); } break; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java index 7f671e6660..6ab304f8c1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/VoterSetHistory.java @@ -29,10 +29,10 @@ * evaluating the latest set of voters. */ public final class VoterSetHistory { - private final Optional staticVoterSet; + private final VoterSet staticVoterSet; private final LogHistory votersHistory = new TreeMapLogHistory<>(); - VoterSetHistory(Optional staticVoterSet) { + VoterSetHistory(VoterSet staticVoterSet) { this.staticVoterSet = staticVoterSet; } @@ -85,13 +85,9 @@ public Optional valueAtOrBefore(long offset) { * Returns the latest set of voters. */ public VoterSet lastValue() { - Optional> result = votersHistory.lastEntry(); - if (result.isPresent()) { - return result.get().value(); - } - - return staticVoterSet - .orElseThrow(() -> new IllegalStateException("No voter set found")); + return votersHistory.lastEntry() + .map(LogHistory.Entry::value) + .orElse(staticVoterSet); } /** diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 0d46af767a..77825f5328 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -179,7 +179,6 @@ public void testLeaderDoesNotBootstrapRecordsWithKraftVersion0() throws Exceptio RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) .withStaticVoters(voters) - .withBootstrapSnapshot(Optional.empty()) .withUnknownLeader(0) .build(); @@ -201,19 +200,6 @@ public void testLeaderDoesNotBootstrapRecordsWithKraftVersion0() throws Exceptio ) ); - // check the bootstrap snapshot exists but is empty - assertEquals(BOOTSTRAP_SNAPSHOT_ID, context.log.latestSnapshotId().get()); - try (SnapshotReader reader = RecordsSnapshotReader.of( - context.log.latestSnapshot().get(), - context.serde, - BufferSupplier.NO_CACHING, - KafkaRaftClient.MAX_BATCH_SIZE_BYTES, - false - ) - ) { - SnapshotWriterReaderTest.assertControlSnapshot(expectedBootstrapRecords, reader); - } - // check leader does not write bootstrap records to log context.becomeLeader(); @@ -2245,6 +2231,25 @@ void testUpdateVoterResponseCausesEpochChange() throws Exception { assertEquals(voter2.id(), fetchRequest.destination().id()); } + @Test + void testObserverDiscoversLeaderWithUnknownVoters() throws Exception { + ReplicaKey local = replicaKey(randomeReplicaId(), true); + InetSocketAddress bootstrapAdddress = InetSocketAddress.createUnresolved("localhost", 1234); + int epoch = 3; + + RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.empty()) + .withUnknownLeader(epoch) + .withBootstrapServers(Optional.of(Collections.singletonList(bootstrapAdddress))) + .build(); + + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + assertEquals(-2, fetchRequest.destination().id()); + } + private static void verifyVotersRecord( VoterSet expectedVoterSet, ByteBuffer recordKey, diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 9106115d6a..62d81621cd 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -69,6 +69,9 @@ import static java.util.Collections.singletonList; import static org.apache.kafka.raft.RaftClientTestContext.Builder.DEFAULT_ELECTION_TIMEOUT_MS; import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.in; +import static org.hamcrest.Matchers.is; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -793,7 +796,7 @@ public void testCannotResignIfObserver(boolean withKip853Rpc) throws Exception { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destination().id())); + assertThat(fetchRequest.destination().id(), is(in(voters))); context.assertFetchRequestData(fetchRequest, 0, 0L, 0); context.deliverResponse( @@ -1784,7 +1787,7 @@ public void testInitializeObserverNoPreviousState(boolean withKip853Rpc) throws context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - assertTrue(voters.contains(fetchRequest.destination().id())); + assertThat(fetchRequest.destination().id(), is(in(voters))); context.assertFetchRequestData(fetchRequest, 0, 0L, 0); context.deliverResponse( @@ -1810,7 +1813,7 @@ public void testObserverQuorumDiscoveryFailure(boolean withKip853Rpc) throws Exc .collect(Collectors.toList()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withBootstrapServers(bootstrapServers) + .withBootstrapServers(Optional.of(bootstrapServers)) .withKip853Rpc(withKip853Rpc) .build(); @@ -1857,7 +1860,7 @@ public void testObserverSendDiscoveryFetchAfterFetchTimeout(boolean withKip853Rp .collect(Collectors.toList()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withBootstrapServers(bootstrapServers) + .withBootstrapServers(Optional.of(bootstrapServers)) .withKip853Rpc(withKip853Rpc) .build(); @@ -1900,7 +1903,7 @@ public void testObserverHandleRetryFetchtToBootstrapServer(boolean withKip853Rpc .collect(Collectors.toList()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withBootstrapServers(bootstrapServers) + .withBootstrapServers(Optional.of(bootstrapServers)) .withKip853Rpc(withKip853Rpc) .build(); @@ -1974,7 +1977,7 @@ public void testObserverHandleRetryFetchToLeader(boolean withKip853Rpc) throws E .collect(Collectors.toList()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withBootstrapServers(bootstrapServers) + .withBootstrapServers(Optional.of(bootstrapServers)) .withKip853Rpc(withKip853Rpc) .build(); @@ -2631,7 +2634,7 @@ public void testObserverLeaderRediscoveryAfterBrokerNotAvailableError(boolean wi .collect(Collectors.toList()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withBootstrapServers(bootstrapServers) + .withBootstrapServers(Optional.of(bootstrapServers)) .withKip853Rpc(withKip853Rpc) .build(); @@ -2682,7 +2685,7 @@ public void testObserverLeaderRediscoveryAfterRequestTimeout(boolean withKip853R .collect(Collectors.toList()); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) - .withBootstrapServers(bootstrapServers) + .withBootstrapServers(Optional.of(bootstrapServers)) .withKip853Rpc(withKip853Rpc) .build(); @@ -4208,7 +4211,7 @@ public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Excepti .collect(Collectors.toList()); RaftClientTestContext context = new RaftClientTestContext.Builder(OptionalInt.empty(), voters) - .withBootstrapServers(bootstrapServers) + .withBootstrapServers(Optional.of(bootstrapServers)) .withKip853Rpc(withKip853Rpc) .build(); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index d12140a6c9..e59513cecb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -168,9 +168,9 @@ public static final class Builder { private int electionTimeoutMs = DEFAULT_ELECTION_TIMEOUT_MS; private int appendLingerMs = DEFAULT_APPEND_LINGER_MS; private MemoryPool memoryPool = MemoryPool.NONE; - private List bootstrapServers = Collections.emptyList(); + private Optional> bootstrapServers = Optional.empty(); private boolean kip853Rpc = false; - private Optional startingVoters = Optional.empty(); + private VoterSet startingVoters = VoterSet.empty(); private Endpoints localListeners = Endpoints.empty(); private boolean isStartingVotersStatic = false; @@ -193,15 +193,7 @@ public Builder(OptionalInt localId, Uuid localDirectoryId) { this.localDirectoryId = localDirectoryId; } - private static IllegalStateException missingStartingVoterException() { - return new IllegalStateException( - "The starting voter set must be set with withStaticVoters or withBootstrapSnapshot" - ); - } - - Builder withElectedLeader(int epoch, int leaderId) { - VoterSet startingVoters = this.startingVoters.orElseThrow(Builder::missingStartingVoterException); quorumStateStore.writeElectionState( ElectionState.withElectedLeader(epoch, leaderId, startingVoters.voterIds()), kraftVersion @@ -210,7 +202,6 @@ Builder withElectedLeader(int epoch, int leaderId) { } Builder withUnknownLeader(int epoch) { - VoterSet startingVoters = this.startingVoters.orElseThrow(Builder::missingStartingVoterException); quorumStateStore.writeElectionState( ElectionState.withUnknownLeader(epoch, startingVoters.voterIds()), kraftVersion @@ -219,7 +210,6 @@ Builder withUnknownLeader(int epoch) { } Builder withVotedCandidate(int epoch, ReplicaKey votedKey) { - VoterSet startingVoters = this.startingVoters.orElseThrow(Builder::missingStartingVoterException); quorumStateStore.writeElectionState( ElectionState.withVotedCandidate(epoch, votedKey, startingVoters.voterIds()), kraftVersion @@ -293,7 +283,7 @@ Builder withRequestTimeoutMs(int requestTimeoutMs) { return this; } - Builder withBootstrapServers(List bootstrapServers) { + Builder withBootstrapServers(Optional> bootstrapServers) { this.bootstrapServers = bootstrapServers; return this; } @@ -319,19 +309,20 @@ Builder withStaticVoters(Set staticVoters) { } Builder withStaticVoters(VoterSet staticVoters) { - this.startingVoters = Optional.of(staticVoters); - this.isStartingVotersStatic = true; + startingVoters = staticVoters; + isStartingVotersStatic = true; + kraftVersion = KRaftVersion.KRAFT_VERSION_0; return this; } Builder withBootstrapSnapshot(Optional voters) { + startingVoters = voters.orElse(VoterSet.empty()); + isStartingVotersStatic = false; + if (voters.isPresent()) { kraftVersion = KRaftVersion.KRAFT_VERSION_1; - startingVoters = voters; - isStartingVotersStatic = false; - RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() .setRawSnapshotWriter( log.createNewSnapshotUnchecked(Snapshots.BOOTSTRAP_SNAPSHOT_ID).get() @@ -357,8 +348,6 @@ Builder withLocalListeners(Endpoints localListeners) { } public RaftClientTestContext build() throws IOException { - VoterSet startingVoters = this.startingVoters.orElseThrow(Builder::missingStartingVoterException); - Metrics metrics = new Metrics(time); MockNetworkChannel channel = new MockNetworkChannel(); MockListener listener = new MockListener(localId); @@ -395,6 +384,18 @@ public RaftClientTestContext build() throws IOException { appendLingerMs ); + List computedBootstrapServers = bootstrapServers.orElseGet(() -> { + if (isStartingVotersStatic) { + return Collections.emptyList(); + } else { + return startingVoters + .voterNodes(startingVoters.voterIds().stream(), channel.listenerName()) + .stream() + .map(node -> InetSocketAddress.createUnresolved(node.host(), node.port())) + .collect(Collectors.toList()); + } + }); + KafkaRaftClient client = new KafkaRaftClient<>( localId, localDirectoryId, @@ -407,7 +408,7 @@ public RaftClientTestContext build() throws IOException { new MockExpirationService(time), FETCH_MAX_WAIT_MS, clusterId, - bootstrapServers, + computedBootstrapServers, localListeners, Features.KRAFT_VERSION.supportedVersionRange(), logContext, @@ -436,7 +437,7 @@ public RaftClientTestContext build() throws IOException { startingVoters, IntStream .iterate(-2, id -> id - 1) - .limit(bootstrapServers.size()) + .limit(bootstrapServers.map(List::size).orElse(0)) .boxed() .collect(Collectors.toSet()), kip853Rpc, diff --git a/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java index 1f41f4fd22..26f2dc662b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java @@ -48,12 +48,12 @@ public final class VoterSetTest { @Test void testEmptyVoterSet() { - assertThrows(IllegalArgumentException.class, () -> new VoterSet(Collections.emptyMap())); + assertEquals(VoterSet.empty(), VoterSet.fromMap(Collections.emptyMap())); } @Test void testVoterNode() { - VoterSet voterSet = new VoterSet(voterMap(IntStream.of(1, 2, 3), true)); + VoterSet voterSet = VoterSet.fromMap(voterMap(IntStream.of(1, 2, 3), true)); assertEquals( Optional.of(new Node(1, "localhost", 9991)), voterSet.voterNode(1, DEFAULT_LISTENER_NAME) @@ -64,7 +64,7 @@ void testVoterNode() { @Test void testVoterNodes() { - VoterSet voterSet = new VoterSet(voterMap(IntStream.of(1, 2, 3), true)); + VoterSet voterSet = VoterSet.fromMap(voterMap(IntStream.of(1, 2, 3), true)); assertEquals( Utils.mkSet(new Node(1, "localhost", 9991), new Node(2, "localhost", 9992)), @@ -84,33 +84,33 @@ void testVoterNodes() { @Test void testVoterIds() { - VoterSet voterSet = new VoterSet(voterMap(IntStream.of(1, 2, 3), true)); + VoterSet voterSet = VoterSet.fromMap(voterMap(IntStream.of(1, 2, 3), true)); assertEquals(new HashSet<>(Arrays.asList(1, 2, 3)), voterSet.voterIds()); } @Test void testAddVoter() { Map aVoterMap = voterMap(IntStream.of(1, 2, 3), true); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); assertEquals(Optional.empty(), voterSet.addVoter(voterNode(1, true))); VoterSet.VoterNode voter4 = voterNode(4, true); aVoterMap.put(voter4.voterKey().id(), voter4); - assertEquals(Optional.of(new VoterSet(new HashMap<>(aVoterMap))), voterSet.addVoter(voter4)); + assertEquals(Optional.of(VoterSet.fromMap(new HashMap<>(aVoterMap))), voterSet.addVoter(voter4)); } @Test void testRemoveVoter() { Map aVoterMap = voterMap(IntStream.of(1, 2, 3), true); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, ReplicaKey.NO_DIRECTORY_ID))); assertEquals(Optional.empty(), voterSet.removeVoter(ReplicaKey.of(4, Uuid.randomUuid()))); VoterSet.VoterNode voter3 = aVoterMap.remove(3); assertEquals( - Optional.of(new VoterSet(new HashMap<>(aVoterMap))), + Optional.of(VoterSet.fromMap(new HashMap<>(aVoterMap))), voterSet.removeVoter(voter3.voterKey()) ); } @@ -118,7 +118,7 @@ void testRemoveVoter() { @Test void testUpdateVoter() { Map aVoterMap = voterMap(IntStream.of(1, 2, 3), true); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); assertEquals(Optional.empty(), voterSet.updateVoter(voterNode(4, true))); assertFalse(voterSet.voterNodeNeedsUpdate(voterNode(4, true))); @@ -140,7 +140,7 @@ void testUpdateVoter() { assertTrue(voterSet.voterNodeNeedsUpdate(newVoter3)); assertEquals( - Optional.of(new VoterSet(new HashMap<>(aVoterMap))), + Optional.of(VoterSet.fromMap(new HashMap<>(aVoterMap))), voterSet.updateVoter(newVoter3) ); } @@ -149,7 +149,7 @@ void testUpdateVoter() { @Test void testCannotRemoveToEmptyVoterSet() { Map aVoterMap = voterMap(IntStream.of(1), true); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); ReplicaKey voter1 = aVoterMap.get(1).voterKey(); assertTrue(voterSet.isVoter(voter1)); @@ -159,7 +159,7 @@ void testCannotRemoveToEmptyVoterSet() { @Test void testIsVoterWithDirectoryId() { Map aVoterMap = voterMap(IntStream.of(1, 2, 3), true); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); assertTrue(voterSet.isVoter(aVoterMap.get(1).voterKey())); assertFalse(voterSet.isVoter(ReplicaKey.of(1, Uuid.randomUuid()))); @@ -176,7 +176,7 @@ void testIsVoterWithDirectoryId() { @Test void testIsVoterWithoutDirectoryId() { Map aVoterMap = voterMap(IntStream.of(1, 2, 3), false); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); assertTrue(voterSet.isVoter(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID))); assertTrue(voterSet.isVoter(ReplicaKey.of(1, Uuid.randomUuid()))); @@ -212,7 +212,7 @@ void testVoterNodeIsVoterWithoutDirectoryId() { @ValueSource(booleans = { true, false }) void testEndpoints(boolean withDirectoryId) { Map aVoterMap = voterMap(IntStream.of(1, 2, 3), withDirectoryId); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); assertNotEquals(Endpoints.empty(), voterSet.listeners(1)); assertNotEquals(Endpoints.empty(), voterSet.listeners(2)); @@ -223,7 +223,7 @@ void testEndpoints(boolean withDirectoryId) { @Test void testIsOnlyVoterInStandalone() { Map aVoterMap = voterMap(IntStream.of(1), true); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); assertTrue(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey())); assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Uuid.randomUuid()))); @@ -237,7 +237,7 @@ void testIsOnlyVoterInStandalone() { @Test void testIsOnlyVoterInNotStandalone() { Map aVoterMap = voterMap(IntStream.of(1, 2), true); - VoterSet voterSet = new VoterSet(new HashMap<>(aVoterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); assertFalse(voterSet.isOnlyVoter(aVoterMap.get(1).voterKey())); assertFalse(voterSet.isOnlyVoter(ReplicaKey.of(1, Uuid.randomUuid()))); @@ -253,7 +253,7 @@ void testIsOnlyVoterInNotStandalone() { @Test void testRecordRoundTrip() { - VoterSet voterSet = new VoterSet(voterMap(IntStream.of(1, 2, 3), true)); + VoterSet voterSet = VoterSet.fromMap(voterMap(IntStream.of(1, 2, 3), true)); assertEquals(voterSet, VoterSet.fromVotersRecord(voterSet.toVotersRecord((short) 0))); } @@ -375,7 +375,7 @@ public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { } public static VoterSet voterSet(Map voters) { - return new VoterSet(voters); + return VoterSet.fromMap(voters); } public static VoterSet voterSet(Stream voterKeys) { diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java index e3673bef52..99226efae2 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java @@ -44,7 +44,7 @@ private static MockLog buildLog() { return new MockLog(new TopicPartition("partition", 0), Uuid.randomUuid(), new LogContext()); } - private static KRaftControlRecordStateMachine buildPartitionListener(MockLog log, Optional staticVoterSet) { + private static KRaftControlRecordStateMachine buildPartitionListener(MockLog log, VoterSet staticVoterSet) { return new KRaftControlRecordStateMachine( staticVoterSet, log, @@ -60,7 +60,7 @@ void testEmptyPartition() { MockLog log = buildLog(); VoterSet voterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); - KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(voterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, voterSet); // This should be a no-op operation partitionState.updateState(); @@ -68,6 +68,18 @@ void testEmptyPartition() { assertEquals(voterSet, partitionState.lastVoterSet()); } + @Test + void testEmptyPartitionWithNoStaticVoters() { + MockLog log = buildLog(); + + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, VoterSet.empty()); + + // This should be a no-op operation + partitionState.updateState(); + + assertEquals(VoterSet.empty(), partitionState.lastVoterSet()); + } + @Test void testUpdateWithoutSnapshot() { MockLog log = buildLog(); @@ -75,7 +87,7 @@ void testUpdateWithoutSnapshot() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, staticVoterSet); // Append the kraft.version control record KRaftVersion kraftVersion = KRaftVersion.KRAFT_VERSION_1; @@ -118,7 +130,7 @@ void testUpdateWithEmptySnapshot() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, staticVoterSet); // Create a snapshot that doesn't have any kraft.version or voter set control records RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() @@ -168,7 +180,7 @@ void testUpdateWithSnapshot() { VoterSet staticVoterSet = VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); int epoch = 1; - KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, staticVoterSet); // Create a snapshot that has kraft.version and voter set control records KRaftVersion kraftVersion = KRaftVersion.KRAFT_VERSION_1; @@ -198,7 +210,7 @@ void testUpdateWithSnapshotAndLogOverride() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, staticVoterSet); // Create a snapshot that has kraft.version and voter set control records KRaftVersion kraftVersion = KRaftVersion.KRAFT_VERSION_1; @@ -245,7 +257,7 @@ void testTruncateTo() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, staticVoterSet); // Append the kraft.version control record KRaftVersion kraftVersion = KRaftVersion.KRAFT_VERSION_1; @@ -313,7 +325,7 @@ void testTrimPrefixTo() { BufferSupplier bufferSupplier = BufferSupplier.NO_CACHING; int epoch = 1; - KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, Optional.of(staticVoterSet)); + KRaftControlRecordStateMachine partitionState = buildPartitionListener(log, staticVoterSet); // Append the kraft.version control record long kraftVersionOffset = log.endOffset().offset(); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index 84904032a5..a8aa1a6e6a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -210,7 +210,7 @@ public void testControlRecordIterationWithKraftVersion0() { @Test public void testControlRecordIterationWithKraftVersion1() { AtomicReference buffer = new AtomicReference<>(null); - VoterSet voterSet = new VoterSet( + VoterSet voterSet = VoterSet.fromMap( VoterSetTest.voterMap(IntStream.of(1, 2, 3), true) ); RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder() diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java index f40311d8b7..04f8aa8d36 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/VoterSetHistoryTest.java @@ -32,8 +32,8 @@ public final class VoterSetHistoryTest { @Test void testStaticVoterSet() { - VoterSet staticVoterSet = new VoterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); - VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + VoterSet staticVoterSet = VoterSet.fromMap(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true)); + VoterSetHistory votersHistory = new VoterSetHistory(staticVoterSet); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); @@ -54,27 +54,27 @@ void testStaticVoterSet() { @Test void TestNoStaticVoterSet() { - VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); + VoterSetHistory votersHistory = new VoterSetHistory(VoterSet.empty()); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(0)); assertEquals(Optional.empty(), votersHistory.valueAtOrBefore(100)); - assertThrows(IllegalStateException.class, votersHistory::lastValue); + assertEquals(VoterSet.empty(), votersHistory.lastValue()); } @Test void testAddAt() { Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); - VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); - VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + VoterSet staticVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(staticVoterSet); assertThrows( IllegalArgumentException.class, - () -> votersHistory.addAt(-2, new VoterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true))) + () -> votersHistory.addAt(-2, VoterSet.fromMap(VoterSetTest.voterMap(IntStream.of(1, 2, 3), true))) ); assertEquals(staticVoterSet, votersHistory.lastValue()); voterMap.put(4, VoterSetTest.voterNode(4, true)); - VoterSet addedVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSet addedVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(100, addedVoterSet); assertEquals(addedVoterSet, votersHistory.lastValue()); @@ -82,7 +82,7 @@ void testAddAt() { assertEquals(Optional.of(addedVoterSet), votersHistory.valueAtOrBefore(100)); voterMap.remove(4); - VoterSet removedVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSet removedVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(200, removedVoterSet); assertEquals(removedVoterSet, votersHistory.lastValue()); @@ -94,8 +94,8 @@ void testAddAt() { @Test void testBootstrapAddAt() { Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); - VoterSet bootstrapVoterSet = new VoterSet(new HashMap<>(voterMap)); - VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); + VoterSet bootstrapVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(VoterSet.empty()); votersHistory.addAt(-1, bootstrapVoterSet); assertEquals(bootstrapVoterSet, votersHistory.lastValue()); @@ -103,7 +103,7 @@ void testBootstrapAddAt() { assertEquals(Optional.of(bootstrapVoterSet), votersHistory.valueAtOrBefore(-1)); voterMap.put(4, VoterSetTest.voterNode(4, true)); - VoterSet addedVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSet addedVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(100, addedVoterSet); assertEquals(addedVoterSet, votersHistory.lastValue()); @@ -112,7 +112,7 @@ void testBootstrapAddAt() { assertEquals(Optional.of(addedVoterSet), votersHistory.valueAtOrBefore(100)); voterMap.remove(4); - VoterSet removedVoterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSet removedVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(200, removedVoterSet); assertEquals(removedVoterSet, votersHistory.lastValue()); @@ -124,10 +124,10 @@ void testBootstrapAddAt() { @Test void testAddAtNonOverlapping() { - VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); + VoterSetHistory votersHistory = new VoterSetHistory(VoterSet.empty()); Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); - VoterSet voterSet = new VoterSet(new HashMap<>(voterMap)); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(voterMap)); // Add a starting voter to the history votersHistory.addAt(100, voterSet); @@ -159,8 +159,8 @@ void testAddAtNonOverlapping() { @Test void testNonoverlappingFromStaticVoterSet() { Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); - VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); - VoterSetHistory votersHistory = new VoterSetHistory(Optional.empty()); + VoterSet staticVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(VoterSet.empty()); // Remove voter so that it doesn't overlap VoterSet nonoverlappingRemovedSet = staticVoterSet @@ -174,17 +174,17 @@ void testNonoverlappingFromStaticVoterSet() { @Test void testTruncateTo() { Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); - VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); - VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + VoterSet staticVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(staticVoterSet); // Add voter 4 to the voter set and voter set history voterMap.put(4, VoterSetTest.voterNode(4, true)); - VoterSet voterSet1234 = new VoterSet(new HashMap<>(voterMap)); + VoterSet voterSet1234 = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(100, voterSet1234); // Add voter 5 to the voter set and voter set history voterMap.put(5, VoterSetTest.voterNode(5, true)); - VoterSet voterSet12345 = new VoterSet(new HashMap<>(voterMap)); + VoterSet voterSet12345 = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(200, voterSet12345); votersHistory.truncateNewEntries(201); @@ -200,17 +200,17 @@ void testTruncateTo() { @Test void testTrimPrefixTo() { Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); - VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); - VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + VoterSet staticVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(staticVoterSet); // Add voter 4 to the voter set and voter set history voterMap.put(4, VoterSetTest.voterNode(4, true)); - VoterSet voterSet1234 = new VoterSet(new HashMap<>(voterMap)); + VoterSet voterSet1234 = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(100, voterSet1234); // Add voter 5 to the voter set and voter set history voterMap.put(5, VoterSetTest.voterNode(5, true)); - VoterSet voterSet12345 = new VoterSet(new HashMap<>(voterMap)); + VoterSet voterSet12345 = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(200, voterSet12345); votersHistory.truncateOldEntries(99); @@ -233,17 +233,17 @@ void testTrimPrefixTo() { @Test void testClear() { Map voterMap = VoterSetTest.voterMap(IntStream.of(1, 2, 3), true); - VoterSet staticVoterSet = new VoterSet(new HashMap<>(voterMap)); - VoterSetHistory votersHistory = new VoterSetHistory(Optional.of(staticVoterSet)); + VoterSet staticVoterSet = VoterSet.fromMap(new HashMap<>(voterMap)); + VoterSetHistory votersHistory = new VoterSetHistory(staticVoterSet); // Add voter 4 to the voter set and voter set history voterMap.put(4, VoterSetTest.voterNode(4, true)); - VoterSet voterSet1234 = new VoterSet(new HashMap<>(voterMap)); + VoterSet voterSet1234 = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(100, voterSet1234); // Add voter 5 to the voter set and voter set history voterMap.put(5, VoterSetTest.voterNode(5, true)); - VoterSet voterSet12345 = new VoterSet(new HashMap<>(voterMap)); + VoterSet voterSet12345 = VoterSet.fromMap(new HashMap<>(voterMap)); votersHistory.addAt(200, voterSet12345); votersHistory.clear(); From 313af4e83de20a686222f1efce22f5b3e9cbc1a0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Mon, 19 Aug 2024 20:44:23 -0400 Subject: [PATCH 054/123] KAFKA-17332; Controller always flush and can call resign on observers (#16907) This change includes two improvements. When the leader removes itself from the voters set clients of RaftClient may call resign. In those cases the leader is not in the voter set and should not throw an exception. Controllers that are observers must flush the log on every append because leader may be trying to add them to the voter set. Leader always assume that voters flush their disk before sending a Fetch request. Reviewers: David Arthur , Alyssa Huang --- .../main/scala/kafka/raft/RaftManager.scala | 2 + .../apache/kafka/raft/KafkaRaftClient.java | 17 ++- .../raft/KafkaRaftClientReconfigTest.java | 109 ++++++++++-------- .../kafka/raft/KafkaRaftClientTest.java | 32 +++++ .../kafka/raft/RaftClientTestContext.java | 16 ++- .../kafka/raft/RaftEventSimulationTest.java | 1 + 6 files changed, 120 insertions(+), 57 deletions(-) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 25c8e8b294..d248443e61 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -235,6 +235,8 @@ class KafkaRaftManager[T]( time, expirationService, logContext, + // Controllers should always flush the log on replication because they may become voters + config.processRoles.contains(ProcessRole.ControllerRole), clusterId, bootstrapServers, localListeners, diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 4d151b3128..b4ccdfbc80 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -173,6 +173,7 @@ public final class KafkaRaftClient implements RaftClient { private final Logger logger; private final Time time; private final int fetchMaxWaitMs; + private final boolean followersAlwaysFlush; private final String clusterId; private final Endpoints localListeners; private final SupportedVersionRange localSupportedKRaftVersion; @@ -219,6 +220,8 @@ public final class KafkaRaftClient implements RaftClient { * * Note that if the node ID is empty, then the client will behave as a * non-participating observer. + * + * @param followersAlwaysFlush instruct followers to always fsync when appending to the log */ public KafkaRaftClient( OptionalInt nodeId, @@ -229,6 +232,7 @@ public KafkaRaftClient( Time time, ExpirationService expirationService, LogContext logContext, + boolean followersAlwaysFlush, String clusterId, Collection bootstrapServers, Endpoints localListeners, @@ -246,6 +250,7 @@ public KafkaRaftClient( time, expirationService, MAX_FETCH_WAIT_MS, + followersAlwaysFlush, clusterId, bootstrapServers, localListeners, @@ -267,6 +272,7 @@ public KafkaRaftClient( Time time, ExpirationService expirationService, int fetchMaxWaitMs, + boolean followersAlwaysFlush, String clusterId, Collection bootstrapServers, Endpoints localListeners, @@ -290,6 +296,7 @@ public KafkaRaftClient( this.localListeners = localListeners; this.localSupportedKRaftVersion = localSupportedKRaftVersion; this.fetchMaxWaitMs = fetchMaxWaitMs; + this.followersAlwaysFlush = followersAlwaysFlush; this.logger = logContext.logger(KafkaRaftClient.class); this.random = random; this.quorumConfig = quorumConfig; @@ -1684,9 +1691,11 @@ private void appendAsFollower( Records records ) { LogAppendInfo info = log.appendAsFollower(records); - if (quorum.isVoter()) { - // the leader only requires that voters have flushed their log before sending - // a Fetch request + if (quorum.isVoter() || followersAlwaysFlush) { + // the leader only requires that voters have flushed their log before sending a Fetch + // request. Because of reconfiguration some observers (that are getting added to the + // voter set) need to flush the disk because the leader may assume that they are in the + // set of voters. log.flush(false); } @@ -3355,8 +3364,6 @@ public void resign(int epoch) { throw new IllegalArgumentException("Attempt to resign from an invalid negative epoch " + epoch); } else if (!isInitialized()) { throw new IllegalStateException("Replica needs to be initialized before resigning"); - } else if (!quorum.isVoter()) { - throw new IllegalStateException("Attempt to resign by a non-voter"); } LeaderAndEpoch leaderAndEpoch = leaderAndEpoch(); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 77825f5328..63e7f046b1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -67,7 +67,7 @@ public class KafkaRaftClientReconfigTest { @Test public void testLeaderWritesBootstrapRecords() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -142,7 +142,7 @@ record = recordIterator.next(); @Test public void testBootstrapCheckpointIsNotReturnedOnFetch() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -172,7 +172,7 @@ public void testBootstrapCheckpointIsNotReturnedOnFetch() throws Exception { @Test public void testLeaderDoesNotBootstrapRecordsWithKraftVersion0() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -220,7 +220,7 @@ public void testLeaderDoesNotBootstrapRecordsWithKraftVersion0() throws Exceptio @Test public void testFollowerDoesNotRequestLeaderBootstrapSnapshot() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey leader = replicaKey(local.id() + 1, true); int epoch = 1; @@ -250,7 +250,7 @@ public void testFollowerDoesNotRequestLeaderBootstrapSnapshot() throws Exception @Test public void testFollowerReadsKRaftBootstrapRecords() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey leader = replicaKey(local.id() + 1, true); ReplicaKey follower = replicaKey(local.id() + 2, true); VoterSet voterSet = VoterSetTest.voterSet(Stream.of(local, leader)); @@ -320,7 +320,7 @@ public void testFollowerReadsKRaftBootstrapRecords() throws Exception { @Test public void testAddVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -399,7 +399,7 @@ public void testAddVoter() throws Exception { @Test void testAddVoterInvalidClusterId() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -436,7 +436,7 @@ void testAddVoterInvalidClusterId() throws Exception { @Test void testAddVoterToNotLeader() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -464,7 +464,7 @@ void testAddVoterToNotLeader() throws Exception { @Test void testAddVoterWithMissingDefaultListener() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -494,7 +494,7 @@ void testAddVoterWithMissingDefaultListener() throws Exception { @Test void testAddVoterWithPendingAddVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -550,7 +550,7 @@ void testAddVoterWithPendingAddVoter() throws Exception { @Test void testAddVoterWithoutFencedPreviousLeaders() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -588,7 +588,7 @@ void testAddVoterWithoutFencedPreviousLeaders() throws Exception { @Test void testAddVoterWithKraftVersion0() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -633,7 +633,7 @@ void testAddVoterWithKraftVersion0() throws Exception { @Test void testAddVoterWithExistingVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -678,7 +678,7 @@ void testAddVoterWithExistingVoter() throws Exception { @Test void testAddVoterTimeout() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -750,7 +750,7 @@ void testAddVoterTimeout() throws Exception { @Test void testAddVoterWithApiVersionsFromIncorrectNode() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -810,7 +810,7 @@ void testAddVoterWithApiVersionsFromIncorrectNode() throws Exception { @Test void testAddVoterInvalidFeatureVersion() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -870,7 +870,7 @@ void testAddVoterInvalidFeatureVersion() throws Exception { @Test void testAddVoterWithLaggingNewVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -923,7 +923,7 @@ void testAddVoterWithLaggingNewVoter() throws Exception { @Test void testAddVoterFailsWhenLosingLeadership() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -972,7 +972,7 @@ void testAddVoterFailsWhenLosingLeadership() throws Exception { @Test void testAddVoterWithMissingDirectoryId() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1010,7 +1010,7 @@ void testAddVoterWithMissingDirectoryId() throws Exception { @Test public void testRemoveVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1059,7 +1059,7 @@ public void testRemoveVoter() throws Exception { @Test public void testRemoveVoterIsLeader() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1112,9 +1112,16 @@ public void testRemoveVoterIsLeader() throws Exception { // Expect END_QUORUM_EPOCH requests context.pollUntilRequest(); - context.collectEndQuorumRequests(epoch, new HashSet<>(Arrays.asList(follower1.id(), follower2.id())), Optional.empty()); + context.collectEndQuorumRequests( + epoch, + new HashSet<>(Arrays.asList(follower1.id(), follower2.id())), + Optional.empty() + ); + + // Calls to resign should be allowed and not throw an exception + context.client.resign(epoch); - // Election timeout is randome numer in [electionTimeoutMs, 2 * electionTimeoutMs) + // Election timeout is random numer in [electionTimeoutMs, 2 * electionTimeoutMs) context.time.sleep(2 * context.electionTimeoutMs()); context.client.poll(); @@ -1124,7 +1131,7 @@ public void testRemoveVoterIsLeader() throws Exception { @Test public void testRemoveVoterInvalidClusterId() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1153,7 +1160,7 @@ public void testRemoveVoterInvalidClusterId() throws Exception { @Test void testRemoveVoterToNotLeader() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1173,7 +1180,7 @@ void testRemoveVoterToNotLeader() throws Exception { @Test void testRemoveVoterWithPendingRemoveVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1211,7 +1218,7 @@ void testRemoveVoterWithPendingRemoveVoter() throws Exception { @Test void testRemoveVoterWithoutFencedPreviousLeaders() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1233,7 +1240,7 @@ void testRemoveVoterWithoutFencedPreviousLeaders() throws Exception { @Test void testRemoveVoterWithKraftVersion0() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1263,7 +1270,7 @@ void testRemoveVoterWithKraftVersion0() throws Exception { @Test void testRemoveVoterWithNoneVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1293,7 +1300,7 @@ void testRemoveVoterWithNoneVoter() throws Exception { @Test void testRemoveVoterWithNoneVoterId() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1327,7 +1334,7 @@ void testRemoveVoterWithNoneVoterId() throws Exception { @Test void testRemoveVoterToEmptyVoterSet() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local)); @@ -1345,7 +1352,7 @@ void testRemoveVoterToEmptyVoterSet() throws Exception { @Test void testRemoveVoterTimedOut() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1389,7 +1396,7 @@ void testRemoveVoterTimedOut() throws Exception { @Test void testRemoveVoterFailsWhenLosingLeadership() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1431,7 +1438,7 @@ void testRemoveVoterFailsWhenLosingLeadership() throws Exception { @Test void testAddVoterWithPendingRemoveVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower1 = replicaKey(local.id() + 1, true); ReplicaKey follower2 = replicaKey(local.id() + 2, true); @@ -1477,7 +1484,7 @@ void testAddVoterWithPendingRemoveVoter() throws Exception { @Test void testRemoveVoterWithPendingAddVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1525,7 +1532,7 @@ void testRemoveVoterWithPendingAddVoter() throws Exception { @Test void testUpdateVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1583,7 +1590,7 @@ void testUpdateVoter() throws Exception { @Test void testLeaderUpdatesVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1630,7 +1637,7 @@ void testLeaderUpdatesVoter() throws Exception { @Test public void testUpdateVoterInvalidClusterId() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1681,7 +1688,7 @@ public void testUpdateVoterInvalidClusterId() throws Exception { @Test void testUpdateVoterOldEpoch() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1714,7 +1721,7 @@ void testUpdateVoterOldEpoch() throws Exception { @Test void testUpdateVoterNewEpoch() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1747,7 +1754,7 @@ void testUpdateVoterNewEpoch() throws Exception { @Test void testUpdateVoterToNotLeader() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1776,7 +1783,7 @@ void testUpdateVoterToNotLeader() throws Exception { @Test void testUpdateVoterWithoutFencedPreviousLeaders() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1821,7 +1828,7 @@ void testUpdateVoterWithoutFencedPreviousLeaders() throws Exception { // KAFKA-16538 is going to allow UpdateVoter RPC when the kraft.version is 0 @Test void testUpdateVoterWithKraftVersion0() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1872,7 +1879,7 @@ void testUpdateVoterWithKraftVersion0() throws Exception { @Test void testUpdateVoterWithNoneVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1923,7 +1930,7 @@ void testUpdateVoterWithNoneVoter() throws Exception { @Test void testUpdateVoterWithNoneVoterId() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -1974,7 +1981,7 @@ void testUpdateVoterWithNoneVoterId() throws Exception { @Test void testUpdateVoterWithPendingAddVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey follower = replicaKey(local.id() + 1, true); VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); @@ -2044,7 +2051,7 @@ void testUpdateVoterWithPendingAddVoter() throws Exception { @Test void testFollowerSendsUpdateVoter() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey voter1 = replicaKey(local.id() + 1, true); ReplicaKey voter2 = replicaKey(local.id() + 2, true); @@ -2116,7 +2123,7 @@ void testFollowerSendsUpdateVoter() throws Exception { @Test void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey voter1 = replicaKey(local.id() + 1, true); ReplicaKey voter2 = replicaKey(local.id() + 2, true); @@ -2160,7 +2167,7 @@ void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { @Test void testUpdateVoterResponseCausesEpochChange() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); ReplicaKey voter1 = replicaKey(local.id() + 1, true); ReplicaKey voter2 = replicaKey(local.id() + 2, true); @@ -2233,7 +2240,7 @@ void testUpdateVoterResponseCausesEpochChange() throws Exception { @Test void testObserverDiscoversLeaderWithUnknownVoters() throws Exception { - ReplicaKey local = replicaKey(randomeReplicaId(), true); + ReplicaKey local = replicaKey(randomReplicaId(), true); InetSocketAddress bootstrapAdddress = InetSocketAddress.createUnresolved("localhost", 1234); int epoch = 3; @@ -2273,7 +2280,7 @@ private static void verifyKRaftVersionRecord( assertEquals(expectedKRaftVersion, kRaftVersionRecord.kRaftVersion()); } - private int randomeReplicaId() { + private int randomReplicaId() { return ThreadLocalRandom.current().nextInt(1025); } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 62d81621cd..632acaecf8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -3396,6 +3396,38 @@ public void testFollowerReplication(boolean withKip853Rpc) throws Exception { assertEquals(2L, context.log.firstUnflushedOffset()); } + @ParameterizedTest + @CsvSource({ "true, true", "true, false", "false, true", "false, false" }) + public void testObserverReplication(boolean withKip853Rpc, boolean alwaysFlush) throws Exception { + int localId = randomReplicaId(); + int otherNodeId = localId + 1; + int epoch = 5; + Set voters = Utils.mkSet(otherNodeId); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withElectedLeader(epoch, otherNodeId) + .withKip853Rpc(withKip853Rpc) + .withAlwaysFlush(alwaysFlush) + .build(); + context.assertElectedLeader(epoch, otherNodeId); + + context.pollUntilRequest(); + + RaftRequest.Outbound fetchQuorumRequest = context.assertSentFetchRequest(epoch, 0L, 0); + Records records = context.buildBatch(0L, 3, Arrays.asList("a", "b")); + FetchResponseData response = context.fetchResponse(epoch, otherNodeId, records, 0L, Errors.NONE); + context.deliverResponse( + fetchQuorumRequest.correlationId(), + fetchQuorumRequest.destination(), + response + ); + + context.client.poll(); + assertEquals(2L, context.log.endOffset().offset()); + long firstUnflushedOffset = alwaysFlush ? 2L : 0L; + assertEquals(firstUnflushedOffset, context.log.firstUnflushedOffset()); + } + @ParameterizedTest @ValueSource(booleans = { true, false }) public void testEmptyRecordSetInFetchResponse(boolean withKip853Rpc) throws Exception { diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index e59513cecb..83677dccd1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -137,6 +137,8 @@ public final class RaftClientTestContext { final Set bootstrapIds; // Used to determine which RPC request and response to construct final boolean kip853Rpc; + // Used to determine if the local kraft client was configured to always flush + final boolean alwaysFlush; private final List sentResponses = new ArrayList<>(); @@ -170,6 +172,7 @@ public static final class Builder { private MemoryPool memoryPool = MemoryPool.NONE; private Optional> bootstrapServers = Optional.empty(); private boolean kip853Rpc = false; + private boolean alwaysFlush = false; private VoterSet startingVoters = VoterSet.empty(); private Endpoints localListeners = Endpoints.empty(); private boolean isStartingVotersStatic = false; @@ -293,6 +296,11 @@ Builder withKip853Rpc(boolean kip853Rpc) { return this; } + Builder withAlwaysFlush(boolean alwaysFlush) { + this.alwaysFlush = alwaysFlush; + return this; + } + Builder withStaticVoters(Set staticVoters) { Map staticVoterAddressMap = staticVoters .stream() @@ -407,6 +415,7 @@ public RaftClientTestContext build() throws IOException { time, new MockExpirationService(time), FETCH_MAX_WAIT_MS, + alwaysFlush, clusterId, computedBootstrapServers, localListeners, @@ -441,6 +450,7 @@ public RaftClientTestContext build() throws IOException { .boxed() .collect(Collectors.toSet()), kip853Rpc, + alwaysFlush, metrics, listener ); @@ -468,6 +478,7 @@ private RaftClientTestContext( VoterSet startingVoters, Set bootstrapIds, boolean kip853Rpc, + boolean alwaysFlush, Metrics metrics, MockListener listener ) { @@ -484,6 +495,7 @@ private RaftClientTestContext( this.startingVoters = startingVoters; this.bootstrapIds = bootstrapIds; this.kip853Rpc = kip853Rpc; + this.alwaysFlush = alwaysFlush; this.metrics = metrics; this.listener = listener; } @@ -1531,7 +1543,9 @@ void assertFetchRequestData( assertEquals(localId.orElse(-1), request.replicaState().replicaId()); // Assert that voters have flushed up to the fetch offset - if (localId.isPresent() && startingVoters.voterIds().contains(localId.getAsInt())) { + if ((localId.isPresent() && startingVoters.voterIds().contains(localId.getAsInt())) || + alwaysFlush + ) { assertEquals( log.firstUnflushedOffset(), fetchOffset, diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 9b1a325dd2..cd8743635c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -789,6 +789,7 @@ void start(int nodeId) { time, new MockExpirationService(time), FETCH_MAX_WAIT_MS, + true, clusterId, Collections.emptyList(), endpointsFromId(nodeId, channel.listenerName()), From b7a97e7102c2202c425de347d2f731f97c470dd0 Mon Sep 17 00:00:00 2001 From: Mason Chen Date: Wed, 21 Aug 2024 10:08:28 +1200 Subject: [PATCH 055/123] KAFKA-17169: Add EndpointsTest (#16659) Reviewers: Omnia Ibrahim , Colin P. McCabe --- .../org/apache/kafka/raft/EndpointsTest.java | 363 ++++++++++++++++++ 1 file changed, 363 insertions(+) create mode 100644 raft/src/test/java/org/apache/kafka/raft/EndpointsTest.java diff --git a/raft/src/test/java/org/apache/kafka/raft/EndpointsTest.java b/raft/src/test/java/org/apache/kafka/raft/EndpointsTest.java new file mode 100644 index 0000000000..a4a39de097 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/EndpointsTest.java @@ -0,0 +1,363 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import org.apache.kafka.common.message.BeginQuorumEpochRequestData; +import org.apache.kafka.common.message.BeginQuorumEpochResponseData; +import org.apache.kafka.common.message.EndQuorumEpochRequestData; +import org.apache.kafka.common.message.EndQuorumEpochResponseData; +import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.message.FetchSnapshotResponseData; +import org.apache.kafka.common.message.VoteResponseData; +import org.apache.kafka.common.message.VotersRecord; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.utils.Utils; + +import org.junit.jupiter.api.Test; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; + +final class EndpointsTest { + private ListenerName testListener = ListenerName.normalised("listener"); + private InetSocketAddress testSocketAddress = InetSocketAddress.createUnresolved("localhost", 9092); + + @Test + void testAddressWithValidEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + Optional address = endpoints.address(testListener); + + assertEquals(Optional.of(testSocketAddress), address); + } + + @Test + void testAddressWithEmptyEndpoint() { + Endpoints endpoints = Endpoints.empty(); + + Optional address = endpoints.address(ListenerName.normalised("nonExistentListener")); + + assertEquals(Optional.empty(), address); + } + + @Test + void testVotersRecordEndpointsWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + VotersRecord.Endpoint endpoint = endpoints.votersRecordEndpoints().next(); + + assertEquals(testListener.value(), endpoint.name()); + assertEquals("localhost", endpoint.host()); + assertEquals(9092, endpoint.port()); + } + + @Test + void testVotersRecordEndpointsWithEmptyEndpoint() { + assertFalse(Endpoints.empty().votersRecordEndpoints().hasNext()); + } + + @Test + void testSize() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + + assertEquals(1, Endpoints.fromInetSocketAddresses(endpointMap).size()); + assertEquals(0, Endpoints.empty().size()); + } + + @Test + void testIsEmptyWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + + assertFalse(Endpoints.fromInetSocketAddresses(endpointMap).isEmpty()); + } + + @Test + void testEqualsAndHashCodeWithSameEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + Endpoints sameEndpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + assertEquals(endpoints, sameEndpoints); + assertEquals(endpoints.hashCode(), sameEndpoints.hashCode()); + } + + @Test + void testEqualsAndHashCodeWithDifferentEndpoints() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + Map anotherEndpointMap = Utils.mkMap( + Utils.mkEntry( + ListenerName.normalised("another"), + InetSocketAddress.createUnresolved("localhost", 9093))); + Endpoints differentEndpoints = Endpoints.fromInetSocketAddresses(anotherEndpointMap); + + assertNotEquals(endpoints, differentEndpoints); + assertNotEquals(endpoints.hashCode(), differentEndpoints.hashCode()); + } + + @Test + void testToBeginQuorumEpochRequestWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + BeginQuorumEpochRequestData.LeaderEndpointCollection leaderEndpoints = endpoints.toBeginQuorumEpochRequest(); + + assertEquals(1, leaderEndpoints.size()); + + BeginQuorumEpochRequestData.LeaderEndpoint leaderEndpoint = leaderEndpoints.iterator().next(); + assertEquals(testListener.value(), leaderEndpoint.name()); + assertEquals("localhost", leaderEndpoint.host()); + assertEquals(9092, leaderEndpoint.port()); + } + + @Test + void testToBeginQuorumEpochRequestWithEmptyEndpoint() { + assertEquals(0, Endpoints.empty().toBeginQuorumEpochRequest().size()); + } + + @Test + void testFromInetSocketAddressesWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + + assertEquals(1, Endpoints.fromInetSocketAddresses(endpointMap).size()); + } + + @Test + void testFromVotersRecordEndpointsWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + List votersEndpoints = new ArrayList<>(); + votersEndpoints.add( + new VotersRecord.Endpoint() + .setName("listener") + .setHost("localhost") + .setPort(9092)); + + Endpoints createdEndpoints = Endpoints.fromVotersRecordEndpoints(votersEndpoints); + + assertEquals(endpoints, createdEndpoints); + } + + @Test + void testFromVotersRecordEndpointsWithEmptyEndpoint() { + List votersEndpoints = Collections.emptyList(); + + assertEquals(Endpoints.empty(), Endpoints.fromVotersRecordEndpoints(votersEndpoints)); + } + + @Test + void testFromBeginQuorumEpochRequestWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + BeginQuorumEpochRequestData.LeaderEndpointCollection leaderEndpoints = new BeginQuorumEpochRequestData.LeaderEndpointCollection(); + leaderEndpoints.add( + new BeginQuorumEpochRequestData.LeaderEndpoint() + .setName("listener") + .setHost("localhost") + .setPort(9092)); + + Endpoints createdEndpoints = Endpoints.fromBeginQuorumEpochRequest(leaderEndpoints); + + assertEquals(endpoints, createdEndpoints); + } + + @Test + void testFromBeginQuorumEpochRequestWithEmptyEndpoint() { + BeginQuorumEpochRequestData.LeaderEndpointCollection leaderEndpoints = new BeginQuorumEpochRequestData.LeaderEndpointCollection(); + + Endpoints createdEndpoints = Endpoints.fromBeginQuorumEpochRequest(leaderEndpoints); + + assertEquals(Endpoints.empty(), createdEndpoints); + } + + @Test + void testFromBeginQuorumEpochResponseWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + BeginQuorumEpochResponseData.NodeEndpointCollection nodeEndpointCollection = new BeginQuorumEpochResponseData.NodeEndpointCollection(); + nodeEndpointCollection.add( + new BeginQuorumEpochResponseData.NodeEndpoint() + .setNodeId(1) + .setHost("localhost") + .setPort(9092)); + + Endpoints createdEndpoints = Endpoints.fromBeginQuorumEpochResponse(testListener, 1, nodeEndpointCollection); + + assertEquals(endpoints, createdEndpoints); + } + + @Test + void testFromBeginQuorumEpochResponseWithEmptyEndpoint() { + BeginQuorumEpochResponseData.NodeEndpointCollection nodeEndpointCollection = new BeginQuorumEpochResponseData.NodeEndpointCollection(); + + Endpoints createdEndpoints = Endpoints.fromBeginQuorumEpochResponse(ListenerName.normalised("nonExistListener"), + 1, nodeEndpointCollection); + + assertEquals(Endpoints.empty(), createdEndpoints); + } + + @Test + void testFromEndQuorumEpochRequestWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + EndQuorumEpochRequestData.LeaderEndpointCollection leaderEndpoints = new EndQuorumEpochRequestData.LeaderEndpointCollection(); + leaderEndpoints.add( + new EndQuorumEpochRequestData.LeaderEndpoint() + .setName("listener") + .setHost("localhost") + .setPort(9092)); + + assertEquals(endpoints, Endpoints.fromEndQuorumEpochRequest(leaderEndpoints)); + } + + @Test + void testFromEndQuorumEpochRequestWithEmptyEndpoint() { + EndQuorumEpochRequestData.LeaderEndpointCollection leaderEndpoints = new EndQuorumEpochRequestData.LeaderEndpointCollection(); + + assertEquals(Endpoints.empty(), Endpoints.fromEndQuorumEpochRequest(leaderEndpoints)); + } + + @Test + void testFromEndQuorumEpochResponseWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + EndQuorumEpochResponseData.NodeEndpointCollection nodeEndpointCollection = new EndQuorumEpochResponseData.NodeEndpointCollection(); + nodeEndpointCollection.add( + new EndQuorumEpochResponseData.NodeEndpoint() + .setNodeId(1) + .setHost("localhost") + .setPort(9092)); + + assertEquals(endpoints, Endpoints.fromEndQuorumEpochResponse(testListener, 1, nodeEndpointCollection)); + } + + @Test + void testFromEndQuorumEpochResponseWithEmptyEndpoint() { + EndQuorumEpochResponseData.NodeEndpointCollection nodeEndpointCollection = new EndQuorumEpochResponseData.NodeEndpointCollection(); + + Endpoints createdEndpoints = Endpoints.fromEndQuorumEpochResponse(ListenerName.normalised("nonExistListener"), + 1, nodeEndpointCollection); + + assertEquals(Endpoints.empty(), createdEndpoints); + } + + @Test + void testFromVoteResponseWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + VoteResponseData.NodeEndpointCollection nodeEndpointCollection = new VoteResponseData.NodeEndpointCollection(); + nodeEndpointCollection.add( + new VoteResponseData.NodeEndpoint() + .setNodeId(1) + .setHost("localhost") + .setPort(9092)); + + assertEquals(endpoints, Endpoints.fromVoteResponse(testListener, 1, nodeEndpointCollection)); + } + + @Test + void testFromVoteResponseWithEmptyEndpoint() { + VoteResponseData.NodeEndpointCollection nodeEndpointCollection = new VoteResponseData.NodeEndpointCollection(); + + Endpoints createdEndpoints = Endpoints.fromVoteResponse(ListenerName.normalised("nonExistListener"), 1, + nodeEndpointCollection); + + assertEquals(Endpoints.empty(), createdEndpoints); + } + + @Test + void testFromFetchResponseWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + FetchResponseData.NodeEndpointCollection nodeEndpointCollection = new FetchResponseData.NodeEndpointCollection(); + nodeEndpointCollection.add( + new FetchResponseData.NodeEndpoint() + .setNodeId(1) + .setHost("localhost") + .setPort(9092)); + + assertEquals(endpoints, Endpoints.fromFetchResponse(testListener, 1, nodeEndpointCollection)); + } + + @Test + void testFromFetchResponseWithEmptyEndpoint() { + FetchResponseData.NodeEndpointCollection nodeEndpointCollection = new FetchResponseData.NodeEndpointCollection(); + + assertEquals(Endpoints.empty(), Endpoints.fromFetchResponse(ListenerName.normalised("nonExistListener"), 1, + nodeEndpointCollection)); + } + + @Test + void testFromFetchSnapshotResponseWithEndpoint() { + Map endpointMap = Utils.mkMap( + Utils.mkEntry(testListener, testSocketAddress)); + Endpoints endpoints = Endpoints.fromInetSocketAddresses(endpointMap); + + FetchSnapshotResponseData.NodeEndpointCollection nodeEndpointCollection = new FetchSnapshotResponseData.NodeEndpointCollection(); + nodeEndpointCollection.add( + new FetchSnapshotResponseData.NodeEndpoint() + .setNodeId(1) + .setHost("localhost") + .setPort(9092)); + + assertEquals(endpoints, Endpoints.fromFetchSnapshotResponse(testListener, 1, nodeEndpointCollection)); + } + + @Test + void testFromFetchSnapshotResponseWithEmptyEndpoint() { + FetchSnapshotResponseData.NodeEndpointCollection nodeEndpointCollection = new FetchSnapshotResponseData.NodeEndpointCollection(); + + Endpoints createdEndpoints = Endpoints.fromFetchSnapshotResponse(ListenerName.normalised("nonExistListener"), 1, + nodeEndpointCollection); + + assertEquals(Endpoints.empty(), createdEndpoints); + } +} From 321ab711926427410d74c2922e6050da5e65270b Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Wed, 21 Aug 2024 13:17:31 +0100 Subject: [PATCH 056/123] KAFKA-17279: Handle retriable errors from offset fetches (#16826) (#16934) Handle retriable errors from offset fetches in ConsumerCoordinator. Reviewers: Lianet Magrans , David Jacot --- .../internals/ConsumerCoordinator.java | 9 ++--- .../internals/ConsumerCoordinatorTest.java | 33 ++++++++----------- 2 files changed, 19 insertions(+), 23 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 785df2362a..8647476758 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -1457,15 +1457,16 @@ public void handle(OffsetFetchResponse response, RequestFuture coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE))); } - @Test - public void testRefreshOffsetNotCoordinatorForConsumer() { + @ParameterizedTest + @CsvSource({ + "NOT_COORDINATOR, true", + "COORDINATOR_NOT_AVAILABLE, true", + "COORDINATOR_LOAD_IN_PROGRESS, false", + "NETWORK_EXCEPTION, false", + }) + public void testRefreshOffsetRetriableErrorCoordinatorLookup(Errors error, boolean expectCoordinatorRelookup) { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); - client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + client.prepareResponse(offsetFetchResponse(error, Collections.emptyMap())); + if (expectCoordinatorRelookup) { + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); + } client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); From 283b82d46afe73ed2d864f693eb4fcb484ad62cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 20 Aug 2024 19:24:13 -0400 Subject: [PATCH 057/123] KAFKA-17333; ResignedState should not notify of leader change (#16900) When a voter fails as leader (LeaderState) the quorum-state still states that it is the leader of the epoch. When the voter starts it never starts as leader and instead starts as resigned (ResignedState) if it was previously a leader. This causes the KRaft client to immediately notify the state machine (e.g QuorumController) that it is leader or active. This is incorrect for two reasons. One, the controller cannot be notified of leadership until it has reached the LEO. If the controller is notified before that it will generate and append records that are not based on the latest state. Two, it is not practical to notify of local leadership when it is resigned since any write operation (prepareAppend and schedulePreparedAppend) will fail with NotLeaderException while KRaft is in the resigned state. Reviewers: Colin P. McCabe , David Arthur --- .../apache/kafka/raft/KafkaRaftClient.java | 11 +++++-- .../org/apache/kafka/raft/QuorumState.java | 1 + .../kafka/raft/KafkaRaftClientTest.java | 30 +++++++++++++++++++ .../kafka/raft/RaftClientTestContext.java | 2 +- 4 files changed, 41 insertions(+), 3 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index b4ccdfbc80..e8ab7b5c6d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -684,7 +684,6 @@ private void transitionToResigned(List preferredSuccessors) { fetchPurgatory.completeAllExceptionally( Errors.NOT_LEADER_OR_FOLLOWER.exception("Not handling request since this node is resigning")); quorum.transitionToResigned(preferredSuccessors); - maybeFireLeaderChange(); resetConnections(); } @@ -3178,7 +3177,14 @@ private void pollListeners() { Optional> leaderState = quorum.maybeLeaderState(); if (leaderState.isPresent()) { maybeFireLeaderChange(leaderState.get()); - } else { + } else if (!quorum.isResigned()) { + /* Should not fire leader change while in the resigned state for two reasons. + * 1. The epoch start offset is not tracked but the leader is the local replica. + * Listener cannot be notify of leadership until they have caught to the latest + * epoch. + * 2. It is not pratical to notify of local leadership since any write operation + * (prepareAppend and schedulePreparedAppend) will fail with NotLeaderException + */ maybeFireLeaderChange(); } } @@ -3706,6 +3712,7 @@ private void maybeFireLeaderChange(LeaderAndEpoch leaderAndEpoch, long epochStar // to have consumed up to that new high-watermark. if (shouldFireLeaderChange(leaderAndEpoch) && nextOffset() > epochStartOffset) { lastFiredLeaderChange = leaderAndEpoch; + logger.debug("Notifying listener {} of new leadership {}", listenerName(), leaderAndEpoch); listener.handleLeaderChange(leaderAndEpoch); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index da2b35609e..888273723f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -577,6 +577,7 @@ public LeaderState transitionToLeader(long epochStartOffset, BatchAccumul } private void durableTransitionTo(EpochState newState) { + log.info("Attempting durable transition to {} from {}", newState, state); store.writeElectionState(newState.election(), partitionState.lastKraftVersion()); memoryTransitionTo(newState); } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 632acaecf8..52ffcadc25 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -4230,6 +4230,36 @@ public void testHandleLeaderChangeFiresAfterFollowerRegistration(boolean withKip assertEquals(expectedLeaderAndEpoch, secondListener.currentLeaderAndEpoch()); } + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testHandleLeaderChangeFiresAfterResignRegistration(boolean withKip853Rpc) throws Exception { + // When registering a listener while the replica is resigned, it should not get notified with + // the current leader and epoch + int localId = randomReplicaId(); + int otherNodeId = localId + 1; + int epoch = 7; + Set voters = Utils.mkSet(localId, otherNodeId); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withElectedLeader(epoch, localId) + .withKip853Rpc(withKip853Rpc) + .build(); + + context.client.poll(); + assertTrue(context.client.quorum().isResigned()); + assertEquals(LeaderAndEpoch.UNKNOWN, context.listener.currentLeaderAndEpoch()); + + // Register another listener and verify that it is not notified of latest leader and epoch + RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener( + OptionalInt.of(localId) + ); + context.client.register(secondListener); + context.client.poll(); + + assertTrue(context.client.quorum().isResigned()); + assertEquals(LeaderAndEpoch.UNKNOWN, secondListener.currentLeaderAndEpoch()); + } + @ParameterizedTest @ValueSource(booleans = { true, false }) public void testObserverFetchWithNoLocalId(boolean withKip853Rpc) throws Exception { diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 83677dccd1..0f1cfd6f3c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -1926,7 +1926,7 @@ static class MockListener implements RaftClient.Listener { private final List> commits = new ArrayList<>(); private final List> savedBatches = new ArrayList<>(); private final Map claimedEpochStartOffsets = new HashMap<>(); - private LeaderAndEpoch currentLeaderAndEpoch = new LeaderAndEpoch(OptionalInt.empty(), 0); + private LeaderAndEpoch currentLeaderAndEpoch = LeaderAndEpoch.UNKNOWN; private final OptionalInt localId; private Optional> snapshot = Optional.empty(); private boolean readCommit = true; From b377ea94c5fae39bb3b06f203de8c95afbd7a423 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Wed, 21 Aug 2024 11:14:56 -0700 Subject: [PATCH 058/123] KAFKA-17305; Check broker registrations for missing features (#16848) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When a broker tries to register with the controller quorum, its registration should be rejected if it doesn't support a feature that is currently enabled. (A feature is enabled if it is set to a non-zero feature level.) This is important for the newly added kraft.version feature flag. Reviewers: Colin P. McCabe , José Armando García Sancio --- .../scala/kafka/server/BrokerFeatures.scala | 8 + .../scala/kafka/server/BrokerServer.scala | 8 +- .../main/scala/kafka/server/KafkaServer.scala | 11 +- .../server/BrokerLifecycleManagerTest.scala | 53 ++++++- .../BrokerRegistrationRequestTest.scala | 40 ++++- .../kafka/server/ReplicationQuotasTest.scala | 12 +- .../controller/ClusterControlManager.java | 41 +++-- .../kafka/controller/QuorumController.java | 8 +- .../controller/ClusterControlManagerTest.java | 142 +++++++++++++++++- .../controller/QuorumControllerTest.java | 59 ++++++++ .../apache/kafka/metalog/LocalLogManager.java | 11 +- .../kafka/metalog/LocalLogManagerTestEnv.java | 19 ++- 12 files changed, 371 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 5aab78d4a2..6a8ddcb19c 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -19,6 +19,7 @@ package kafka.server import kafka.utils.Logging import org.apache.kafka.common.feature.{Features, SupportedVersionRange} +import org.apache.kafka.metadata.VersionRange import org.apache.kafka.server.common.Features.PRODUCTION_FEATURES import org.apache.kafka.server.common.MetadataVersion @@ -80,6 +81,13 @@ object BrokerFeatures extends Logging { new BrokerFeatures(defaultSupportedFeatures(unstableFeatureVersionsEnabled)) } + def createDefaultFeatureMap(features: BrokerFeatures): Map[String, VersionRange] = { + features.supportedFeatures.features.asScala.map { + case (name, versionRange) => + (name, VersionRange.of(versionRange.min, versionRange.max)) + }.toMap + } + def defaultSupportedFeatures(unstableFeatureVersionsEnabled: Boolean): Features[SupportedVersionRange] = { val features = new util.HashMap[String, SupportedVersionRange]() features.put(MetadataVersion.FEATURE_NAME, diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 790bc4ccd1..113bd87989 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -27,7 +27,6 @@ import kafka.raft.KafkaRaftManager import kafka.server.metadata.{AclPublisher, BrokerMetadataPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, ScramPublisher} import kafka.utils.CoreUtils import org.apache.kafka.common.config.ConfigException -import org.apache.kafka.common.feature.SupportedVersionRange import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName @@ -38,7 +37,7 @@ import org.apache.kafka.common.{ClusterResource, TopicPartition, Uuid} import org.apache.kafka.coordinator.group.metrics.{GroupCoordinatorMetrics, GroupCoordinatorRuntimeMetrics} import org.apache.kafka.coordinator.group.{CoordinatorRecord, GroupCoordinator, GroupCoordinatorService, CoordinatorRecordSerde} import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher} -import org.apache.kafka.metadata.{BrokerState, ListenerInfo, VersionRange} +import org.apache.kafka.metadata.{BrokerState, ListenerInfo} import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.{AssignmentsManager, ClientMetricsManager, NodeToControllerChannelManager} import org.apache.kafka.server.authorizer.Authorizer @@ -357,10 +356,7 @@ class BrokerServer( ConfigType.BROKER -> new BrokerConfigHandler(config, quotaManagers), ConfigType.CLIENT_METRICS -> new ClientMetricsConfigHandler(clientMetricsManager)) - val featuresRemapped = brokerFeatures.supportedFeatures.features().asScala.map { - case (k: String, v: SupportedVersionRange) => - k -> VersionRange.of(v.min, v.max) - }.asJava + val featuresRemapped = BrokerFeatures.createDefaultFeatureMap(brokerFeatures).asJava val brokerLifecycleChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider, diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 78ddc408d6..4b1dec3212 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -474,18 +474,19 @@ class KafkaServer( setSecurityProtocol(ep.securityProtocol.id)) } - // Even though ZK brokers don't use "metadata.version" feature, we send our IBP here as part of the broker registration + val features = BrokerFeatures.createDefaultFeatureMap(BrokerFeatures.createDefault(config.unstableFeatureVersionsEnabled)) + + // Even though ZK brokers don't use "metadata.version" feature, we need to overwrite it with our IBP as part of registration // so the KRaft controller can verify that all brokers are on the same IBP before starting the migration. - val ibpAsFeature = - java.util.Collections.singletonMap(MetadataVersion.FEATURE_NAME, - VersionRange.of(config.interBrokerProtocolVersion.featureLevel(), config.interBrokerProtocolVersion.featureLevel())) + val featuresRemapped = features + (MetadataVersion.FEATURE_NAME -> + VersionRange.of(config.interBrokerProtocolVersion.featureLevel(), config.interBrokerProtocolVersion.featureLevel())) lifecycleManager.start( () => listener.highestOffset, brokerToQuorumChannelManager, clusterId, networkListeners, - ibpAsFeature, + featuresRemapped.asJava, OptionalLong.empty() ) logger.debug("Start RaftManager") diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index 71bfbefa30..8e88eea3c4 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -24,11 +24,15 @@ import org.apache.kafka.common.Uuid import org.apache.kafka.common.message.{BrokerHeartbeatResponseData, BrokerRegistrationResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse} -import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.metadata.{BrokerState, VersionRange} import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} +import org.apache.kafka.server.common.{Features, KRaftVersion, MetadataVersion} +import org.apache.kafka.server.common.MetadataVersion.{IBP_3_8_IV0, IBP_3_9_IV0} +import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test, Timeout} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import java.util.concurrent.{CompletableFuture, Future} import scala.jdk.CollectionConverters._ @@ -56,6 +60,15 @@ class BrokerLifecycleManagerTest { properties } + def migrationConfigProperties(ibp: MetadataVersion) = { + val migrationConfigProperties = configProperties + migrationConfigProperties.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true") + migrationConfigProperties.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") + migrationConfigProperties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "") + migrationConfigProperties.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, ibp.toString) + migrationConfigProperties + } + @Test def testCreateAndClose(): Unit = { val context = new RegistrationTestContext(configProperties) @@ -99,6 +112,42 @@ class BrokerLifecycleManagerTest { } } + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testSuccessfulRegistrationDuringMigration(nonInitialKraftVersion: Boolean): Unit = { + val ibp = if (nonInitialKraftVersion) IBP_3_9_IV0 else IBP_3_8_IV0 + val context = new RegistrationTestContext(migrationConfigProperties(ibp)) + manager = new BrokerLifecycleManager(context.config, context.time, "successful-registration-", isZkBroker = false, Set(Uuid.fromString("gCpDJgRlS2CBCpxoP2VMsQ"))) + val controllerNode = new Node(3000, "localhost", 8021) + context.controllerNodeProvider.node.set(controllerNode) + val features = BrokerFeatures.createDefaultFeatureMap(BrokerFeatures.createDefault(true)) + + // Even though ZK brokers don't use "metadata.version" feature, we need to overwrite it with our IBP as part of registration + // so the KRaft controller can verify that all brokers are on the same IBP before starting the migration. + val featuresRemapped = features + (MetadataVersion.FEATURE_NAME -> VersionRange.of(ibp.featureLevel(), ibp.featureLevel())) + + manager.start(() => context.highestMetadataOffset.get(), + context.mockChannelManager, context.clusterId, context.advertisedListeners, + featuresRemapped.asJava, OptionalLong.of(10L)) + TestUtils.retry(60000) { + assertEquals(1, context.mockChannelManager.unsentQueue.size) + val sentBrokerRegistrationData = context.mockChannelManager.unsentQueue.getFirst.request.build().asInstanceOf[BrokerRegistrationRequest].data() + assertEquals(10L, sentBrokerRegistrationData.previousBrokerEpoch()) + assertEquals(ibp.featureLevel(), sentBrokerRegistrationData.features().find(MetadataVersion.FEATURE_NAME).maxSupportedVersion()) + if (nonInitialKraftVersion) { + val sentKraftVersion = sentBrokerRegistrationData.features().find(KRaftVersion.FEATURE_NAME) + assertEquals(Features.KRAFT_VERSION.minimumProduction(), sentKraftVersion.minSupportedVersion()) + assertEquals(Features.KRAFT_VERSION.latestTesting(), sentKraftVersion.maxSupportedVersion()) + } + } + context.mockClient.prepareResponseFrom(new BrokerRegistrationResponse( + new BrokerRegistrationResponseData().setBrokerEpoch(1000)), controllerNode) + TestUtils.retry(10000) { + context.poll() + assertEquals(1000L, manager.brokerEpoch) + } + } + @Test def testRegistrationTimeout(): Unit = { val context = new RegistrationTestContext(configProperties) diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala index 247973255b..f60aac80c9 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala @@ -32,11 +32,12 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} -import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.{Features, MetadataVersion} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.Timeout import org.junit.jupiter.api.extension.ExtendWith +import java.util.Collections import java.util.concurrent.{CompletableFuture, TimeUnit, TimeoutException} /** @@ -98,7 +99,6 @@ class BrokerRegistrationRequestTest { ibpToSend: Option[(MetadataVersion, MetadataVersion)] ): Errors = { val features = new BrokerRegistrationRequestData.FeatureCollection() - ibpToSend foreach { case (min, max) => features.add(new BrokerRegistrationRequestData.Feature() @@ -107,9 +107,17 @@ class BrokerRegistrationRequestTest { .setMaxSupportedVersion(max.featureLevel()) ) } + Features.PRODUCTION_FEATURES.stream().filter(_.featureName != MetadataVersion.FEATURE_NAME).forEach { + feature => + features.add(new BrokerRegistrationRequestData.Feature() + .setName(feature.featureName) + .setMinSupportedVersion(feature.minimumProduction()) + .setMaxSupportedVersion(feature.latestTesting())) + } val req = new BrokerRegistrationRequestData() .setBrokerId(brokerId) + .setLogDirs(Collections.singletonList(Uuid.randomUuid())) .setClusterId(clusterId) .setIncarnationId(Uuid.randomUuid()) .setIsMigratingZkBroker(zkEpoch.isDefined) @@ -229,6 +237,34 @@ class BrokerRegistrationRequestTest { } } + @ClusterTest( + types = Array(Type.KRAFT), + brokers = 1, + controllers = 1, + metadataVersion = MetadataVersion.IBP_3_9_IV0, + autoStart = AutoStart.NO, + serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true"))) + def testRegisterZkWithKRaftMigrationEnabledKRaftV1(clusterInstance: ClusterInstance): Unit = { + clusterInstance.asInstanceOf[RaftClusterInstance].controllers().values().forEach(_.startup()) + + val clusterId = clusterInstance.clusterId() + val channelManager = brokerToControllerChannelManager(clusterInstance) + try { + channelManager.start() + + assertEquals( + Errors.NONE, + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_9_IV0, MetadataVersion.IBP_3_9_IV0)))) + + // Cannot register KRaft broker when in pre-migration + assertEquals( + Errors.BROKER_ID_NOT_REGISTERED, + registerBroker(channelManager, clusterId, 100, None, Some((MetadataVersion.IBP_3_9_IV0, MetadataVersion.IBP_3_9_IV0)))) + } finally { + channelManager.shutdown() + } + } + /** * Start a KRaft cluster with migrations enabled, verify that the controller does not accept metadata changes * through the RPCs. The migration never proceeds past pre-migration since no ZK brokers are registered. diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index 3353c6efcc..ff697dad37 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -35,7 +35,7 @@ import org.apache.kafka.common.message.BrokerRegistrationRequestData.{Listener, import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol.PLAINTEXT import org.apache.kafka.controller.ControllerRequestContextUtil -import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.common.{Features, KRaftVersion, MetadataVersion, TransactionVersion} import org.apache.kafka.server.config.QuotaConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.AfterEach @@ -291,10 +291,18 @@ class ReplicationQuotasTest extends QuorumTestHarness { val listeners = new ListenerCollection() listeners.add(new Listener().setName(PLAINTEXT.name).setHost("localhost").setPort(9092 + id)) val features = new BrokerRegistrationRequestData.FeatureCollection() + features.add(new BrokerRegistrationRequestData.Feature() + .setName(KRaftVersion.FEATURE_NAME) + .setMinSupportedVersion(KRaftVersion.KRAFT_VERSION_0.featureLevel()) + .setMaxSupportedVersion(Features.KRAFT_VERSION.latestTesting())) features.add(new BrokerRegistrationRequestData.Feature() .setName(MetadataVersion.FEATURE_NAME) - .setMinSupportedVersion(MetadataVersion.IBP_3_0_IV1.featureLevel()) + .setMinSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()) .setMaxSupportedVersion(MetadataVersion.latestTesting().featureLevel())) + features.add(new BrokerRegistrationRequestData.Feature() + .setName(TransactionVersion.FEATURE_NAME) + .setMinSupportedVersion(TransactionVersion.TV_0.featureLevel()) + .setMaxSupportedVersion(Features.TRANSACTION_VERSION.latestTesting())) controllerServer.controller.registerBroker( ControllerRequestContextUtil.ANONYMOUS_CONTEXT, new BrokerRegistrationRequestData() diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 7f77455abc..77ffd1036b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -50,7 +50,6 @@ import org.apache.kafka.metadata.placement.StripedReplicaPlacer; import org.apache.kafka.metadata.placement.UsableBroker; import org.apache.kafka.server.common.ApiMessageAndVersion; -import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; @@ -406,18 +405,34 @@ public ControllerResult registerBroker( setRack(request.rack()). setEndPoints(listenerInfo.toBrokerRegistrationRecord()); + // Track which finalized features we have not yet verified are supported by the broker. + Map unverifiedFeatures = new HashMap<>(finalizedFeatures.featureMap()); + + // Check every broker feature version range includes the finalized version. for (BrokerRegistrationRequestData.Feature feature : request.features()) { record.features().add(processRegistrationFeature(brokerId, finalizedFeatures, feature)); + unverifiedFeatures.remove(feature.name()); } + // Brokers that don't send a supported metadata.version range are assumed to only + // support the original metadata.version. if (request.features().find(MetadataVersion.FEATURE_NAME) == null) { - // Brokers that don't send a supported metadata.version range are assumed to only - // support the original metadata.version. record.features().add(processRegistrationFeature(brokerId, finalizedFeatures, + new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). + setMaxSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()))); + unverifiedFeatures.remove(MetadataVersion.FEATURE_NAME); + } + // We also need to check every controller feature is supported by the broker. + unverifiedFeatures.forEach((featureName, finalizedVersion) -> { + if (finalizedVersion != 0 && request.features().findAll(featureName).isEmpty()) { + processRegistrationFeature(brokerId, finalizedFeatures, new BrokerRegistrationRequestData.Feature(). - setName(MetadataVersion.FEATURE_NAME). - setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()). - setMaxSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()))); - } + setName(featureName). + setMinSupportedVersion((short) 0). + setMaxSupportedVersion((short) 0)); + } + }); if (featureControl.metadataVersion().isDirectoryAssignmentSupported()) { record.setLogDirs(request.logDirs()); } @@ -490,15 +505,15 @@ BrokerFeature processRegistrationFeature( short finalized = finalizedFeatures.versionOrDefault(feature.name(), (short) defaultVersion); if (!VersionRange.of(feature.minSupportedVersion(), feature.maxSupportedVersion()).contains(finalized)) { throw new UnsupportedVersionException("Unable to register because the broker " + - "does not support version " + finalized + " of " + feature.name() + - ". It wants a version between " + feature.minSupportedVersion() + " and " + + "does not support finalized version " + finalized + " of " + feature.name() + + ". The broker wants a version between " + feature.minSupportedVersion() + " and " + feature.maxSupportedVersion() + ", inclusive."); } // A feature is not found in the finalizedFeature map if it is unknown to the controller or set to 0 (feature not enabled). - // Only log if the feature name is not known by the controller. - if (!Features.PRODUCTION_FEATURE_NAMES.contains(feature.name())) - log.warn("Broker {} registered with feature {} that is unknown to the controller", - brokerId, feature.name()); + if (!finalizedFeatures.featureNames().contains(feature.name())) + log.debug("Broker {} registered with version range ({}, {}] of feature {} which controller does not know " + + "or has finalized version of 0.", + brokerId, feature.minSupportedVersion(), feature.maxSupportedVersion(), feature.name()); return new BrokerFeature(). setName(feature.name()). setMinSupportedVersion(feature.minSupportedVersion()). diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 56cbb80d54..3f7bb4df74 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -119,6 +119,7 @@ import org.apache.kafka.server.authorizer.AclCreateResult; import org.apache.kafka.server.authorizer.AclDeleteResult; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.FaultHandlerException; @@ -2171,11 +2172,14 @@ public CompletableFuture registerBroker( ControllerRequestContext context, BrokerRegistrationRequestData request ) { + // populate finalized features map with latest known kraft version for validation + Map controllerFeatures = new HashMap<>(featureControl.finalizedFeatures(Long.MAX_VALUE).featureMap()); + controllerFeatures.put(KRaftVersion.FEATURE_NAME, raftClient.kraftVersion().featureLevel()); return appendWriteEvent("registerBroker", context.deadlineNs(), () -> { ControllerResult result = clusterControl. - registerBroker(request, offsetControl.nextWriteOffset(), featureControl. - finalizedFeatures(Long.MAX_VALUE)); + registerBroker(request, offsetControl.nextWriteOffset(), + new FinalizedControllerFeatures(controllerFeatures, Long.MAX_VALUE)); rescheduleMaybeFenceStaleBrokers(); return result; }, diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index ddc7506b0d..0646d4aaa5 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.message.BrokerRegistrationRequestData; import org.apache.kafka.common.message.ControllerRegistrationRequestData; import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord; +import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.metadata.FenceBrokerRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; @@ -49,7 +50,9 @@ import org.apache.kafka.metadata.placement.PlacementSpec; import org.apache.kafka.metadata.placement.UsableBroker; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.common.TestFeatureVersion; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; @@ -62,9 +65,11 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Stream; @@ -531,6 +536,135 @@ public void testRegistrationsToRecords(MetadataVersion metadataVersion) { clusterControl.brokerRegistrations().get(2).toRecord(options)); } + @Test + public void testRegistrationWithUnsupportedFeature() { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + Map supportedFeatures = new HashMap<>(); + supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( + MetadataVersion.IBP_3_1_IV0.featureLevel(), + MetadataVersion.IBP_3_7_IV0.featureLevel())); + supportedFeatures.put(TestFeatureVersion.FEATURE_NAME, VersionRange.of( + TestFeatureVersion.TEST_0.featureLevel(), + TestFeatureVersion.TEST_1.featureLevel())); + FeatureControlManager featureControl = new FeatureControlManager.Builder(). + setSnapshotRegistry(snapshotRegistry). + setQuorumFeatures(new QuorumFeatures(0, supportedFeatures, Collections.singletonList(0))). + setMetadataVersion(MetadataVersion.IBP_3_7_IV0). + build(); + ClusterControlManager clusterControl = new ClusterControlManager.Builder(). + setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). + setTime(new MockTime(0, 0, 0)). + setSnapshotRegistry(snapshotRegistry). + setFeatureControlManager(featureControl). + setBrokerUncleanShutdownHandler((brokerId, records) -> { }). + build(); + clusterControl.activate(); + FeatureLevelRecord testFeatureRecord = new FeatureLevelRecord(). + setName(TestFeatureVersion.FEATURE_NAME).setFeatureLevel((short) 1); + featureControl.replay(testFeatureRecord); + + List logDirs = asList(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ")); + BrokerRegistrationRequestData baseRequest = new BrokerRegistrationRequestData(). + setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). + setBrokerId(0). + setRack(null). + setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")). + setLogDirs(logDirs); + + assertEquals("Unable to register because the broker does not support finalized version 1 of " + + "test.feature.version. The broker wants a version between 0 and 0, inclusive.", + assertThrows(UnsupportedVersionException.class, + () -> clusterControl.registerBroker( + baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.IBP_3_1_IV0.featureLevel()). + setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), + 123L, + featureControl.finalizedFeatures(Long.MAX_VALUE))).getMessage()); + } + + @Test + public void testRegistrationWithUnsupportedKraftVersion() { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + Map supportedFeatures = new HashMap<>(); + supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( + MetadataVersion.IBP_3_1_IV0.featureLevel(), + MetadataVersion.IBP_3_9_IV0.featureLevel())); + supportedFeatures.put(KRaftVersion.FEATURE_NAME, VersionRange.of( + KRaftVersion.KRAFT_VERSION_1.featureLevel(), + KRaftVersion.KRAFT_VERSION_1.featureLevel())); + FeatureControlManager featureControl = new FeatureControlManager.Builder(). + setSnapshotRegistry(snapshotRegistry). + setQuorumFeatures(new QuorumFeatures(0, supportedFeatures, Collections.singletonList(0))). + setMetadataVersion(MetadataVersion.IBP_3_9_IV0). + build(); + ClusterControlManager clusterControl = new ClusterControlManager.Builder(). + setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). + setTime(new MockTime(0, 0, 0)). + setSnapshotRegistry(snapshotRegistry). + setFeatureControlManager(featureControl). + setBrokerUncleanShutdownHandler((brokerId, records) -> { }). + build(); + clusterControl.activate(); + + List logDirs = asList(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ")); + BrokerRegistrationRequestData baseRequest = new BrokerRegistrationRequestData(). + setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). + setBrokerId(0). + setRack(null). + setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")). + setLogDirs(logDirs); + + // quorum controller passes in the latest kraft version to populate finalized features + Map updatedFeaturesMap = new HashMap<>(featureControl.finalizedFeatures(Long.MAX_VALUE).featureMap()); + updatedFeaturesMap.put(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()); + FinalizedControllerFeatures updatedFinalizedFeatures = new FinalizedControllerFeatures(updatedFeaturesMap, Long.MAX_VALUE); + + assertEquals("Unable to register because the broker does not support finalized version 1 of " + + "kraft.version. The broker wants a version between 0 and 0, inclusive.", + assertThrows(UnsupportedVersionException.class, + () -> clusterControl.registerBroker( + baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Collections.singleton(new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()). + setMaxSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel())).iterator())), + 123L, + updatedFinalizedFeatures)).getMessage()); + + assertEquals("Unable to register because the broker does not support finalized version 1 of " + + "kraft.version. The broker wants a version between 0 and 0, inclusive.", + assertThrows(UnsupportedVersionException.class, + () -> clusterControl.registerBroker( + baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Arrays.asList( + new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()). + setMaxSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()), + new BrokerRegistrationRequestData.Feature(). + setName(KRaftVersion.FEATURE_NAME). + setMinSupportedVersion(KRaftVersion.KRAFT_VERSION_0.featureLevel()). + setMaxSupportedVersion(KRaftVersion.KRAFT_VERSION_0.featureLevel())).iterator())), + 123L, + updatedFinalizedFeatures)).getMessage()); + + clusterControl.registerBroker( + baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( + Arrays.asList( + new BrokerRegistrationRequestData.Feature(). + setName(MetadataVersion.FEATURE_NAME). + setMinSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()). + setMaxSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()), + new BrokerRegistrationRequestData.Feature(). + setName(KRaftVersion.FEATURE_NAME). + setMinSupportedVersion(KRaftVersion.KRAFT_VERSION_1.featureLevel()). + setMaxSupportedVersion(KRaftVersion.KRAFT_VERSION_1.featureLevel())).iterator())), + 123L, + updatedFinalizedFeatures); + } + @Test public void testRegistrationWithUnsupportedMetadataVersion() { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); @@ -552,8 +686,8 @@ public void testRegistrationWithUnsupportedMetadataVersion() { build(); clusterControl.activate(); - assertEquals("Unable to register because the broker does not support version 4 of " + - "metadata.version. It wants a version between 1 and 1, inclusive.", + assertEquals("Unable to register because the broker does not support finalized version 4 of " + + "metadata.version. The broker wants a version between 1 and 1, inclusive.", assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerBroker( new BrokerRegistrationRequestData(). @@ -564,8 +698,8 @@ public void testRegistrationWithUnsupportedMetadataVersion() { 123L, featureControl.finalizedFeatures(Long.MAX_VALUE))).getMessage()); - assertEquals("Unable to register because the broker does not support version 4 of " + - "metadata.version. It wants a version between 7 and 7, inclusive.", + assertEquals("Unable to register because the broker does not support finalized version 4 of " + + "metadata.version. The broker wants a version between 7 and 7, inclusive.", assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerBroker( new BrokerRegistrationRequestData(). diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 23f8bdbd66..08213dfdfe 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.errors.BrokerIdNotRegisteredException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.AllocateProducerIdsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignableTopic; @@ -106,6 +107,8 @@ import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.Features; +import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.TopicIdPartition; import org.apache.kafka.server.fault.FaultHandlerException; @@ -117,6 +120,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; @@ -685,6 +689,61 @@ public void testNoOpRecordWriteAfterTimeout() throws Throwable { } } + @ParameterizedTest + @CsvSource(value = {"0, 0", "0, 1", "1, 0", "1, 1"}) + public void testRegisterBrokerKRaftVersions(short finalizedKraftVersion, short brokerMaxSupportedKraftVersion) throws Throwable { + try ( + LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). + setLastKRaftVersion(KRaftVersion.fromFeatureLevel(finalizedKraftVersion)). + build(); + QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). + setControllerBuilderInitializer(controllerBuilder -> + controllerBuilder.setConfigSchema(SCHEMA)). + setBootstrapMetadata(SIMPLE_BOOTSTRAP). + build() + ) { + ListenerCollection listeners = new ListenerCollection(); + listeners.add(new Listener().setName("PLAINTEXT"). + setHost("localhost").setPort(9092)); + QuorumController active = controlEnv.activeController(); + BrokerRegistrationRequestData.FeatureCollection brokerFeatures = new BrokerRegistrationRequestData.FeatureCollection(); + brokerFeatures.add(new BrokerRegistrationRequestData.Feature() + .setName(MetadataVersion.FEATURE_NAME) + .setMinSupportedVersion(MetadataVersion.IBP_3_0_IV1.featureLevel()) + .setMaxSupportedVersion(MetadataVersion.latestTesting().featureLevel())); + // broker registration requests do not include initial versions of features + if (brokerMaxSupportedKraftVersion != 0) { + brokerFeatures.add(new BrokerRegistrationRequestData.Feature() + .setName(KRaftVersion.FEATURE_NAME) + .setMinSupportedVersion(Features.KRAFT_VERSION.minimumProduction()) + .setMaxSupportedVersion(brokerMaxSupportedKraftVersion)); + } + BrokerRegistrationRequestData request = new BrokerRegistrationRequestData(). + setBrokerId(0). + setClusterId(active.clusterId()). + setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). + setFeatures(brokerFeatures). + setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). + setListeners(listeners); + + if (brokerMaxSupportedKraftVersion < finalizedKraftVersion) { + Throwable exception = assertThrows(ExecutionException.class, () -> active.registerBroker( + ANONYMOUS_CONTEXT, + request).get()); + assertEquals(UnsupportedVersionException.class, exception.getCause().getClass()); + assertEquals("Unable to register because the broker does not support finalized version " + + finalizedKraftVersion + " of kraft.version. The broker wants a version between 0 and " + + brokerMaxSupportedKraftVersion + ", inclusive.", + exception.getCause().getMessage()); + } else { + BrokerRegistrationReply reply = active.registerBroker( + ANONYMOUS_CONTEXT, + request).get(); + assertTrue(reply.epoch() >= 5, "Unexpected broker epoch " + reply.epoch()); + } + } + } + @Test public void testUnregisterBroker() throws Throwable { try ( diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index 2f24d3f966..add04f0603 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -470,6 +470,11 @@ void beginShutdown() { */ private final EventQueue eventQueue; + /** + * The latest kraft version used by this local log manager. + */ + private final KRaftVersion lastKRaftVersion; + /** * Whether this LocalLogManager has been shut down. */ @@ -499,13 +504,15 @@ void beginShutdown() { public LocalLogManager(LogContext logContext, int nodeId, SharedLogData shared, - String threadNamePrefix) { + String threadNamePrefix, + KRaftVersion lastKRaftVersion) { this.log = logContext.logger(LocalLogManager.class); this.nodeId = nodeId; this.shared = shared; this.maxReadOffset = shared.initialMaxReadOffset(); this.eventQueue = new KafkaEventQueue(Time.SYSTEM, logContext, threadNamePrefix, new ShutdownEvent()); + this.lastKRaftVersion = lastKRaftVersion; shared.registerLogManager(this); } @@ -840,6 +847,6 @@ public void throwOnNextAppend() { @Override public KRaftVersion kraftVersion() { - return KRaftVersion.KRAFT_VERSION_0; + return lastKRaftVersion; } } diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java index dd6ba56a3b..661c96a3a8 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTestEnv.java @@ -25,6 +25,7 @@ import org.apache.kafka.metalog.LocalLogManager.SharedLogData; import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.test.TestUtils; @@ -71,6 +72,7 @@ public static class Builder { private final int numManagers; private Optional snapshotReader = Optional.empty(); private Consumer sharedLogDataInitializer = __ -> { }; + private KRaftVersion lastKRaftVersion = KRaftVersion.KRAFT_VERSION_0; public Builder(int numManagers) { this.numManagers = numManagers; @@ -86,11 +88,20 @@ public Builder setSharedLogDataInitializer(Consumer sharedLogData return this; } + /** + * Used to mock the latest KRaft version that would be returned from RaftClient.kraftVersion() + */ + public Builder setLastKRaftVersion(KRaftVersion kraftVersion) { + this.lastKRaftVersion = kraftVersion; + return this; + } + public LocalLogManagerTestEnv build() { return new LocalLogManagerTestEnv( numManagers, snapshotReader, - sharedLogDataInitializer); + sharedLogDataInitializer, + lastKRaftVersion); } public LocalLogManagerTestEnv buildWithMockListeners() { @@ -114,7 +125,8 @@ public LocalLogManagerTestEnv buildWithMockListeners() { private LocalLogManagerTestEnv( int numManagers, Optional snapshotReader, - Consumer sharedLogDataInitializer + Consumer sharedLogDataInitializer, + KRaftVersion lastKRaftVersion ) { clusterId = Uuid.randomUuid().toString(); dir = TestUtils.tempDirectory(); @@ -127,7 +139,8 @@ private LocalLogManagerTestEnv( new LogContext(String.format("[LocalLogManager %d] ", nodeId)), nodeId, shared, - String.format("LocalLogManager-%d_", nodeId))); + String.format("LocalLogManager-%d_", nodeId), + lastKRaftVersion)); } } catch (Throwable t) { for (LocalLogManager logManager : newLogManagers) { From 29bbb6555c1c867865ee8464218160d40ffa84b1 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Fri, 23 Aug 2024 03:47:55 +0800 Subject: [PATCH 059/123] KAFKA-17336 Add IT to make sure the production MV does not use unstable version of LIST_OFFSET (#16893) - due to the server config UNSTABLE_API_VERSIONS_ENABLE_CONFIG is true, so we can't test the scenario of ListOffsetsRequest is unstable version. We want to test this case in this PR - get the MV from metadataCache.metadataVersion() instead of config.interBrokerProtocolVersion since MV can be set dynamically. Reviewers: Jun Rao , Chia-Ping Tsai --- .../main/scala/kafka/cluster/Partition.scala | 2 +- .../TransactionMarkerChannelManager.scala | 9 +++----- .../main/scala/kafka/server/KafkaApis.scala | 12 +++++------ ...ransactionCoordinatorConcurrencyTest.scala | 3 +++ .../TransactionMarkerChannelManagerTest.scala | 3 +++ .../unit/kafka/server/KafkaApisTest.scala | 21 ++++++++++++++++++- .../kafka/server/common/MetadataVersion.java | 8 +++++++ .../server/common/MetadataVersionTest.java | 15 +++++++++++++ 8 files changed, 59 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index e6783b7d19..454eec0d94 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -139,7 +139,7 @@ object Partition { new Partition(topicPartition, _topicId = topicId, replicaLagTimeMaxMs = replicaManager.config.replicaLagTimeMaxMs, - interBrokerProtocolVersion = replicaManager.config.interBrokerProtocolVersion, + interBrokerProtocolVersion = replicaManager.metadataCache.metadataVersion(), localBrokerId = replicaManager.config.brokerId, localBrokerEpochSupplier = replicaManager.brokerEpochSupplier, time = time, diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 44176d2276..688de61849 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -33,7 +33,6 @@ import org.apache.kafka.common.requests.{TransactionResult, WriteTxnMarkersReque import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{Node, Reconfigurable, TopicPartition} -import org.apache.kafka.server.common.MetadataVersion.IBP_2_8_IV0 import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletionHandler} @@ -180,10 +179,6 @@ class TransactionMarkerChannelManager( private val transactionsWithPendingMarkers = new ConcurrentHashMap[String, PendingCompleteTxn] - private val writeTxnMarkersRequestVersion: Short = - if (config.interBrokerProtocolVersion.isAtLeast(IBP_2_8_IV0)) 1 - else 0 - metricsGroup.newGauge(UnknownDestinationQueueSizeMetricName, () => markersQueueForUnknownBroker.totalNumMarkers) metricsGroup.newGauge(LogAppendRetryQueueSizeMetricName, () => txnLogAppendRetryQueue.size) @@ -261,7 +256,9 @@ class TransactionMarkerChannelManager( }.filter { case (_, entries) => !entries.isEmpty }.map { case (node, entries) => val markersToSend = entries.asScala.map(_.txnMarkerEntry).asJava val requestCompletionHandler = new TransactionMarkerRequestCompletionHandler(node.id, txnStateManager, this, entries) - val request = new WriteTxnMarkersRequest.Builder(writeTxnMarkersRequestVersion, markersToSend) + val request = new WriteTxnMarkersRequest.Builder( + metadataCache.metadataVersion().writeTxnMarkersRequestVersion(), markersToSend + ) new RequestAndCompletionHandler( currentTimeMs, diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 8335cd54f8..cdd2d06ee2 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -451,7 +451,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (!authHelper.authorize(request.context, READ, GROUP, offsetCommitRequest.data.groupId)) { requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) - } else if (offsetCommitRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion.isLessThan(IBP_2_3_IV0)) { + } else if (offsetCommitRequest.data.groupInstanceId != null && metadataCache.metadataVersion().isLessThan(IBP_2_3_IV0)) { // Only enable static membership when IBP >= 2.3, because it is not safe for the broker to use the static member logic // until we are sure that all brokers support it. If static group being loaded by an older coordinator, it will discard // the group.instance.id field, so static members could accidentally become "dynamic", which leads to wrong states. @@ -1807,7 +1807,7 @@ class KafkaApis(val requestChannel: RequestChannel, ): CompletableFuture[Unit] = { val joinGroupRequest = request.body[JoinGroupRequest] - if (joinGroupRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion.isLessThan(IBP_2_3_IV0)) { + if (joinGroupRequest.data.groupInstanceId != null && metadataCache.metadataVersion().isLessThan(IBP_2_3_IV0)) { // Only enable static membership when IBP >= 2.3, because it is not safe for the broker to use the static member logic // until we are sure that all brokers support it. If static group being loaded by an older coordinator, it will discard // the group.instance.id field, so static members could accidentally become "dynamic", which leads to wrong states. @@ -1837,7 +1837,7 @@ class KafkaApis(val requestChannel: RequestChannel, ): CompletableFuture[Unit] = { val syncGroupRequest = request.body[SyncGroupRequest] - if (syncGroupRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion.isLessThan(IBP_2_3_IV0)) { + if (syncGroupRequest.data.groupInstanceId != null && metadataCache.metadataVersion().isLessThan(IBP_2_3_IV0)) { // Only enable static membership when IBP >= 2.3, because it is not safe for the broker to use the static member logic // until we are sure that all brokers support it. If static group being loaded by an older coordinator, it will discard // the group.instance.id field, so static members could accidentally become "dynamic", which leads to wrong states. @@ -1906,7 +1906,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleHeartbeatRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { val heartbeatRequest = request.body[HeartbeatRequest] - if (heartbeatRequest.data.groupInstanceId != null && config.interBrokerProtocolVersion.isLessThan(IBP_2_3_IV0)) { + if (heartbeatRequest.data.groupInstanceId != null && metadataCache.metadataVersion().isLessThan(IBP_2_3_IV0)) { // Only enable static membership when IBP >= 2.3, because it is not safe for the broker to use the static member logic // until we are sure that all brokers support it. If static group being loaded by an older coordinator, it will discard // the group.instance.id field, so static members could accidentally become "dynamic", which leads to wrong states. @@ -2537,8 +2537,8 @@ class KafkaApis(val requestChannel: RequestChannel, } def ensureInterBrokerVersion(version: MetadataVersion): Unit = { - if (config.interBrokerProtocolVersion.isLessThan(version)) - throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion} is less than the required version: ${version}") + if (metadataCache.metadataVersion().isLessThan(version)) + throw new UnsupportedVersionException(s"metadata.version: ${metadataCache.metadataVersion()} is less than the required version: ${version}") } def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = { diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index 3de08c1af7..44a35e981a 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -92,6 +92,9 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren ) } + when(metadataCache.metadataVersion()) + .thenReturn(MetadataVersion.latestProduction()) + txnStateManager = new TransactionStateManager(0, scheduler, replicaManager, metadataCache, txnConfig, time, new Metrics()) txnStateManager.startup(() => zkClient.getTopicPartitionCount(TRANSACTION_STATE_TOPIC_NAME).get, diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala index 3356c4f9e3..d40932f322 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{RequestHeader, TransactionResult, WriteTxnMarkersRequest, WriteTxnMarkersResponse} import org.apache.kafka.common.utils.MockTime import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.util.RequestAndCompletionHandler import org.junit.jupiter.api.Assertions._ @@ -86,6 +87,8 @@ class TransactionMarkerChannelManagerTest { .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata1)))) when(txnStateManager.getTransactionState(ArgumentMatchers.eq(transactionalId2))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata2)))) + when(metadataCache.metadataVersion()) + .thenReturn(MetadataVersion.latestProduction()) } @Test diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 91b45dcc03..9b45fbd6a4 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -124,7 +124,7 @@ class KafkaApisTest extends Logging { private val brokerId = 1 // KRaft tests should override this with a KRaftMetadataCache private var metadataCache: MetadataCache = MetadataCache.zkMetadataCache(brokerId, MetadataVersion.latestTesting()) - private val brokerEpochManager: ZkBrokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) + private var brokerEpochManager: ZkBrokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) private val clientQuotaManager: ClientQuotaManager = mock(classOf[ClientQuotaManager]) private val clientRequestQuotaManager: ClientRequestQuotaManager = mock(classOf[ClientRequestQuotaManager]) private val clientControllerQuotaManager: ControllerMutationQuotaManager = mock(classOf[ControllerMutationQuotaManager]) @@ -2829,6 +2829,8 @@ class KafkaApisTest extends Logging { @Test def shouldThrowUnsupportedVersionExceptionOnHandleAddOffsetToTxnRequestWhenInterBrokerProtocolNotSupported(): Unit = { + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_0_10_2_IV0) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_0_10_2_IV0) assertThrows(classOf[UnsupportedVersionException], () => kafkaApis.handleAddOffsetsToTxnRequest(null, RequestLocal.withThreadConfinedCaching)) @@ -2836,6 +2838,8 @@ class KafkaApisTest extends Logging { @Test def shouldThrowUnsupportedVersionExceptionOnHandleAddPartitionsToTxnRequestWhenInterBrokerProtocolNotSupported(): Unit = { + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_0_10_2_IV0) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_0_10_2_IV0) assertThrows(classOf[UnsupportedVersionException], () => kafkaApis.handleAddPartitionsToTxnRequest(null, RequestLocal.withThreadConfinedCaching)) @@ -2843,6 +2847,8 @@ class KafkaApisTest extends Logging { @Test def shouldThrowUnsupportedVersionExceptionOnHandleTxnOffsetCommitRequestWhenInterBrokerProtocolNotSupported(): Unit = { + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_0_10_2_IV0) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_0_10_2_IV0) assertThrows(classOf[UnsupportedVersionException], () => kafkaApis.handleAddPartitionsToTxnRequest(null, RequestLocal.withThreadConfinedCaching)) @@ -2850,6 +2856,8 @@ class KafkaApisTest extends Logging { @Test def shouldThrowUnsupportedVersionExceptionOnHandleEndTxnRequestWhenInterBrokerProtocolNotSupported(): Unit = { + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_0_10_2_IV0) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_0_10_2_IV0) assertThrows(classOf[UnsupportedVersionException], () => kafkaApis.handleEndTxnRequest(null, RequestLocal.withThreadConfinedCaching)) @@ -2857,6 +2865,8 @@ class KafkaApisTest extends Logging { @Test def shouldThrowUnsupportedVersionExceptionOnHandleWriteTxnMarkersRequestWhenInterBrokerProtocolNotSupported(): Unit = { + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_0_10_2_IV0) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_0_10_2_IV0) assertThrows(classOf[UnsupportedVersionException], () => kafkaApis.handleWriteTxnMarkersRequest(null, RequestLocal.withThreadConfinedCaching)) @@ -4807,6 +4817,8 @@ class KafkaApisTest extends Logging { ).build() val requestChannelRequest = buildRequest(joinGroupRequest) + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_2_2_IV1) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_2_2_IV1) kafkaApis.handleJoinGroupRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching) @@ -4825,6 +4837,8 @@ class KafkaApisTest extends Logging { ).build() val requestChannelRequest = buildRequest(syncGroupRequest) + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_2_2_IV1) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_2_2_IV1) kafkaApis.handleSyncGroupRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching) @@ -4842,6 +4856,8 @@ class KafkaApisTest extends Logging { .setGenerationId(1) ).build() val requestChannelRequest = buildRequest(heartbeatRequest) + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_2_2_IV1) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_2_2_IV1) kafkaApis.handleHeartbeatRequest(requestChannelRequest) @@ -4871,6 +4887,9 @@ class KafkaApisTest extends Logging { ).build() val requestChannelRequest = buildRequest(offsetCommitRequest) + + metadataCache = MetadataCache.zkMetadataCache(brokerId, IBP_2_2_IV1) + brokerEpochManager = new ZkBrokerEpochManager(metadataCache, controller, None) kafkaApis = createKafkaApis(IBP_2_2_IV1) kafkaApis.handleOffsetCommitRequest(requestChannelRequest, RequestLocal.withThreadConfinedCaching) diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index 94dff4c91f..0574b97cde 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -646,6 +646,14 @@ private static boolean checkIfMetadataChangedOrdered(MetadataVersion highVersion return version != lowVersion; } + public short writeTxnMarkersRequestVersion() { + if (isAtLeast(IBP_2_8_IV0)) { + return 1; + } else { + return 0; + } + } + public boolean isAtLeast(MetadataVersion otherVersion) { return this.compareTo(otherVersion) >= 0; } diff --git a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java index 3908dd1cf6..d2867756bf 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/MetadataVersionTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.server.common; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.record.RecordVersion; import org.junit.jupiter.api.Test; @@ -466,6 +467,20 @@ public void assertLatestProductionIsLessThanLatest() { " to be less than the latest of " + MetadataVersion.latestTesting()); } + /** + * We need to ensure that the latest production MV doesn't inadvertently rely on an unstable + * request version. Currently, the broker selects the version for some inter-broker RPCs based on the MV + * rather than using the supported version from the ApiResponse. + */ + @Test + public void testProductionMetadataDontUseUnstableApiVersion() { + MetadataVersion mv = MetadataVersion.latestProduction(); + assertTrue(mv.listOffsetRequestVersion() <= ApiKeys.LIST_OFFSETS.latestVersion(false)); + assertTrue(mv.fetchRequestVersion() <= ApiKeys.FETCH.latestVersion(false)); + assertTrue(mv.offsetForLeaderEpochRequestVersion() <= ApiKeys.OFFSET_FOR_LEADER_EPOCH.latestVersion(false)); + assertTrue(mv.writeTxnMarkersRequestVersion() <= ApiKeys.WRITE_TXN_MARKERS.latestVersion(false)); + } + @Test public void assertLatestProductionIsProduction() { assertTrue(LATEST_PRODUCTION.isProduction()); From 1f6d5aec82908177c1d66eb2920d2644d00f7df8 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Thu, 22 Aug 2024 00:24:16 -0700 Subject: [PATCH 060/123] MINOR: fix HTML for topology.optimization config (#16953) The HTML rendering broke via https://issues.apache.org/jira/browse/KAFKA-14209 in 3.4 release. The currently shown value is some garbage org.apache.kafka.streams.StreamsConfig$$Lambda$20/0x0000000800c0cf18@b1bc7ed cf https://kafka.apache.org/documentation/#streamsconfigs_topology.optimization Verified the fix via running StreamsConfig#main() locally. Reviewers: Bill Bejeck , Chia-Ping Tsai --- .../java/org/apache/kafka/streams/StreamsConfig.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index d1ffd6a587..61cb804b46 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1035,7 +1035,17 @@ public class StreamsConfig extends AbstractConfig { .define(TOPOLOGY_OPTIMIZATION_CONFIG, Type.STRING, NO_OPTIMIZATION, - (name, value) -> verifyTopologyOptimizationConfigs((String) value), + new ConfigDef.Validator() { + @Override + public void ensureValid(final String name, final Object value) { + verifyTopologyOptimizationConfigs((String) value); + } + + @Override + public String toString() { + return TOPOLOGY_OPTIMIZATION_CONFIGS.toString(); + } + }, Importance.MEDIUM, TOPOLOGY_OPTIMIZATION_DOC) From dcb4578903983ad0dbb7038326a1e2ad210248ff Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Sun, 25 Aug 2024 07:29:58 +0800 Subject: [PATCH 061/123] KAFKA-17315 Fix the behavior of delegation tokens that expire immediately upon creation in KRaft mode (#16858) In kraft mode, expiring delegation token (`expiryTimePeriodMs` < 0) has following different behavior to zk mode. 1. `ExpiryTimestampMs` is set to "expiryTimePeriodMs" [0] rather than "now" [1] 2. it throws exception directly if the token is expired already [2]. By contrast, zk mode does not. [3] [0] https://github.com/apache/kafka/blob/49fc14f6116a697550339a8804177bd9290d15db/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java#L316 [1] https://github.com/apache/kafka/blob/49fc14f6116a697550339a8804177bd9290d15db/core/src/main/scala/kafka/server/DelegationTokenManagerZk.scala#L292 [2] https://github.com/apache/kafka/blob/49fc14f6116a697550339a8804177bd9290d15db/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java#L305 [3] https://github.com/apache/kafka/blob/49fc14f6116a697550339a8804177bd9290d15db/core/src/main/scala/kafka/server/DelegationTokenManagerZk.scala#L293 Reviewers: Chia-Ping Tsai --- .../api/SaslSslAdminIntegrationTest.scala | 53 ++++++++++++++++++- .../DelegationTokenControlManager.java | 8 ++- 2 files changed, 54 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 642bb4b69b..ad397be325 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -21,13 +21,14 @@ import org.apache.kafka.common.acl._ import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE, IDEMPOTENT_WRITE} import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} import org.apache.kafka.common.config.{ConfigResource, SaslConfigs, TopicConfig} -import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidRequestException, TopicAuthorizationException, UnknownTopicOrPartitionException} +import org.apache.kafka.common.errors.{ClusterAuthorizationException, DelegationTokenExpiredException, DelegationTokenNotFoundException, InvalidRequestException, TopicAuthorizationException, UnknownTopicOrPartitionException} import org.apache.kafka.common.resource.PatternType.LITERAL import org.apache.kafka.common.resource.ResourceType.{GROUP, TOPIC} import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} +import org.apache.kafka.common.security.token.delegation.DelegationToken import org.apache.kafka.security.authorizer.AclEntry.{WILDCARD_HOST, WILDCARD_PRINCIPAL_STRING} -import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, ServerConfigs, ZkConfigs} import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions._ @@ -66,6 +67,10 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu this.serverConfig.setProperty(AclAuthorizer.SuperUsersProp, kafkaPrincipal.toString) } + // Enable delegationTokenControlManager + serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "123") + serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG, "5000") + setUpSasl() super.setUp(testInfo) setInitialAcls() @@ -519,6 +524,50 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu } } + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testExpireDelegationToken(quorum: String): Unit = { + client = createAdminClient + val createDelegationTokenOptions = new CreateDelegationTokenOptions() + + // Test expiration for non-exists token + TestUtils.assertFutureExceptionTypeEquals( + client.expireDelegationToken("".getBytes()).expiryTimestamp(), + classOf[DelegationTokenNotFoundException] + ) + + // Test expiring the token immediately + val token1 = client.createDelegationToken(createDelegationTokenOptions).delegationToken().get() + TestUtils.retry(maxWaitMs = 1000) { assertTrue(expireTokenOrFailWithAssert(token1, -1) < System.currentTimeMillis()) } + + // Test expiring the expired token + val token2 = client.createDelegationToken(createDelegationTokenOptions.maxlifeTimeMs(1000)).delegationToken().get() + // Ensure current time > maxLifeTimeMs of token + Thread.sleep(1000) + TestUtils.assertFutureExceptionTypeEquals( + client.expireDelegationToken(token2.hmac(), new ExpireDelegationTokenOptions().expiryTimePeriodMs(1)).expiryTimestamp(), + classOf[DelegationTokenExpiredException] + ) + + // Ensure expiring the expired token with negative expiryTimePeriodMs will not throw exception + assertDoesNotThrow(() => expireTokenOrFailWithAssert(token2, -1)) + + // Test shortening the expiryTimestamp + val token3 = client.createDelegationToken(createDelegationTokenOptions).delegationToken().get() + TestUtils.retry(1000) { assertTrue(expireTokenOrFailWithAssert(token3, 200) < token3.tokenInfo().expiryTimestamp()) } + } + + private def expireTokenOrFailWithAssert(token: DelegationToken, expiryTimePeriodMs: Long): Long = { + try { + client.expireDelegationToken(token.hmac(), new ExpireDelegationTokenOptions().expiryTimePeriodMs(expiryTimePeriodMs)) + .expiryTimestamp().get() + } catch { + // If metadata is not synced yet, the response will contain an errorCode, causing an exception to be thrown. + // This wrapper is designed to work with TestUtils.retry + case _: ExecutionException => throw new AssertionError("Metadata not sync yet.") + } + } + private def describeConfigs(topic: String): Iterable[ConfigEntry] = { val topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic) var configEntries: Iterable[ConfigEntry] = null diff --git a/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java index fa50570a48..04a69a1e36 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java @@ -302,10 +302,6 @@ public ControllerResult expireDelegationToken return ControllerResult.atomicOf(records, responseData.setErrorCode(DELEGATION_TOKEN_NOT_FOUND.code())); } - if (myTokenInformation.maxTimestamp() < now || myTokenInformation.expiryTimestamp() < now) { - return ControllerResult.atomicOf(records, responseData.setErrorCode(DELEGATION_TOKEN_EXPIRED.code())); - } - if (!allowedToRenew(myTokenInformation, context.principal())) { return ControllerResult.atomicOf(records, responseData.setErrorCode(DELEGATION_TOKEN_OWNER_MISMATCH.code())); } @@ -313,9 +309,11 @@ public ControllerResult expireDelegationToken if (requestData.expiryTimePeriodMs() < 0) { // expire immediately responseData .setErrorCode(NONE.code()) - .setExpiryTimestampMs(requestData.expiryTimePeriodMs()); + .setExpiryTimestampMs(now); records.add(new ApiMessageAndVersion(new RemoveDelegationTokenRecord(). setTokenId(myTokenInformation.tokenId()), (short) 0)); + } else if (myTokenInformation.maxTimestamp() < now || myTokenInformation.expiryTimestamp() < now) { + responseData.setErrorCode(DELEGATION_TOKEN_EXPIRED.code()); } else { long expiryTimestamp = Math.min(myTokenInformation.maxTimestamp(), now + requestData.expiryTimePeriodMs()); From db686fb9645612ca0a44535b409c671904124084 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Sun, 25 Aug 2024 17:39:28 +0800 Subject: [PATCH 062/123] KAFKA-17331 Set correct version for EarliestLocalSpec and LatestTieredSpec (#16876) Add the version check to client side when building ListOffsetRequest for the specific timestamp: 1) the version must be >=8 if timestamp=-4L (EARLIEST_LOCAL_TIMESTAMP) 2) the version must be >=9 if timestamp=-5L (LATEST_TIERED_TIMESTAMP) Reviewers: PoAn Yang , Chia-Ping Tsai --- checkstyle/import-control.xml | 1 + .../admin/internals/ListOffsetsHandler.java | 17 ++++--- .../consumer/internals/OffsetFetcher.java | 2 +- .../internals/OffsetsRequestManager.java | 2 +- .../common/requests/ListOffsetsRequest.java | 12 ++++- .../clients/admin/KafkaAdminClientTest.java | 2 +- .../internals/ListOffsetsHandlerTest.java | 14 +++++- .../requests/ListOffsetsRequestTest.java | 44 +++++++++++++++++-- .../common/requests/RequestResponseTest.java | 6 +-- .../kafka/api/AuthorizerIntegrationTest.scala | 4 +- .../unit/kafka/server/KafkaApisTest.scala | 4 +- .../kafka/server/ListOffsetsRequestTest.scala | 20 +++++---- .../unit/kafka/server/LogOffsetTest.scala | 2 +- .../unit/kafka/server/RequestQuotaTest.scala | 2 +- .../common/ListOffsetRequestBenchmark.java | 2 +- 15 files changed, 101 insertions(+), 33 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 280dc1dc3c..a3b1c69796 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -49,6 +49,7 @@ + diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java index 7dfcb22afb..338c41a8fe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java @@ -93,13 +93,20 @@ ListOffsetsRequest.Builder buildBatchedRequest(int brokerId, Set .stream() .anyMatch(key -> offsetTimestampsByPartition.get(key) == ListOffsetsRequest.MAX_TIMESTAMP); + boolean requireEarliestLocalTimestamp = keys + .stream() + .anyMatch(key -> offsetTimestampsByPartition.get(key) == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP); + boolean requireTieredStorageTimestamp = keys .stream() - .anyMatch(key -> offsetTimestampsByPartition.get(key) == ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP || offsetTimestampsByPartition.get(key) == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP); - - return ListOffsetsRequest.Builder - .forConsumer(true, options.isolationLevel(), supportsMaxTimestamp, requireTieredStorageTimestamp) - .setTargetTimes(new ArrayList<>(topicsByName.values())); + .anyMatch(key -> offsetTimestampsByPartition.get(key) == ListOffsetsRequest.LATEST_TIERED_TIMESTAMP); + + return ListOffsetsRequest.Builder.forConsumer(true, + options.isolationLevel(), + supportsMaxTimestamp, + requireEarliestLocalTimestamp, + requireTieredStorageTimestamp) + .setTargetTimes(new ArrayList<>(topicsByName.values())); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java index ec0bfe2fc1..56c77ec6e4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetFetcher.java @@ -391,7 +391,7 @@ private RequestFuture sendListOffsetRequest(final Node node, final Map timestampsToSearch, boolean requireTimestamp) { ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder - .forConsumer(requireTimestamp, isolationLevel, false, false) + .forConsumer(requireTimestamp, isolationLevel) .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(timestampsToSearch)); log.debug("Sending ListOffsetRequest {} to broker {}", builder, node); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java index 71da2f5bf6..c219f1d98e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java @@ -337,7 +337,7 @@ private CompletableFuture buildListOffsetRequestToNode( boolean requireTimestamps, List unsentRequests) { ListOffsetsRequest.Builder builder = ListOffsetsRequest.Builder - .forConsumer(requireTimestamps, isolationLevel, false, false) + .forConsumer(requireTimestamps, isolationLevel) .setTargetTimes(ListOffsetsRequest.toListOffsetsTopics(targetTimes)); log.debug("Creating ListOffset request {} for broker {} to reset positions", builder, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java index 8ebf0886be..ce95a4ae68 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsRequest.java @@ -58,17 +58,21 @@ public class ListOffsetsRequest extends AbstractRequest { public static class Builder extends AbstractRequest.Builder { private final ListOffsetsRequestData data; - public static Builder forReplica(short allowedVersion, int replicaId) { - return new Builder((short) 0, allowedVersion, replicaId, IsolationLevel.READ_UNCOMMITTED); + public static Builder forConsumer(boolean requireTimestamp, + IsolationLevel isolationLevel) { + return forConsumer(requireTimestamp, isolationLevel, false, false, false); } public static Builder forConsumer(boolean requireTimestamp, IsolationLevel isolationLevel, boolean requireMaxTimestamp, + boolean requireEarliestLocalTimestamp, boolean requireTieredStorageTimestamp) { short minVersion = 0; if (requireTieredStorageTimestamp) minVersion = 9; + else if (requireEarliestLocalTimestamp) + minVersion = 8; else if (requireMaxTimestamp) minVersion = 7; else if (isolationLevel == IsolationLevel.READ_COMMITTED) @@ -78,6 +82,10 @@ else if (requireTimestamp) return new Builder(minVersion, ApiKeys.LIST_OFFSETS.latestVersion(), CONSUMER_REPLICA_ID, isolationLevel); } + public static Builder forReplica(short allowedVersion, int replicaId) { + return new Builder((short) 0, allowedVersion, replicaId, IsolationLevel.READ_UNCOMMITTED); + } + private Builder(short oldestAllowedVersion, short latestAllowedVersion, int replicaId, diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index fbae396830..5018952966 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -5883,7 +5883,7 @@ public void testListOffsetsEarliestLocalSpecMinVersion() throws Exception { env.adminClient().listOffsets(Collections.singletonMap(tp0, OffsetSpec.earliestLocal())); TestUtils.waitForCondition(() -> env.kafkaClient().requests().stream().anyMatch(request -> - request.requestBuilder().apiKey().messageType == ApiMessageType.LIST_OFFSETS && request.requestBuilder().oldestAllowedVersion() == 9 + request.requestBuilder().apiKey().messageType == ApiMessageType.LIST_OFFSETS && request.requestBuilder().oldestAllowedVersion() == 8 ), "no listOffsets request has the expected oldestAllowedVersion"); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandlerTest.java index fe02303706..d3896e0023 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandlerTest.java @@ -60,6 +60,8 @@ public final class ListOffsetsHandlerTest { private final TopicPartition t0p1 = new TopicPartition("t0", 1); private final TopicPartition t1p0 = new TopicPartition("t1", 0); private final TopicPartition t1p1 = new TopicPartition("t1", 1); + private final TopicPartition t2p0 = new TopicPartition("t2", 0); + private final TopicPartition t2p1 = new TopicPartition("t2", 1); private final Node node = new Node(1, "host", 1234); @@ -69,6 +71,8 @@ public final class ListOffsetsHandlerTest { put(t0p1, ListOffsetsRequest.EARLIEST_TIMESTAMP); put(t1p0, 123L); put(t1p1, ListOffsetsRequest.MAX_TIMESTAMP); + put(t2p0, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP); + put(t2p1, ListOffsetsRequest.LATEST_TIERED_TIMESTAMP); } }; @@ -96,14 +100,14 @@ public void testBuildRequestMultipleTopicsWithReadCommitted() { ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), offsetTimestampsByPartition.keySet()).build(); List topics = request.topics(); - assertEquals(2, topics.size()); + assertEquals(3, topics.size()); Map partitions = new HashMap<>(); for (ListOffsetsTopic topic : topics) { for (ListOffsetsPartition partition : topic.partitions()) { partitions.put(new TopicPartition(topic.name(), partition.partitionIndex()), partition); } } - assertEquals(4, partitions.size()); + assertEquals(6, partitions.size()); for (Map.Entry entry : partitions.entrySet()) { assertExpectedTimestamp(entry.getKey(), entry.getValue().timestamp()); } @@ -126,6 +130,12 @@ public void testBuildRequestAllowedVersions() { builder = readCommittedHandler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1, t1p0, t1p1)); assertEquals(7, builder.oldestAllowedVersion()); + + builder = readCommittedHandler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1, t1p0, t1p1, t2p0)); + assertEquals(8, builder.oldestAllowedVersion()); + + builder = readCommittedHandler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1, t1p0, t1p1, t2p0, t2p1)); + assertEquals(9, builder.oldestAllowedVersion()); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java index e9a2bec2ff..3888f0cd32 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ListOffsetsRequestTest.java @@ -16,8 +16,11 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.ListOffsetsRequestData; import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic; @@ -36,11 +39,16 @@ import java.util.List; import java.util.Map; +import static org.apache.kafka.common.requests.ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP; +import static org.apache.kafka.common.requests.ListOffsetsRequest.LATEST_TIERED_TIMESTAMP; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class ListOffsetsRequestTest { + private final NodeApiVersions versionInfo = new NodeApiVersions(new ApiVersionsResponseData.ApiVersionCollection(), Collections.emptyList(), false); + @Test public void testDuplicatePartitions() { List topics = Collections.singletonList( @@ -68,7 +76,7 @@ public void testGetErrorResponse() { new ListOffsetsPartition() .setPartitionIndex(0)))); ListOffsetsRequest request = ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_COMMITTED, false, false) + .forConsumer(true, IsolationLevel.READ_COMMITTED) .setTargetTimes(topics) .build(version); ListOffsetsResponse response = (ListOffsetsResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception()); @@ -101,7 +109,7 @@ public void testGetErrorResponseV0() { new ListOffsetsPartition() .setPartitionIndex(0)))); ListOffsetsRequest request = ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_UNCOMMITTED, false, false) + .forConsumer(true, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(topics) .build((short) 0); ListOffsetsResponse response = (ListOffsetsResponse) request.getErrorResponse(0, Errors.NOT_LEADER_OR_FOLLOWER.exception()); @@ -146,4 +154,34 @@ public void testToListOffsetsTopics() { assertTrue(topic.partitions().contains(lop1)); } -} + @Test + public void testCheckEarliestLocalTimestampVersion() { + int maxVersion = ApiKeys.LIST_OFFSETS.latestVersion(); + for (int i = 0; i <= maxVersion; i++) { + testUnsupportedVersion(i, EARLIEST_LOCAL_TIMESTAMP); + } + } + + @Test + public void testCheckLatestTieredTimestampVersion() { + int maxVersion = ApiKeys.LIST_OFFSETS.latestVersion(); + for (int i = 0; i <= maxVersion; i++) { + testUnsupportedVersion(i, LATEST_TIERED_TIMESTAMP); + } + } + + private void testUnsupportedVersion(int version, long timestamp) { + if (timestamp == EARLIEST_LOCAL_TIMESTAMP && version < 8) { + assertUnsupportedVersion(version); + } else if (timestamp == LATEST_TIERED_TIMESTAMP && version < 9) { + assertUnsupportedVersion(version); + } + } + + private void assertUnsupportedVersion(int version) { + ApiKeys apiKey = ApiKeys.LIST_OFFSETS; + UnsupportedVersionException exception = assertThrows(UnsupportedVersionException.class, + () -> versionInfo.latestUsableVersion(apiKey, (short) version, apiKey.latestVersion())); + assertEquals("The node does not support " + apiKey, exception.getMessage()); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 7aee499163..5a9c8193d1 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -2355,7 +2355,7 @@ private ListOffsetsRequest createListOffsetRequest(short version) { .setMaxNumOffsets(10) .setCurrentLeaderEpoch(5))); return ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(singletonList(topic)) .build(version); } else if (version == 1) { @@ -2366,7 +2366,7 @@ private ListOffsetsRequest createListOffsetRequest(short version) { .setTimestamp(1000000L) .setCurrentLeaderEpoch(5))); return ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_UNCOMMITTED, false, false) + .forConsumer(true, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(singletonList(topic)) .build(version); } else if (version >= 2 && version <= LIST_OFFSETS.latestVersion()) { @@ -2379,7 +2379,7 @@ private ListOffsetsRequest createListOffsetRequest(short version) { .setName("test") .setPartitions(singletonList(partition)); return ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_COMMITTED, false, false) + .forConsumer(true, IsolationLevel.READ_COMMITTED) .setTargetTimes(singletonList(topic)) .build(version); } else { diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index c58caaae8b..1b992e6c35 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -287,8 +287,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } private def createListOffsetsRequest = { - requests.ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false).setTargetTimes( - List(new ListOffsetsTopic() + requests.ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) + .setTargetTimes(List(new ListOffsetsTopic() .setName(tp.topic) .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(tp.partition) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 9b45fbd6a4..be1afc3494 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -4010,7 +4010,7 @@ class KafkaApisTest extends Logging { .setPartitionIndex(tp.partition) .setTimestamp(ListOffsetsRequest.EARLIEST_TIMESTAMP) .setCurrentLeaderEpoch(currentLeaderEpoch.get)).asJava)).asJava - val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel, false, false) + val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel) .setTargetTimes(targetTimes).build() val request = buildRequest(listOffsetRequest) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), @@ -6170,7 +6170,7 @@ class KafkaApisTest extends Logging { .setPartitions(List(new ListOffsetsPartition() .setPartitionIndex(tp.partition) .setTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)).asJava)).asJava - val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel, false, false) + val listOffsetRequest = ListOffsetsRequest.Builder.forConsumer(true, isolationLevel) .setTargetTimes(targetTimes).build() val request = buildRequest(listOffsetRequest) kafkaApis = createKafkaApis() diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala index f48e3546d8..03585be97c 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala @@ -54,7 +54,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { .setCurrentLeaderEpoch(0)).asJava)).asJava val consumerRequest = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(targetTimes) .build() @@ -96,24 +96,28 @@ class ListOffsetsRequestTest extends BaseRequestTest { @ValueSource(strings = Array("zk", "kraft")) def testListOffsetsRequestOldestVersion(): Unit = { val consumerRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) val requireTimestampRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(true, IsolationLevel.READ_UNCOMMITTED, false, false) + .forConsumer(true, IsolationLevel.READ_UNCOMMITTED) val requestCommittedRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_COMMITTED, false, false) + .forConsumer(false, IsolationLevel.READ_COMMITTED) val maxTimestampRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, true, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, true, false, false) + + val requireEarliestLocalTimestampRequestBuilder = ListOffsetsRequest.Builder + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, true, false) val requireTieredStorageTimestampRequestBuilder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, true) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false, true) assertEquals(0.toShort, consumerRequestBuilder.oldestAllowedVersion()) assertEquals(1.toShort, requireTimestampRequestBuilder.oldestAllowedVersion()) assertEquals(2.toShort, requestCommittedRequestBuilder.oldestAllowedVersion()) assertEquals(7.toShort, maxTimestampRequestBuilder.oldestAllowedVersion()) + assertEquals(8.toShort, requireEarliestLocalTimestampRequestBuilder.oldestAllowedVersion()) assertEquals(9.toShort, requireTieredStorageTimestampRequestBuilder.oldestAllowedVersion()) } @@ -127,7 +131,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { .setName(topic) .setPartitions(List(listOffsetPartition).asJava)).asJava val request = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(targetTimes) .build() assertResponseError(error, brokerId, request) @@ -171,7 +175,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { .setTimestamp(timestamp)).asJava)).asJava val builder = ListOffsetsRequest.Builder - .forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + .forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(targetTimes) val request = if (version == -1) builder.build() else builder.build(version) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 61fcd6fcf2..177f59221a 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -60,7 +60,7 @@ class LogOffsetTest extends BaseRequestTest { @ValueSource(strings = Array("zk", "kraft")) def testGetOffsetsForUnknownTopic(quorum: String): Unit = { val topicPartition = new TopicPartition("foo", 0) - val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.LATEST_TIMESTAMP, 10).asJava).build(0) val response = sendListOffsetsRequest(request) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, findPartition(response.topics.asScala, topicPartition).errorCode) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index fbffa4ce37..218c0cf5fb 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -288,7 +288,7 @@ class RequestQuotaTest extends BaseRequestTest { .setPartitionIndex(tp.partition) .setTimestamp(0L) .setCurrentLeaderEpoch(15)).asJava) - ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(List(topic).asJava) case ApiKeys.LEADER_AND_ISR => diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java index 786648f904..f5e8b3e459 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ListOffsetRequestBenchmark.java @@ -71,7 +71,7 @@ public void setup() { } } - this.offsetRequest = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED, false, false) + this.offsetRequest = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .build(ApiKeys.LIST_OFFSETS.latestVersion()); } From 6d2b81e07f22086e3e752024168772657294c4d9 Mon Sep 17 00:00:00 2001 From: Kuan-Po Tseng Date: Sun, 25 Aug 2024 19:46:15 +0800 Subject: [PATCH 063/123] KAFKA-17360 local log retention ms/bytes "-2" is not treated correctly (#16932) 1) When the local.retention.ms/bytes is set to -2, we didn't replace it with the server-side retention.ms/bytes config, so the -2 local retention won't take effect. 2) When setting retention.ms/bytes to -2, we can notice this log message: ``` Deleting segment LogSegment(baseOffset=10045, size=1037087, lastModifiedTime=1724040653922, largestRecordTimestamp=1724040653835) due to local log retention size -2 breach. Local log size after deletion will be 13435280. (kafka.log.UnifiedLog) [kafka-scheduler-6] ``` This is not helpful for users. We should replace -2 with real retention value when logging. Reviewers: Luke Chen , Chia-Ping Tsai --- .../src/main/scala/kafka/log/UnifiedLog.scala | 20 ++++---- .../scala/unit/kafka/log/UnifiedLogTest.scala | 48 +++++++++++++++++++ .../storage/internals/log/LogConfig.java | 14 +++--- .../integration/BaseDeleteSegmentsTest.java | 5 +- 4 files changed, 68 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 69b6787ca6..092b8c92aa 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -1586,7 +1586,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, startMs - segment.largestTimestamp > retentionMs } - deleteOldSegments(shouldDelete, RetentionMsBreach(this, remoteLogEnabled())) + deleteOldSegments(shouldDelete, RetentionMsBreach(this, remoteLogEnabledAndRemoteCopyEnabled())) } private def deleteRetentionSizeBreachedSegments(): Int = { @@ -1602,7 +1602,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, } } - deleteOldSegments(shouldDelete, RetentionSizeBreach(this, remoteLogEnabled())) + deleteOldSegments(shouldDelete, RetentionSizeBreach(this, remoteLogEnabledAndRemoteCopyEnabled())) } private def deleteLogStartOffsetBreachedSegments(): Int = { @@ -2347,11 +2347,11 @@ object UnifiedLog extends Logging { } private[log] def localRetentionMs(config: LogConfig, remoteLogEnabledAndRemoteCopyEnabled: Boolean): Long = { - if (remoteLogEnabledAndRemoteCopyEnabled) config.remoteLogConfig.localRetentionMs else config.retentionMs + if (remoteLogEnabledAndRemoteCopyEnabled) config.localRetentionMs else config.retentionMs } private[log] def localRetentionSize(config: LogConfig, remoteLogEnabledAndRemoteCopyEnabled: Boolean): Long = { - if (remoteLogEnabledAndRemoteCopyEnabled) config.remoteLogConfig.localRetentionBytes else config.retentionSize + if (remoteLogEnabledAndRemoteCopyEnabled) config.localRetentionBytes else config.retentionSize } } @@ -2367,19 +2367,19 @@ object LogMetricNames { } } -case class RetentionMsBreach(log: UnifiedLog, remoteLogEnabled: Boolean) extends SegmentDeletionReason { +case class RetentionMsBreach(log: UnifiedLog, remoteLogEnabledAndRemoteCopyEnabled: Boolean) extends SegmentDeletionReason { override def logReason(toDelete: List[LogSegment]): Unit = { - val retentionMs = UnifiedLog.localRetentionMs(log.config, remoteLogEnabled) + val retentionMs = UnifiedLog.localRetentionMs(log.config, remoteLogEnabledAndRemoteCopyEnabled) toDelete.foreach { segment => if (segment.largestRecordTimestamp.isPresent) - if (remoteLogEnabled) + if (remoteLogEnabledAndRemoteCopyEnabled) log.info(s"Deleting segment $segment due to local log retention time ${retentionMs}ms breach based on the largest " + s"record timestamp in the segment") else log.info(s"Deleting segment $segment due to log retention time ${retentionMs}ms breach based on the largest " + s"record timestamp in the segment") else { - if (remoteLogEnabled) + if (remoteLogEnabledAndRemoteCopyEnabled) log.info(s"Deleting segment $segment due to local log retention time ${retentionMs}ms breach based on the " + s"last modified time of the segment") else @@ -2390,12 +2390,12 @@ case class RetentionMsBreach(log: UnifiedLog, remoteLogEnabled: Boolean) extends } } -case class RetentionSizeBreach(log: UnifiedLog, remoteLogEnabled: Boolean) extends SegmentDeletionReason { +case class RetentionSizeBreach(log: UnifiedLog, remoteLogEnabledAndRemoteCopyEnabled: Boolean) extends SegmentDeletionReason { override def logReason(toDelete: List[LogSegment]): Unit = { var size = log.size toDelete.foreach { segment => size -= segment.size - if (remoteLogEnabled) log.info(s"Deleting segment $segment due to local log retention size ${UnifiedLog.localRetentionSize(log.config, remoteLogEnabled)} breach. " + + if (remoteLogEnabledAndRemoteCopyEnabled) log.info(s"Deleting segment $segment due to local log retention size ${UnifiedLog.localRetentionSize(log.config, remoteLogEnabledAndRemoteCopyEnabled)} breach. " + s"Local log size after deletion will be $size.") else log.info(s"Deleting segment $segment due to log retention size ${log.config.retentionSize} breach. Log size " + s"after deletion will be $size.") diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index b45c9171d6..6d7614d9b5 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -4187,6 +4187,54 @@ class UnifiedLogTest { assertEquals(1, log.logSegments.size) } + @Test + def testRetentionOnLocalLogDeletionWhenRemoteLogCopyEnabledAndDefaultLocalRetentionBytes(): Unit = { + def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) + val segmentBytes = createRecords.sizeInBytes() + val retentionBytesConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, retentionBytes = 1, + fileDeleteDelayMs = 0, remoteLogStorageEnable = true) + val log = createLog(logDir, retentionBytesConfig, remoteStorageSystemEnable = true) + + // Given 6 segments of 1 message each + for (_ <- 0 until 6) { + log.appendAsLeader(createRecords, leaderEpoch = 0) + } + assertEquals(6, log.logSegments.size) + + log.updateHighWatermark(log.logEndOffset) + // simulate calls to upload 2 segments to remote storage + log.updateHighestOffsetInRemoteStorage(1) + log.deleteOldSegments() + assertEquals(4, log.logSegments.size()) + assertEquals(0, log.logStartOffset) + assertEquals(2, log.localLogStartOffset()) + } + + @Test + def testRetentionOnLocalLogDeletionWhenRemoteLogCopyEnabledAndDefaultLocalRetentionMs(): Unit = { + def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) + val segmentBytes = createRecords.sizeInBytes() + val retentionBytesConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, retentionMs = 1000, + fileDeleteDelayMs = 0, remoteLogStorageEnable = true) + val log = createLog(logDir, retentionBytesConfig, remoteStorageSystemEnable = true) + + // Given 6 segments of 1 message each + for (_ <- 0 until 6) { + log.appendAsLeader(createRecords, leaderEpoch = 0) + } + assertEquals(6, log.logSegments.size) + + log.updateHighWatermark(log.logEndOffset) + // simulate calls to upload 2 segments to remote storage + log.updateHighestOffsetInRemoteStorage(1) + + mockTime.sleep(1001) + log.deleteOldSegments() + assertEquals(4, log.logSegments.size()) + assertEquals(0, log.logStartOffset) + assertEquals(2, log.localLogStartOffset()) + } + @Test def testRetentionOnLocalLogDeletionWhenRemoteLogCopyDisabled(): Unit = { def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index f3f151ddc1..3299b6e89d 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -108,13 +108,13 @@ public String topicWarningMessage(String topicName) { } } - public static class RemoteLogConfig { + private static class RemoteLogConfig { - public final boolean remoteStorageEnable; - public final boolean remoteLogDeleteOnDisable; - public final boolean remoteLogCopyDisable; - public final long localRetentionMs; - public final long localRetentionBytes; + private final boolean remoteStorageEnable; + private final boolean remoteLogDeleteOnDisable; + private final boolean remoteLogCopyDisable; + private final long localRetentionMs; + private final long localRetentionBytes; private RemoteLogConfig(LogConfig config) { this.remoteStorageEnable = config.getBoolean(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG); @@ -375,8 +375,8 @@ public Optional serverConfigName(String configName) { public final List leaderReplicationThrottledReplicas; public final List followerReplicationThrottledReplicas; public final boolean messageDownConversionEnable; - public final RemoteLogConfig remoteLogConfig; + private final RemoteLogConfig remoteLogConfig; private final int maxMessageSize; private final Map props; diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseDeleteSegmentsTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseDeleteSegmentsTest.java index 0147c9f7c5..11c3ca7ad0 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseDeleteSegmentsTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/BaseDeleteSegmentsTest.java @@ -23,6 +23,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent.EventType.DELETE_SEGMENT; @@ -54,8 +55,8 @@ protected void writeTestSpecifications(TieredStorageTestBuilder builder) { .expectSegmentToBeOffloaded(broker0, topicA, p0, 1, new KeyValueSpec("k1", "v1")) .expectSegmentToBeOffloaded(broker0, topicA, p0, 2, new KeyValueSpec("k2", "v2")) .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 3L) - .produce(topicA, p0, new KeyValueSpec("k0", "v0"), new KeyValueSpec("k1", "v1"), - new KeyValueSpec("k2", "v2"), new KeyValueSpec("k3", "v3")) + .produceWithTimestamp(topicA, p0, new KeyValueSpec("k0", "v0"), new KeyValueSpec("k1", "v1"), + new KeyValueSpec("k2", "v2"), new KeyValueSpec("k3", "v3", System.currentTimeMillis() + TimeUnit.DAYS.toMillis(1))) // update the topic config such that it triggers the deletion of segments .updateTopicConfig(topicA, configsToBeAdded(), Collections.emptyList()) // expect that the three offloaded remote log segments are deleted From a19792fbd747e761ba02a4b62a12f9ffc4340715 Mon Sep 17 00:00:00 2001 From: Arpit Goyal Date: Wed, 28 Aug 2024 08:14:57 +0530 Subject: [PATCH 064/123] KAFKA-17422: Adding copySegmentLatch countdown after expiration task is over (#17012) The given test took 5 seconds as the logic was waiting completely for 5 seconds for the expiration task to be completed. Adding copySegmentLatch countdown after expiration task is over Reviewers: Luke Chen , Chia-Ping Tsai --- core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 302b6adfe4..d6d2ff0ecb 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -2133,6 +2133,7 @@ long findLogStartOffset(TopicIdPartition topicIdPartition, UnifiedLog log) { // wait until copy thread has started copying segment data copySegmentDataLatch.await(); expirationTask.cleanupExpiredRemoteLogSegments(); + copyLogSegmentLatch.countDown(); } catch (RemoteStorageException | ExecutionException | InterruptedException e) { throw new RuntimeException(e); } From a87b501a47a55a1f9038b6c52e31b71590c3a80a Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 28 Aug 2024 08:11:41 -0700 Subject: [PATCH 065/123] KAFKA-17335 Lack of default for URL encoding configuration for OAuth causes NPE (#16990) AccessTokenRetrieverFactory uses the value of sasl.oauthbearer.header.urlencode provided by the user, or null if no value was provided for that configuration. When the HttpAccessTokenRetriever is created the JVM attempts to unbox the value into a boolean, a NullPointerException is thrown. The fix is to explicitly check the Boolean, and if it's null, use Boolean.FALSE. Reviewers: bachmanity1 <81428651+bachmanity1@users.noreply.github.com>, Chia-Ping Tsai --- .../secured/AccessTokenRetrieverFactory.java | 26 ++++++++++++++++- .../internals/secured/ConfigurationUtils.java | 29 ++++++++++++------- .../AccessTokenRetrieverFactoryTest.java | 23 +++++++++++++++ 3 files changed, 67 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java index a0ebc586eb..0ed4a1a230 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactory.java @@ -17,12 +17,15 @@ package org.apache.kafka.common.security.oauthbearer.internals.secured; +import org.apache.kafka.common.config.SaslConfigs; + import java.net.URL; import java.util.Locale; import java.util.Map; import javax.net.ssl.SSLSocketFactory; +import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_CONNECT_TIMEOUT_MS; import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_READ_TIMEOUT_MS; import static org.apache.kafka.common.config.SaslConfigs.SASL_LOGIN_RETRY_BACKOFF_MAX_MS; @@ -70,6 +73,8 @@ public static AccessTokenRetriever create(Map configs, if (jou.shouldCreateSSLSocketFactory(tokenEndpointUrl)) sslSocketFactory = jou.createSSLSocketFactory(); + boolean urlencodeHeader = validateUrlencodeHeader(cu); + return new HttpAccessTokenRetriever(clientId, clientSecret, scope, @@ -79,8 +84,27 @@ public static AccessTokenRetriever create(Map configs, cu.validateLong(SASL_LOGIN_RETRY_BACKOFF_MAX_MS), cu.validateInteger(SASL_LOGIN_CONNECT_TIMEOUT_MS, false), cu.validateInteger(SASL_LOGIN_READ_TIMEOUT_MS, false), - cu.get(SASL_OAUTHBEARER_HEADER_URLENCODE)); + urlencodeHeader); } } + /** + * In some cases, the incoming {@link Map} doesn't contain a value for + * {@link SaslConfigs#SASL_OAUTHBEARER_HEADER_URLENCODE}. Returning {@code null} from {@link Map#get(Object)} + * will cause a {@link NullPointerException} when it is later unboxed. + * + *

    + * + * This utility method ensures that we have a non-{@code null} value to use in the + * {@link HttpAccessTokenRetriever} constructor. + */ + static boolean validateUrlencodeHeader(ConfigurationUtils configurationUtils) { + Boolean urlencodeHeader = configurationUtils.validateBoolean(SASL_OAUTHBEARER_HEADER_URLENCODE, false); + + if (urlencodeHeader != null) + return urlencodeHeader; + else + return DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; + } + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java index 5b878130ba..0be91cdfb5 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/secured/ConfigurationUtils.java @@ -72,17 +72,17 @@ public Path validateFile(String name) { try { file = new File(url.toURI().getRawPath()).getAbsoluteFile(); } catch (URISyntaxException e) { - throw new ConfigException(name, url.toString(), String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, url, e.getMessage())); + throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, url, e.getMessage())); } if (!file.exists()) - throw new ConfigException(name, file, String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", name, file)); + throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't exist", name, file)); if (!file.canRead()) - throw new ConfigException(name, file, String.format("The OAuth configuration option %s contains a file (%s) that doesn't have read permission", name, file)); + throw new ConfigException(String.format("The OAuth configuration option %s contains a file (%s) that doesn't have read permission", name, file)); if (file.isDirectory()) - throw new ConfigException(name, file, String.format("The OAuth configuration option %s references a directory (%s), not a file", name, file)); + throw new ConfigException(String.format("The OAuth configuration option %s references a directory (%s), not a file", name, file)); return file.toPath(); } @@ -104,7 +104,7 @@ public Integer validateInteger(String name, boolean isRequired) { if (value == null) { if (isRequired) - throw new ConfigException(name, null, String.format("The OAuth configuration option %s must be non-null", name)); + throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); else return null; } @@ -137,13 +137,13 @@ public Long validateLong(String name, boolean isRequired, Long min) { if (value == null) { if (isRequired) - throw new ConfigException(name, null, String.format("The OAuth configuration option %s must be non-null", name)); + throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); else return null; } if (min != null && value < min) - throw new ConfigException(name, value, String.format("The OAuth configuration option %s value must be at least %s", name, min)); + throw new ConfigException(String.format("The OAuth configuration option %s value must be at least %s", name, min)); return value; } @@ -167,18 +167,18 @@ public URL validateUrl(String name) { try { url = new URL(value); } catch (MalformedURLException e) { - throw new ConfigException(name, value, String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, value, e.getMessage())); + throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is malformed: %s", name, value, e.getMessage())); } String protocol = url.getProtocol(); if (protocol == null || protocol.trim().isEmpty()) - throw new ConfigException(name, value, String.format("The OAuth configuration option %s contains a URL (%s) that is missing the protocol", name, value)); + throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that is missing the protocol", name, value)); protocol = protocol.toLowerCase(Locale.ROOT); if (!(protocol.equals("http") || protocol.equals("https") || protocol.equals("file"))) - throw new ConfigException(name, value, String.format("The OAuth configuration option %s contains a URL (%s) that contains an invalid protocol (%s); only \"http\", \"https\", and \"file\" protocol are supported", name, value, protocol)); + throw new ConfigException(String.format("The OAuth configuration option %s contains a URL (%s) that contains an invalid protocol (%s); only \"http\", \"https\", and \"file\" protocol are supported", name, value, protocol)); return url; } @@ -209,6 +209,15 @@ public String validateString(String name, boolean isRequired) throws ValidateExc return value; } + public Boolean validateBoolean(String name, boolean isRequired) { + Boolean value = get(name); + + if (value == null && isRequired) + throw new ConfigException(String.format("The OAuth configuration option %s must be non-null", name)); + + return value; + } + @SuppressWarnings("unchecked") public T get(String name) { T value = (T) configs.get(prefix + name); diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java index 0ba706d898..478e2baba1 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/AccessTokenRetrieverFactoryTest.java @@ -20,11 +20,17 @@ import org.apache.kafka.common.config.ConfigException; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.util.Collections; import java.util.Map; +import java.util.stream.Stream; +import static org.apache.kafka.common.config.SaslConfigs.DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE; +import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE; import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -64,4 +70,21 @@ public void testConfigureRefreshingFileAccessTokenRetrieverWithInvalidFile() thr assertThrowsWithMessage(ConfigException.class, () -> AccessTokenRetrieverFactory.create(configs, jaasConfig), "that doesn't exist"); } + @ParameterizedTest + @MethodSource("urlencodeHeaderSupplier") + public void testUrlencodeHeader(Map configs, boolean expectedValue) { + ConfigurationUtils cu = new ConfigurationUtils(configs); + boolean actualValue = AccessTokenRetrieverFactory.validateUrlencodeHeader(cu); + assertEquals(expectedValue, actualValue); + } + + private static Stream urlencodeHeaderSupplier() { + return Stream.of( + Arguments.of(Collections.emptyMap(), DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE), + Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, null), DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE), + Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, true), true), + Arguments.of(Collections.singletonMap(SASL_OAUTHBEARER_HEADER_URLENCODE, false), false) + ); + } + } From 145fa49e54460acdf2d4dba6423f42476872926e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 28 Aug 2024 14:31:58 -0400 Subject: [PATCH 066/123] KAFKA-17426; Check node directory id for KRaft (#17017) Reviewers: Colin P. McCabe --- .../java/org/apache/kafka/raft/KafkaRaftClient.java | 13 +++++++++---- .../org/apache/kafka/raft/KafkaRaftClientTest.java | 8 ++++++++ 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index e8ab7b5c6d..d75417a77a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -221,6 +221,7 @@ public final class KafkaRaftClient implements RaftClient { * Note that if the node ID is empty, then the client will behave as a * non-participating observer. * + * @param nodeDirectoryId the node directory id, cannot be the zero uuid * @param followersAlwaysFlush instruct followers to always fsync when appending to the log */ public KafkaRaftClient( @@ -281,6 +282,10 @@ public KafkaRaftClient( Random random, QuorumConfig quorumConfig ) { + if (nodeDirectoryId.equals(Uuid.ZERO_UUID)) { + throw new IllegalArgumentException("The node directory id must be set and not be the zero uuid"); + } + this.nodeId = nodeId; this.nodeDirectoryId = nodeDirectoryId; this.logContext = logContext; @@ -3178,11 +3183,11 @@ private void pollListeners() { if (leaderState.isPresent()) { maybeFireLeaderChange(leaderState.get()); } else if (!quorum.isResigned()) { - /* Should not fire leader change while in the resigned state for two reasons. + /* Should not fire leader change while in the resigned state for two reasons: * 1. The epoch start offset is not tracked but the leader is the local replica. - * Listener cannot be notify of leadership until they have caught to the latest - * epoch. - * 2. It is not pratical to notify of local leadership since any write operation + * Listener cannot be notified of leadership until they have caught to the latest + * epoch and LEO. + * 2. It is not practical to notify of local leadership since any write operation * (prepareAppend and schedulePreparedAppend) will fail with NotLeaderException */ maybeFireLeaderChange(); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 52ffcadc25..1477ee0a39 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -81,6 +81,14 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class KafkaRaftClientTest { + @Test + public void testNodeDirectoryId() { + int localId = randomReplicaId(); + assertThrows( + IllegalArgumentException.class, + new RaftClientTestContext.Builder(localId, Uuid.ZERO_UUID)::build + ); + } @ParameterizedTest @ValueSource(booleans = { true, false }) From e5f47ba35019db8f6059b0acf068379334964fc1 Mon Sep 17 00:00:00 2001 From: kevin-wu24 <66326898+kevin-wu24@users.noreply.github.com> Date: Tue, 27 Aug 2024 18:39:31 -0500 Subject: [PATCH 067/123] KAFKA-15406: Add the ForwardingManager metrics from KIP-938 (#16904) Implement the remaining ForwardingManager metrics from KIP-938: Add more metrics for measuring KRaft performance: kafka.server:type=ForwardingManager,name=QueueTimeMs.p99 kafka.server:type=ForwardingManager,name=QueueTimeMs.p999 kafka.server:type=ForwardingManager,name=QueueLength kafka.server:type=ForwardingManager,name=RemoteTimeMs.p99 kafka.server:type=ForwardingManager,name=RemoteTimeMs.p999 Reviewers: Colin P. McCabe --- .../scala/kafka/server/BrokerServer.scala | 5 +- .../kafka/server/ForwardingManager.scala | 31 ++++- .../server/ForwardingManagerMetrics.scala | 100 +++++++++++++++ .../main/scala/kafka/server/KafkaServer.scala | 5 +- .../NodeToControllerChannelManager.scala | 2 + .../server/ForwardingManagerMetricsTest.scala | 114 ++++++++++++++++++ .../kafka/server/ForwardingManagerTest.scala | 53 +++++++- .../MockNodeToControllerChannelManager.scala | 2 + .../NodeToControllerChannelManager.java | 2 + .../kafka/server/AssignmentsManagerTest.java | 5 + 10 files changed, 312 insertions(+), 7 deletions(-) create mode 100644 core/src/main/scala/kafka/server/ForwardingManagerMetrics.scala create mode 100644 core/src/test/scala/unit/kafka/server/ForwardingManagerMetricsTest.scala diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 113bd87989..1143abcd42 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -234,7 +234,7 @@ class BrokerServer( retryTimeoutMs = 60000 ) clientToControllerChannelManager.start() - forwardingManager = new ForwardingManagerImpl(clientToControllerChannelManager) + forwardingManager = new ForwardingManagerImpl(clientToControllerChannelManager, metrics) clientMetricsManager = new ClientMetricsManager(clientMetricsReceiverPlugin, config.clientTelemetryMaxBytes, time, metrics) val apiVersionManager = ApiVersionManager( @@ -675,6 +675,9 @@ class BrokerServer( if (alterPartitionManager != null) CoreUtils.swallow(alterPartitionManager.shutdown(), this) + if (forwardingManager != null) + CoreUtils.swallow(forwardingManager.close(), this) + if (clientToControllerChannelManager != null) CoreUtils.swallow(clientToControllerChannelManager.shutdown(), this) diff --git a/core/src/main/scala/kafka/server/ForwardingManager.scala b/core/src/main/scala/kafka/server/ForwardingManager.scala index 7d7b6eba02..85f48f4b2d 100644 --- a/core/src/main/scala/kafka/server/ForwardingManager.scala +++ b/core/src/main/scala/kafka/server/ForwardingManager.scala @@ -22,13 +22,17 @@ import kafka.network.RequestChannel import kafka.utils.Logging import org.apache.kafka.clients.{ClientResponse, NodeApiVersions} import org.apache.kafka.common.errors.TimeoutException +import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, EnvelopeRequest, EnvelopeResponse, RequestContext, RequestHeader} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} +import java.util.concurrent.TimeUnit import scala.compat.java8.OptionConverters._ trait ForwardingManager { + def close(): Unit + def forwardRequest( originalRequest: RequestChannel.Request, responseCallback: Option[AbstractResponse] => Unit @@ -37,6 +41,7 @@ trait ForwardingManager { buffer.flip() forwardRequest(originalRequest.context, buffer, + originalRequest.startTimeNanos, originalRequest.body[AbstractRequest], () => originalRequest.toString, responseCallback) @@ -50,6 +55,7 @@ trait ForwardingManager { val buffer = newRequestBody.serializeWithHeader(originalRequest.header) forwardRequest(originalRequest.context, buffer, + originalRequest.startTimeNanos, newRequestBody, () => originalRequest.toString, responseCallback) @@ -73,6 +79,7 @@ trait ForwardingManager { def forwardRequest( requestContext: RequestContext, requestBufferCopy: ByteBuffer, + requestCreationNs: Long, requestBody: AbstractRequest, requestToString: () => String, responseCallback: Option[AbstractResponse] => Unit @@ -83,9 +90,10 @@ trait ForwardingManager { object ForwardingManager { def apply( - channelManager: NodeToControllerChannelManager + channelManager: NodeToControllerChannelManager, + metrics: Metrics ): ForwardingManager = { - new ForwardingManagerImpl(channelManager) + new ForwardingManagerImpl(channelManager, metrics) } private[server] def buildEnvelopeRequest(context: RequestContext, @@ -104,21 +112,30 @@ object ForwardingManager { } class ForwardingManagerImpl( - channelManager: NodeToControllerChannelManager -) extends ForwardingManager with Logging { + channelManager: NodeToControllerChannelManager, + metrics: Metrics +) extends ForwardingManager with AutoCloseable with Logging { + + val forwardingManagerMetrics: ForwardingManagerMetrics = ForwardingManagerMetrics(metrics, channelManager.getTimeoutMs) override def forwardRequest( requestContext: RequestContext, requestBufferCopy: ByteBuffer, + requestCreationNs: Long, requestBody: AbstractRequest, requestToString: () => String, responseCallback: Option[AbstractResponse] => Unit ): Unit = { val envelopeRequest = ForwardingManager.buildEnvelopeRequest(requestContext, requestBufferCopy) + val requestCreationTimeMs = TimeUnit.NANOSECONDS.toMillis(requestCreationNs) class ForwardingResponseHandler extends ControllerRequestCompletionHandler { override def onComplete(clientResponse: ClientResponse): Unit = { + forwardingManagerMetrics.queueLength.getAndDecrement() + forwardingManagerMetrics.remoteTimeMsHist.record(clientResponse.requestLatencyMs()) + forwardingManagerMetrics.queueTimeMsHist.record(clientResponse.receivedTimeMs() - clientResponse.requestLatencyMs() - requestCreationTimeMs) + if (clientResponse.versionMismatch != null) { debug(s"Returning `UNKNOWN_SERVER_ERROR` in response to ${requestToString()} " + s"due to unexpected version error", clientResponse.versionMismatch) @@ -156,14 +173,20 @@ class ForwardingManagerImpl( override def onTimeout(): Unit = { debug(s"Forwarding of the request ${requestToString()} failed due to timeout exception") + forwardingManagerMetrics.queueLength.getAndDecrement() + forwardingManagerMetrics.queueTimeMsHist.record(channelManager.getTimeoutMs) val response = requestBody.getErrorResponse(new TimeoutException()) responseCallback(Option(response)) } } + forwardingManagerMetrics.queueLength.getAndIncrement() channelManager.sendRequest(envelopeRequest, new ForwardingResponseHandler) } + override def close(): Unit = + forwardingManagerMetrics.close() + override def controllerApiVersions: Option[NodeApiVersions] = channelManager.controllerApiVersions.asScala diff --git a/core/src/main/scala/kafka/server/ForwardingManagerMetrics.scala b/core/src/main/scala/kafka/server/ForwardingManagerMetrics.scala new file mode 100644 index 0000000000..a846f8c495 --- /dev/null +++ b/core/src/main/scala/kafka/server/ForwardingManagerMetrics.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.{Gauge, MetricConfig, Metrics} +import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing +import org.apache.kafka.common.metrics.stats.{Percentile, Percentiles} + +import java.util.concurrent.atomic.AtomicInteger + +final class ForwardingManagerMetrics private ( + metrics: Metrics, + timeoutMs: Long, +) extends AutoCloseable { + import ForwardingManagerMetrics._ + + /** + * A histogram describing the amount of time in milliseconds each admin request spends in the broker's forwarding manager queue, waiting to be sent to the controller. + * This does not include the time that the request spends waiting for a response from the controller. + */ + val queueTimeMsHist: LatencyHistogram = new LatencyHistogram(metrics, queueTimeMsName, metricGroupName, timeoutMs) + + /** + * A histogram describing the amount of time in milliseconds each request sent by the ForwardingManager spends waiting for a response. + * This does not include the time spent in the queue. + */ + val remoteTimeMsHist: LatencyHistogram = new LatencyHistogram(metrics, remoteTimeMsName, metricGroupName, timeoutMs) + + val queueLengthName: MetricName = metrics.metricName( + "QueueLength", + metricGroupName, + "The current number of RPCs that are waiting in the broker's forwarding manager queue, waiting to be sent to the controller." + ) + val queueLength: AtomicInteger = new AtomicInteger(0) + metrics.addMetric(queueLengthName, new FuncGauge(_ => queueLength.get())) + + override def close(): Unit = { + queueTimeMsHist.close() + remoteTimeMsHist.close() + metrics.removeMetric(queueLengthName) + } +} + +object ForwardingManagerMetrics { + + val metricGroupName = "ForwardingManager" + val queueTimeMsName = "QueueTimeMs" + val remoteTimeMsName = "RemoteTimeMs" + + final class LatencyHistogram ( + metrics: Metrics, + name: String, + group: String, + maxLatency: Long + ) extends AutoCloseable { + private val sensor = metrics.sensor(name) + val latencyP99Name: MetricName = metrics.metricName(s"$name.p99", group) + val latencyP999Name: MetricName = metrics.metricName(s"$name.p999", group) + + sensor.add(new Percentiles( + 4000, + maxLatency, + BucketSizing.CONSTANT, + new Percentile(latencyP99Name, 99), + new Percentile(latencyP999Name, 99.9) + )) + + override def close(): Unit = { + metrics.removeSensor(name) + metrics.removeMetric(latencyP99Name) + metrics.removeMetric(latencyP999Name) + } + + def record(latencyMs: Long): Unit = sensor.record(latencyMs) + } + + private final class FuncGauge[T](func: Long => T) extends Gauge[T] { + override def value(config: MetricConfig, now: Long): T = { + func(now) + } + } + + def apply(metrics: Metrics, timeoutMs: Long): ForwardingManagerMetrics = new ForwardingManagerMetrics(metrics, timeoutMs) +} diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 4b1dec3212..b1883d2a4a 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -364,7 +364,7 @@ class KafkaServer( /* start forwarding manager */ var autoTopicCreationChannel = Option.empty[NodeToControllerChannelManager] if (enableForwarding) { - this.forwardingManager = Some(ForwardingManager(clientToControllerChannelManager)) + this.forwardingManager = Some(ForwardingManager(clientToControllerChannelManager, metrics)) autoTopicCreationChannel = Some(clientToControllerChannelManager) } @@ -1032,6 +1032,9 @@ class KafkaServer( if (alterPartitionManager != null) CoreUtils.swallow(alterPartitionManager.shutdown(), this) + if (forwardingManager.isDefined) + CoreUtils.swallow(forwardingManager.get.close(), this) + if (clientToControllerChannelManager != null) CoreUtils.swallow(clientToControllerChannelManager.shutdown(), this) diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index 6ce6e9e0a4..a70b5545fe 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -233,6 +233,8 @@ class NodeToControllerChannelManagerImpl( Option(apiVersions.get(activeController.idString)) }.asJava } + + def getTimeoutMs: Long = retryTimeoutMs } case class NodeToControllerQueueItem( diff --git a/core/src/test/scala/unit/kafka/server/ForwardingManagerMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ForwardingManagerMetricsTest.scala new file mode 100644 index 0000000000..2c10decb3e --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ForwardingManagerMetricsTest.scala @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package unit.kafka.server + +import kafka.server.ForwardingManagerMetrics +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.Metrics +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Test + +import java.util.Collections +import scala.jdk.CollectionConverters._ + +final class ForwardingManagerMetricsTest { + @Test + def testMetricsNames(): Unit = { + val metrics = new Metrics() + val expectedGroup = "ForwardingManager" + + val expectedMetrics = Set( + new MetricName("QueueTimeMs.p99", expectedGroup, "", Collections.emptyMap()), + new MetricName("QueueTimeMs.p999", expectedGroup, "", Collections.emptyMap()), + new MetricName("QueueLength", expectedGroup, "", Collections.emptyMap()), + new MetricName("RemoteTimeMs.p99", expectedGroup, "", Collections.emptyMap()), + new MetricName("RemoteTimeMs.p999", expectedGroup, "", Collections.emptyMap()) + ) + + var metricsMap = metrics.metrics().asScala.filter { case (name, _) => name.group == expectedGroup } + assertEquals(0, metricsMap.size) + + ForwardingManagerMetrics(metrics, 1000) + metricsMap = metrics.metrics().asScala.filter { case (name, _) => name.group == expectedGroup } + assertEquals(metricsMap.size, expectedMetrics.size) + metricsMap.foreach { case (name, _) => + assertTrue(expectedMetrics.contains(name)) + } + } + + @Test + def testQueueTimeMs(): Unit = { + val metrics = new Metrics() + + val forwardingManagerMetrics = ForwardingManagerMetrics(metrics, 1000) + val queueTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist.latencyP99Name) + val queueTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist.latencyP999Name) + assertEquals(Double.NaN, queueTimeMsP99.metricValue.asInstanceOf[Double]) + assertEquals(Double.NaN, queueTimeMsP999.metricValue.asInstanceOf[Double]) + for(i <- 0 to 999) { + forwardingManagerMetrics.queueTimeMsHist.record(i) + } + assertEquals(990.0, queueTimeMsP99.metricValue.asInstanceOf[Double]) + assertEquals(999.0, queueTimeMsP999.metricValue.asInstanceOf[Double]) + } + + @Test + def testQueueLength(): Unit = { + val metrics = new Metrics() + + val forwardingManagerMetrics = ForwardingManagerMetrics(metrics, 1000) + val queueLength = metrics.metrics().get(forwardingManagerMetrics.queueLengthName) + assertEquals(0, queueLength.metricValue.asInstanceOf[Int]) + forwardingManagerMetrics.queueLength.getAndIncrement() + assertEquals(1, queueLength.metricValue.asInstanceOf[Int]) + } + + @Test + def testRemoteTimeMs(): Unit = { + val metrics = new Metrics() + + val forwardingManagerMetrics = ForwardingManagerMetrics(metrics, 1000) + val remoteTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.remoteTimeMsHist.latencyP99Name) + val remoteTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.remoteTimeMsHist.latencyP999Name) + assertEquals(Double.NaN, remoteTimeMsP99.metricValue.asInstanceOf[Double]) + assertEquals(Double.NaN, remoteTimeMsP999.metricValue.asInstanceOf[Double]) + for (i <- 0 to 999) { + forwardingManagerMetrics.remoteTimeMsHist.record(i) + } + assertEquals(990.0, remoteTimeMsP99.metricValue.asInstanceOf[Double]) + assertEquals(999.0, remoteTimeMsP999.metricValue.asInstanceOf[Double]) + } + + @Test + def testTimeoutMs(): Unit = { + val metrics = new Metrics() + val timeoutMs = 500 + val forwardingManagerMetrics = ForwardingManagerMetrics(metrics, timeoutMs) + val queueTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist.latencyP99Name) + val queueTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist.latencyP999Name) + assertEquals(Double.NaN, queueTimeMsP99.metricValue.asInstanceOf[Double]) + assertEquals(Double.NaN, queueTimeMsP999.metricValue.asInstanceOf[Double]) + for(i <- 0 to 99) { + forwardingManagerMetrics.queueTimeMsHist.record(i) + } + forwardingManagerMetrics.queueTimeMsHist.record(1000) + + assertEquals(99, queueTimeMsP99.metricValue.asInstanceOf[Double]) + assertEquals(timeoutMs * 0.999, queueTimeMsP999.metricValue.asInstanceOf[Double]) + } +} diff --git a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala index 2214d948e4..971eee41d7 100644 --- a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala @@ -29,6 +29,7 @@ import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.{AlterConfigsResponseData, ApiVersionsResponseData} +import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ClientInformation, ListenerName} import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AlterConfigsRequest, AlterConfigsResponse, EnvelopeRequest, EnvelopeResponse, RequestContext, RequestHeader, RequestTestUtils} @@ -47,8 +48,12 @@ class ForwardingManagerTest { private val controllerNodeProvider = Mockito.mock(classOf[ControllerNodeProvider]) private val brokerToController = new MockNodeToControllerChannelManager( client, time, controllerNodeProvider, controllerApiVersions) - private val forwardingManager = new ForwardingManagerImpl(brokerToController) + private val metrics = new Metrics() + private val forwardingManager = new ForwardingManagerImpl(brokerToController, metrics) private val principalBuilder = new DefaultKafkaPrincipalBuilder(null, null) + private val queueTimeMsP999 = metrics.metrics().get(forwardingManager.forwardingManagerMetrics.queueTimeMsHist.latencyP999Name) + private val queueLength = metrics.metrics().get(forwardingManager.forwardingManagerMetrics.queueLengthName) + private val remoteTimeMsP999 = metrics.metrics().get(forwardingManager.forwardingManagerMetrics.remoteTimeMsHist.latencyP999Name) private def controllerApiVersions: NodeApiVersions = { // The Envelope API is not yet included in the standard set of APIs @@ -202,6 +207,52 @@ class ForwardingManagerTest { assertEquals(Map(Errors.UNKNOWN_SERVER_ERROR -> 1).asJava, alterConfigResponse.errorCounts) } + @Test + def testForwardingManagerMetricsOnComplete(): Unit = { + val requestCorrelationId = 27 + val clientPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "client") + val (requestHeader, requestBuffer) = buildRequest(testAlterConfigRequest, requestCorrelationId) + val request = buildRequest(requestHeader, requestBuffer, clientPrincipal) + + val responseBody = new AlterConfigsResponse(new AlterConfigsResponseData()) + val responseBuffer = RequestTestUtils.serializeResponseWithHeader(responseBody, + requestHeader.apiVersion, requestCorrelationId) + + Mockito.when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo) + val isEnvelopeRequest: RequestMatcher = request => request.isInstanceOf[EnvelopeRequest] + client.prepareResponse(isEnvelopeRequest, new EnvelopeResponse(responseBuffer, Errors.UNSUPPORTED_VERSION)) + + val responseOpt = new AtomicReference[Option[AbstractResponse]]() + forwardingManager.forwardRequest(request, responseOpt.set) + assertEquals(1, queueLength.metricValue.asInstanceOf[Int]) + + brokerToController.poll() + client.poll(10000, time.milliseconds()) + assertEquals(0, queueLength.metricValue.asInstanceOf[Int]) + assertNotEquals(Double.NaN, queueTimeMsP999.metricValue.asInstanceOf[Double]) + assertNotEquals(Double.NaN, remoteTimeMsP999.metricValue.asInstanceOf[Double]) + } + + @Test + def testForwardingManagerMetricsOnTimeout(): Unit = { + val requestCorrelationId = 27 + val clientPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "client") + val (requestHeader, requestBuffer) = buildRequest(testAlterConfigRequest, requestCorrelationId) + val request = buildRequest(requestHeader, requestBuffer, clientPrincipal) + + Mockito.when(controllerNodeProvider.getControllerInfo()).thenReturn(controllerInfo) + + val response = new AtomicReference[AbstractResponse]() + forwardingManager.forwardRequest(request, res => res.foreach(response.set)) + assertEquals(1, queueLength.metricValue.asInstanceOf[Int]) + + time.sleep(brokerToController.retryTimeoutMs) + brokerToController.poll() + assertEquals(0, queueLength.metricValue.asInstanceOf[Int]) + assertEquals(brokerToController.retryTimeoutMs * 0.999, queueTimeMsP999.metricValue.asInstanceOf[Double]) + assertEquals(Double.NaN, remoteTimeMsP999.metricValue.asInstanceOf[Double]) + } + private def buildRequest( body: AbstractRequest, correlationId: Int diff --git a/core/src/test/scala/unit/kafka/server/MockNodeToControllerChannelManager.scala b/core/src/test/scala/unit/kafka/server/MockNodeToControllerChannelManager.scala index c3265d6be7..58f4c920cb 100644 --- a/core/src/test/scala/unit/kafka/server/MockNodeToControllerChannelManager.scala +++ b/core/src/test/scala/unit/kafka/server/MockNodeToControllerChannelManager.scala @@ -55,6 +55,8 @@ class MockNodeToControllerChannelManager( Optional.of(controllerApiVersions) } + override def getTimeoutMs: Long = retryTimeoutMs.toLong + private[server] def handleResponse(request: NodeToControllerQueueItem)(response: ClientResponse): Unit = { if (response.authenticationException != null || response.versionMismatch != null) { request.callback.onComplete(response) diff --git a/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManager.java b/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManager.java index cb855fdad1..032340a20e 100644 --- a/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManager.java +++ b/server/src/main/java/org/apache/kafka/server/NodeToControllerChannelManager.java @@ -34,4 +34,6 @@ void sendRequest( AbstractRequest.Builder request, ControllerRequestCompletionHandler callback ); + + long getTimeoutMs(); } diff --git a/server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java b/server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java index f819e37f19..49e3ef5f37 100644 --- a/server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java +++ b/server/src/test/java/org/apache/kafka/server/AssignmentsManagerTest.java @@ -170,6 +170,11 @@ public void sendRequest( } } + @Override + public long getTimeoutMs() { + return 0L; + } + void completeCallback(Function> completionist) throws InterruptedException { Map.Entry entry = callbacks.take(); Optional clientResponse = completionist.apply(entry.getKey()); From a4728f566f90150916126037dc8748ae74be4f4d Mon Sep 17 00:00:00 2001 From: Dmitry Werner Date: Tue, 30 Jul 2024 19:04:16 +0500 Subject: [PATCH 068/123] KAFKA-15746: KRaft support in ControllerMutationQuotaTest (#16620) Reviewers: Mickael Maison --- .../server/ControllerMutationQuotaTest.scala | 89 +++++++++++++------ 1 file changed, 61 insertions(+), 28 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala index 86a32d126c..3c06b33640 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala @@ -26,6 +26,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.DeleteTopicsRequestData import org.apache.kafka.common.metrics.KafkaMetric +import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.quota.ClientQuotaAlteration @@ -47,8 +48,11 @@ import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertTrue import org.junit.jupiter.api.Assertions.fail -import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{BeforeEach, TestInfo} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource +import scala.collection.Seq import scala.jdk.CollectionConverters._ object ControllerMutationQuotaTest { @@ -106,6 +110,12 @@ class ControllerMutationQuotaTest extends BaseRequestTest { properties.put(QuotaConfigs.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS_CONFIG, ControllerQuotaWindowSizeSeconds.toString) } + override def kraftControllerConfigs(): Seq[Properties] = { + val props = super.kraftControllerConfigs() + props.head.setProperty(QuotaConfigs.NUM_CONTROLLER_QUOTA_SAMPLES_CONFIG, ControllerQuotaSamples.toString) + props + } + @BeforeEach override def setUp(testInfo: TestInfo): Unit = { super.setUp(testInfo) @@ -115,8 +125,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { waitUserQuota(ThrottledPrincipal.getName, ControllerMutationRate) } - @Test - def testSetUnsetQuota(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testSetUnsetQuota(quorum: String): Unit = { val rate = 1.5 val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "User") // Default Value @@ -131,8 +142,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { waitUserQuota(principal.getName, Long.MaxValue) } - @Test - def testQuotaMetric(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testQuotaMetric(quorum: String): Unit = { asPrincipal(ThrottledPrincipal) { // Metric is lazily created assertTrue(quotaMetric(principal.getName).isEmpty) @@ -153,8 +165,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testStrictCreateTopicsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testStrictCreateTopicsRequest(quorum: String): Unit = { asPrincipal(ThrottledPrincipal) { // Create two topics worth of 30 partitions each. As we use a strict quota, we // expect one to be created and one to be rejected. @@ -176,8 +189,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testPermissiveCreateTopicsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testPermissiveCreateTopicsRequest(quorum: String): Unit = { asPrincipal(ThrottledPrincipal) { // Create two topics worth of 30 partitions each. As we use a permissive quota, we // expect both topics to be created. @@ -189,8 +203,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testUnboundedCreateTopicsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testUnboundedCreateTopicsRequest(quorum: String): Unit = { asPrincipal(UnboundedPrincipal) { // Create two topics worth of 30 partitions each. As we use an user without quota, we // expect both topics to be created. The throttle time should be equal to 0. @@ -200,8 +215,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testStrictDeleteTopicsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testStrictDeleteTopicsRequest(quorum: String): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion) } @@ -227,8 +243,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testPermissiveDeleteTopicsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testPermissiveDeleteTopicsRequest(quorum: String): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion) } @@ -244,8 +261,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testUnboundedDeleteTopicsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testUnboundedDeleteTopicsRequest(quorum: String): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion) @@ -257,8 +275,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testStrictCreatePartitionsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testStrictCreatePartitionsRequest(quorum: String): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion) } @@ -284,8 +303,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testPermissiveCreatePartitionsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testPermissiveCreatePartitionsRequest(quorum: String): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion) } @@ -301,8 +321,9 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @Test - def testUnboundedCreatePartitionsRequest(): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testUnboundedCreatePartitionsRequest(quorum: String): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion) @@ -364,23 +385,31 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } private def waitUserQuota(user: String, expectedQuota: Double): Unit = { - val quotaManager = servers.head.quotaManagers.controllerMutation + val quotaManager = brokers.head.quotaManagers.controllerMutation + val controllerQuotaManager = + if (isKRaftTest()) Option(controllerServers.head.quotaManagers.controllerMutation) + else Option.empty var actualQuota = Double.MinValue TestUtils.waitUntilTrue(() => { actualQuota = quotaManager.quota(user, "").bound() - expectedQuota == actualQuota + if (controllerQuotaManager.isDefined) + expectedQuota == actualQuota && expectedQuota == controllerQuotaManager.get.quota(user, "").bound() + else + expectedQuota == actualQuota }, s"Quota of $user is not $expectedQuota but $actualQuota") } private def quotaMetric(user: String): Option[KafkaMetric] = { - val metrics = servers.head.metrics + val metrics = + if (isKRaftTest()) controllerServers.head.metrics + else brokers.head.metrics val metricName = metrics.metricName( "tokens", QuotaType.ControllerMutation.toString, "Tracking remaining tokens in the token bucket per user/client-id", Map(DefaultTags.User -> user, DefaultTags.ClientId -> "").asJava) - Option(servers.head.metrics.metric(metricName)) + Option(metrics.metric(metricName)) } private def waitQuotaMetric(user: String, expectedQuota: Double): Unit = { @@ -416,6 +445,10 @@ class ControllerMutationQuotaTest extends BaseRequestTest { private def sendAlterClientQuotasRequest(entries: Iterable[ClientQuotaAlteration]): AlterClientQuotasResponse = { val request = new AlterClientQuotasRequest.Builder(entries.asJavaCollection, false).build() - connectAndReceive[AlterClientQuotasResponse](request, destination = controllerSocketServer) + connectAndReceive[AlterClientQuotasResponse]( + request, + destination = controllerSocketServer, + if (isKRaftTest()) ListenerName.normalised("CONTROLLER") else listenerName + ) } } From 7356328f53088e1eaac93c6722033dd6efb9a2c3 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Wed, 28 Aug 2024 14:13:20 -0700 Subject: [PATCH 069/123] KAFKA-12670: Support configuring unclean leader election in KRaft (#16866) Previously in KRaft mode, we could request an unclean leader election for a specific topic using the electLeaders API. This PR adds an additional way to trigger unclean leader election when in KRaft mode via the static controller configuration and various dynamic configurations. In order to support all possible configuration methods, we have to do a multi-step configuration lookup process: 1. check the dynamic topic configuration for the topic. 2. check the dynamic node configuration. 3. check the dynamic cluster configuration. 4. check the controller's static configuration. Fortunately, we already have the logic to do this multi-step lookup in KafkaConfigSchema.java. This PR reuses that logic. It also makes setting a configuration schema in ConfigurationControlManager mandatory. Previously, it was optional for unit tests. Of course, the dynamic configuration can change over time, or the active controller can change to a different one with a different configuration. These changes can make unclean leader elections possible for partitions that they were not previously possible for. In order to address this, I added a periodic background task which scans leaderless partitions to check if they are eligible for an unclean leader election. Finally, this PR adds the UncleanLeaderElectionsPerSec metric. Co-authored-by: Luke Chen showuon@gmail.com Reviewers: Igor Soarez , Luke Chen --- .../scala/kafka/server/ControllerServer.scala | 2 +- .../main/scala/kafka/server/KafkaConfig.scala | 1 + .../AdminFenceProducersIntegrationTest.scala | 2 +- .../kafka/admin/RemoteTopicCrudTest.scala | 2 +- .../AbstractAuthorizerIntegrationTest.scala | 4 +- ...minClientWithPoliciesIntegrationTest.scala | 2 +- .../kafka/api/BaseAdminIntegrationTest.scala | 2 +- .../api/GroupAuthorizerIntegrationTest.scala | 4 +- .../kafka/api/IntegrationTestHarness.scala | 2 +- .../kafka/api/TransactionsTest.scala | 2 +- .../kafka/server/QuorumTestHarness.scala | 4 +- .../UncleanLeaderElectionTest.scala | 98 +++++++++---- ...mCredentialsRequestNotAuthorizedTest.scala | 5 +- ...AlterUserScramCredentialsRequestTest.scala | 4 +- .../server/ControllerMutationQuotaTest.scala | 4 +- .../CreateTopicsRequestWithPolicyTest.scala | 3 +- ...opicsRequestWithDeletionDisabledTest.scala | 5 +- ...istOffsetsRequestWithRemoteStoreTest.scala | 3 +- .../unit/kafka/server/RequestQuotaTest.scala | 2 +- .../ConfigurationControlManager.java | 50 +++++-- .../kafka/controller/QuorumController.java | 92 +++++++++++-- .../controller/ReplicationControlManager.java | 87 ++++++++---- .../metrics/ControllerMetadataMetrics.java | 15 +- .../metrics/ControllerMetricsChanges.java | 10 ++ .../kafka/metadata/KafkaConfigSchema.java | 28 +++- .../ConfigurationControlManagerTest.java | 4 +- .../controller/QuorumControllerTest.java | 40 ------ .../controller/QuorumControllerTestEnv.java | 2 + .../ReplicationControlManagerTest.java | 130 ++++++++++++++++-- .../ControllerMetadataMetricsTest.java | 23 +++- .../kafka/metadata/FakeKafkaConfigSchema.java | 60 ++++++++ .../server/config/ReplicationConfigs.java | 7 + .../storage/TieredStorageTestHarness.java | 2 +- 33 files changed, 530 insertions(+), 171 deletions(-) create mode 100644 metadata/src/test/java/org/apache/kafka/metadata/FakeKafkaConfigSchema.java diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 1008fe2af7..0e4321cffb 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -247,7 +247,6 @@ class ControllerServer( setQuorumFeatures(quorumFeatures). setDefaultReplicationFactor(config.defaultReplicationFactor.toShort). setDefaultNumPartitions(config.numPartitions.intValue()). - setDefaultMinIsr(config.minInSyncReplicas.intValue()). setSessionTimeoutNs(TimeUnit.NANOSECONDS.convert(config.brokerSessionTimeoutMs.longValue(), TimeUnit.MILLISECONDS)). setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs). @@ -266,6 +265,7 @@ class ControllerServer( setDelegationTokenMaxLifeMs(config.delegationTokenMaxLifeMs). setDelegationTokenExpiryTimeMs(config.delegationTokenExpiryTimeMs). setDelegationTokenExpiryCheckIntervalMs(config.delegationTokenExpiryCheckIntervalMs). + setUncleanLeaderElectionCheckIntervalMs(config.uncleanLeaderElectionCheckIntervalMs). setEligibleLeaderReplicasEnabled(config.elrEnabled) } controller = controllerBuilder.build() diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index fb71ad84c0..1edb310016 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -532,6 +532,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) val autoLeaderRebalanceEnable = getBoolean(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG) val leaderImbalancePerBrokerPercentage = getInt(ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG) val leaderImbalanceCheckIntervalSeconds: Long = getLong(ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG) + val uncleanLeaderElectionCheckIntervalMs: Long = getLong(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_INTERVAL_MS_CONFIG) def uncleanLeaderElectionEnable: java.lang.Boolean = getBoolean(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG) // We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0` diff --git a/core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala index cbfeee5098..8a611c592f 100644 --- a/core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/admin/AdminFenceProducersIntegrationTest.scala @@ -71,7 +71,7 @@ class AdminFenceProducersIntegrationTest extends IntegrationTestHarness { props.foreach(p => p.putAll(overridingProps())) } - override protected def kraftControllerConfigs(): Seq[Properties] = { + override protected def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { Seq(overridingProps()) } diff --git a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala index 2983ba0e30..1880024ac8 100644 --- a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala +++ b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala @@ -56,7 +56,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { props.foreach(p => p.putAll(overrideProps())) } - override protected def kraftControllerConfigs(): Seq[Properties] = { + override protected def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { Seq(overrideProps()) } diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala index e45689b67f..5e181b9e82 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala @@ -98,8 +98,8 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { addNodeProperties(properties) } - override def kraftControllerConfigs(): collection.Seq[Properties] = { - val controllerConfigs = super.kraftControllerConfigs() + override def kraftControllerConfigs(testInfo: TestInfo): collection.Seq[Properties] = { + val controllerConfigs = super.kraftControllerConfigs(testInfo) controllerConfigs.foreach(addNodeProperties) controllerConfigs } diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index 3e99a0316d..3b2c8d9739 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -70,7 +70,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with configs.map(KafkaConfig.fromProps) } - override def kraftControllerConfigs(): Seq[Properties] = { + override def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { val props = new Properties() overrideNodeConfigs(props) Seq(props) diff --git a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala index 436673806a..ab47edc8cd 100644 --- a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala @@ -218,7 +218,7 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg } } - override def kraftControllerConfigs(): Seq[Properties] = { + override def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { val controllerConfig = new Properties() val controllerConfigs = Seq(controllerConfig) modifyConfigs(controllerConfigs) diff --git a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala index 02c8399fa0..e053966c07 100644 --- a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala @@ -70,8 +70,8 @@ class GroupAuthorizerIntegrationTest extends BaseRequestTest { def brokerPrincipal: KafkaPrincipal = BrokerPrincipal def clientPrincipal: KafkaPrincipal = ClientPrincipal - override def kraftControllerConfigs(): collection.Seq[Properties] = { - val controllerConfigs = super.kraftControllerConfigs() + override def kraftControllerConfigs(testInfo: TestInfo): collection.Seq[Properties] = { + val controllerConfigs = super.kraftControllerConfigs(testInfo) controllerConfigs.foreach(addNodeProperties) controllerConfigs } diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index bbdc01e72a..1c41cbc4fd 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -82,7 +82,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { cfgs.map(KafkaConfig.fromProps) } - override protected def kraftControllerConfigs(): Seq[Properties] = { + override protected def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { Seq(controllerConfig) } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 09462d361f..9c8b93012a 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -78,7 +78,7 @@ class TransactionsTest extends IntegrationTestHarness { props.foreach(p => p.putAll(overridingProps())) } - override protected def kraftControllerConfigs(): Seq[Properties] = { + override protected def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { Seq(overridingProps()) } diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 188b684c51..e1f7415ca3 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -174,7 +174,7 @@ abstract class QuorumTestHarness extends Logging { */ protected val controllerListenerSecurityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT - protected def kraftControllerConfigs(): Seq[Properties] = { + protected def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { Seq(new Properties()) } @@ -315,7 +315,7 @@ abstract class QuorumTestHarness extends Logging { } protected def newKRaftQuorum(overridingProps: Properties): KRaftQuorumImplementation = { - val propsList = kraftControllerConfigs() + val propsList = kraftControllerConfigs(testInfo) if (propsList.size != 1) { throw new RuntimeException("Only one KRaft controller is supported for now.") } diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index 828f6eb111..2a23023c89 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ExecutionException import scala.util.Random import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} -import kafka.server.{KafkaBroker, KafkaConfig, QuorumTestHarness} +import kafka.server.{KafkaBroker, KafkaConfig, MetadataCache, QuorumTestHarness} import kafka.utils.{CoreUtils, TestUtils} import kafka.utils.TestUtils._ import org.apache.kafka.common.TopicPartition @@ -40,6 +40,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource import com.yammer.metrics.core.Meter +import org.apache.kafka.metadata.LeaderConstants class UncleanLeaderElectionTest extends QuorumTestHarness { val brokerId1 = 0 @@ -97,6 +98,15 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { super.tearDown() } + override def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { + val properties = new Properties() + if (testInfo.getTestMethod.get().getName.contains("testUncleanLeaderElectionEnabled")) { + properties.setProperty("unclean.leader.election.enable", "true") + } + properties.setProperty("unclean.leader.election.interval.ms", "10") + Seq(properties) + } + private def startBrokers(cluster: Seq[Properties]): Unit = { for (props <- cluster) { val config = KafkaConfig.fromProps(props) @@ -110,7 +120,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { } @ParameterizedTest - @ValueSource(strings = Array("zk")) + @ValueSource(strings = Array("zk", "kraft")) def testUncleanLeaderElectionEnabled(quorum: String): Unit = { // enable unclean leader election configProps1.put("unclean.leader.election.enable", "true") @@ -123,8 +133,8 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { } @ParameterizedTest - @ValueSource(strings = Array("zk")) - def testUncleanLeaderElectionDisabled(): Unit = { + @ValueSource(strings = Array("zk", "kraft")) + def testUncleanLeaderElectionDisabled(quorum: String): Unit = { // unclean leader election is disabled by default startBrokers(Seq(configProps1, configProps2)) @@ -135,8 +145,8 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { } @ParameterizedTest - @ValueSource(strings = Array("zk")) - def testUncleanLeaderElectionEnabledByTopicOverride(): Unit = { + @ValueSource(strings = Array("zk", "kraft")) + def testUncleanLeaderElectionEnabledByTopicOverride(quorum: String): Unit = { // disable unclean leader election globally, but enable for our specific test topic configProps1.put("unclean.leader.election.enable", "false") configProps2.put("unclean.leader.election.enable", "false") @@ -151,8 +161,8 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { } @ParameterizedTest - @ValueSource(strings = Array("zk")) - def testUncleanLeaderElectionDisabledByTopicOverride(): Unit = { + @ValueSource(strings = Array("zk", "kraft")) + def testUncleanLeaderElectionDisabledByTopicOverride(quorum: String): Unit = { // enable unclean leader election globally, but disable for our specific test topic configProps1.put("unclean.leader.election.enable", "true") configProps2.put("unclean.leader.election.enable", "true") @@ -167,8 +177,8 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { } @ParameterizedTest - @ValueSource(strings = Array("zk")) - def testUncleanLeaderElectionInvalidTopicOverride(): Unit = { + @ValueSource(strings = Array("zk", "kraft")) + def testUncleanLeaderElectionInvalidTopicOverride(quorum: String): Unit = { startBrokers(Seq(configProps1)) // create topic with an invalid value for unclean leader election @@ -205,7 +215,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { //verify that unclean election metric count is 0 val uncleanLeaderElectionsPerSecGauge = getGauge("UncleanLeaderElectionsPerSec") @volatile var uncleanLeaderElectionsPerSec = uncleanLeaderElectionsPerSecGauge.count() - assert(uncleanLeaderElectionsPerSec == 0) + assertEquals(0, uncleanLeaderElectionsPerSec) // shutdown leader and then restart follower brokers.filter(_.config.brokerId == leaderId).map(shutdownBroker) @@ -215,7 +225,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { // wait until new leader is (uncleanly) elected awaitLeaderChange(brokers, topicPartition, expectedLeaderOpt = Some(followerId), timeout = 30000) uncleanLeaderElectionsPerSec = uncleanLeaderElectionsPerSecGauge.count() - assert(uncleanLeaderElectionsPerSec == 1) + assertEquals(1, uncleanLeaderElectionsPerSec) produceMessage(brokers, topic, "third") @@ -247,17 +257,18 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { //remove any previous unclean election metric val uncleanLeaderElectionsPerSecGauge = getGauge("UncleanLeaderElectionsPerSec") @volatile var uncleanLeaderElectionsPerSec = uncleanLeaderElectionsPerSecGauge.count() - assert(uncleanLeaderElectionsPerSec == 0) + assertEquals(0, uncleanLeaderElectionsPerSec) // shutdown leader and then restart follower brokers.filter(_.config.brokerId == leaderId).map(shutdownBroker) val followerServer = brokers.find(_.config.brokerId == followerId).get followerServer.startup() - // verify that unclean election to non-ISR follower does not occur - awaitLeaderChange(brokers, topicPartition, expectedLeaderOpt = Some(leaderId)) + // verify that unclean election to non-ISR follower does not occur. + // That is, leader should be NO_LEADER(-1) and the ISR should has only old leaderId. + waitForNoLeaderAndIsrHasOldLeaderId(followerServer.replicaManager.metadataCache, leaderId) uncleanLeaderElectionsPerSec = uncleanLeaderElectionsPerSecGauge.count() - assert(uncleanLeaderElectionsPerSec == 0) + assertEquals(0, uncleanLeaderElectionsPerSec) // message production and consumption should both fail while leader is down val e = assertThrows(classOf[ExecutionException], () => produceMessage(brokers, topic, "third", deliveryTimeoutMs = 1000, requestTimeoutMs = 1000)) @@ -312,8 +323,8 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { } @ParameterizedTest - @ValueSource(strings = Array("zk")) - def testTopicUncleanLeaderElectionEnableWithAlterTopicConfigs(): Unit = { + @ValueSource(strings = Array("zk", "kraft")) + def testTopicUncleanLeaderElectionEnableWithAlterTopicConfigs(quorum: String): Unit = { // unclean leader election is disabled by default startBrokers(Seq(configProps1, configProps2)) @@ -330,26 +341,45 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { waitForPartitionMetadata(brokers, topic, partitionId) assertEquals(List("first"), consumeAllMessages(topic, 1)) + // Verify the "unclean.leader.election.enable" won't be triggered even if it is enabled/disabled dynamically, + // because the leader is still alive + val adminClient = createAdminClient() + try { + val newProps = new Properties + newProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true") + alterTopicConfigs(adminClient, topic, newProps).all.get + // leader should not change to followerId + awaitLeaderChange(brokers, topicPartition, expectedLeaderOpt = Some(leaderId), timeout = 10000) + + newProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false") + alterTopicConfigs(adminClient, topic, newProps).all.get + // leader should not change to followerId + awaitLeaderChange(brokers, topicPartition, expectedLeaderOpt = Some(leaderId), timeout = 10000) + } finally { + adminClient.close() + } + // shutdown follower server brokers.filter(broker => broker.config.brokerId == followerId).map(broker => shutdownBroker(broker)) produceMessage(brokers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic, 2)) - //verify that unclean election metric count is 0 + // verify that unclean election metric count is 0 val uncleanLeaderElectionsPerSecGauge = getGauge("UncleanLeaderElectionsPerSec") @volatile var uncleanLeaderElectionsPerSec = uncleanLeaderElectionsPerSecGauge.count() - assert(uncleanLeaderElectionsPerSec == 0) + assertEquals(0, uncleanLeaderElectionsPerSec) // shutdown leader and then restart follower brokers.filter(_.config.brokerId == leaderId).map(shutdownBroker) val followerBroker = brokers.find(_.config.brokerId == followerId).get followerBroker.startup() - // leader should not change - awaitLeaderChange(brokers, topicPartition, expectedLeaderOpt = Some(leaderId), timeout = 30000) + // verify that unclean election to non-ISR follower does not occur. + // That is, leader should be NO_LEADER(-1) and the ISR should has only old leaderId. + waitForNoLeaderAndIsrHasOldLeaderId(followerBroker.replicaManager.metadataCache, leaderId) uncleanLeaderElectionsPerSec = uncleanLeaderElectionsPerSecGauge.count() - assert(uncleanLeaderElectionsPerSec == 0) + assertEquals(0, uncleanLeaderElectionsPerSec) // message production and consumption should both fail while leader is down val e = assertThrows(classOf[ExecutionException], () => produceMessage(brokers, topic, "third", deliveryTimeoutMs = 1000, requestTimeoutMs = 1000)) @@ -358,16 +388,19 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { assertEquals(List.empty[String], consumeAllMessages(topic, 0)) // Enable unclean leader election for topic - val adminClient = createAdminClient() - val newProps = new Properties - newProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true") - alterTopicConfigs(adminClient, topic, newProps).all.get - adminClient.close() + val adminClient2 = createAdminClient() + try { + val newProps = new Properties + newProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true") + alterTopicConfigs(adminClient2, topic, newProps).all.get + } finally { + adminClient2.close() + } // wait until new leader is (uncleanly) elected awaitLeaderChange(brokers, topicPartition, expectedLeaderOpt = Some(followerId), timeout = 30000) uncleanLeaderElectionsPerSec = uncleanLeaderElectionsPerSecGauge.count() - assert(uncleanLeaderElectionsPerSec == 1) + assertEquals(1, uncleanLeaderElectionsPerSec) produceMessage(brokers, topic, "third") @@ -389,4 +422,11 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { config.put(AdminClientConfig.METADATA_MAX_AGE_CONFIG, "10") Admin.create(config) } + + private def waitForNoLeaderAndIsrHasOldLeaderId(metadataCache: MetadataCache, leaderId: Int): Unit = { + waitUntilTrue(() => metadataCache.getPartitionInfo(topic, partitionId).isDefined && + metadataCache.getPartitionInfo(topic, partitionId).get.leader() == LeaderConstants.NO_LEADER && + java.util.Arrays.asList(leaderId).equals(metadataCache.getPartitionInfo(topic, partitionId).get.isr()), + "Timed out waiting for broker metadata cache updates the info for topic partition:" + topicPartition) + } } diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala index 9469961d2d..220770c37e 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse} import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.TestInfo import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -47,8 +48,8 @@ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) } - override def kraftControllerConfigs(): collection.Seq[Properties] = { - val controllerConfigs = super.kraftControllerConfigs() + override def kraftControllerConfigs(testInfo: TestInfo): collection.Seq[Properties] = { + val controllerConfigs = super.kraftControllerConfigs(testInfo) controllerConfigs.head.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestStandardAuthorizer].getName) controllerConfigs.head.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) controllerConfigs diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala index c55c83ccfb..feb914a252 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala @@ -71,8 +71,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { super.setUp(testInfo) } - override def kraftControllerConfigs(): collection.Seq[Properties] = { - val controllerConfigs = super.kraftControllerConfigs() + override def kraftControllerConfigs(testInfo: TestInfo): collection.Seq[Properties] = { + val controllerConfigs = super.kraftControllerConfigs(testInfo) controllerConfigs.head.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestStandardAuthorizer].getName) controllerConfigs.head.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) controllerConfigs diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala index 3c06b33640..7886b851b9 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala @@ -110,8 +110,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { properties.put(QuotaConfigs.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS_CONFIG, ControllerQuotaWindowSizeSeconds.toString) } - override def kraftControllerConfigs(): Seq[Properties] = { - val props = super.kraftControllerConfigs() + override def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { + val props = super.kraftControllerConfigs(testInfo) props.head.setProperty(QuotaConfigs.NUM_CONTROLLER_QUOTA_SAMPLES_CONFIG, ControllerQuotaSamples.toString) props } diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala index 0f03b4069b..9e82d5d3ec 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala @@ -26,6 +26,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.server.config.ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG import org.apache.kafka.server.policy.CreateTopicPolicy import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata +import org.junit.jupiter.api.TestInfo import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -39,7 +40,7 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest properties.put(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[Policy].getName) } - override def kraftControllerConfigs(): Seq[Properties] = { + override def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { val properties = new Properties() properties.put(CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, classOf[Policy].getName) Seq(properties) diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala index 91ea2e8840..5d0ec2ec60 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse} import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.TestInfo import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -32,8 +33,8 @@ class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest { override def brokerCount: Int = 1 - override def kraftControllerConfigs() = { - val props = super.kraftControllerConfigs() + override def kraftControllerConfigs(testInfo: TestInfo) = { + val props = super.kraftControllerConfigs(testInfo) props.head.setProperty(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "false") props } diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestWithRemoteStoreTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestWithRemoteStoreTest.scala index 2f477486c6..970f3f7489 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestWithRemoteStoreTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestWithRemoteStoreTest.scala @@ -18,13 +18,14 @@ package kafka.server import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig} +import org.junit.jupiter.api.TestInfo import java.util.Properties import scala.collection.Seq class ListOffsetsRequestWithRemoteStoreTest extends ListOffsetsRequestTest { - override def kraftControllerConfigs(): Seq[Properties] = { + override def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { val props = new Properties brokerPropertyOverrides(props) Seq(props) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 218c0cf5fb..c7526e808b 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -93,7 +93,7 @@ class RequestQuotaTest extends BaseRequestTest { } } - override def kraftControllerConfigs(): Seq[Properties] = { + override def kraftControllerConfigs(testInfo: TestInfo): Seq[Properties] = { val properties = new Properties() properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName) Seq(properties) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index b10614cc2e..29b334c977 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -50,6 +50,7 @@ import java.util.function.Consumer; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.APPEND; +import static org.apache.kafka.common.config.TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG; import static org.apache.kafka.common.protocol.Errors.INVALID_CONFIG; import static org.apache.kafka.controller.QuorumController.MAX_RECORDS_PER_USER_OP; @@ -70,7 +71,7 @@ public class ConfigurationControlManager { static class Builder { private LogContext logContext = null; private SnapshotRegistry snapshotRegistry = null; - private KafkaConfigSchema configSchema = KafkaConfigSchema.EMPTY; + private KafkaConfigSchema configSchema = null; private Consumer existenceChecker = __ -> { }; private Optional alterConfigPolicy = Optional.empty(); private ConfigurationValidator validator = ConfigurationValidator.NO_OP; @@ -120,6 +121,9 @@ Builder setNodeId(int nodeId) { ConfigurationControlManager build() { if (logContext == null) logContext = new LogContext(); if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext); + if (configSchema == null) { + throw new RuntimeException("You must set the configSchema."); + } return new ConfigurationControlManager( logContext, snapshotRegistry, @@ -440,22 +444,24 @@ Map getConfigs(ConfigResource configResource) { } /** - * Get the config value for the give topic and give config key. + * Get the config value for the given topic and given config key. + * The check order is: + * 1. dynamic topic overridden configs + * 2. dynamic node overridden configs + * 3. dynamic cluster overridden configs + * 4. static configs * If the config value is not found, return null. * * @param topicName The topic name for the config. * @param configKey The key for the config. + * @return the config value for the provided config key in the topic */ - String getTopicConfig(String topicName, String configKey) throws NoSuchElementException { - Map map = configData.get(new ConfigResource(Type.TOPIC, topicName)); - if (map == null || !map.containsKey(configKey)) { - Map effectiveConfigMap = computeEffectiveTopicConfigs(Collections.emptyMap()); - if (!effectiveConfigMap.containsKey(configKey)) { - return null; - } - return effectiveConfigMap.get(configKey).value(); - } - return map.get(configKey); + ConfigEntry getTopicConfig(String topicName, String configKey) throws NoSuchElementException { + ConfigEntry result = configSchema.resolveEffectiveTopicConfig(configKey, + staticConfig, + clusterConfig(), + currentControllerConfig(), currentTopicConfig(topicName)); + return result; } public Map>> describeConfigs( @@ -496,8 +502,19 @@ void deleteTopicConfigs(String name) { configData.remove(new ConfigResource(Type.TOPIC, name)); } - boolean uncleanLeaderElectionEnabledForTopic(String name) { - return false; // TODO: support configuring unclean leader election. + /** + * Check if this topic has "unclean.leader.election.enable" set to true. + * + * @param topicName The topic name for the config. + * @return true if this topic has uncleanLeaderElection enabled + */ + boolean uncleanLeaderElectionEnabledForTopic(String topicName) { + String uncleanLeaderElection = getTopicConfig(topicName, UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG).value(); + if (!uncleanLeaderElection.isEmpty()) { + return Boolean.parseBoolean(uncleanLeaderElection); + } + + return false; } Map computeEffectiveTopicConfigs(Map creationConfigs) { @@ -514,4 +531,9 @@ Map currentControllerConfig() { Map result = configData.get(currentController); return (result == null) ? Collections.emptyMap() : result; } + + Map currentTopicConfig(String topicName) { + Map result = configData.get(new ConfigResource(Type.TOPIC, topicName)); + return (result == null) ? Collections.emptyMap() : result; + } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 3f7bb4df74..64723b86c0 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -206,7 +206,6 @@ public static class Builder { private QuorumFeatures quorumFeatures = null; private short defaultReplicationFactor = 3; private int defaultNumPartitions = 1; - private int defaultMinIsr = 1; private ReplicaPlacer replicaPlacer = new StripedReplicaPlacer(new Random()); private OptionalLong leaderImbalanceCheckIntervalNs = OptionalLong.empty(); private OptionalLong maxIdleIntervalNs = OptionalLong.empty(); @@ -225,6 +224,7 @@ public static class Builder { private long delegationTokenMaxLifeMs; private long delegationTokenExpiryTimeMs; private long delegationTokenExpiryCheckIntervalMs; + private long uncleanLeaderElectionCheckIntervalMs = TimeUnit.MINUTES.toMillis(5); public Builder(int nodeId, String clusterId) { this.nodeId = nodeId; @@ -285,11 +285,6 @@ public Builder setDefaultNumPartitions(int defaultNumPartitions) { return this; } - public Builder setDefaultMinIsr(int defaultMinIsr) { - this.defaultMinIsr = defaultMinIsr; - return this; - } - public Builder setReplicaPlacer(ReplicaPlacer replicaPlacer) { this.replicaPlacer = replicaPlacer; return this; @@ -380,6 +375,11 @@ public Builder setDelegationTokenExpiryCheckIntervalMs(long delegationTokenExpir return this; } + public Builder setUncleanLeaderElectionCheckIntervalMs(long uncleanLeaderElectionCheckIntervalMs) { + this.uncleanLeaderElectionCheckIntervalMs = uncleanLeaderElectionCheckIntervalMs; + return this; + } + public QuorumController build() throws Exception { if (raftClient == null) { throw new IllegalStateException("You must set a raft client."); @@ -419,7 +419,6 @@ public QuorumController build() throws Exception { quorumFeatures, defaultReplicationFactor, defaultNumPartitions, - defaultMinIsr, replicaPlacer, leaderImbalanceCheckIntervalNs, maxIdleIntervalNs, @@ -437,7 +436,8 @@ public QuorumController build() throws Exception { delegationTokenMaxLifeMs, delegationTokenExpiryTimeMs, delegationTokenExpiryCheckIntervalMs, - eligibleLeaderReplicasEnabled + eligibleLeaderReplicasEnabled, + uncleanLeaderElectionCheckIntervalMs ); } catch (Exception e) { Utils.closeQuietly(queue, "event queue"); @@ -850,6 +850,9 @@ public void run() throws Exception { // with leader that is not the preferred leader. maybeScheduleNextBalancePartitionLeaders(); + // Schedule a new unclean leader election if there are partitions that do not have a leader. + maybeScheduleNextElectUncleanLeaders(); + // Remember the latest offset and future if it is not already completed if (!future.isDone()) { if (featureControl.inPreMigrationMode() && flags.contains(RUNS_IN_PREMIGRATION)) { @@ -1251,6 +1254,7 @@ public void processBatchEndOffset(long offset) { // metadata.version and other in-memory state. maybeScheduleNextExpiredDelegationTokenSweep(); maybeScheduleNextBalancePartitionLeaders(); + maybeScheduleNextElectUncleanLeaders(); maybeScheduleNextWriteNoOpRecord(); } } @@ -1271,6 +1275,7 @@ void renounce() { clusterControl.deactivate(); cancelMaybeFenceReplicas(); cancelMaybeBalancePartitionLeaders(); + cancelMaybeNextElectUncleanLeaders(); cancelNextWriteNoOpRecord(); } catch (Throwable e) { fatalFaultHandler.handleFault("exception while renouncing leadership", e); @@ -1345,7 +1350,13 @@ private void maybeScheduleNextBalancePartitionLeaders() { ); ControllerWriteEvent event = new ControllerWriteEvent<>(MAYBE_BALANCE_PARTITION_LEADERS, () -> { + long startTimeNs = time.nanoseconds(); ControllerResult result = replicationControl.maybeBalancePartitionLeaders(); + long endTimeNs = time.nanoseconds(); + long durationNs = endTimeNs - startTimeNs; + log.info("maybeBalancePartitionLeaders: generated {} records in {} microseconds.{}", + result.records().size(), NANOSECONDS.toMicros(durationNs), + result.response() ? " Rescheduling immediately." : ""); // reschedule the operation after the leaderImbalanceCheckIntervalNs interval. // Mark the imbalance event as completed and reschedule if necessary @@ -1383,6 +1394,54 @@ private void cancelMaybeBalancePartitionLeaders() { queue.cancelDeferred(MAYBE_BALANCE_PARTITION_LEADERS); } + private static final String MAYBE_ELECT_UNCLEAN_LEADERS = "maybeElectUncleanLeaders"; + + private void maybeScheduleNextElectUncleanLeaders() { + if (uncleanScheduled != ImbalanceSchedule.SCHEDULED && + replicationControl.areSomePartitionsLeaderless()) { + log.debug( + "Scheduling write event for {} because scheduled ({}), and areSomePartitionsLeaderless ({})", + MAYBE_ELECT_UNCLEAN_LEADERS, + uncleanScheduled, + replicationControl.areSomePartitionsLeaderless() + ); + + ControllerWriteEvent event = new ControllerWriteEvent<>(MAYBE_ELECT_UNCLEAN_LEADERS, () -> { + long startTimeNs = time.nanoseconds(); + ControllerResult result = replicationControl.maybeElectUncleanLeaders(); + long endTimeNs = time.nanoseconds(); + long durationNs = endTimeNs - startTimeNs; + log.info("maybeElectUncleanLeaders: generated {} records in {} microseconds.{}", + result.records().size(), NANOSECONDS.toMicros(durationNs), + result.response() ? " Rescheduling immediately." : ""); + if (result.response()) { + uncleanScheduled = ImbalanceSchedule.IMMEDIATELY; + } else { + uncleanScheduled = ImbalanceSchedule.DEFERRED; + } + return result; + }, EnumSet.of(DOES_NOT_UPDATE_QUEUE_TIME)); + + long delayNs = time.nanoseconds(); + if (uncleanScheduled == ImbalanceSchedule.DEFERRED) { + delayNs += uncleanLeaderElectionCheckIntervalNs; + } else { + // The current implementation of KafkaEventQueue always picks from the deferred collection of operations + // before picking from the non-deferred collection of operations. This can result in some unfairness if + // deferred operation are scheduled for immediate execution. This delays them by a small amount of time. + delayNs += NANOSECONDS.convert(10, TimeUnit.MILLISECONDS); + } + queue.scheduleDeferred(MAYBE_ELECT_UNCLEAN_LEADERS, new EarliestDeadlineFunction(delayNs), event); + uncleanScheduled = ImbalanceSchedule.SCHEDULED; + } + } + + + private void cancelMaybeNextElectUncleanLeaders() { + uncleanScheduled = ImbalanceSchedule.DEFERRED; + queue.cancelDeferred(MAYBE_ELECT_UNCLEAN_LEADERS); + } + private static final String WRITE_NO_OP_RECORD = "writeNoOpRecord"; private void maybeScheduleNextWriteNoOpRecord() { @@ -1742,6 +1801,11 @@ private enum ImbalanceSchedule { */ private ImbalanceSchedule imbalancedScheduled = ImbalanceSchedule.DEFERRED; + /** + * Tracks the scheduling state for unclean leader election operations. + */ + private ImbalanceSchedule uncleanScheduled = ImbalanceSchedule.DEFERRED; + /** * Tracks if the write of the NoOpRecord has been scheduled. */ @@ -1758,6 +1822,11 @@ private enum ImbalanceSchedule { private final boolean eligibleLeaderReplicasEnabled; + /** + * The number of nanoseconds between unclean leader election checks. + */ + private final long uncleanLeaderElectionCheckIntervalNs; + /** * The maximum number of records per batch to allow. */ @@ -1781,7 +1850,6 @@ private QuorumController( QuorumFeatures quorumFeatures, short defaultReplicationFactor, int defaultNumPartitions, - int defaultMinIsr, ReplicaPlacer replicaPlacer, OptionalLong leaderImbalanceCheckIntervalNs, OptionalLong maxIdleIntervalNs, @@ -1799,7 +1867,8 @@ private QuorumController( long delegationTokenMaxLifeMs, long delegationTokenExpiryTimeMs, long delegationTokenExpiryCheckIntervalMs, - boolean eligibleLeaderReplicasEnabled + boolean eligibleLeaderReplicasEnabled, + long uncleanLeaderElectionCheckIntervalMs ) { this.nonFatalFaultHandler = nonFatalFaultHandler; this.fatalFaultHandler = fatalFaultHandler; @@ -1869,7 +1938,6 @@ private QuorumController( setLogContext(logContext). setDefaultReplicationFactor(defaultReplicationFactor). setDefaultNumPartitions(defaultNumPartitions). - setDefaultMinIsr(defaultMinIsr). setEligibleLeaderReplicasEnabled(eligibleLeaderReplicasEnabled). setMaxElectionsPerImbalance(ReplicationControlManager.MAX_ELECTIONS_PER_IMBALANCE). setConfigurationControl(configurationControl). @@ -1905,6 +1973,8 @@ private QuorumController( this.zkMigrationEnabled = zkMigrationEnabled; this.recordRedactor = new RecordRedactor(configSchema); this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; + this.uncleanLeaderElectionCheckIntervalNs = + TimeUnit.MILLISECONDS.toNanos(uncleanLeaderElectionCheckIntervalMs); log.info("Creating new QuorumController with clusterId {}.{}{}", clusterId, zkMigrationEnabled ? " ZK migration mode is enabled." : "", diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index 81abfbeb53..e7a4359fb5 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -159,7 +159,6 @@ static class Builder { private short defaultReplicationFactor = (short) 3; private int defaultNumPartitions = 1; - private int defaultMinIsr = 1; private int maxElectionsPerImbalance = MAX_ELECTIONS_PER_IMBALANCE; private ConfigurationControlManager configurationControl = null; private ClusterControlManager clusterControl = null; @@ -187,11 +186,6 @@ Builder setDefaultNumPartitions(int defaultNumPartitions) { return this; } - Builder setDefaultMinIsr(int defaultMinIsr) { - this.defaultMinIsr = defaultMinIsr; - return this; - } - Builder setEligibleLeaderReplicasEnabled(boolean eligibleLeaderReplicasEnabled) { this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; return this; @@ -237,7 +231,6 @@ ReplicationControlManager build() { logContext, defaultReplicationFactor, defaultNumPartitions, - defaultMinIsr, maxElectionsPerImbalance, eligibleLeaderReplicasEnabled, configurationControl, @@ -311,11 +304,6 @@ static Map translateCreationConfigs(CreatableTopicConfigCollecti */ private final int defaultNumPartitions; - /** - * The default min ISR that is used if a CreateTopics request does not specify one. - */ - private final int defaultMinIsr; - /** * True if eligible leader replicas is enabled. */ @@ -409,7 +397,6 @@ private ReplicationControlManager( LogContext logContext, short defaultReplicationFactor, int defaultNumPartitions, - int defaultMinIsr, int maxElectionsPerImbalance, boolean eligibleLeaderReplicasEnabled, ConfigurationControlManager configurationControl, @@ -421,7 +408,6 @@ private ReplicationControlManager( this.log = logContext.logger(ReplicationControlManager.class); this.defaultReplicationFactor = defaultReplicationFactor; this.defaultNumPartitions = defaultNumPartitions; - this.defaultMinIsr = defaultMinIsr; this.maxElectionsPerImbalance = maxElectionsPerImbalance; this.eligibleLeaderReplicasEnabled = eligibleLeaderReplicasEnabled; this.configurationControl = configurationControl; @@ -1631,6 +1617,10 @@ boolean arePartitionLeadersImbalanced() { return !imbalancedPartitions.isEmpty(); } + boolean areSomePartitionsLeaderless() { + return brokersToIsrs.partitionsWithNoLeader().hasNext(); + } + /** * Attempt to elect a preferred leader for all topic partitions which have a leader that is not the preferred replica. * @@ -1641,12 +1631,17 @@ boolean arePartitionLeadersImbalanced() { */ ControllerResult maybeBalancePartitionLeaders() { List records = new ArrayList<>(); + maybeTriggerLeaderChangeForPartitionsWithoutPreferredLeader(records, maxElectionsPerImbalance); + return ControllerResult.of(records, records.size() >= maxElectionsPerImbalance); + } - boolean rescheduleImmediately = false; + void maybeTriggerLeaderChangeForPartitionsWithoutPreferredLeader( + List records, + int maxElections + ) { for (TopicIdPartition topicPartition : imbalancedPartitions) { - if (records.size() >= maxElectionsPerImbalance) { - rescheduleImmediately = true; - break; + if (records.size() >= maxElections) { + return; } TopicControlInfo topic = topics.get(topicPartition.topicId()); @@ -1676,8 +1671,52 @@ ControllerResult maybeBalancePartitionLeaders() { .setDefaultDirProvider(clusterDescriber) .build().ifPresent(records::add); } + } + + /** + * Check if we can do an unclean election for partitions with no leader. + * + * The response() method in the return object is true if this method returned without electing all possible preferred replicas. + * The quorum controller should reschedule this operation immediately if it is true. + * + * @return All of the election records and true if there may be more elections to be done. + */ + ControllerResult maybeElectUncleanLeaders() { + List records = new ArrayList<>(); + maybeTriggerUncleanLeaderElectionForLeaderlessPartitions(records, maxElectionsPerImbalance); + return ControllerResult.of(records, records.size() >= maxElectionsPerImbalance); + } - return ControllerResult.of(records, rescheduleImmediately); + /** + * Trigger unclean leader election for partitions without leader (visiable for testing) + * + * @param records The record list to append to. + * @param maxElections The maximum number of elections to perform. + */ + void maybeTriggerUncleanLeaderElectionForLeaderlessPartitions( + List records, + int maxElections + ) { + Iterator iterator = brokersToIsrs.partitionsWithNoLeader(); + while (iterator.hasNext() && records.size() < maxElections) { + TopicIdPartition topicIdPartition = iterator.next(); + TopicControlInfo topic = topics.get(topicIdPartition.topicId()); + if (configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name)) { + ApiError result = electLeader(topic.name, topicIdPartition.partitionId(), + ElectionType.UNCLEAN, records); + if (result.error().equals(Errors.NONE)) { + log.error("Triggering unclean leader election for offline partition {}-{}.", + topic.name, topicIdPartition.partitionId()); + } else { + log.warn("Cannot trigger unclean leader election for offline partition {}-{}: {}", + topic.name, topicIdPartition.partitionId(), result.error()); + } + } else if (log.isDebugEnabled()) { + log.debug("Cannot trigger unclean leader election for offline partition {}-{} " + + "because unclean leader election is disabled for this topic.", + topic.name, topicIdPartition.partitionId()); + } + } } ControllerResult> createPartitions( @@ -2222,14 +2261,8 @@ private void listReassigningTopic(ListPartitionReassignmentsResponseData respons // Visible to test. int getTopicEffectiveMinIsr(String topicName) { - int currentMinIsr = defaultMinIsr; - String minIsrConfig = configurationControl.getTopicConfig(topicName, MIN_IN_SYNC_REPLICAS_CONFIG); - if (minIsrConfig != null) { - currentMinIsr = Integer.parseInt(minIsrConfig); - } else { - log.debug("Can't find the min isr config for topic: " + topicName + ". Use default value " + defaultMinIsr); - } - + String minIsrConfig = configurationControl.getTopicConfig(topicName, MIN_IN_SYNC_REPLICAS_CONFIG).value(); + int currentMinIsr = Integer.parseInt(minIsrConfig); Uuid topicId = topicsByName.get(topicName); int replicationFactor = topics.get(topicId).parts.get(0).replicas.length; return Math.min(currentMinIsr, replicationFactor); diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java index bbe2634547..884bc709cf 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java @@ -20,11 +20,13 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics; import com.yammer.metrics.core.Gauge; +import com.yammer.metrics.core.Meter; import com.yammer.metrics.core.MetricName; import com.yammer.metrics.core.MetricsRegistry; import java.util.Arrays; import java.util.Optional; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; /** @@ -54,6 +56,8 @@ public final class ControllerMetadataMetrics implements AutoCloseable { "KafkaController", "MetadataErrorCount"); private static final MetricName ZK_MIGRATION_STATE = getMetricName( "KafkaController", "ZkMigrationState"); + private static final MetricName UNCLEAN_LEADER_ELECTIONS_PER_SEC = getMetricName( + "ControllerStats", "UncleanLeaderElectionsPerSec"); private final Optional registry; private final AtomicInteger fencedBrokerCount = new AtomicInteger(0); @@ -65,6 +69,8 @@ public final class ControllerMetadataMetrics implements AutoCloseable { private final AtomicInteger preferredReplicaImbalanceCount = new AtomicInteger(0); private final AtomicInteger metadataErrorCount = new AtomicInteger(0); private final AtomicInteger zkMigrationState = new AtomicInteger(-1); + private Optional uncleanLeaderElectionMeter = Optional.empty(); + /** * Create a new ControllerMetadataMetrics object. @@ -129,6 +135,8 @@ public Integer value() { } })); + registry.ifPresent(r -> uncleanLeaderElectionMeter = + Optional.of(registry.get().newMeter(UNCLEAN_LEADER_ELECTIONS_PER_SEC, "elections", TimeUnit.SECONDS))); } public void setFencedBrokerCount(int brokerCount) { @@ -231,6 +239,10 @@ public byte zkMigrationState() { return zkMigrationState.byteValue(); } + public void updateUncleanLeaderElection(int count) { + this.uncleanLeaderElectionMeter.ifPresent(m -> m.mark(count)); + } + @Override public void close() { registry.ifPresent(r -> Arrays.asList( @@ -242,7 +254,8 @@ public void close() { OFFLINE_PARTITION_COUNT, PREFERRED_REPLICA_IMBALANCE_COUNT, METADATA_ERROR_COUNT, - ZK_MIGRATION_STATE + ZK_MIGRATION_STATE, + UNCLEAN_LEADER_ELECTIONS_PER_SEC ).forEach(r::removeMetric)); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java index 12956b3d61..a4b30a9f9f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java @@ -48,6 +48,7 @@ static int delta(boolean prev, boolean next) { private int globalPartitionsChange = 0; private int offlinePartitionsChange = 0; private int partitionsWithoutPreferredLeaderChange = 0; + private int uncleanLeaderElection = 0; public int fencedBrokersChange() { return fencedBrokersChange; @@ -135,6 +136,11 @@ void handlePartitionChange(PartitionRegistration prev, PartitionRegistration nex isPresent = true; isOffline = !next.hasLeader(); isWithoutPreferredLeader = !next.hasPreferredLeader(); + // take current all replicas as ISR if prev is null (new created partition), so we won't treat it as unclean election. + int[] prevIsr = prev != null ? prev.isr : next.replicas; + if (!PartitionRegistration.electionWasClean(next.leader, prevIsr)) { + uncleanLeaderElection++; + } } globalPartitionsChange += delta(wasPresent, isPresent); offlinePartitionsChange += delta(wasOffline, isOffline); @@ -166,5 +172,9 @@ void apply(ControllerMetadataMetrics metrics) { if (partitionsWithoutPreferredLeaderChange != 0) { metrics.addToPreferredReplicaImbalanceCount(partitionsWithoutPreferredLeaderChange); } + if (uncleanLeaderElection > 0) { + metrics.updateUncleanLeaderElection(uncleanLeaderElection); + uncleanLeaderElection = 0; + } } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java index 642c697b8b..d2d2521c12 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java @@ -173,11 +173,29 @@ public Map resolveEffectiveTopicConfigs( return effectiveConfigs; } - private ConfigEntry resolveEffectiveTopicConfig(ConfigDef.ConfigKey configKey, - Map staticNodeConfig, - Map dynamicClusterConfigs, - Map dynamicNodeConfigs, - Map dynamicTopicConfigs) { + public ConfigEntry resolveEffectiveTopicConfig( + String keyName, + Map staticNodeConfig, + Map dynamicClusterConfigs, + Map dynamicNodeConfigs, + Map dynamicTopicConfigs + ) { + ConfigDef configDef = configDefs.getOrDefault(ConfigResource.Type.TOPIC, EMPTY_CONFIG_DEF); + ConfigDef.ConfigKey configKey = configDef.configKeys().get(keyName); + return resolveEffectiveTopicConfig(configKey, + staticNodeConfig, + dynamicClusterConfigs, + dynamicNodeConfigs, + dynamicTopicConfigs); + } + + public ConfigEntry resolveEffectiveTopicConfig( + ConfigDef.ConfigKey configKey, + Map staticNodeConfig, + Map dynamicClusterConfigs, + Map dynamicNodeConfigs, + Map dynamicTopicConfigs + ) { if (dynamicTopicConfigs.containsKey(configKey.name)) { return toConfigEntry(configKey, dynamicTopicConfigs.get(configKey.name), diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index 0efdd13b11..2ce417f63a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -56,7 +56,6 @@ import static org.apache.kafka.common.metadata.MetadataRecordType.CONFIG_RECORD; import static org.apache.kafka.server.config.ConfigSynonym.HOURS_TO_MILLISECONDS; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; @Timeout(value = 40) @@ -136,8 +135,7 @@ public void testReplay() throws Exception { setName("def").setValue("blah")); assertEquals(toMap(entry("abc", "x,y,z"), entry("def", "blah")), manager.getConfigs(MYTOPIC)); - assertEquals("x,y,z", manager.getTopicConfig(MYTOPIC.name(), "abc")); - assertNull(manager.getTopicConfig(MYTOPIC.name(), "none-exists")); + assertEquals("x,y,z", manager.getTopicConfig(MYTOPIC.name(), "abc").value()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 08213dfdfe..0b754ec124 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -158,7 +158,6 @@ import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; import static org.apache.kafka.controller.ConfigurationControlManagerTest.BROKER0; -import static org.apache.kafka.controller.ConfigurationControlManagerTest.SCHEMA; import static org.apache.kafka.controller.ConfigurationControlManagerTest.entry; import static org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT; import static org.apache.kafka.controller.ControllerRequestContextUtil.anonymousContextFor; @@ -191,9 +190,6 @@ public void testConfigurationOperations() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). build() ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, @@ -235,9 +231,6 @@ public void testDelayedConfigurationOperations() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). build() ) { controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT, @@ -282,9 +275,6 @@ public void testFenceMultipleBrokers() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). setSessionTimeoutMillis(OptionalLong.of(sessionTimeoutMillis)). setBootstrapMetadata(SIMPLE_BOOTSTRAP). build() @@ -374,9 +364,6 @@ public void testUncleanShutdownBroker() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). setSessionTimeoutMillis(OptionalLong.of(sessionTimeoutMillis)). setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_4_0_IV0, "test-provided bootstrap ELR enabled")). @@ -509,9 +496,6 @@ public void testBalancePartitionLeaders() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). setSessionTimeoutMillis(OptionalLong.of(sessionTimeoutMillis)). setLeaderImbalanceCheckIntervalNs(OptionalLong.of(leaderImbalanceCheckIntervalNs)). setBootstrapMetadata(SIMPLE_BOOTSTRAP). @@ -651,7 +635,6 @@ public void testNoOpRecordWriteAfterTimeout() throws Throwable { build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). setControllerBuilderInitializer(controllerBuilder -> { - controllerBuilder.setConfigSchema(SCHEMA); controllerBuilder.setMaxIdleIntervalNs(OptionalLong.of(maxIdleIntervalNs)); }). build() @@ -697,8 +680,6 @@ public void testRegisterBrokerKRaftVersions(short finalizedKraftVersion, short b setLastKRaftVersion(KRaftVersion.fromFeatureLevel(finalizedKraftVersion)). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA)). setBootstrapMetadata(SIMPLE_BOOTSTRAP). build() ) { @@ -750,9 +731,6 @@ public void testUnregisterBroker() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). build() ) { ListenerCollection listeners = new ListenerCollection(); @@ -832,9 +810,6 @@ public void testSnapshotSaveAndLoad() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). setBootstrapMetadata(SIMPLE_BOOTSTRAP). build() ) { @@ -1042,9 +1017,6 @@ public void testTimeouts() throws Throwable { try ( LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). build() ) { QuorumController controller = controlEnv.activeController(); @@ -1107,9 +1079,6 @@ public void testEarlyControllerResults() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). build() ) { QuorumController controller = controlEnv.activeController(); @@ -1149,9 +1118,6 @@ public void testConfigResourceExistenceChecker() throws Throwable { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). build() ) { QuorumController active = controlEnv.activeController(); @@ -1320,9 +1286,6 @@ public void testUpgradeFromPreProductionVersion() throws Exception { initialSnapshot.tempDir.toPath(), new OffsetAndEpoch(0, 0))). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). setBootstrapMetadata(COMPLEX_BOOTSTRAP). build() ) { @@ -1345,9 +1308,6 @@ public void testInsertBootstrapRecordsToEmptyLog() throws Exception { LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3). build(); QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). - setControllerBuilderInitializer(controllerBuilder -> - controllerBuilder.setConfigSchema(SCHEMA) - ). setBootstrapMetadata(COMPLEX_BOOTSTRAP). build() ) { diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java index dcea7ef21d..8bece2bb86 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java @@ -17,6 +17,7 @@ package org.apache.kafka.controller; +import org.apache.kafka.metadata.FakeKafkaConfigSchema; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metalog.LocalLogManagerTestEnv; import org.apache.kafka.raft.LeaderAndEpoch; @@ -115,6 +116,7 @@ private QuorumControllerTestEnv( MockFaultHandler nonFatalFaultHandler = new MockFaultHandler("nonFatalFaultHandler"); builder.setNonFatalFaultHandler(nonFatalFaultHandler); builder.setEligibleLeaderReplicasEnabled(eligibleLeaderReplicasEnabled); + builder.setConfigSchema(FakeKafkaConfigSchema.INSTANCE); nonFatalFaultHandlers.put(nodeId, fatalFaultHandler); controllerBuilderInitializer.accept(builder); this.controllers.add(builder.build()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index df62af828e..b50d32e16b 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.controller; +import org.apache.kafka.clients.admin.AlterConfigOp; import org.apache.kafka.common.DirectoryId; import org.apache.kafka.common.ElectionType; import org.apache.kafka.common.TopicPartition; @@ -83,6 +84,7 @@ import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.metadata.BrokerRegistrationInControlledShutdownChange; +import org.apache.kafka.metadata.FakeKafkaConfigSchema; import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.RecordTestUtils; @@ -104,6 +106,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -171,6 +174,7 @@ private static class Builder { private MetadataVersion metadataVersion = MetadataVersion.latestTesting(); private MockTime mockTime = new MockTime(); private boolean isElrEnabled = false; + private final Map staticConfig = new HashMap<>(); Builder setCreateTopicPolicy(CreateTopicPolicy createTopicPolicy) { this.createTopicPolicy = Optional.of(createTopicPolicy); @@ -187,6 +191,11 @@ Builder setIsElrEnabled(Boolean isElrEnabled) { return this; } + Builder setStaticConfig(String key, Object value) { + this.staticConfig.put(key, value); + return this; + } + Builder setMockTime(MockTime mockTime) { this.mockTime = mockTime; return this; @@ -196,14 +205,8 @@ ReplicationControlTestContext build() { return new ReplicationControlTestContext(metadataVersion, createTopicPolicy, mockTime, - isElrEnabled); - } - - ReplicationControlTestContext build(MetadataVersion metadataVersion) { - return new ReplicationControlTestContext(metadataVersion, - createTopicPolicy, - mockTime, - isElrEnabled); + isElrEnabled, + staticConfig); } } @@ -213,9 +216,7 @@ ReplicationControlTestContext build(MetadataVersion metadataVersion) { final MockRandom random = new MockRandom(); final FeatureControlManager featureControl; final ClusterControlManager clusterControl; - final ConfigurationControlManager configurationControl = new ConfigurationControlManager.Builder(). - setSnapshotRegistry(snapshotRegistry). - build(); + final ConfigurationControlManager configurationControl; final ReplicationControlManager replicationControl; void replay(List records) { @@ -228,9 +229,15 @@ private ReplicationControlTestContext( MetadataVersion metadataVersion, Optional createTopicPolicy, MockTime time, - Boolean isElrEnabled + boolean isElrEnabled, + Map staticConfig ) { this.time = time; + this.configurationControl = new ConfigurationControlManager.Builder(). + setSnapshotRegistry(snapshotRegistry). + setStaticConfig(staticConfig). + setKafkaConfigSchema(FakeKafkaConfigSchema.INSTANCE). + build(); this.featureControl = new FeatureControlManager.Builder(). setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, @@ -483,6 +490,10 @@ void alterTopicConfig( replay(singletonList(new ApiMessageAndVersion(configRecord, (short) 0))); } + void fenceBrokers(Integer... brokerIds) { + fenceBrokers(Utils.mkSet(brokerIds)); + } + void fenceBrokers(Set brokerIds) { time.sleep(BROKER_SESSION_TIMEOUT_MS); @@ -524,6 +535,11 @@ ControllerResult assignReplicasToDirs(int brok } } + static CreateTopicsResponseData withoutConfigs(CreateTopicsResponseData data) { + data.topics().forEach(t -> t.configs().clear()); + return data; + } + private static class MockCreateTopicPolicy implements CreateTopicPolicy { private final List expecteds; private final AtomicLong index = new AtomicLong(0); @@ -598,7 +614,7 @@ public void testCreateTopics() { setNumPartitions(1).setReplicationFactor((short) 3). setErrorMessage(null).setErrorCode((short) 0). setTopicId(result3.response().topics().find("foo").topicId())); - assertEquals(expectedResponse3, result3.response()); + assertEquals(expectedResponse3, withoutConfigs(result3.response())); ctx.replay(result3.records()); assertEquals(new PartitionRegistration.Builder().setReplicas(new int[] {1, 2, 0}). setDirectories(new Uuid[] { @@ -660,6 +676,9 @@ public void testCreateTopicsISRInvariants() { setNumPartitions(1).setReplicationFactor((short) 3). setErrorMessage(null).setErrorCode((short) 0). setTopicId(result.response().topics().find("foo").topicId())); + for (CreatableTopicResult topic : result.response().topics()) { + topic.configs().clear(); + } assertEquals(expectedResponse, result.response()); ctx.replay(result.records()); @@ -1380,7 +1399,7 @@ public void testDeleteTopics() { setNumPartitions(3).setReplicationFactor((short) 2). setErrorMessage(null).setErrorCode((short) 0). setTopicId(topicId)); - assertEquals(expectedResponse, createResult.response()); + assertEquals(expectedResponse, withoutConfigs(createResult.response())); // Until the records are replayed, no changes are made assertNull(replicationControl.getPartition(topicId, 0)); assertEmptyTopicConfigs(ctx, "foo"); @@ -2245,7 +2264,9 @@ private void assertLeaderAndIsr( @ParameterizedTest @ValueSource(booleans = {true, false}) public void testElectUncleanLeaders_WithoutElr(boolean electAllPartitions) { - ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder().build(MetadataVersion.IBP_3_6_IV1); + ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder(). + setMetadataVersion(MetadataVersion.IBP_3_6_IV1). + build(); ReplicationControlManager replication = ctx.replicationControl; ctx.registerBrokers(0, 1, 2, 3, 4); ctx.unfenceBrokers(0, 1, 2, 3, 4); @@ -2608,6 +2629,85 @@ public void testBalancePartitionLeaders() { assertFalse(balanceResult.response()); } + @ParameterizedTest + @ValueSource(strings = {"none", "static", "dynamic_cluster", "dynamic_node", "dynamic_topic"}) + public void testMaybeTriggerUncleanLeaderElectionForLeaderlessPartitions(String uncleanConfig) { + ReplicationControlTestContext.Builder ctxBuilder = new ReplicationControlTestContext.Builder(); + if (uncleanConfig.equals("static")) { + ctxBuilder.setStaticConfig(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "true"); + } + ReplicationControlTestContext ctx = ctxBuilder.build(); + ReplicationControlManager replication = ctx.replicationControl; + ctx.registerBrokers(0, 1, 2, 3, 4); + ctx.unfenceBrokers(0, 1, 2, 3, 4); + Uuid fooId = ctx.createTestTopic("foo", new int[][]{ + new int[]{1, 2, 4}, new int[]{1, 3, 4}, new int[]{0, 2, 4}}).topicId(); + assertFalse(replication.areSomePartitionsLeaderless()); + ctx.fenceBrokers(0, 1, 2, 3, 4); + assertTrue(replication.areSomePartitionsLeaderless()); + for (int partitionId : Arrays.asList(0, 1, 2)) { + assertArrayEquals(new int[] {4}, ctx.replicationControl.getPartition(fooId, partitionId).isr); + assertEquals(-1, ctx.replicationControl.getPartition(fooId, partitionId).leader); + } + + // Unfence broker 2. It is now available to be the leader for partition 0 and 2, after + // an unclean election. + ctx.unfenceBrokers(2); + + if (uncleanConfig.equals("static")) { + // If we statically configured unclean leader election, the election already happened. + assertArrayEquals(new int[] {2}, ctx.replicationControl.getPartition(fooId, 0).isr); + assertEquals(2, ctx.replicationControl.getPartition(fooId, 0).leader); + assertArrayEquals(new int[] {4}, ctx.replicationControl.getPartition(fooId, 1).isr); + assertEquals(-1, ctx.replicationControl.getPartition(fooId, 1).leader); + assertArrayEquals(new int[] {2}, ctx.replicationControl.getPartition(fooId, 2).isr); + assertEquals(2, ctx.replicationControl.getPartition(fooId, 2).leader); + } else { + // Otherwise, check that the election did NOT happen. + for (int partitionId : Arrays.asList(0, 1, 2)) { + assertArrayEquals(new int[] {4}, ctx.replicationControl.getPartition(fooId, partitionId).isr); + assertEquals(-1, ctx.replicationControl.getPartition(fooId, partitionId).leader); + } + } + + // If we're setting unclean leader election dynamically, do that here. + if (uncleanConfig.equals("dynamic_cluster")) { + ctx.replay(ctx.configurationControl.incrementalAlterConfigs( + Collections.singletonMap(new ConfigResource(ConfigResource.Type.BROKER, ""), + Collections.singletonMap(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, + new AbstractMap.SimpleImmutableEntry<>(AlterConfigOp.OpType.SET, "true"))), + true).records()); + } else if (uncleanConfig.equals("dynamic_node")) { + ctx.replay(ctx.configurationControl.incrementalAlterConfigs( + Collections.singletonMap(new ConfigResource(ConfigResource.Type.BROKER, "0"), + Collections.singletonMap(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, + new AbstractMap.SimpleImmutableEntry<>(AlterConfigOp.OpType.SET, "true"))), + true).records()); + } else if (uncleanConfig.equals("dynamic_topic")) { + ctx.replay(ctx.configurationControl.incrementalAlterConfigs( + Collections.singletonMap(new ConfigResource(ConfigResource.Type.TOPIC, "foo"), + Collections.singletonMap(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, + new AbstractMap.SimpleImmutableEntry<>(AlterConfigOp.OpType.SET, "true"))), + true).records()); + } + ControllerResult balanceResult = replication.maybeElectUncleanLeaders(); + assertFalse(balanceResult.response()); + if (uncleanConfig.equals("none") || uncleanConfig.equals("static")) { + assertEquals(0, balanceResult.records().size(), "Expected no records, but " + + balanceResult.records().size() + " were found."); + } else { + assertNotEquals(0, balanceResult.records().size(), "Expected some records, but " + + "none were found."); + ctx.replay(balanceResult.records()); + assertArrayEquals(new int[] {2}, ctx.replicationControl.getPartition(fooId, 0).isr); + assertEquals(2, ctx.replicationControl.getPartition(fooId, 0).leader); + assertArrayEquals(new int[] {4}, ctx.replicationControl.getPartition(fooId, 1).isr); + assertEquals(-1, ctx.replicationControl.getPartition(fooId, 1).leader); + assertArrayEquals(new int[] {2}, ctx.replicationControl.getPartition(fooId, 2).isr); + assertEquals(2, ctx.replicationControl.getPartition(fooId, 2).leader); + } + } + private void assertElectLeadersResponse( ElectLeadersResponseData expected, ElectLeadersResponseData actual diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java index 984ad68d0c..345f0264c5 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java @@ -17,7 +17,10 @@ package org.apache.kafka.controller.metrics; +import org.apache.kafka.common.utils.MockTime; + import com.yammer.metrics.core.Gauge; +import com.yammer.metrics.core.Meter; import com.yammer.metrics.core.MetricName; import com.yammer.metrics.core.MetricsRegistry; @@ -48,7 +51,8 @@ public void testMetricNames() { "kafka.controller:type=KafkaController,name=MetadataErrorCount", "kafka.controller:type=KafkaController,name=OfflinePartitionsCount", "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount", - "kafka.controller:type=KafkaController,name=ZkMigrationState" + "kafka.controller:type=KafkaController,name=ZkMigrationState", + "kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec" ))); } ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "KafkaController", @@ -174,4 +178,21 @@ public void testPreferredReplicaImbalanceCountMetric() { (m, v) -> m.addToPreferredReplicaImbalanceCount(v) ); } + + @SuppressWarnings("LocalVariableName") + @Test + public void testUpdateUncleanLeaderElection() { + MetricsRegistry registry = new MetricsRegistry(); + MockTime time = new MockTime(); + try (ControllerMetadataMetrics metrics = new ControllerMetadataMetrics(Optional.of(registry))) { + Meter UncleanLeaderElectionsPerSec = (Meter) registry + .allMetrics() + .get(metricName("ControllerStats", "UncleanLeaderElectionsPerSec")); + assertEquals(0, UncleanLeaderElectionsPerSec.count()); + metrics.updateUncleanLeaderElection(2); + assertEquals(2, UncleanLeaderElectionsPerSec.count()); + } finally { + registry.shutdown(); + } + } } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/FakeKafkaConfigSchema.java b/metadata/src/test/java/org/apache/kafka/metadata/FakeKafkaConfigSchema.java new file mode 100644 index 0000000000..c535e10887 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/metadata/FakeKafkaConfigSchema.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.metadata; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.server.config.ConfigSynonym; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; +import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; +import static org.apache.kafka.common.config.ConfigDef.Type.INT; +import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; +import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; + +/** + * A fake KafkaConfigSchema object that can be used in tests. + */ +public class FakeKafkaConfigSchema { + public static final Map CONFIGS = new HashMap<>(); + + static { + CONFIGS.put(BROKER, new ConfigDef(). + define("unclean.leader.election.enable", BOOLEAN, "false", HIGH, ""). + define("min.insync.replicas", INT, "1", HIGH, "")); + CONFIGS.put(TOPIC, new ConfigDef(). + define("unclean.leader.election.enable", BOOLEAN, "false", HIGH, ""). + define("min.insync.replicas", INT, "1", HIGH, "")); + } + + public static final Map> SYNONYMS = new HashMap<>(); + + static { + SYNONYMS.put("unclean.leader.election.enable", + Arrays.asList(new ConfigSynonym("unclean.leader.election.enable"))); + SYNONYMS.put("min.insync.replicas", + Arrays.asList(new ConfigSynonym("min.insync.replicas"))); + } + + public static final KafkaConfigSchema INSTANCE = new KafkaConfigSchema(CONFIGS, SYNONYMS); +} diff --git a/server/src/main/java/org/apache/kafka/server/config/ReplicationConfigs.java b/server/src/main/java/org/apache/kafka/server/config/ReplicationConfigs.java index aa061a0afd..3bce284e47 100644 --- a/server/src/main/java/org/apache/kafka/server/config/ReplicationConfigs.java +++ b/server/src/main/java/org/apache/kafka/server/config/ReplicationConfigs.java @@ -25,6 +25,8 @@ import org.apache.kafka.server.common.MetadataVersionValidator; import org.apache.kafka.storage.internals.log.LogConfig; +import java.util.concurrent.TimeUnit; + import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; @@ -115,6 +117,10 @@ public class ReplicationConfigs { public static final int LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_DEFAULT = 300; public static final String LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_DOC = "The frequency with which the partition rebalance check is triggered by the controller"; + public static final String UNCLEAN_LEADER_ELECTION_INTERVAL_MS_CONFIG = "unclean.leader.election.interval.ms"; + public static final long UNCLEAN_LEADER_ELECTION_INTERVAL_MS_DEFAULT = TimeUnit.MINUTES.toMillis(5); + public static final String UNCLEAN_LEADER_ELECTION_INTERVAL_MS_DOC = "The frequency with which the controller checks if it should perform an unclean leader election for leaderless partitions."; + public static final String UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG); public static final String UNCLEAN_LEADER_ELECTION_ENABLE_DOC = "Indicates whether to enable replicas not in the ISR set to be elected as leader as a last resort, even though doing so may result in data loss"; @@ -159,6 +165,7 @@ public class ReplicationConfigs { .define(AUTO_LEADER_REBALANCE_ENABLE_CONFIG, BOOLEAN, AUTO_LEADER_REBALANCE_ENABLE_DEFAULT, HIGH, AUTO_LEADER_REBALANCE_ENABLE_DOC) .define(LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG, INT, LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_DEFAULT, HIGH, LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_DOC) .define(LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG, LONG, LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_DEFAULT, atLeast(1), HIGH, LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_DOC) + .defineInternal(UNCLEAN_LEADER_ELECTION_INTERVAL_MS_CONFIG, LONG, UNCLEAN_LEADER_ELECTION_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, UNCLEAN_LEADER_ELECTION_INTERVAL_MS_DOC) .define(UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, BOOLEAN, LogConfig.DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE, HIGH, UNCLEAN_LEADER_ELECTION_ENABLE_DOC) .define(INTER_BROKER_SECURITY_PROTOCOL_CONFIG, STRING, INTER_BROKER_SECURITY_PROTOCOL_DEFAULT, ConfigDef.ValidString.in(Utils.enumOptions(SecurityProtocol.class)), MEDIUM, INTER_BROKER_SECURITY_PROTOCOL_DOC) .define(INTER_BROKER_PROTOCOL_VERSION_CONFIG, STRING, INTER_BROKER_PROTOCOL_VERSION_DEFAULT, new MetadataVersionValidator(), MEDIUM, INTER_BROKER_PROTOCOL_VERSION_DOC) diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java index 91b24fc2b0..fc21758518 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java @@ -72,7 +72,7 @@ public void modifyConfigs(Seq props) { @SuppressWarnings("deprecation") @Override - public Seq kraftControllerConfigs() { + public Seq kraftControllerConfigs(TestInfo testInfo) { return JavaConverters.asScalaBuffer(Collections.singletonList(overridingProps())).toSeq(); } From fbaea5ff6a4c6fd6124244e13ba534b2c0eddd4b Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Thu, 29 Aug 2024 15:09:55 +0900 Subject: [PATCH 070/123] KAFKA-17062: handle dangling "copy_segment_start" state when deleting remote logs (#16959) The COPY_SEGMENT_STARTED state segments are counted when calculating remote retention size. This causes unexpected retention size breached segment deletion. This PR fixes it by 1. only counting COPY_SEGMENT_FINISHED and DELETE_SEGMENT_STARTED state segments when calculating remote log size. 2. During copy Segment, if we encounter errors, we will delete the segment immediately. 3. Tests added. Co-authored-by: Guillaume Mallet <> Reviewers: Kamal Chandraprakash, Satish Duggana , Guillaume Mallet <> --- checkstyle/suppressions.xml | 2 +- .../kafka/log/remote/RemoteLogManager.java | 29 ++- .../log/remote/RemoteLogManagerTest.java | 241 ++++++++++++++++-- 3 files changed, 243 insertions(+), 29 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index def3e2bc0d..756ed349c2 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -39,7 +39,7 @@ - + customMetadata = remoteLogStorageManager.copyLogSegmentData(copySegmentStartedRlsm, segmentData); + Optional customMetadata = Optional.empty(); + try { + customMetadata = remoteLogStorageManager.copyLogSegmentData(copySegmentStartedRlsm, segmentData); + } catch (RemoteStorageException e) { + try { + remoteLogStorageManager.deleteLogSegmentData(copySegmentStartedRlsm); + logger.info("Successfully cleaned segment {} after failing to copy segment", segmentId); + } catch (RemoteStorageException e1) { + logger.error("Error while cleaning segment {}, consider cleaning manually", segmentId, e1); + } + throw e; + } RemoteLogSegmentMetadataUpdate copySegmentFinishedRlsm = new RemoteLogSegmentMetadataUpdate(segmentId, time.milliseconds(), customMetadata, RemoteLogSegmentState.COPY_SEGMENT_FINISHED, brokerId); @@ -1374,10 +1385,18 @@ private Optional buildRetentionSizeData(long retentionSize, Iterator segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch); while (segmentsIterator.hasNext()) { RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next(); - RemoteLogSegmentId segmentId = segmentMetadata.remoteLogSegmentId(); - if (!visitedSegmentIds.contains(segmentId) && isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) { - remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes(); - visitedSegmentIds.add(segmentId); + // Only count the size of "COPY_SEGMENT_FINISHED" and "DELETE_SEGMENT_STARTED" state segments + // because "COPY_SEGMENT_STARTED" means copy didn't complete, and "DELETE_SEGMENT_FINISHED" means delete did complete. + // Note: there might be some "COPY_SEGMENT_STARTED" segments not counted here. + // Either they are being copied and will be counted next time or they are dangling and will be cleaned elsewhere, + // either way, this won't cause more segment deletion. + if (segmentMetadata.state().equals(RemoteLogSegmentState.COPY_SEGMENT_FINISHED) || + segmentMetadata.state().equals(RemoteLogSegmentState.DELETE_SEGMENT_STARTED)) { + RemoteLogSegmentId segmentId = segmentMetadata.remoteLogSegmentId(); + if (!visitedSegmentIds.contains(segmentId) && isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) { + remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes(); + visitedSegmentIds.add(segmentId); + } } } } diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index d6d2ff0ecb..81d7e3f100 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -112,6 +112,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -164,6 +165,7 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; @@ -676,10 +678,90 @@ void testCustomMetadataSizeExceedsLimit() throws Exception { // The metadata update should not be posted. verify(remoteLogMetadataManager, never()).updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class)); - // Verify the metric for remote writes are not updated. + // Verify the metrics + assertEquals(1, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyRequestRate().count()); + assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyBytesRate().count()); + assertEquals(1, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).failedRemoteCopyRequestRate().count()); + // Verify aggregate metrics + assertEquals(1, brokerTopicStats.allTopicsStats().remoteCopyRequestRate().count()); + assertEquals(0, brokerTopicStats.allTopicsStats().remoteCopyBytesRate().count()); + assertEquals(1, brokerTopicStats.allTopicsStats().failedRemoteCopyRequestRate().count()); + } + + @Test + void testFailedCopyShouldDeleteTheDanglingSegment() throws Exception { + long oldSegmentStartOffset = 0L; + long nextSegmentStartOffset = 150L; + long lastStableOffset = 150L; + long logEndOffset = 150L; + + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + + // leader epoch preparation + checkpoint.write(totalEpochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint, scheduler); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())).thenReturn(Optional.of(-1L)); + + File tempFile = TestUtils.tempFile(); + File mockProducerSnapshotIndex = TestUtils.tempFile(); + File tempDir = TestUtils.tempDirectory(); + // create 2 log segments, with 0 and 150 as log start offset + LogSegment oldSegment = mock(LogSegment.class); + LogSegment activeSegment = mock(LogSegment.class); + + when(oldSegment.baseOffset()).thenReturn(oldSegmentStartOffset); + when(activeSegment.baseOffset()).thenReturn(nextSegmentStartOffset); + verify(oldSegment, times(0)).readNextOffset(); + verify(activeSegment, times(0)).readNextOffset(); + + FileRecords fileRecords = mock(FileRecords.class); + when(oldSegment.log()).thenReturn(fileRecords); + when(fileRecords.file()).thenReturn(tempFile); + when(fileRecords.sizeInBytes()).thenReturn(10); + when(oldSegment.readNextOffset()).thenReturn(nextSegmentStartOffset); + + when(mockLog.activeSegment()).thenReturn(activeSegment); + when(mockLog.logStartOffset()).thenReturn(oldSegmentStartOffset); + when(mockLog.logSegments(anyLong(), anyLong())).thenReturn(JavaConverters.collectionAsScalaIterable(Arrays.asList(oldSegment, activeSegment))); + + ProducerStateManager mockStateManager = mock(ProducerStateManager.class); + when(mockLog.producerStateManager()).thenReturn(mockStateManager); + when(mockStateManager.fetchSnapshot(anyLong())).thenReturn(Optional.of(mockProducerSnapshotIndex)); + when(mockLog.lastStableOffset()).thenReturn(lastStableOffset); + when(mockLog.logEndOffset()).thenReturn(logEndOffset); + + OffsetIndex idx = LazyIndex.forOffset(LogFileUtils.offsetIndexFile(tempDir, oldSegmentStartOffset, ""), oldSegmentStartOffset, 1000).get(); + TimeIndex timeIdx = LazyIndex.forTime(LogFileUtils.timeIndexFile(tempDir, oldSegmentStartOffset, ""), oldSegmentStartOffset, 1500).get(); + File txnFile = UnifiedLog.transactionIndexFile(tempDir, oldSegmentStartOffset, ""); + txnFile.createNewFile(); + TransactionIndex txnIndex = new TransactionIndex(oldSegmentStartOffset, txnFile); + when(oldSegment.timeIndex()).thenReturn(timeIdx); + when(oldSegment.offsetIndex()).thenReturn(idx); + when(oldSegment.txnIndex()).thenReturn(txnIndex); + + CompletableFuture dummyFuture = new CompletableFuture<>(); + dummyFuture.complete(null); + when(remoteLogMetadataManager.addRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadata.class))).thenReturn(dummyFuture); + when(rlmCopyQuotaManager.getThrottleTimeMs()).thenReturn(quotaAvailableThrottleTime); + + // throw exception when copyLogSegmentData + when(remoteStorageManager.copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class))) + .thenThrow(new RemoteStorageException("test")); + RemoteLogManager.RLMCopyTask task = remoteLogManager.new RLMCopyTask(leaderTopicIdPartition, 128); + task.copyLogSegmentsToRemote(mockLog); + + ArgumentCaptor remoteLogSegmentMetadataArg = ArgumentCaptor.forClass(RemoteLogSegmentMetadata.class); + verify(remoteLogMetadataManager).addRemoteLogSegmentMetadata(remoteLogSegmentMetadataArg.capture()); + // verify the segment is deleted + verify(remoteStorageManager, times(1)).deleteLogSegmentData(eq(remoteLogSegmentMetadataArg.getValue())); + + // The metadata update should not be posted. + verify(remoteLogMetadataManager, never()).updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class)); + + // Verify the metrics assertEquals(1, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyRequestRate().count()); assertEquals(0, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteCopyBytesRate().count()); - // Verify we did not report any failure for remote writes assertEquals(1, brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).failedRemoteCopyRequestRate().count()); // Verify aggregate metrics assertEquals(1, brokerTopicStats.allTopicsStats().remoteCopyRequestRate().count()); @@ -2303,34 +2385,16 @@ public void testRemoteDeleteLagsOnRetentionBreachedSegments(long retentionSize, .thenReturn(CompletableFuture.runAsync(() -> { })); doAnswer(ans -> { - assertEquals(2048, safeLongYammerMetricValue("RemoteDeleteLagBytes"), - String.format("Expected to find 2048 for RemoteDeleteLagBytes metric value, but found %d", safeLongYammerMetricValue("RemoteDeleteLagBytes"))); - assertEquals(2048, safeLongYammerMetricValue("RemoteDeleteLagBytes,topic=" + leaderTopic), - String.format("Expected to find 2048 for RemoteDeleteLagBytes for 'Leader' topic metric value, but found %d", safeLongYammerMetricValue("RemoteDeleteLagBytes,topic=" + leaderTopic))); - assertEquals(2, safeLongYammerMetricValue("RemoteDeleteLagSegments"), - String.format("Expected to find 2 for RemoteDeleteLagSegments metric value, but found %d", safeLongYammerMetricValue("RemoteDeleteLagSegments"))); - assertEquals(2, safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic), - String.format("Expected to find 2 for RemoteDeleteLagSegments for 'Leader' topic metric value, but found %d", safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic))); + verifyRemoteDeleteMetrics(2048L, 2L); return Optional.empty(); }).doAnswer(ans -> { - assertEquals(1024, safeLongYammerMetricValue("RemoteDeleteLagBytes"), - String.format("Expected to find 1024 for RemoteDeleteLagBytes metric value, but found %d", safeLongYammerMetricValue("RemoteDeleteLagBytes"))); - assertEquals(1, safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic), - String.format("Expected to find 1 for RemoteDeleteLagSegments for 'Leader' topic metric value, but found %d", safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic))); - assertEquals(1024, safeLongYammerMetricValue("RemoteDeleteLagBytes"), - String.format("Expected to find 1024 for RemoteDeleteLagBytes metric value, but found %d", safeLongYammerMetricValue("RemoteDeleteLagBytes"))); - assertEquals(1, safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic), - String.format("Expected to find 1 for RemoteDeleteLagSegments for 'Leader' topic metric value, but found %d", safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic))); + verifyRemoteDeleteMetrics(1024L, 1L); return Optional.empty(); }).when(remoteStorageManager).deleteLogSegmentData(any(RemoteLogSegmentMetadata.class)); RemoteLogManager.RLMExpirationTask task = remoteLogManager.new RLMExpirationTask(leaderTopicIdPartition); - assertEquals(0L, yammerMetricValue("RemoteDeleteLagBytes")); - assertEquals(0L, yammerMetricValue("RemoteDeleteLagSegments")); - - assertEquals(0L, safeLongYammerMetricValue("RemoteDeleteLagBytes,topic=" + leaderTopic)); - assertEquals(0L, safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic)); + verifyRemoteDeleteMetrics(0L, 0L); task.cleanupExpiredRemoteLogSegments(); @@ -2339,6 +2403,98 @@ public void testRemoteDeleteLagsOnRetentionBreachedSegments(long retentionSize, verify(remoteStorageManager).deleteLogSegmentData(metadataList.get(1)); } + @Test + public void testRemoteLogSizeRetentionShouldFilterOutCopySegmentStartState() + throws RemoteStorageException, ExecutionException, InterruptedException { + int segmentSize = 1024; + Map logProps = new HashMap<>(); + // set the retention.bytes to 10 segment size + logProps.put("retention.bytes", segmentSize * 10L); + logProps.put("retention.ms", -1L); + LogConfig mockLogConfig = new LogConfig(logProps); + when(mockLog.config()).thenReturn(mockLogConfig); + + List epochEntries = Collections.singletonList(epochEntry0); + checkpoint.write(epochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, scheduler); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + when(mockLog.logEndOffset()).thenReturn(2000L); + + // creating remote log metadata list: + // s1. One segment with "COPY_SEGMENT_STARTED" state to simulate the segment was failing on copying to remote storage. + // it should be ignored for both remote log size calculation, but get deleted in the 1st run. + // s2. One segment with "DELETE_SEGMENT_FINISHED" state to simulate the remoteLogMetadataManager doesn't filter it out and returned. + // We should filter it out when calculating remote storage log size and deletion + // s3. One segment with "DELETE_SEGMENT_STARTED" state to simulate the segment was failing on deleting remote log. + // We should count it in when calculating remote storage log size. + // s4. Another segment with "COPY_SEGMENT_STARTED" state to simulate the segment is copying to remote storage. + // The segment state will change to "COPY_SEGMENT_FINISHED" state before checking deletion. + // In the 1st run, this segment should be skipped when calculating remote storage size. + // In the 2nd run, we should count it in when calculating remote storage log size. + // s5. 11 segments with "COPY_SEGMENT_FINISHED" state. These are expected to be counted in when calculating remote storage log size + // + // Expected results (retention.size is 10240 (10 segments)): + // In the 1st run, the total remote storage size should be 1024 * 12 (s3, s5), so 2 segments (s1, s3) will be deleted + // due to retention size breached. s1 will be deleted even though it is not included in size calculation. But it's fine. + // The segment intended to be deleted will be deleted in the next run. + // In the 2nd run, the total remote storage size should be 1024 * 12 (s4, s5) + // so 2 segments will be deleted due to retention size breached. + RemoteLogSegmentMetadata s1 = createRemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), + 0, 99, segmentSize, epochEntries, RemoteLogSegmentState.COPY_SEGMENT_STARTED); + RemoteLogSegmentMetadata s2 = createRemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), + 0, 99, segmentSize, epochEntries, RemoteLogSegmentState.DELETE_SEGMENT_FINISHED); + RemoteLogSegmentMetadata s3 = createRemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), + 0, 99, segmentSize, epochEntries, RemoteLogSegmentState.DELETE_SEGMENT_STARTED); + RemoteLogSegmentMetadata s4CopyStarted = createRemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), + 200, 299, segmentSize, epochEntries, RemoteLogSegmentState.COPY_SEGMENT_STARTED); + RemoteLogSegmentMetadata s4CopyFinished = createRemoteLogSegmentMetadata(s4CopyStarted.remoteLogSegmentId(), + s4CopyStarted.startOffset(), s4CopyStarted.endOffset(), segmentSize, epochEntries, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + List s5 = + listRemoteLogSegmentMetadata(leaderTopicIdPartition, 11, 100, 1024, epochEntries, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + + List metadataList = new LinkedList<>(); + metadataList.addAll(Arrays.asList(s1, s2, s3, s4CopyStarted)); + metadataList.addAll(s5); + + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)) + .thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 0)) + .thenReturn(metadataList.iterator()).thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))) + .thenReturn(CompletableFuture.runAsync(() -> { })); + doNothing().when(remoteStorageManager).deleteLogSegmentData(any(RemoteLogSegmentMetadata.class)); + + // RUN 1 + RemoteLogManager.RLMExpirationTask task = remoteLogManager.new RLMExpirationTask(leaderTopicIdPartition); + task.cleanupExpiredRemoteLogSegments(); + verify(remoteStorageManager, times(2)).deleteLogSegmentData(any(RemoteLogSegmentMetadata.class)); + verify(remoteStorageManager).deleteLogSegmentData(s1); + // make sure the s2 segment with "DELETE_SEGMENT_FINISHED" state is not invoking "deleteLogSegmentData" + verify(remoteStorageManager, never()).deleteLogSegmentData(s2); + verify(remoteStorageManager).deleteLogSegmentData(s3); + + clearInvocations(remoteStorageManager); + + // RUN 2 + // update the metadata list to remove deleted s1, s3, and set the state in s4 to COPY_SEGMENT_FINISHED + List updatedMetadataList = new LinkedList<>(); + updatedMetadataList.addAll(Arrays.asList(s2, s4CopyFinished)); + updatedMetadataList.addAll(s5); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)) + .thenReturn(updatedMetadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 0)) + .thenAnswer(ans -> updatedMetadataList.iterator()); + + doNothing().when(remoteStorageManager).deleteLogSegmentData(any(RemoteLogSegmentMetadata.class)); + task.cleanupExpiredRemoteLogSegments(); + + // make sure 2 segments got deleted + verify(remoteStorageManager, times(2)).deleteLogSegmentData(any(RemoteLogSegmentMetadata.class)); + verify(remoteStorageManager).deleteLogSegmentData(s4CopyFinished); + verify(remoteStorageManager).deleteLogSegmentData(s5.get(0)); + } + @Test public void testDeleteRetentionMsBeingCancelledBeforeSecondDelete() throws RemoteStorageException, ExecutionException, InterruptedException { RemoteLogManager.RLMExpirationTask leaderTask = remoteLogManager.new RLMExpirationTask(leaderTopicIdPartition); @@ -2453,6 +2609,11 @@ public void testFailedDeleteExpiredSegments(long retentionSize, // Verify aggregate metrics assertEquals(1, brokerTopicStats.allTopicsStats().remoteDeleteRequestRate().count()); assertEquals(1, brokerTopicStats.allTopicsStats().failedRemoteDeleteRequestRate().count()); + + // make sure we'll retry the deletion in next run + doNothing().when(remoteStorageManager).deleteLogSegmentData(any()); + task.cleanupExpiredRemoteLogSegments(); + verify(remoteStorageManager).deleteLogSegmentData(metadataList.get(0)); } @ParameterizedTest(name = "testDeleteLogSegmentDueToRetentionSizeBreach segmentCount={0} deletableSegmentCount={1}") @@ -2531,6 +2692,21 @@ public void testDeleteLogSegmentDueToRetentionTimeBreach(int segmentCount, verifyDeleteLogSegment(segmentMetadataList, deletableSegmentCount, currentLeaderEpoch); } + private void verifyRemoteDeleteMetrics(long remoteDeleteLagBytes, long remoteDeleteLagSegments) { + assertEquals(remoteDeleteLagBytes, safeLongYammerMetricValue("RemoteDeleteLagBytes"), + String.format("Expected to find %d for RemoteDeleteLagBytes metric value, but found %d", + remoteDeleteLagBytes, safeLongYammerMetricValue("RemoteDeleteLagBytes"))); + assertEquals(remoteDeleteLagSegments, safeLongYammerMetricValue("RemoteDeleteLagSegments"), + String.format("Expected to find %d for RemoteDeleteLagSegments metric value, but found %d", + remoteDeleteLagSegments, safeLongYammerMetricValue("RemoteDeleteLagSegments"))); + assertEquals(remoteDeleteLagBytes, safeLongYammerMetricValue("RemoteDeleteLagBytes,topic=" + leaderTopic), + String.format("Expected to find %d for RemoteDeleteLagBytes for 'Leader' topic metric value, but found %d", + remoteDeleteLagBytes, safeLongYammerMetricValue("RemoteDeleteLagBytes,topic=" + leaderTopic))); + assertEquals(remoteDeleteLagSegments, safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic), + String.format("Expected to find %d for RemoteDeleteLagSegments for 'Leader' topic metric value, but found %d", + remoteDeleteLagSegments, safeLongYammerMetricValue("RemoteDeleteLagSegments,topic=" + leaderTopic))); + } + private void verifyDeleteLogSegment(List segmentMetadataList, int deletableSegmentCount, int currentLeaderEpoch) @@ -2665,6 +2841,25 @@ private List listRemoteLogSegmentMetadataByTime(TopicI return segmentMetadataList; } + private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(RemoteLogSegmentId segmentID, + long startOffset, + long endOffset, + int segmentSize, + List epochEntries, + RemoteLogSegmentState state) { + return new RemoteLogSegmentMetadata( + segmentID, + startOffset, + endOffset, + time.milliseconds(), + brokerId, + time.milliseconds(), + segmentSize, + Optional.empty(), + state, + truncateAndGetLeaderEpochs(epochEntries, startOffset, endOffset)); + } + private Map truncateAndGetLeaderEpochs(List entries, Long startOffset, Long endOffset) { From d9ebb2e79b80a14a93d77e78a1815b7345756c74 Mon Sep 17 00:00:00 2001 From: Krishna Agarwal <62741600+kagarwal06@users.noreply.github.com> Date: Thu, 29 Aug 2024 17:33:11 +0530 Subject: [PATCH 071/123] MINOR: Add experimental message for the native docker image (#17041) The docker image for Native Apache Kafka was introduced with KIP-974 and was first release with 3.8 AK release. The docker image for Native Apache Kafka is currently intended for local development and testing purposes. This PR intends to add a logline indicating the same during docker image startup. Reviewers: Manikumar Reddy --- docker/native/launch | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/native/launch b/docker/native/launch index 05fa4132cb..93de7aa948 100755 --- a/docker/native/launch +++ b/docker/native/launch @@ -46,5 +46,7 @@ result=$(/opt/kafka/kafka.Kafka setup \ echo $result | grep -i "already formatted" || \ { echo $result && (exit 1) } +echo "WARNING: THIS IS AN EXPERIMENTAL DOCKER IMAGE RECOMMENDED FOR LOCAL TESTING AND DEVELOPMENT PURPOSES." + KAFKA_LOG4J_CMD_OPTS="-Dkafka.logs.dir=/opt/kafka/logs/ -Dlog4j.configuration=file:/opt/kafka/config/log4j.properties" exec /opt/kafka/kafka.Kafka start --config /opt/kafka/config/server.properties $KAFKA_LOG4J_CMD_OPTS $KAFKA_JMX_OPTS ${KAFKA_OPTS-} From c7cc4d0b687ab83c0992ae800fa3e76a15cd3ad0 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Thu, 29 Aug 2024 12:51:42 -0700 Subject: [PATCH 072/123] KAFKA-17434: Do not test impossible scenarios in upgrade_test.py (#17024) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Because of KIP-902 (Upgrade Zookeeper version to 3.8.2), it is not possible to upgrade from a Kafka version earlier than 2.4 to a version later than 2.4. Therefore, we should not test these upgrade scenarios in upgrade_test.py. They do happen to work sometimes, but only in the trivial case where we don't create topics or make changes during the upgrade (which would reveal the ZK incompatibility). Instead, we should test only supported scenarios. Reviewers: Reviewers: José Armando García Sancio --- tests/kafkatest/tests/core/downgrade_test.py | 12 +---- tests/kafkatest/tests/core/upgrade_test.py | 49 ++------------------ 2 files changed, 5 insertions(+), 56 deletions(-) diff --git a/tests/kafkatest/tests/core/downgrade_test.py b/tests/kafkatest/tests/core/downgrade_test.py index 68f102e874..e35f6f7515 100644 --- a/tests/kafkatest/tests/core/downgrade_test.py +++ b/tests/kafkatest/tests/core/downgrade_test.py @@ -19,7 +19,7 @@ from kafkatest.services.kafka import config_property from kafkatest.tests.end_to_end import EndToEndTest -from kafkatest.version import LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, \ +from kafkatest.version import LATEST_2_4, LATEST_2_5, \ LATEST_2_6, LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ LATEST_3_6, LATEST_3_7, LATEST_3_8, DEV_BRANCH, KafkaVersion @@ -122,16 +122,6 @@ def wait_until_rejoin(self): # required static membership to be enabled @parametrize(version=str(LATEST_2_4), compression_types=["none"], static_membership=True) @parametrize(version=str(LATEST_2_4), compression_types=["zstd"], security_protocol="SASL_SSL", static_membership=True) - @parametrize(version=str(LATEST_2_3), compression_types=["none"]) - @parametrize(version=str(LATEST_2_3), compression_types=["zstd"], security_protocol="SASL_SSL") - @parametrize(version=str(LATEST_2_2), compression_types=["none"]) - @parametrize(version=str(LATEST_2_2), compression_types=["zstd"], security_protocol="SASL_SSL") - @parametrize(version=str(LATEST_2_1), compression_types=["none"]) - @parametrize(version=str(LATEST_2_1), compression_types=["lz4"], security_protocol="SASL_SSL") - @parametrize(version=str(LATEST_2_0), compression_types=["none"]) - @parametrize(version=str(LATEST_2_0), compression_types=["snappy"], security_protocol="SASL_SSL") - @parametrize(version=str(LATEST_1_1), compression_types=["none"]) - @parametrize(version=str(LATEST_1_1), compression_types=["lz4"], security_protocol="SASL_SSL") def test_upgrade_and_downgrade(self, version, compression_types, security_protocol="PLAINTEXT", static_membership=False): """Test upgrade and downgrade of Kafka cluster from old versions to the current version diff --git a/tests/kafkatest/tests/core/upgrade_test.py b/tests/kafkatest/tests/core/upgrade_test.py index f4548b3c87..279579363b 100644 --- a/tests/kafkatest/tests/core/upgrade_test.py +++ b/tests/kafkatest/tests/core/upgrade_test.py @@ -24,10 +24,9 @@ from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int from kafkatest.utils.remote_account import java_version -from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10, LATEST_0_10_0, LATEST_0_10_1, LATEST_0_10_2, \ - LATEST_0_11_0, LATEST_1_0, LATEST_1_1, LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, \ - LATEST_2_6, LATEST_2_7, LATEST_2_8, LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, LATEST_3_7, LATEST_3_8, V_0_11_0_0, V_2_8_0, V_3_0_0, DEV_BRANCH, KafkaVersion +from kafkatest.version import LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \ + LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, \ + LATEST_3_7, LATEST_3_8, V_2_8_0, DEV_BRANCH, KafkaVersion from kafkatest.services.kafka.util import new_jdk_not_supported class TestUpgrade(ProduceConsumeValidateTest): @@ -86,9 +85,6 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None): if to_message_format_version is None: del node.config[config_property.MESSAGE_FORMAT_VERSION] else: - # older message formats are not supported with IBP 3.0 or higher - if to_message_format_version < V_0_11_0_0: - node.config[config_property.INTER_BROKER_PROTOCOL_VERSION] = str(V_2_8_0) node.config[config_property.MESSAGE_FORMAT_VERSION] = to_message_format_version self.kafka.start_node(node) self.wait_until_rejoin() @@ -134,38 +130,6 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None): @parametrize(from_kafka_version=str(LATEST_2_5), to_message_format_version=None, compression_types=["zstd"]) @parametrize(from_kafka_version=str(LATEST_2_4), to_message_format_version=None, compression_types=["none"]) @parametrize(from_kafka_version=str(LATEST_2_4), to_message_format_version=None, compression_types=["zstd"]) - @parametrize(from_kafka_version=str(LATEST_2_3), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_2_3), to_message_format_version=None, compression_types=["zstd"]) - @parametrize(from_kafka_version=str(LATEST_2_2), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_2_2), to_message_format_version=None, compression_types=["zstd"]) - @parametrize(from_kafka_version=str(LATEST_2_1), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_2_1), to_message_format_version=None, compression_types=["lz4"]) - @parametrize(from_kafka_version=str(LATEST_2_0), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_2_0), to_message_format_version=None, compression_types=["snappy"]) - @parametrize(from_kafka_version=str(LATEST_1_1), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_1_1), to_message_format_version=None, compression_types=["lz4"]) - @parametrize(from_kafka_version=str(LATEST_1_0), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_1_0), to_message_format_version=None, compression_types=["snappy"]) - @parametrize(from_kafka_version=str(LATEST_0_11_0), to_message_format_version=None, compression_types=["gzip"]) - @parametrize(from_kafka_version=str(LATEST_0_11_0), to_message_format_version=None, compression_types=["lz4"]) - @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=str(LATEST_0_9), compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=str(LATEST_0_10), compression_types=["snappy"]) - @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_0_10_2), to_message_format_version=None, compression_types=["lz4"]) - @parametrize(from_kafka_version=str(LATEST_0_10_1), to_message_format_version=None, compression_types=["lz4"]) - @parametrize(from_kafka_version=str(LATEST_0_10_1), to_message_format_version=None, compression_types=["snappy"]) - @parametrize(from_kafka_version=str(LATEST_0_10_0), to_message_format_version=None, compression_types=["snappy"]) - @parametrize(from_kafka_version=str(LATEST_0_10_0), to_message_format_version=None, compression_types=["lz4"]) - @cluster(num_nodes=7) - @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["none"], security_protocol="SASL_SSL") - @cluster(num_nodes=6) - @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["snappy"]) - @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["lz4"]) - @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=str(LATEST_0_9), compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=str(LATEST_0_9), compression_types=["lz4"]) - @cluster(num_nodes=7) - @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["none"]) - @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["snappy"]) def test_upgrade(self, from_kafka_version, to_message_format_version, compression_types, security_protocol="PLAINTEXT"): """Test upgrade of Kafka broker cluster from various versions to the current version @@ -213,12 +177,7 @@ def test_upgrade(self, from_kafka_version, to_message_format_version, compressio compression_types=compression_types, version=KafkaVersion(from_kafka_version)) - if from_kafka_version <= LATEST_0_10_0: - assert self.kafka.cluster_id() is None - - # With older message formats before KIP-101, message loss may occur due to truncation - # after leader change. Tolerate limited data loss for this case to avoid transient test failures. - self.may_truncate_acked_records = False if from_kafka_version >= V_0_11_0_0 else True + self.may_truncate_acked_records = False new_consumer = fromKafkaVersion.consumer_supports_bootstrap_server() # TODO - reduce the timeout From a954ad1c67ce213d40cb20c64b90b39e451645d9 Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Sun, 1 Sep 2024 21:13:40 +0800 Subject: [PATCH 073/123] KAFKA-17331 Throw unsupported version exception if the server does NOT support EarliestLocalSpec and LatestTieredSpec (#16873) Add the version check to server side for the specific timestamp: - the version must be >=8 if timestamp=-4L - the version must be >=9 if timestamp=-5L Reviewers: Chia-Ping Tsai --- .../src/main/scala/kafka/log/UnifiedLog.scala | 2 - .../main/scala/kafka/server/KafkaApis.scala | 70 ++++++++++++------- .../unit/kafka/server/KafkaApisTest.scala | 49 +++++++++++++ .../kafka/server/ListOffsetsRequestTest.scala | 19 +++-- .../unit/kafka/server/LogOffsetTest.scala | 18 +++-- 5 files changed, 115 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 092b8c92aa..450f18b4ef 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -1400,8 +1400,6 @@ class UnifiedLog(@volatile var logStartOffset: Long, startIndex = offsetTimeArray.length - 1 case ListOffsetsRequest.EARLIEST_TIMESTAMP => startIndex = 0 - case ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP => - startIndex = 0 case _ => var isFound = false debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2))) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index cdd2d06ee2..f68fec5695 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -84,7 +84,7 @@ import java.util.concurrent.{CompletableFuture, ConcurrentHashMap} import java.util.{Collections, Optional, OptionalInt} import scala.annotation.nowarn import scala.collection.mutable.ArrayBuffer -import scala.collection.{Map, Seq, Set, mutable} +import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} @@ -1111,35 +1111,43 @@ class KafkaApis(val requestChannel: RequestChannel, val responseTopics = authorizedRequestInfo.map { topic => val responsePartitions = topic.partitions.asScala.map { partition => - val topicPartition = new TopicPartition(topic.name, partition.partitionIndex) - - try { - val offsets = replicaManager.legacyFetchOffsetsForTimestamp( - topicPartition = topicPartition, - timestamp = partition.timestamp, - maxNumOffsets = partition.maxNumOffsets, - isFromConsumer = offsetRequest.replicaId == ListOffsetsRequest.CONSUMER_REPLICA_ID, - fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetsRequest.DEBUGGING_REPLICA_ID) + if (partition.timestamp() < ListOffsetsRequest.EARLIEST_TIMESTAMP) { + // Negative timestamps are reserved for some functions. + // For v0 requests, negative timestamps only support LATEST_TIMESTAMP (-1) and EARLIEST_TIMESTAMP (-2). new ListOffsetsPartitionResponse() .setPartitionIndex(partition.partitionIndex) - .setErrorCode(Errors.NONE.code) - .setOldStyleOffsets(offsets.map(JLong.valueOf).asJava) - } catch { - // NOTE: UnknownTopicOrPartitionException and NotLeaderOrFollowerException are special cases since these error messages - // are typically transient and there is no value in logging the entire stack trace for the same - case e @ (_ : UnknownTopicOrPartitionException | - _ : NotLeaderOrFollowerException | - _ : KafkaStorageException) => - debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( - correlationId, clientId, topicPartition, e.getMessage)) - new ListOffsetsPartitionResponse() - .setPartitionIndex(partition.partitionIndex) - .setErrorCode(Errors.forException(e).code) - case e: Throwable => - error("Error while responding to offset request", e) + .setErrorCode(Errors.UNSUPPORTED_VERSION.code) + } else { + val topicPartition = new TopicPartition(topic.name, partition.partitionIndex) + + try { + val offsets = replicaManager.legacyFetchOffsetsForTimestamp( + topicPartition = topicPartition, + timestamp = partition.timestamp, + maxNumOffsets = partition.maxNumOffsets, + isFromConsumer = offsetRequest.replicaId == ListOffsetsRequest.CONSUMER_REPLICA_ID, + fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetsRequest.DEBUGGING_REPLICA_ID) new ListOffsetsPartitionResponse() .setPartitionIndex(partition.partitionIndex) - .setErrorCode(Errors.forException(e).code) + .setErrorCode(Errors.NONE.code) + .setOldStyleOffsets(offsets.map(JLong.valueOf).asJava) + } catch { + // NOTE: UnknownTopicOrPartitionException and NotLeaderOrFollowerException are special cases since these error messages + // are typically transient and there is no value in logging the entire stack trace for the same + case e @ (_ : UnknownTopicOrPartitionException | + _ : NotLeaderOrFollowerException | + _ : KafkaStorageException) => + debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + correlationId, clientId, topicPartition, e.getMessage)) + new ListOffsetsPartitionResponse() + .setPartitionIndex(partition.partitionIndex) + .setErrorCode(Errors.forException(e).code) + case e: Throwable => + error("Error while responding to offset request", e) + new ListOffsetsPartitionResponse() + .setPartitionIndex(partition.partitionIndex) + .setErrorCode(Errors.forException(e).code) + } } } new ListOffsetsTopicResponse().setName(topic.name).setPartitions(responsePartitions.asJava) @@ -1152,6 +1160,13 @@ class KafkaApis(val requestChannel: RequestChannel, val clientId = request.header.clientId val offsetRequest = request.body[ListOffsetsRequest] val version = request.header.apiVersion + val timestampMinSupportedVersion = immutable.Map[Long, Short]( + ListOffsetsRequest.EARLIEST_TIMESTAMP -> 1.toShort, + ListOffsetsRequest.LATEST_TIMESTAMP -> 1.toShort, + ListOffsetsRequest.MAX_TIMESTAMP -> 7.toShort, + ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP -> 8.toShort, + ListOffsetsRequest.LATEST_TIERED_TIMESTAMP -> 9.toShort + ) def buildErrorResponse(e: Errors, partition: ListOffsetsPartition): ListOffsetsPartitionResponse = { new ListOffsetsPartitionResponse() @@ -1178,6 +1193,9 @@ class KafkaApis(val requestChannel: RequestChannel, debug(s"OffsetRequest with correlation id $correlationId from client $clientId on partition $topicPartition " + s"failed because the partition is duplicated in the request.") buildErrorResponse(Errors.INVALID_REQUEST, partition) + } else if (partition.timestamp() < 0 && + (!timestampMinSupportedVersion.contains(partition.timestamp()) || version < timestampMinSupportedVersion(partition.timestamp()))) { + buildErrorResponse(Errors.UNSUPPORTED_VERSION, partition) } else { try { val fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetsRequest.DEBUGGING_REPLICA_ID diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index be1afc3494..7487d05ede 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -4039,6 +4039,31 @@ class KafkaApisTest extends Logging { testConsumerListOffsetLatest(IsolationLevel.READ_COMMITTED) } + @Test + def testListOffsetMaxTimestampWithUnsupportedVersion(): Unit = { + testConsumerListOffsetWithUnsupportedVersion(ListOffsetsRequest.MAX_TIMESTAMP, 6) + } + + @Test + def testListOffsetEarliestLocalTimestampWithUnsupportedVersion(): Unit = { + testConsumerListOffsetWithUnsupportedVersion(ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, 7) + } + + @Test + def testListOffsetLatestTieredTimestampWithUnsupportedVersion(): Unit = { + testConsumerListOffsetWithUnsupportedVersion(ListOffsetsRequest.LATEST_TIERED_TIMESTAMP, 8) + } + + @Test + def testListOffsetNegativeTimestampWithZeroVersion(): Unit = { + testConsumerListOffsetWithUnsupportedVersion(-3, 0) + } + + @Test + def testListOffsetNegativeTimestampWithOneOrAboveVersion(): Unit = { + testConsumerListOffsetWithUnsupportedVersion(-6, 1) + } + /** * Verifies that the metadata response is correct if the broker listeners are inconsistent (i.e. one broker has * more listeners than another) and the request is sent on the listener that exists in both brokers. @@ -6152,6 +6177,30 @@ class KafkaApisTest extends Logging { verifyNoThrottling[MetadataResponse](requestChannelRequest) } + private def testConsumerListOffsetWithUnsupportedVersion(timestamp: Long, version: Short): Unit = { + val tp = new TopicPartition("foo", 0) + val targetTimes = List(new ListOffsetsTopic() + .setName(tp.topic) + .setPartitions(List(new ListOffsetsPartition() + .setPartitionIndex(tp.partition) + .setTimestamp(timestamp)).asJava)).asJava + + val data = new ListOffsetsRequestData().setTopics(targetTimes).setReplicaId(ListOffsetsRequest.CONSUMER_REPLICA_ID) + val listOffsetRequest = ListOffsetsRequest.parse(MessageUtil.toByteBuffer(data, version), version) + val request = buildRequest(listOffsetRequest) + + kafkaApis = createKafkaApis() + kafkaApis.handleListOffsetRequest(request) + + val response = verifyNoThrottling[ListOffsetsResponse](request) + val partitionDataOptional = response.topics.asScala.find(_.name == tp.topic).get + .partitions.asScala.find(_.partitionIndex == tp.partition) + assertTrue(partitionDataOptional.isDefined) + + val partitionData = partitionDataOptional.get + assertEquals(Errors.UNSUPPORTED_VERSION.code, partitionData.errorCode) + } + private def testConsumerListOffsetLatest(isolationLevel: IsolationLevel): Unit = { val tp = new TopicPartition("foo", 0) val latestOffset = 15L diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala index 03585be97c..9c40efd29e 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala @@ -258,24 +258,33 @@ class ListOffsetsRequestTest extends BaseRequestTest { if (version == 0) { assertEquals((-1L, -1), fetchOffsetAndEpoch(firstLeaderId, 0L, version.toShort)) assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, version.toShort)) - assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) assertEquals((10L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, version.toShort)) + assertEquals((-1L, -1, Errors.UNSUPPORTED_VERSION.code()), fetchOffsetAndEpochWithError(firstLeaderId, ListOffsetsRequest.MAX_TIMESTAMP, version.toShort)) + assertEquals((-1L, -1, Errors.UNSUPPORTED_VERSION.code()), fetchOffsetAndEpochWithError(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) } else if (version >= 1 && version <= 3) { assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, 0L, version.toShort)) assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, version.toShort)) - assertEquals((0L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) assertEquals((10L, -1), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, version.toShort)) + assertEquals((-1L, -1, Errors.UNSUPPORTED_VERSION.code()), fetchOffsetAndEpochWithError(firstLeaderId, ListOffsetsRequest.MAX_TIMESTAMP, version.toShort)) + assertEquals((-1L, -1, Errors.UNSUPPORTED_VERSION.code()), fetchOffsetAndEpochWithError(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) } else if (version >= 4 && version <= 6) { assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, 0L, version.toShort)) assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, version.toShort)) - assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) assertEquals((10L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, version.toShort)) - } else if (version >= 7) { + assertEquals((-1L, -1, Errors.UNSUPPORTED_VERSION.code()), fetchOffsetAndEpochWithError(firstLeaderId, ListOffsetsRequest.MAX_TIMESTAMP, version.toShort)) + assertEquals((-1L, -1, Errors.UNSUPPORTED_VERSION.code()), fetchOffsetAndEpochWithError(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) + } else if (version == 7) { assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, 0L, version.toShort)) assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, version.toShort)) - assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) assertEquals((10L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, version.toShort)) assertEquals((9L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.MAX_TIMESTAMP, version.toShort)) + assertEquals((-1L, -1, Errors.UNSUPPORTED_VERSION.code()), fetchOffsetAndEpochWithError(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) + } else if (version >= 8) { + assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, 0L, version.toShort)) + assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_TIMESTAMP, version.toShort)) + assertEquals((10L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.LATEST_TIMESTAMP, version.toShort)) + assertEquals((9L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.MAX_TIMESTAMP, version.toShort)) + assertEquals((0L, 0), fetchOffsetAndEpoch(firstLeaderId, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP, version.toShort)) } } } diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 177f59221a..3bcdf2b259 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -250,17 +250,15 @@ class LogOffsetTest extends BaseRequestTest { log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0) log.flush(false) - for (timestamp <- Seq(ListOffsetsRequest.EARLIEST_TIMESTAMP, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP)) { - val offsets = log.legacyFetchOffsetsBefore(timestamp, 10) - assertEquals(Seq(0L), offsets) + val offsets = log.legacyFetchOffsetsBefore(ListOffsetsRequest.EARLIEST_TIMESTAMP, 10) + assertEquals(Seq(0L), offsets) - TestUtils.waitUntilTrue(() => isLeaderLocalOnBroker(topic, topicPartition.partition, broker), - "Leader should be elected") - val request = ListOffsetsRequest.Builder.forReplica(0, 0) - .setTargetTimes(buildTargetTimes(topicPartition, timestamp, 10).asJava).build() - val consumerOffsets = findPartition(sendListOffsetsRequest(request).topics.asScala, topicPartition).oldStyleOffsets.asScala - assertEquals(Seq(0L), consumerOffsets) - } + TestUtils.waitUntilTrue(() => isLeaderLocalOnBroker(topic, topicPartition.partition, broker), + "Leader should be elected") + val request = ListOffsetsRequest.Builder.forReplica(0, 0) + .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.EARLIEST_TIMESTAMP, 10).asJava).build() + val consumerOffsets = findPartition(sendListOffsetsRequest(request).topics.asScala, topicPartition).oldStyleOffsets.asScala + assertEquals(Seq(0L), consumerOffsets) } /* We test that `fetchOffsetsBefore` works correctly if `LogSegment.size` changes after each invocation (simulating From 28f5ff6039be9c75f7d53e4119613acd4517ce55 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Wed, 4 Sep 2024 08:11:46 +0900 Subject: [PATCH 074/123] KAFKA-17412: add doc for `unclean.leader.election.enable` in KRaft (#17051) Reviewers: Colin P. McCabe --- .../java/org/apache/kafka/common/config/TopicConfig.java | 4 +++- docs/upgrade.html | 3 +++ .../org/apache/kafka/server/config/ReplicationConfigs.java | 5 ++++- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index b3cea19c55..1437577ed0 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -165,7 +165,9 @@ public class TopicConfig { public static final String UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG = "unclean.leader.election.enable"; public static final String UNCLEAN_LEADER_ELECTION_ENABLE_DOC = "Indicates whether to enable replicas " + "not in the ISR set to be elected as leader as a last resort, even though doing so may result in data " + - "loss."; + "loss.

    Note: In KRaft mode, when enabling this config dynamically, it needs to wait for the unclean leader election" + + "thread to trigger election periodically (default is 5 minutes). Please run `kafka-leader-election.sh` with `unclean` option " + + "to trigger the unclean leader election immediately if needed.

    "; public static final String MIN_IN_SYNC_REPLICAS_CONFIG = "min.insync.replicas"; public static final String MIN_IN_SYNC_REPLICAS_DOC = "When a producer sets acks to \"all\" (or \"-1\"), " + diff --git a/docs/upgrade.html b/docs/upgrade.html index 95e94515a2..049a8b4f9a 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -32,6 +32,9 @@
    Notable changes in 3 See KIP-950 for more details.
  • Tiered storage quota is implemented. Users can set an "upper bound" on the rate at which logs are copied/read to/from the remote storage. See KIP-956 for more details.
  • +
  • unclean.leader.election.enable config is supported in KRaft. Compared with ZK mode, there is one behavior change in KRaft mode + when dynamically enabling unclean.leader.election.enable config. Please check + here for more details.
  • Upgrading to 3.8.0 from any version 0.8.x through 3.7.x

    diff --git a/server/src/main/java/org/apache/kafka/server/config/ReplicationConfigs.java b/server/src/main/java/org/apache/kafka/server/config/ReplicationConfigs.java index 3bce284e47..5106b81025 100644 --- a/server/src/main/java/org/apache/kafka/server/config/ReplicationConfigs.java +++ b/server/src/main/java/org/apache/kafka/server/config/ReplicationConfigs.java @@ -122,7 +122,10 @@ public class ReplicationConfigs { public static final String UNCLEAN_LEADER_ELECTION_INTERVAL_MS_DOC = "The frequency with which the controller checks if it should perform an unclean leader election for leaderless partitions."; public static final String UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG); - public static final String UNCLEAN_LEADER_ELECTION_ENABLE_DOC = "Indicates whether to enable replicas not in the ISR set to be elected as leader as a last resort, even though doing so may result in data loss"; + public static final String UNCLEAN_LEADER_ELECTION_ENABLE_DOC = "Indicates whether to enable replicas not in the ISR set to be elected as leader as a last resort, even though doing so may result in data loss" + + "

    Note: In KRaft mode, when enabling this config dynamically, it needs to wait for the unclean leader election " + + "thread to trigger election periodically (default is 5 minutes). Please run `kafka-leader-election.sh` with `unclean` option " + + "to trigger the unclean leader election immediately if needed.

    "; public static final String INTER_BROKER_PROTOCOL_VERSION_CONFIG = "inter.broker.protocol.version"; public static final String INTER_BROKER_PROTOCOL_VERSION_DEFAULT = MetadataVersion.latestProduction().version(); From 4756402f75fef1a656e15602553ec6177d23baff Mon Sep 17 00:00:00 2001 From: Justine Olshan Date: Mon, 12 Aug 2024 09:51:04 -0700 Subject: [PATCH 075/123] MINOR: Reduce log levels for transactions_mixed_versions_test with 3.2 due to bug in that version (#16787) https://github.com/apache/kafka/commit/7496e6243410ca851f4b8502270cf4222bd89a2b fixed an error that caused an exception to be thrown on broker startup when debug logs were on. This made it to every version except 3.2. The Kraft upgrade tests totally turn off debug logs, but I think we only need to remove them for the broken version. Note: this bug is also present in 3.1, but there is no logging on startup like in subsequent versions. Reviewers: Chia-Ping Tsai , David Jacot --- tests/kafkatest/tests/core/transactions_mixed_versions_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/kafkatest/tests/core/transactions_mixed_versions_test.py b/tests/kafkatest/tests/core/transactions_mixed_versions_test.py index e573cacb9b..77041b15d3 100644 --- a/tests/kafkatest/tests/core/transactions_mixed_versions_test.py +++ b/tests/kafkatest/tests/core/transactions_mixed_versions_test.py @@ -198,7 +198,7 @@ def test_transactions_mixed_versions(self, old_kafka_version, metadata_quorum=qu self.kafka.interbroker_security_protocol = security_protocol self.kafka.logs["kafka_data_1"]["collect_default"] = True self.kafka.logs["kafka_data_2"]["collect_default"] = True - self.kafka.logs["kafka_operational_logs_debug"]["collect_default"] = True + self.kafka.logs["kafka_operational_logs_debug"]["collect_default"] = False if old_kafka_version == str(LATEST_3_2) else True self.setup_topics() self.kafka.start() From 14e1ebee9e79f9f5950bbc0a730442a6ff97c160 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Wed, 4 Sep 2024 15:09:03 +0800 Subject: [PATCH 076/123] KAFKA-17454 Fix failed transactions_mixed_versions_test.py when running with 3.2 (#17067) why https://github.com/apache/kafka/commit/df04887ba597941674a28d54bd35202bbd21631e does not fix it? The fix of https://github.com/apache/kafka/commit/df04887ba597941674a28d54bd35202bbd21631e is to NOT collect the log from path `/mnt/kafka/kafka-operational-logs/debug/xxxx.log`if the task is successful. It does not change the log level. see ducktape https://github.com/confluentinc/ducktape/blob/b2ad7693f2750a2286a5099882ecc774fd7138b2/ducktape/tests/test.py#L181 why https://github.com/apache/kafka/commit/df04887ba597941674a28d54bd35202bbd21631e does not see the error of "sort" https://github.com/apache/kafka/commit/df04887ba597941674a28d54bd35202bbd21631e does NOT show the error since the number of features is only "one" (only metadata.version). Hence, the bug is not triggered as it does not need to "sort". Now, we have two features - metadata.version and krafe.version - so the sort is executed and then we see the "hello bug" why we should change the kafka.log_level to INFO? the template of log4j.properties is controlled by `log_level` (https://github.com/apache/kafka/blob/trunk/tests/kafkatest/services/kafka/templates/log4j.properties#L16), and the bug happens in writing debug message (https://github.com/apache/kafka/blob/e4ca066680296ea29d443efb626baecc837083f6/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala#L274). Hence, changing the log level to DEBUG can avoid triggering the bug. Reviewers: Justine Olshan , Chia-Ping Tsai --- tests/kafkatest/tests/core/transactions_mixed_versions_test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/kafkatest/tests/core/transactions_mixed_versions_test.py b/tests/kafkatest/tests/core/transactions_mixed_versions_test.py index 77041b15d3..a0f298b1fd 100644 --- a/tests/kafkatest/tests/core/transactions_mixed_versions_test.py +++ b/tests/kafkatest/tests/core/transactions_mixed_versions_test.py @@ -199,6 +199,9 @@ def test_transactions_mixed_versions(self, old_kafka_version, metadata_quorum=qu self.kafka.logs["kafka_data_1"]["collect_default"] = True self.kafka.logs["kafka_data_2"]["collect_default"] = True self.kafka.logs["kafka_operational_logs_debug"]["collect_default"] = False if old_kafka_version == str(LATEST_3_2) else True + # The 3.2.3 release does not include KAFKA-14259, which will cause an exception when writing debug messages. + # Therefore, changing the log level to INFO can avoid triggering the bug. + self.kafka.log_level = "INFO" if old_kafka_version == str(LATEST_3_2) else "DEBUG" self.setup_topics() self.kafka.start() From 3cabf333cedf556b94f799c542b22c699a1476c9 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Thu, 5 Sep 2024 19:26:23 +0900 Subject: [PATCH 077/123] MINOR: Update doc for tiered storage GA (#17088) Reviewers: Satish Duggana --- docs/ops.html | 7 +++---- docs/upgrade.html | 14 ++++++++++---- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index 5d39899c44..8f8b0b95a4 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4222,8 +4222,6 @@

    < Please check KIP-405 for more information.

    -

    Note: Tiered storage is considered as an early access feature, and is not recommended for use in production environments

    -

    Configuration

    Broker Configurations
    @@ -4365,14 +4363,15 @@

    Limitations

    -

    While the early access release of Tiered Storage offers the opportunity to try out this new feature, it is important to be aware of the following limitations: +

    While the Tiered Storage works for most use cases, it is still important to be aware of the following limitations:

    • No support for compacted topics
    • Deleting tiered storage enabled topics is required before disabling tiered storage at the broker level
    • Admin actions related to tiered storage feature are only supported on clients from version 3.0 onwards
    • +
    • No support for log segments missing producer snapshot file. It can happen when topic is created before v2.8.0.
    -

    For more information, please check Tiered Storage Early Access Release Note. +

    For more information, please check Kafka Tiered Storage GA Release Notes.

    diff --git a/docs/upgrade.html b/docs/upgrade.html index 049a8b4f9a..faa4862154 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -28,13 +28,19 @@

    Notable changes in 3 For example, this could be done via the KAFKA_OPTS environment variable like follows: export KAFKA_OPTS="-DZstdTempFolder=/opt/kafka/tmp -Dorg.xerial.snappy.tempdir=/opt/kafka/tmp". This is a known issue for version 3.8.0 as well. -
  • In KRaft mode, the tiered storage feature can be dynamically disabled and then re-enabled on topic level. - See KIP-950 for more details.
  • -
  • Tiered storage quota is implemented. Users can set an "upper bound" on the rate at which logs are copied/read to/from the remote storage. - See KIP-956 for more details.
  • unclean.leader.election.enable config is supported in KRaft. Compared with ZK mode, there is one behavior change in KRaft mode when dynamically enabling unclean.leader.election.enable config. Please check here for more details.
  • +
  • Tiered storage is now a production ready feature. You can check + Kafka Tiered Storage GA Release Notes for more details. + The below enhancements are added in this release. +
      +
    • In KRaft mode, the tiered storage feature can be dynamically disabled and then re-enabled on topic level. + See KIP-950 for more details.
    • +
    • With the tiered storage quota feature, users can define a maximum limit on the rate at which log segments are transferred to or retrieved from the remote storage. + See KIP-956 for more details.
    • +
    +
  • Upgrading to 3.8.0 from any version 0.8.x through 3.7.x

    From 74d55ca6393ced994caf900a72e558c1535109fc Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Thu, 5 Sep 2024 17:42:52 +0200 Subject: [PATCH 078/123] KAFKA-16448: Add timestamp to error handler context (#17054) Part of KIP-1033. Co-authored-by: Dabz Co-authored-by: loicgreffier Reviewers: Matthias J. Sax --- .../streams/errors/ErrorHandlerContext.java | 33 +++++++++++++++++++ .../internals/DefaultErrorHandlerContext.java | 11 ++++++- .../processor/internals/ProcessorNode.java | 3 +- .../internals/RecordCollectorImpl.java | 6 ++-- .../internals/RecordDeserializer.java | 3 +- .../processor/internals/StreamTask.java | 3 +- ...essingExceptionHandlerIntegrationTest.java | 29 ++++++++-------- .../internals/ProcessorNodeTest.java | 1 + .../internals/RecordCollectorTest.java | 1 + .../internals/RecordDeserializerTest.java | 1 + 10 files changed, 72 insertions(+), 19 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java index 6c5e4f1959..af67c8f03a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java @@ -18,8 +18,14 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; +import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; + /** * This interface allows user code to inspect the context of a record that has failed processing. @@ -102,4 +108,31 @@ public interface ErrorHandlerContext { * @return the task ID */ TaskId taskId(); + + /** + * Return the current timestamp. + * + *

    If it is triggered while processing a record streamed from the source processor, + * timestamp is defined as the timestamp of the current input record; the timestamp is extracted from + * {@link org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} by {@link TimestampExtractor}. + * Note, that an upstream {@link Processor} might have set a new timestamp by calling + * {@link ProcessorContext#forward(Record) forward(record.withTimestamp(...))}. + * In particular, some Kafka Streams DSL operators set result record timestamps explicitly, + * to guarantee deterministic results. + * + *

    If it is triggered while processing a record generated not from the source processor (for example, + * if this method is invoked from the punctuate call): + *

      + *
    • In case of {@link PunctuationType#STREAM_TIME} timestamp is defined as the current task's stream time, + * which is defined as the largest timestamp of any record processed by the task + *
    • In case of {@link PunctuationType#WALL_CLOCK_TIME} timestamp is defined the current system time + *
    + * + *

    If it is triggered from a deserialization failure, timestamp is defined as the timestamp of the + * current rawRecord + * {@link org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} + * + * @return the timestamp + */ + long timestamp(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java index 77500ce3c3..fc44e9c95f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java @@ -33,6 +33,8 @@ public class DefaultErrorHandlerContext implements ErrorHandlerContext { private final Headers headers; private final String processorNodeId; private final TaskId taskId; + + private final long timestamp; private ProcessorContext processorContext; public DefaultErrorHandlerContext(final ProcessorContext processorContext, @@ -41,7 +43,8 @@ public DefaultErrorHandlerContext(final ProcessorContext processorContext, final long offset, final Headers headers, final String processorNodeId, - final TaskId taskId) { + final TaskId taskId, + final long timestamp) { this.topic = topic; this.partition = partition; this.offset = offset; @@ -49,6 +52,7 @@ public DefaultErrorHandlerContext(final ProcessorContext processorContext, this.processorNodeId = processorNodeId; this.taskId = taskId; this.processorContext = processorContext; + this.timestamp = timestamp; } @Override @@ -81,6 +85,11 @@ public TaskId taskId() { return taskId; } + @Override + public long timestamp() { + return timestamp; + } + @Override public String toString() { // we do exclude headers on purpose, to not accidentally log user data diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 5df1a1bd00..2b945f2da2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -211,7 +211,8 @@ public void process(final Record record) { internalProcessorContext.offset(), internalProcessorContext.headers(), internalProcessorContext.currentNode().name(), - internalProcessorContext.taskId()); + internalProcessorContext.taskId(), + internalProcessorContext.timestamp()); final ProcessingExceptionHandler.ProcessingHandlerResponse response; try { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index ced6fdef77..7525513ea2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -308,7 +308,8 @@ private void handleException(final ProductionExceptionHandler.Serializati context.recordContext().offset(), context.recordContext().headers(), processorNodeId, - taskId + taskId, + context.recordContext().timestamp() ); final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); @@ -405,7 +406,8 @@ private void recordSendError(final String topic, context.recordContext().offset(), context.recordContext().headers(), processorNodeId, - taskId + taskId, + context.recordContext().timestamp() ); final ProductionExceptionHandlerResponse response; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 5fc03352ec..aefa15da66 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -92,7 +92,8 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa rawRecord.offset(), rawRecord.headers(), sourceNodeName, - processorContext.taskId()); + processorContext.taskId(), + rawRecord.timestamp()); final DeserializationHandlerResponse response; try { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index f08cfa7fd6..6a4e97e470 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -946,7 +946,8 @@ record = null; recordContext.offset(), recordContext.headers(), node.name(), - id() + id(), + recordContext.timestamp() ); final ProcessingExceptionHandler.ProcessingHandlerResponse response; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index d0c3231055..d23b9ba0a6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -61,6 +61,8 @@ public class ProcessingExceptionHandlerIntegrationTest { private final String threadId = Thread.currentThread().getName(); + private static final Instant TIMESTAMP = Instant.now(); + @Test public void shouldFailWhenProcessingExceptionOccursIfExceptionHandlerReturnsFail() { final List> events = Arrays.asList( @@ -71,7 +73,7 @@ public void shouldFailWhenProcessingExceptionOccursIfExceptionHandlerReturnsFail ); final List> expectedProcessedRecords = Collections.singletonList( - new KeyValueTimestamp<>("ID123-1", "ID123-A1", 0) + new KeyValueTimestamp<>("ID123-1", "ID123-A1", TIMESTAMP.toEpochMilli()) ); final MockProcessorSupplier processor = new MockProcessorSupplier<>(); @@ -90,7 +92,7 @@ public void shouldFailWhenProcessingExceptionOccursIfExceptionHandlerReturnsFail final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); final StreamsException exception = assertThrows(StreamsException.class, - () -> inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO)); + () -> inputTopic.pipeKeyValueList(events, TIMESTAMP, Duration.ZERO)); assertTrue(exception.getMessage().contains("Exception caught in process. " + "taskId=0_0, processor=KSTREAM-SOURCE-0000000000, topic=TOPIC_NAME, " @@ -118,10 +120,10 @@ public void shouldContinueWhenProcessingExceptionOccursIfExceptionHandlerReturns ); final List> expectedProcessedRecords = Arrays.asList( - new KeyValueTimestamp<>("ID123-1", "ID123-A1", 0), - new KeyValueTimestamp<>("ID123-3", "ID123-A3", 0), - new KeyValueTimestamp<>("ID123-4", "ID123-A4", 0), - new KeyValueTimestamp<>("ID123-6", "ID123-A6", 0) + new KeyValueTimestamp<>("ID123-1", "ID123-A1", TIMESTAMP.toEpochMilli()), + new KeyValueTimestamp<>("ID123-3", "ID123-A3", TIMESTAMP.toEpochMilli()), + new KeyValueTimestamp<>("ID123-4", "ID123-A4", TIMESTAMP.toEpochMilli()), + new KeyValueTimestamp<>("ID123-6", "ID123-A6", TIMESTAMP.toEpochMilli()) ); final MockProcessorSupplier processor = new MockProcessorSupplier<>(); @@ -138,7 +140,7 @@ public void shouldContinueWhenProcessingExceptionOccursIfExceptionHandlerReturns try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); - inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); + inputTopic.pipeKeyValueList(events, TIMESTAMP, Duration.ZERO); assertEquals(expectedProcessedRecords.size(), processor.theCapturedProcessor().processed().size()); assertIterableEquals(expectedProcessedRecords, processor.theCapturedProcessor().processed()); @@ -176,10 +178,10 @@ public void shouldStopOnFailedProcessorWhenProcessingExceptionOccursInFailProces try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); isExecuted.set(false); - inputTopic.pipeInput(event.key, event.value, Instant.EPOCH); + inputTopic.pipeInput(event.key, event.value, TIMESTAMP); assertTrue(isExecuted.get()); isExecuted.set(false); - final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); + final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, TIMESTAMP)); assertTrue(e.getMessage().contains("Exception caught in process. " + "taskId=0_0, processor=KSTREAM-SOURCE-0000000000, topic=TOPIC_NAME, " + "partition=0, offset=1")); @@ -212,10 +214,10 @@ public void shouldStopOnFailedProcessorWhenProcessingExceptionOccursInContinuePr try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); isExecuted.set(false); - inputTopic.pipeInput(event.key, event.value, Instant.EPOCH); + inputTopic.pipeInput(event.key, event.value, TIMESTAMP); assertTrue(isExecuted.get()); isExecuted.set(false); - inputTopic.pipeInput(eventFalse.key, eventFalse.value, Instant.EPOCH); + inputTopic.pipeInput(eventFalse.key, eventFalse.value, TIMESTAMP); assertFalse(isExecuted.get()); } } @@ -245,7 +247,7 @@ public void shouldStopProcessingWhenProcessingExceptionHandlerReturnsNull() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); isExecuted.set(false); - inputTopic.pipeInput(event.key, event.value, Instant.EPOCH); + inputTopic.pipeInput(event.key, event.value, TIMESTAMP); assertTrue(isExecuted.get()); isExecuted.set(false); final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); @@ -281,7 +283,7 @@ public void shouldStopProcessingWhenFatalUserExceptionProcessingExceptionHandler try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); isExecuted.set(false); - inputTopic.pipeInput(event.key, event.value, Instant.EPOCH); + inputTopic.pipeInput(event.key, event.value, TIMESTAMP); assertTrue(isExecuted.get()); isExecuted.set(false); final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); @@ -328,6 +330,7 @@ private static void assertProcessingExceptionHandlerInputs(final ErrorHandlerCon assertTrue(Arrays.asList("ID123-A2", "ID123-A5").contains((String) record.value())); assertEquals("TOPIC_NAME", context.topic()); assertEquals("KSTREAM-PROCESSOR-0000000003", context.processorNodeId()); + assertEquals(TIMESTAMP.toEpochMilli(), context.timestamp()); assertTrue(exception.getMessage().contains("Exception should be handled by processing exception handler")); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 9fe9244e0a..7f4e2d0849 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -356,6 +356,7 @@ public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHa assertEquals(internalProcessorContext.offset(), context.offset()); assertEquals(internalProcessorContext.currentNode().name(), context.processorNodeId()); assertEquals(internalProcessorContext.taskId(), context.taskId()); + assertEquals(internalProcessorContext.timestamp(), context.timestamp()); assertEquals(KEY, record.key()); assertEquals(VALUE, record.value()); assertInstanceOf(RuntimeException.class, exception); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index c4479bbcc5..9ed655332d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -1892,6 +1892,7 @@ private void assertInputs(final ErrorHandlerContext context, final Exception exc assertEquals(expectedContext.recordContext().offset(), context.offset()); assertEquals(expectedProcessorNodeId, context.processorNodeId()); assertEquals(expectedTaskId, context.taskId()); + assertEquals(expectedContext.recordContext().timestamp(), context.timestamp()); assertInstanceOf(RuntimeException.class, exception); assertEquals("KABOOM!", exception.getMessage()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index 1bca1c9e37..662af063c0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -280,6 +280,7 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, assertEquals(expectedRecord.offset(), context.offset()); assertEquals(expectedProcessorNodeId, context.processorNodeId()); assertEquals(expectedTaskId, context.taskId()); + assertEquals(expectedRecord.timestamp(), context.timestamp()); assertEquals(expectedRecord, record); assertInstanceOf(RuntimeException.class, exception); assertEquals("KABOOM!", exception.getMessage()); From d067ed0a2f1e841f7d591dc0901065f750227117 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 6 Sep 2024 13:28:53 -0400 Subject: [PATCH 079/123] KAFKA-17457 Don't allow ZK migration to start without transactions (#17094) This patch raises the minimum MetadataVersion for migrations to 3.6-IV1 (metadata transactions). This is only enforced on the controller during bootstrap (when the log is empty). If the log is not empty on controller startup, as in the case of a software upgrade, we allow the migration to continue where it left off. The broker will log an ERROR message if migrations are enabled and the IBP is not at least 3.6-IV1. Reviewers: Colin P. McCabe --- .../main/scala/kafka/server/KafkaServer.scala | 6 ++ .../KafkaServerKRaftRegistrationTest.scala | 6 +- .../kafka/zk/ZkMigrationIntegrationTest.scala | 25 ++++---- .../BrokerRegistrationRequestTest.scala | 6 +- .../ActivationRecordsGenerator.java | 5 +- .../ActivationRecordsGeneratorTest.java | 58 +++++++++---------- .../controller/QuorumControllerTest.java | 23 ++++---- 7 files changed, 70 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index b1883d2a4a..6026b566bf 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -419,6 +419,12 @@ class KafkaServer( if (config.migrationEnabled) { logger.info("Starting up additional components for ZooKeeper migration") + if (!config.interBrokerProtocolVersion.isMetadataTransactionSupported) { + logger.error("Caution! Enabled ZooKeeper migration without metadata transactions support. " + + "The controller will not allow the migration to begin. If a migration is in progress, it will be able to " + + "continue, but without the fault tolerance afforded by metadata transactions." + ) + } lifecycleManager = new BrokerLifecycleManager(config, time, s"zk-broker-${config.nodeId}-", diff --git a/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala b/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala index adcb24a3b4..2b6f05bbfc 100644 --- a/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala +++ b/core/src/test/scala/integration/kafka/server/KafkaServerKRaftRegistrationTest.scala @@ -45,7 +45,7 @@ import scala.jdk.CollectionConverters._ @ExtendWith(value = Array(classOf[ClusterTestExtensions])) class KafkaServerKRaftRegistrationTest { - @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_4_IV0, serverProperties = Array( + @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_6_IV1, serverProperties = Array( new ClusterConfigProperty(key = "inter.broker.listener.name", value = "EXTERNAL"), new ClusterConfigProperty(key = "listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), new ClusterConfigProperty(key = "advertised.listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), @@ -57,7 +57,7 @@ class KafkaServerKRaftRegistrationTest { // Bootstrap the ZK cluster ID into KRaft val kraftCluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_3_4_IV0). + setBootstrapMetadataVersion(MetadataVersion.IBP_3_6_IV1). setClusterId(clusterId). setNumBrokerNodes(0). setNumControllerNodes(1).build()) @@ -99,7 +99,7 @@ class KafkaServerKRaftRegistrationTest { val clusterId = zkCluster.clusterId() val kraftCluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_3_4_IV0). + setBootstrapMetadataVersion(MetadataVersion.IBP_3_6_IV1). setClusterId(clusterId). setNumBrokerNodes(0). setNumControllerNodes(1).build()) diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index d0107d73c2..502b138668 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -65,8 +65,7 @@ import scala.jdk.CollectionConverters._ object ZkMigrationIntegrationTest { def zkClustersForAllMigrationVersions(): java.util.List[ClusterConfig] = { Seq( - MetadataVersion.IBP_3_4_IV0, - MetadataVersion.IBP_3_5_IV2, + MetadataVersion.IBP_3_6_IV1, MetadataVersion.IBP_3_6_IV2, MetadataVersion.IBP_3_7_IV0, MetadataVersion.IBP_3_7_IV1, @@ -115,7 +114,7 @@ class ZkMigrationIntegrationTest { @ClusterTest( brokers = 3, types = Array(Type.ZK), autoStart = AutoStart.YES, - metadataVersion = MetadataVersion.IBP_3_4_IV0, + metadataVersion = MetadataVersion.IBP_3_6_IV1, serverProperties = Array( new ClusterConfigProperty(key="authorizer.class.name", value="kafka.security.authorizer.AclAuthorizer"), new ClusterConfigProperty(key="super.users", value="User:ANONYMOUS") @@ -158,7 +157,7 @@ class ZkMigrationIntegrationTest { @ClusterTest( brokers = 3, types = Array(Type.ZK), autoStart = AutoStart.YES, - metadataVersion = MetadataVersion.IBP_3_4_IV0, + metadataVersion = MetadataVersion.IBP_3_6_IV1, serverProperties = Array( new ClusterConfigProperty(key = "authorizer.class.name", value = "kafka.security.authorizer.AclAuthorizer"), new ClusterConfigProperty(key = "super.users", value = "User:ANONYMOUS"), @@ -173,7 +172,7 @@ class ZkMigrationIntegrationTest { val clusterId = zkCluster.clusterId() val kraftCluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_3_4_IV0). + setBootstrapMetadataVersion(MetadataVersion.IBP_3_6_IV1). setClusterId(clusterId). setNumBrokerNodes(0). setNumControllerNodes(1).build()) @@ -220,7 +219,7 @@ class ZkMigrationIntegrationTest { * and modifies data using AdminClient. The ZkMigrationClient is then used to read the metadata from ZK * as would happen during a migration. The generated records are then verified. */ - @ClusterTest(brokers = 3, types = Array(Type.ZK), metadataVersion = MetadataVersion.IBP_3_4_IV0) + @ClusterTest(brokers = 3, types = Array(Type.ZK), metadataVersion = MetadataVersion.IBP_3_6_IV1) def testMigrate(clusterInstance: ClusterInstance): Unit = { val admin = clusterInstance.createAdminClient() val newTopics = new util.ArrayList[NewTopic]() @@ -430,7 +429,7 @@ class ZkMigrationIntegrationTest { } // SCRAM and Quota are intermixed. Test SCRAM Only here - @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_5_IV2, serverProperties = Array( + @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_6_IV1, serverProperties = Array( new ClusterConfigProperty(key = "inter.broker.listener.name", value = "EXTERNAL"), new ClusterConfigProperty(key = "listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), new ClusterConfigProperty(key = "advertised.listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), @@ -447,7 +446,7 @@ class ZkMigrationIntegrationTest { val clusterId = zkCluster.clusterId() val kraftCluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_3_5_IV2). + setBootstrapMetadataVersion(MetadataVersion.IBP_3_6_IV1). setClusterId(clusterId). setNumBrokerNodes(0). setNumControllerNodes(1).build()) @@ -666,7 +665,7 @@ class ZkMigrationIntegrationTest { } // SCRAM and Quota are intermixed. Test both here - @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_5_IV2, serverProperties = Array( + @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_6_IV1, serverProperties = Array( new ClusterConfigProperty(key = "inter.broker.listener.name", value = "EXTERNAL"), new ClusterConfigProperty(key = "listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), new ClusterConfigProperty(key = "advertised.listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), @@ -683,7 +682,7 @@ class ZkMigrationIntegrationTest { val clusterId = zkCluster.clusterId() val kraftCluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_3_5_IV2). + setBootstrapMetadataVersion(MetadataVersion.IBP_3_6_IV1). setClusterId(clusterId). setNumBrokerNodes(0). setNumControllerNodes(1).build()) @@ -731,7 +730,7 @@ class ZkMigrationIntegrationTest { } } - @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_4_IV0, serverProperties = Array( + @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_6_IV1, serverProperties = Array( new ClusterConfigProperty(key = "inter.broker.listener.name", value = "EXTERNAL"), new ClusterConfigProperty(key = "listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), new ClusterConfigProperty(key = "advertised.listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), @@ -747,7 +746,7 @@ class ZkMigrationIntegrationTest { val clusterId = zkCluster.clusterId() val kraftCluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setBootstrapMetadataVersion(MetadataVersion.IBP_3_4_IV0). + setBootstrapMetadataVersion(MetadataVersion.IBP_3_6_IV1). setClusterId(clusterId). setNumBrokerNodes(0). setNumControllerNodes(1).build()) @@ -894,7 +893,7 @@ class ZkMigrationIntegrationTest { } } - @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_4_IV0, serverProperties = Array( + @ClusterTest(types = Array(Type.ZK), brokers = 3, metadataVersion = MetadataVersion.IBP_3_6_IV1, serverProperties = Array( new ClusterConfigProperty(key = "inter.broker.listener.name", value = "EXTERNAL"), new ClusterConfigProperty(key = "listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), new ClusterConfigProperty(key = "advertised.listeners", value = "PLAINTEXT://localhost:0,EXTERNAL://localhost:0"), diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala index f60aac80c9..18625fd1a6 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala @@ -205,7 +205,7 @@ class BrokerRegistrationRequestTest { types = Array(Type.KRAFT), brokers = 1, controllers = 1, - metadataVersion = MetadataVersion.IBP_3_4_IV0, + metadataVersion = MetadataVersion.IBP_3_6_IV1, autoStart = AutoStart.NO, serverProperties = Array(new ClusterConfigProperty(key = "zookeeper.metadata.migration.enable", value = "true"))) def testRegisterZkWithKRaftMigrationEnabled(clusterInstance: ClusterInstance): Unit = { @@ -218,6 +218,10 @@ class BrokerRegistrationRequestTest { assertEquals( Errors.NONE, + registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_6_IV1)))) + + assertEquals( + Errors.UNSUPPORTED_VERSION, registerBroker(channelManager, clusterId, 100, Some(1), Some((MetadataVersion.IBP_3_4_IV0, MetadataVersion.IBP_3_4_IV0)))) assertEquals( diff --git a/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java b/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java index d81d230984..eedaab3880 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ActivationRecordsGenerator.java @@ -88,7 +88,8 @@ static ControllerResult recordsForEmptyLog( // initialization, etc. records.addAll(bootstrapMetadata.records()); - if (metadataVersion.isMigrationSupported()) { + // In 3.9, we moved the minimum MV for migrations to 3.6. For empty logs, we only allow the newer 3.6 MV + if (metadataVersion.isMetadataTransactionSupported()) { if (zkMigrationEnabled) { logMessageBuilder.append("Putting the controller into pre-migration mode. No metadata updates " + "will be allowed until the ZK metadata has been migrated. "); @@ -107,6 +108,7 @@ static ControllerResult recordsForEmptyLog( activationMessageConsumer.accept(logMessageBuilder.toString().trim()); if (metadataVersion.isMetadataTransactionSupported()) { + // End marker for bootstrap records records.add(new ApiMessageAndVersion(new EndTransactionRecord(), (short) 0)); return ControllerResult.of(records, null); } else { @@ -149,6 +151,7 @@ static ControllerResult recordsForNonEmptyLog( .append(". "); } + // In 3.9, we moved the minimum MV for migrations to 3.6. For non-empty logs, we allow the older 3.4 MV if (zkMigrationEnabled && !metadataVersion.isMigrationSupported()) { throw new RuntimeException("Should not have ZK migrations enabled on a cluster running " + "metadata.version " + featureControl.metadataVersion()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java index 614908a3da..60e7a8a99d 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ActivationRecordsGeneratorTest.java @@ -55,28 +55,14 @@ public void testActivationMessageForEmptyLog() { result = ActivationRecordsGenerator.recordsForEmptyLog( logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + "Appending 1 bootstrap record(s) at metadata.version 3.4-IV0 from bootstrap " + - "source 'test'. Setting the ZK migration state to NONE since this is a de-novo KRaft cluster.", logMsg), + "source 'test'.", logMsg), -1L, false, BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"), MetadataVersion.IBP_3_4_IV0 ); assertTrue(result.isAtomic()); - assertEquals(2, result.records().size()); - - - result = ActivationRecordsGenerator.recordsForEmptyLog( - logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + - "Appending 1 bootstrap record(s) at metadata.version 3.4-IV0 from bootstrap " + - "source 'test'. Putting the controller into pre-migration mode. No metadata updates will be allowed " + - "until the ZK metadata has been migrated.", logMsg), - -1L, - true, - BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"), - MetadataVersion.IBP_3_4_IV0 - ); - assertTrue(result.isAtomic()); - assertEquals(2, result.records().size()); + assertEquals(1, result.records().size()); assertEquals( "The bootstrap metadata.version 3.3-IV2 does not support ZK migrations. Cannot continue with ZK migrations enabled.", @@ -90,6 +76,18 @@ public void testActivationMessageForEmptyLog() { )).getMessage() ); + assertEquals( + "The bootstrap metadata.version 3.4-IV0 does not support ZK migrations. Cannot continue with ZK migrations enabled.", + assertThrows(RuntimeException.class, () -> + ActivationRecordsGenerator.recordsForEmptyLog( + logMsg -> fail(), + -1L, + true, + BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_4_IV0, "test"), + MetadataVersion.IBP_3_4_IV0 + )).getMessage() + ); + result = ActivationRecordsGenerator.recordsForEmptyLog( logMsg -> assertEquals("Performing controller activation. The metadata log appears to be empty. " + "Appending 1 bootstrap record(s) in metadata transaction at metadata.version 3.6-IV1 from bootstrap " + @@ -253,21 +251,21 @@ public void testActivationMessageForNonEmptyLogWithMigrations() { true, buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.empty()), MetadataVersion.IBP_3_4_IV0 - ) - ).getMessage() + )).getMessage() ); - result = ActivationRecordsGenerator.recordsForNonEmptyLog( - logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " + - "PRE_MIGRATION. Activating pre-migration controller without empty log. There may be a partial " + - "migration.", logMsg), - -1L, - true, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.PRE_MIGRATION)), - MetadataVersion.IBP_3_4_IV0 + assertEquals( + "Should not have ZK migrations enabled on a cluster that was created in KRaft mode.", + assertThrows(RuntimeException.class, () -> + ActivationRecordsGenerator.recordsForNonEmptyLog( + logMsg -> fail(), + -1L, + true, + buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.empty()), + MetadataVersion.IBP_3_6_IV1 + ) + ).getMessage() ); - assertTrue(result.isAtomic()); - assertEquals(0, result.records().size()); result = ActivationRecordsGenerator.recordsForNonEmptyLog( logMsg -> assertEquals("Performing controller activation. Loaded ZK migration state of " + @@ -285,8 +283,8 @@ public void testActivationMessageForNonEmptyLogWithMigrations() { "Staying in ZK migration mode since 'zookeeper.metadata.migration.enable' is still 'true'.", logMsg), -1L, true, - buildFeatureControl(MetadataVersion.IBP_3_4_IV0, Optional.of(ZkMigrationState.MIGRATION)), - MetadataVersion.IBP_3_4_IV0 + buildFeatureControl(MetadataVersion.IBP_3_6_IV1, Optional.of(ZkMigrationState.MIGRATION)), + MetadataVersion.IBP_3_6_IV1 ); assertTrue(result.isAtomic()); assertEquals(0, result.records().size()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 0b754ec124..fc1a4f19cb 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -1385,18 +1385,18 @@ public void testAppendRecordsAtomically() { @Test public void testBootstrapZkMigrationRecord() throws Exception { assertEquals(ZkMigrationState.PRE_MIGRATION, - checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_4_IV0, true)); + checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_6_IV1, true)); assertEquals(ZkMigrationState.NONE, - checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_4_IV0, false)); + checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_6_IV1, false)); assertEquals(ZkMigrationState.NONE, - checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_3_IV0, false)); + checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_4_IV0, false)); assertEquals( - "The bootstrap metadata.version 3.3-IV0 does not support ZK migrations. Cannot continue with ZK migrations enabled.", + "The bootstrap metadata.version 3.4-IV0 does not support ZK migrations. Cannot continue with ZK migrations enabled.", assertThrows(FaultHandlerException.class, () -> - checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_3_IV0, true)).getCause().getMessage() + checkBootstrapZkMigrationRecord(MetadataVersion.IBP_3_4_IV0, true)).getCause().getMessage() ); } @@ -1499,9 +1499,10 @@ public void testActivationRecords33() { public void testActivationRecords34() { FeatureControlManager featureControl; - featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), true); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); - assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState()); + assertEquals( + "The bootstrap metadata.version 3.4-IV0 does not support ZK migrations. Cannot continue with ZK migrations enabled.", + assertThrows(RuntimeException.class, () -> getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), true)).getMessage() + ); featureControl = getActivationRecords(MetadataVersion.IBP_3_4_IV0, Optional.empty(), false); assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); @@ -1540,8 +1541,8 @@ public void testActivationRecords34() { @Test public void testActivationRecordsNonEmptyLog() { FeatureControlManager featureControl = getActivationRecords( - MetadataVersion.IBP_3_4_IV0, Optional.empty(), true); - assertEquals(MetadataVersion.IBP_3_4_IV0, featureControl.metadataVersion()); + MetadataVersion.IBP_3_6_IV1, Optional.empty(), true); + assertEquals(MetadataVersion.IBP_3_6_IV1, featureControl.metadataVersion()); assertEquals(ZkMigrationState.PRE_MIGRATION, featureControl.zkMigrationState()); } @@ -1732,7 +1733,7 @@ public void testFailoverDuringMigrationTransaction() throws Exception { } @ParameterizedTest - @EnumSource(value = MetadataVersion.class, names = {"IBP_3_4_IV0", "IBP_3_5_IV0", "IBP_3_6_IV0", "IBP_3_6_IV1"}) + @EnumSource(value = MetadataVersion.class, names = {"IBP_3_6_IV1", "IBP_3_6_IV2", "IBP_3_7_IV0"}) public void testBrokerHeartbeatDuringMigration(MetadataVersion metadataVersion) throws Exception { try ( LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build() From 034780dce90002406ccbef4ef6a7799c8c857429 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 6 Sep 2024 13:54:03 -0400 Subject: [PATCH 080/123] KAFKA-15648 Update leader volatile before handleLeaderChange in LocalLogManager (#17118) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Update the leader before calling handleLeaderChange and use the given epoch in LocalLogManager#prepareAppend. This should hopefully fix several flaky QuorumControllerTest tests. Reviewers: José Armando García Sancio --- .../test/java/org/apache/kafka/metalog/LocalLogManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index add04f0603..0b9f8b18d6 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -563,10 +563,10 @@ private void scheduleLogCheck() { if (batch.newLeader.equals(sharedLeader)) { log.debug("Node {}: Executing handleLeaderChange {}", nodeId, sharedLeader); - listenerData.handleLeaderChange(entryOffset, batch.newLeader); if (batch.newLeader.epoch() > leader.epoch()) { leader = batch.newLeader; } + listenerData.handleLeaderChange(entryOffset, batch.newLeader); } else { log.debug("Node {}: Ignoring {} since it doesn't match the latest known leader {}", nodeId, batch.newLeader, sharedLeader); @@ -733,7 +733,7 @@ public long prepareAppend( throw new BufferAllocationException("Test asked to fail the next prepareAppend"); } - return shared.tryAppend(nodeId, leader.epoch(), batch); + return shared.tryAppend(nodeId, epoch, batch); } @Override From 4b6437e6a523f72ef9d1572a77933057dbafafdd Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Tue, 10 Sep 2024 12:14:47 +0800 Subject: [PATCH 081/123] KAFKA-17497 Add e2e for zk migration with old controller (#17131) Reviewers: Chia-Ping Tsai --- .../tests/core/zookeeper_migration_test.py | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/tests/kafkatest/tests/core/zookeeper_migration_test.py b/tests/kafkatest/tests/core/zookeeper_migration_test.py index 5ddd86236a..390f899dba 100644 --- a/tests/kafkatest/tests/core/zookeeper_migration_test.py +++ b/tests/kafkatest/tests/core/zookeeper_migration_test.py @@ -17,19 +17,19 @@ import time from ducktape.utils.util import wait_until -from ducktape.mark import parametrize +from ducktape.mark import parametrize, matrix from ducktape.mark.resource import cluster from ducktape.errors import TimeoutError from kafkatest.services.console_consumer import ConsoleConsumer -from kafkatest.services.kafka import KafkaService +from kafkatest.services.kafka import KafkaService, config_property from kafkatest.services.kafka.config_property import CLUSTER_ID from kafkatest.services.kafka.quorum import isolated_kraft, ServiceQuorumInfo, zk from kafkatest.services.verifiable_producer import VerifiableProducer from kafkatest.services.zookeeper import ZookeeperService from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int -from kafkatest.version import DEV_BRANCH, LATEST_3_4 +from kafkatest.version import DEV_BRANCH, LATEST_3_4, LATEST_3_7, KafkaVersion class TestMigration(ProduceConsumeValidateTest): @@ -51,10 +51,10 @@ def wait_until_rejoin(self): wait_until(lambda: len(self.kafka.isr_idx_list(self.topic, partition)) == self.replication_factor, timeout_sec=60, backoff_sec=1, err_msg="Replicas did not rejoin the ISR in a reasonable amount of time") - def do_migration(self, roll_controller = False, downgrade_to_zk = False): + def do_migration(self, roll_controller=False, downgrade_to_zk=False, from_kafka_version=str(DEV_BRANCH)): # Start up KRaft controller in migration mode remote_quorum = partial(ServiceQuorumInfo, isolated_kraft) - controller = KafkaService(self.test_context, num_nodes=1, zk=self.zk, version=DEV_BRANCH, + controller = KafkaService(self.test_context, num_nodes=1, zk=self.zk, version=KafkaVersion(from_kafka_version), allow_zk_with_kraft=True, isolated_kafka=self.kafka, server_prop_overrides=[["zookeeper.connect", self.zk.connect_setting()], @@ -86,20 +86,23 @@ def do_migration(self, roll_controller = False, downgrade_to_zk = False): controller.start_node(node) @cluster(num_nodes=7) - @parametrize(roll_controller = True) - @parametrize(roll_controller = False) - def test_online_migration(self, roll_controller): + @matrix(roll_controller=[True, False], from_kafka_version=[str(DEV_BRANCH), str(LATEST_3_7)]) + def test_online_migration(self, roll_controller, from_kafka_version): zk_quorum = partial(ServiceQuorumInfo, zk) self.zk = ZookeeperService(self.test_context, num_nodes=1, version=DEV_BRANCH) + + server_prop_overrides = [["zookeeper.metadata.migration.enable", "false"]] + + if from_kafka_version != str(DEV_BRANCH): + server_prop_overrides.append([config_property.INTER_BROKER_PROTOCOL_VERSION, from_kafka_version]) + self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=DEV_BRANCH, quorum_info_provider=zk_quorum, allow_zk_with_kraft=True, - server_prop_overrides=[ - ["zookeeper.metadata.migration.enable", "false"], - ]) + server_prop_overrides=server_prop_overrides) self.kafka.security_protocol = "PLAINTEXT" self.kafka.interbroker_security_protocol = "PLAINTEXT" self.zk.start() @@ -128,7 +131,7 @@ def test_online_migration(self, roll_controller): self.topic, consumer_timeout_ms=30000, message_validator=is_int, version=DEV_BRANCH) - self.run_produce_consume_validate(core_test_action=partial(self.do_migration, roll_controller = roll_controller)) + self.run_produce_consume_validate(core_test_action=partial(self.do_migration, roll_controller=roll_controller, from_kafka_version=from_kafka_version)) self.kafka.stop() @cluster(num_nodes=7) From e885daf0e39ff69f71c1afc2aa0b40689ad8d4c5 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Wed, 11 Sep 2024 01:16:59 +0800 Subject: [PATCH 082/123] KAFKA-17492 skip features with minVersion of 0 instead of replacing 0 with 1 when BrokerRegistrationRequest < 4 (#17128) The 3.8 controller assumes the unknown features have min version = 0, but KAFKA-17011 replace the min=0 by min=1 when BrokerRegistrationRequest < 4. Hence, to support upgrading from 3.8.0 to 3.9, this PR changes the implementation of ApiVersionsResponse (<4) and BrokerRegistrationRequest (<4) to skip features with supported minVersion of 0 instead of replacing 0 with 1 Reviewers: Jun Rao , Colin P. McCabe , Chia-Ping Tsai --- .../kafka/common/requests/ApiVersionsResponse.java | 12 +++++------- .../common/requests/BrokerRegistrationRequest.java | 13 +++---------- .../common/message/ApiVersionsResponse.json | 2 +- .../common/message/BrokerRegistrationRequest.json | 2 +- .../common/requests/ApiVersionsResponseTest.java | 6 +----- .../requests/BrokerRegistrationRequestTest.java | 6 +----- .../server/ApiVersionsResponseIntegrationTest.scala | 12 ++++++++++-- .../tests/core/zookeeper_migration_test.py | 4 ++-- 8 files changed, 24 insertions(+), 33 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index aaa87a817f..0155fb8af5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -289,19 +289,17 @@ private static SupportedFeatureKeyCollection maybeFilterSupportedFeatureKeys( SupportedFeatureKeyCollection converted = new SupportedFeatureKeyCollection(); for (Map.Entry feature : latestSupportedFeatures.features().entrySet()) { final SupportedVersionRange versionRange = feature.getValue(); - final SupportedFeatureKey key = new SupportedFeatureKey(); - key.setName(feature.getKey()); if (alterV0 && versionRange.min() == 0) { // Some older clients will have deserialization problems if a feature's // minimum supported level is 0. Therefore, when preparing ApiVersionResponse - // at versions less than 4, we must set the minimum version for these features - // to 1 rather than 0. See KAFKA-17011 for details. - key.setMinVersion((short) 1); + // at versions less than 4, we must omit these features. See KAFKA-17492. } else { + final SupportedFeatureKey key = new SupportedFeatureKey(); + key.setName(feature.getKey()); key.setMinVersion(versionRange.min()); + key.setMaxVersion(versionRange.max()); + converted.add(key); } - key.setMaxVersion(versionRange.max()); - converted.add(key); } return converted; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java index a75616eb71..93dbf2ac97 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationRequest.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.protocol.Errors; import java.nio.ByteBuffer; -import java.util.Iterator; public class BrokerRegistrationRequest extends AbstractRequest { @@ -47,16 +46,10 @@ public short oldestAllowedVersion() { @Override public BrokerRegistrationRequest build(short version) { if (version < 4) { - // Workaround for KAFKA-17011: for BrokerRegistrationRequest versions older than 4, - // translate minSupportedVersion = 0 to minSupportedVersion = 1. + // Workaround for KAFKA-17492: for BrokerRegistrationRequest versions older than 4, + // remove features with minSupportedVersion = 0. BrokerRegistrationRequestData newData = data.duplicate(); - for (Iterator iter = newData.features().iterator(); - iter.hasNext(); ) { - BrokerRegistrationRequestData.Feature feature = iter.next(); - if (feature.minSupportedVersion() == 0) { - feature.setMinSupportedVersion((short) 1); - } - } + newData.features().removeIf(feature -> feature.minSupportedVersion() == 0); return new BrokerRegistrationRequest(newData, version); } else { return new BrokerRegistrationRequest(data, version); diff --git a/clients/src/main/resources/common/message/ApiVersionsResponse.json b/clients/src/main/resources/common/message/ApiVersionsResponse.json index 570891dd3c..c39d7edcf5 100644 --- a/clients/src/main/resources/common/message/ApiVersionsResponse.json +++ b/clients/src/main/resources/common/message/ApiVersionsResponse.json @@ -47,7 +47,7 @@ "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, { "name": "SupportedFeatures", "type": "[]SupportedFeatureKey", "ignorable": true, "versions": "3+", "tag": 0, "taggedVersions": "3+", - "about": "Features supported by the broker. Note: in v0-v3, features with MinSupportedVersion = 0 show up with MinSupportedVersion = 1.", + "about": "Features supported by the broker. Note: in v0-v3, features with MinSupportedVersion = 0 are omitted.", "fields": [ { "name": "Name", "type": "string", "versions": "3+", "mapKey": true, "about": "The name of the feature." }, diff --git a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json index 294e8549db..0c5153e64e 100644 --- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json +++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json @@ -48,7 +48,7 @@ ] }, { "name": "Features", "type": "[]Feature", - "about": "The features on this broker. Note: in v0-v3, features with MinSupportedVersion = 0 show up with MinSupportedVersion = 1.", "versions": "0+", "fields": [ + "about": "The features on this broker. Note: in v0-v3, features with MinSupportedVersion = 0 are omitted.", "versions": "0+", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "about": "The feature name." }, { "name": "MinSupportedVersion", "type": "int16", "versions": "0+", diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java index af1dda65fc..bfb327e3de 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java @@ -288,11 +288,7 @@ public void testAlterV0Features(boolean alterV0Features) { setAlterFeatureLevel0(alterV0Features). build(); if (alterV0Features) { - assertEquals(new SupportedFeatureKey(). - setName("my.feature"). - setMinVersion((short) 1). - setMaxVersion((short) 1), - response.data().supportedFeatures().find("my.feature")); + assertNull(response.data().supportedFeatures().find("my.feature")); } else { assertEquals(new SupportedFeatureKey(). setName("my.feature"). diff --git a/clients/src/test/java/org/apache/kafka/common/requests/BrokerRegistrationRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/BrokerRegistrationRequestTest.java index 14a6e3ca31..6ce5c73a59 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/BrokerRegistrationRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/BrokerRegistrationRequestTest.java @@ -113,11 +113,7 @@ public void testAlterFeaturesWithMinVersion0BeforeV4(short version) { new BrokerRegistrationRequestData.Feature(). setName("metadata.version"). setMinSupportedVersion((short) 1). - setMaxSupportedVersion((short) 17), - new BrokerRegistrationRequestData.Feature(). - setName("kraft.version"). - setMinSupportedVersion((short) 1). - setMaxSupportedVersion((short) 1)).iterator()), data.features()); + setMaxSupportedVersion((short) 17)).iterator()), data.features()); } else { assertEquals(new BrokerRegistrationRequestData.FeatureCollection( Arrays.asList( diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsResponseIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsResponseIntegrationTest.scala index a5a7564d49..5a1ea6e2bc 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsResponseIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsResponseIntegrationTest.scala @@ -18,7 +18,7 @@ import org.apache.kafka.common.requests.ApiVersionsRequest import org.apache.kafka.common.requests.ApiVersionsResponse import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.config.ServerConfigs -import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull} +import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertNull} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -44,7 +44,7 @@ class ApiVersionsResponseIntegrationTest extends BaseRequestTest { val response = sendApiVersionsRequest(3) if (quorum.equals("kraft")) { assertFeatureHasMinVersion("metadata.version", response.data().supportedFeatures(), 1) - assertFeatureHasMinVersion("kraft.version", response.data().supportedFeatures(), 1) + assertFeatureMissing("kraft.version", response.data().supportedFeatures()) } else { assertEquals(0, response.data().supportedFeatures().size()) } @@ -72,4 +72,12 @@ class ApiVersionsResponseIntegrationTest extends BaseRequestTest { assertEquals(name, key.name()) assertEquals(expectedMinVersion, key.minVersion()) } + + private def assertFeatureMissing( + name: String, + coll: SupportedFeatureKeyCollection, + ): Unit = { + val key = coll.find(name) + assertNull(key) + } } diff --git a/tests/kafkatest/tests/core/zookeeper_migration_test.py b/tests/kafkatest/tests/core/zookeeper_migration_test.py index 390f899dba..ad0168803a 100644 --- a/tests/kafkatest/tests/core/zookeeper_migration_test.py +++ b/tests/kafkatest/tests/core/zookeeper_migration_test.py @@ -29,7 +29,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int -from kafkatest.version import DEV_BRANCH, LATEST_3_4, LATEST_3_7, KafkaVersion +from kafkatest.version import DEV_BRANCH, LATEST_3_4, LATEST_3_7, LATEST_3_8, KafkaVersion class TestMigration(ProduceConsumeValidateTest): @@ -86,7 +86,7 @@ def do_migration(self, roll_controller=False, downgrade_to_zk=False, from_kafka_ controller.start_node(node) @cluster(num_nodes=7) - @matrix(roll_controller=[True, False], from_kafka_version=[str(DEV_BRANCH), str(LATEST_3_7)]) + @matrix(roll_controller=[True, False], from_kafka_version=[str(DEV_BRANCH), str(LATEST_3_7), str(LATEST_3_8)]) def test_online_migration(self, roll_controller, from_kafka_version): zk_quorum = partial(ServiceQuorumInfo, zk) self.zk = ZookeeperService(self.test_context, num_nodes=1, version=DEV_BRANCH) From f7fe4b94410c4e27227641bef4a2febd21eeed94 Mon Sep 17 00:00:00 2001 From: xijiu <422766572@qq.com> Date: Wed, 11 Sep 2024 02:12:25 +0800 Subject: [PATCH 083/123] KAFKA-17458 Add 3.8 to transactions_upgrade_test.py, transactions_mixed_versions_test.py, and kraft_upgrade_test.py (#17084) Reviewers: Chia-Ping Tsai --- tests/kafkatest/tests/core/kraft_upgrade_test.py | 6 +++--- .../tests/core/transactions_mixed_versions_test.py | 4 ++-- tests/kafkatest/tests/core/transactions_upgrade_test.py | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/kafkatest/tests/core/kraft_upgrade_test.py b/tests/kafkatest/tests/core/kraft_upgrade_test.py index 7aab85124f..6975069b73 100644 --- a/tests/kafkatest/tests/core/kraft_upgrade_test.py +++ b/tests/kafkatest/tests/core/kraft_upgrade_test.py @@ -23,7 +23,7 @@ from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int from kafkatest.version import LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, LATEST_3_7, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION + LATEST_3_6, LATEST_3_7, LATEST_3_8, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION # # Test upgrading between different KRaft versions. @@ -109,13 +109,13 @@ def run_upgrade(self, from_kafka_version): assert self.kafka.check_protocol_errors(self) @cluster(num_nodes=5) - @matrix(from_kafka_version=[str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(DEV_BRANCH)], + @matrix(from_kafka_version=[str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(DEV_BRANCH)], metadata_quorum=[combined_kraft]) def test_combined_mode_upgrade(self, from_kafka_version, metadata_quorum, use_new_coordinator=False): self.run_upgrade(from_kafka_version) @cluster(num_nodes=8) - @matrix(from_kafka_version=[str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(DEV_BRANCH)], + @matrix(from_kafka_version=[str(LATEST_3_1), str(LATEST_3_2), str(LATEST_3_3), str(LATEST_3_4), str(LATEST_3_5), str(LATEST_3_6), str(LATEST_3_7), str(LATEST_3_8), str(DEV_BRANCH)], metadata_quorum=[isolated_kraft]) def test_isolated_mode_upgrade(self, from_kafka_version, metadata_quorum, use_new_coordinator=False): self.run_upgrade(from_kafka_version) diff --git a/tests/kafkatest/tests/core/transactions_mixed_versions_test.py b/tests/kafkatest/tests/core/transactions_mixed_versions_test.py index a0f298b1fd..803cd82a9d 100644 --- a/tests/kafkatest/tests/core/transactions_mixed_versions_test.py +++ b/tests/kafkatest/tests/core/transactions_mixed_versions_test.py @@ -21,7 +21,7 @@ from kafkatest.utils import is_int from kafkatest.utils.transactions_utils import create_and_start_copiers from kafkatest.version import LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, LATEST_3_7, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION + LATEST_3_6, LATEST_3_7, LATEST_3_8, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION from ducktape.tests.test import Test from ducktape.mark import matrix @@ -178,7 +178,7 @@ def setup_topics(self): @cluster(num_nodes=8) @matrix( - old_kafka_version=[str(LATEST_3_7), str(LATEST_3_6), str(LATEST_3_5), str(LATEST_3_4), str(LATEST_3_3), str(LATEST_3_2), str(LATEST_3_1)], + old_kafka_version=[str(LATEST_3_8), str(LATEST_3_7), str(LATEST_3_6), str(LATEST_3_5), str(LATEST_3_4), str(LATEST_3_3), str(LATEST_3_2), str(LATEST_3_1)], metadata_quorum=[isolated_kraft], use_new_coordinator=[False], group_protocol=[None] diff --git a/tests/kafkatest/tests/core/transactions_upgrade_test.py b/tests/kafkatest/tests/core/transactions_upgrade_test.py index c6408cb65a..33355fef64 100644 --- a/tests/kafkatest/tests/core/transactions_upgrade_test.py +++ b/tests/kafkatest/tests/core/transactions_upgrade_test.py @@ -21,7 +21,7 @@ from kafkatest.utils import is_int from kafkatest.utils.transactions_utils import create_and_start_copiers from kafkatest.version import LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, \ - LATEST_3_6, LATEST_3_7, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION + LATEST_3_6, LATEST_3_7, LATEST_3_8, DEV_BRANCH, KafkaVersion, LATEST_STABLE_METADATA_VERSION from ducktape.tests.test import Test from ducktape.mark import matrix @@ -199,7 +199,7 @@ def setup_topics(self): @cluster(num_nodes=10) @matrix( - from_kafka_version=[str(LATEST_3_7), str(LATEST_3_6), str(LATEST_3_5), str(LATEST_3_4), str(LATEST_3_3), str(LATEST_3_2), str(LATEST_3_1)], + from_kafka_version=[str(LATEST_3_8), str(LATEST_3_7), str(LATEST_3_6), str(LATEST_3_5), str(LATEST_3_4), str(LATEST_3_3), str(LATEST_3_2), str(LATEST_3_1)], metadata_quorum=[isolated_kraft], use_new_coordinator=[False], group_protocol=[None] From 6d3e77533e287fc023fe8f658bc0165803ea7c46 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 6 Sep 2024 16:57:13 -0400 Subject: [PATCH 084/123] KAFKA-15793 Fix ZkMigrationIntegrationTest#testMigrateTopicDeletions (#17004) Reviewers: Igor Soarez , Ajit Singh <> --- .../kafka/zk/ZkMigrationIntegrationTest.scala | 61 ++++++++++++------- 1 file changed, 40 insertions(+), 21 deletions(-) diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala index 502b138668..eb5922e561 100644 --- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala @@ -91,7 +91,7 @@ object ZkMigrationIntegrationTest { } @ExtendWith(value = Array(classOf[ClusterTestExtensions])) -@Timeout(300) +@Timeout(600) class ZkMigrationIntegrationTest { val log: Logger = LoggerFactory.getLogger(classOf[ZkMigrationIntegrationTest]) @@ -295,13 +295,29 @@ class ZkMigrationIntegrationTest { def testMigrateTopicDeletions(zkCluster: ClusterInstance): Unit = { // Create some topics in ZK mode var admin = zkCluster.createAdminClient() - val newTopics = new util.ArrayList[NewTopic]() - newTopics.add(new NewTopic("test-topic-1", 10, 3.toShort)) - newTopics.add(new NewTopic("test-topic-2", 10, 3.toShort)) - newTopics.add(new NewTopic("test-topic-3", 10, 3.toShort)) - val createTopicResult = admin.createTopics(newTopics) - createTopicResult.all().get(300, TimeUnit.SECONDS) - admin.close() + try { + val newTopics = new util.ArrayList[NewTopic]() + newTopics.add(new NewTopic("test-topic-1", 10, 3.toShort)) + newTopics.add(new NewTopic("test-topic-2", 10, 3.toShort)) + newTopics.add(new NewTopic("test-topic-3", 10, 3.toShort)) + val createTopicResult = admin.createTopics(newTopics) + createTopicResult.all().get(61, TimeUnit.SECONDS) + TestUtils.waitUntilTrue(() => { + val topicDescribe = admin.describeTopics(Seq("test-topic-1", "test-topic-2", "test-topic-3").asJava) + if (topicDescribe.topicNameValues() == null || topicDescribe.topicNameValues().size() < 3) { + false + } else { + topicDescribe.topicNameValues().values().stream().allMatch { + topic => topic.get(62, TimeUnit.SECONDS).partitions().stream().allMatch(part => { + part.leader() != null && part.isr().size() == 3 + }) + } + } + }, msg="waiting for topics to be available", waitTimeMs=303000) + } finally { + admin.close() + } + val zkClient = zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying().zkClient // Bootstrap the ZK cluster ID into KRaft @@ -338,7 +354,7 @@ class ZkMigrationIntegrationTest { zkClient.createDeleteTopicPath("test-topic-3") zkCluster.waitForReadyBrokers() - readyFuture.get(60, TimeUnit.SECONDS) + readyFuture.get(64, TimeUnit.SECONDS) // Only continue with the test if there are some pending deletions to verify. If there are not any pending // deletions, this will mark the test as "skipped" instead of failed. @@ -351,12 +367,13 @@ class ZkMigrationIntegrationTest { TestUtils.waitUntilTrue( () => zkClient.getOrCreateMigrationState(ZkMigrationLeadershipState.EMPTY).initialZkMigrationComplete(), "Timed out waiting for migration to complete", - 30000) + 65000) // At this point, some of the topics may have been deleted by ZK controller and the rest will be // implicitly deleted by the KRaft controller and remove from the ZK brokers as stray partitions def topicsAllDeleted(admin: Admin): Boolean = { - val topics = admin.listTopics().names().get(60, TimeUnit.SECONDS) + val topics = admin.listTopics().names().get(66, TimeUnit.SECONDS) + log.info("Topics are {}", topics) topics.retainAll(util.Arrays.asList( "test-topic-1", "test-topic-2", "test-topic-3" )) @@ -368,18 +385,19 @@ class ZkMigrationIntegrationTest { TestUtils.waitUntilTrue( () => topicsAllDeleted(admin), "Timed out waiting for topics to be deleted", - 30000, + 307000, 1000) + log.info("Topics were deleted. Now re-creating them.") val newTopics = new util.ArrayList[NewTopic]() newTopics.add(new NewTopic("test-topic-1", 2, 3.toShort)) newTopics.add(new NewTopic("test-topic-2", 1, 3.toShort)) newTopics.add(new NewTopic("test-topic-3", 10, 3.toShort)) val createTopicResult = admin.createTopics(newTopics) - createTopicResult.all().get(60, TimeUnit.SECONDS) + createTopicResult.all().get(68, TimeUnit.SECONDS) def topicsAllRecreated(admin: Admin): Boolean = { - val topics = admin.listTopics().names().get(60, TimeUnit.SECONDS) + val topics = admin.listTopics().names().get(69, TimeUnit.SECONDS) topics.retainAll(util.Arrays.asList( "test-topic-1", "test-topic-2", "test-topic-3" )) @@ -390,21 +408,22 @@ class ZkMigrationIntegrationTest { TestUtils.waitUntilTrue( () => topicsAllRecreated(admin), "Timed out waiting for topics to be created", - 30000, + 70000, 1000) - - TestUtils.retry(300000) { + log.info("Topics were re-created. Now waiting for consistent topic state.") + TestUtils.retry(311000) { // Need a retry here since topic metadata may be inconsistent between brokers val topicDescriptions = try { admin.describeTopics(util.Arrays.asList( "test-topic-1", "test-topic-2", "test-topic-3" )).topicNameValues().asScala.map { case (name, description) => - name -> description.get(60, TimeUnit.SECONDS) + name -> description.get(72, TimeUnit.SECONDS) }.toMap } catch { case e: ExecutionException if e.getCause.isInstanceOf[UnknownTopicOrPartitionException] => Map.empty[String, TopicDescription] case t: Throwable => fail("Error describing topics", t.getCause) } + log.debug("Topic describe: {}", topicDescriptions); assertEquals(2, topicDescriptions("test-topic-1").partitions().size()) assertEquals(1, topicDescriptions("test-topic-2").partitions().size()) @@ -416,14 +435,13 @@ class ZkMigrationIntegrationTest { }) } - val absentTopics = admin.listTopics().names().get(60, TimeUnit.SECONDS).asScala + val absentTopics = admin.listTopics().names().get(73, TimeUnit.SECONDS).asScala assertTrue(absentTopics.contains("test-topic-1")) assertTrue(absentTopics.contains("test-topic-2")) assertTrue(absentTopics.contains("test-topic-3")) } - - admin.close() } finally { + admin.close() shutdownInSequence(zkCluster, kraftCluster) } } @@ -1152,6 +1170,7 @@ class ZkMigrationIntegrationTest { def shutdownInSequence(zkCluster: ClusterInstance, kraftCluster: KafkaClusterTestKit): Unit = { zkCluster.brokerIds().forEach(zkCluster.shutdownBroker(_)) + kraftCluster.nonFatalFaultHandler().setIgnore(true) kraftCluster.close() zkCluster.stop() } From 5a4d2b44d2cb963df3cb8123e9adb3e33bd3e6fb Mon Sep 17 00:00:00 2001 From: Vikas Singh Date: Mon, 9 Sep 2024 18:43:33 +0530 Subject: [PATCH 085/123] MINOR: Few cleanups Reviewers: Manikumar Reddy --- .../scram/internals/ScramSaslServer.java | 6 +- .../scram/internals/ScramSaslServerTest.java | 68 +++++++++++++++++++ 2 files changed, 73 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramSaslServer.java b/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramSaslServer.java index cea3ddf71f..e3a300f9a7 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramSaslServer.java +++ b/clients/src/main/java/org/apache/kafka/common/security/scram/internals/ScramSaslServer.java @@ -149,6 +149,9 @@ public byte[] evaluateResponse(byte[] response) throws SaslException, SaslAuthen case RECEIVE_CLIENT_FINAL_MESSAGE: try { ClientFinalMessage clientFinalMessage = new ClientFinalMessage(response); + if (!clientFinalMessage.nonce().endsWith(serverFirstMessage.nonce())) { + throw new SaslException("Invalid client nonce in the final client message."); + } verifyClientProof(clientFinalMessage); byte[] serverKey = scramCredential.serverKey(); byte[] serverSignature = formatter.serverSignature(serverKey, clientFirstMessage, serverFirstMessage, clientFinalMessage); @@ -222,7 +225,8 @@ private void setState(State state) { this.state = state; } - private void verifyClientProof(ClientFinalMessage clientFinalMessage) throws SaslException { + // Visible for testing + void verifyClientProof(ClientFinalMessage clientFinalMessage) throws SaslException { try { byte[] expectedStoredKey = scramCredential.storedKey(); byte[] clientSignature = formatter.clientSignature(expectedStoredKey, clientFirstMessage, serverFirstMessage, clientFinalMessage); diff --git a/clients/src/test/java/org/apache/kafka/common/security/scram/internals/ScramSaslServerTest.java b/clients/src/test/java/org/apache/kafka/common/security/scram/internals/ScramSaslServerTest.java index 1393b26f87..94b95b0cfd 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/scram/internals/ScramSaslServerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/scram/internals/ScramSaslServerTest.java @@ -20,14 +20,23 @@ import org.apache.kafka.common.errors.SaslAuthenticationException; import org.apache.kafka.common.security.authenticator.CredentialCache; import org.apache.kafka.common.security.scram.ScramCredential; +import org.apache.kafka.common.security.scram.internals.ScramMessages.ClientFirstMessage; +import org.apache.kafka.common.security.scram.internals.ScramMessages.ServerFinalMessage; +import org.apache.kafka.common.security.scram.internals.ScramMessages.ServerFirstMessage; import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.nio.charset.StandardCharsets; +import java.util.Base64; import java.util.HashMap; +import javax.security.sasl.SaslException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -67,10 +76,69 @@ public void authorizationIdNotEqualsAuthenticationId() { assertThrows(SaslAuthenticationException.class, () -> saslServer.evaluateResponse(clientFirstMessage(USER_A, USER_B))); } + /** + * Validate that server responds with client's nonce as prefix of its nonce in the + * server first message. + *
    + * In addition, it checks that the client final message has nonce that it sent in its + * first message. + */ + @Test + public void validateNonceExchange() throws SaslException { + ScramSaslServer spySaslServer = Mockito.spy(saslServer); + byte[] clientFirstMsgBytes = clientFirstMessage(USER_A, USER_A); + ClientFirstMessage clientFirstMessage = new ClientFirstMessage(clientFirstMsgBytes); + + byte[] serverFirstMsgBytes = spySaslServer.evaluateResponse(clientFirstMsgBytes); + ServerFirstMessage serverFirstMessage = new ServerFirstMessage(serverFirstMsgBytes); + assertTrue(serverFirstMessage.nonce().startsWith(clientFirstMessage.nonce()), + "Nonce in server message should start with client first message's nonce"); + + byte[] clientFinalMessage = clientFinalMessage(serverFirstMessage.nonce()); + Mockito.doNothing() + .when(spySaslServer).verifyClientProof(Mockito.any(ScramMessages.ClientFinalMessage.class)); + byte[] serverFinalMsgBytes = spySaslServer.evaluateResponse(clientFinalMessage); + ServerFinalMessage serverFinalMessage = new ServerFinalMessage(serverFinalMsgBytes); + assertNull(serverFinalMessage.error(), "Server final message should not contain error"); + } + + @Test + public void validateFailedNonceExchange() throws SaslException { + ScramSaslServer spySaslServer = Mockito.spy(saslServer); + byte[] clientFirstMsgBytes = clientFirstMessage(USER_A, USER_A); + ClientFirstMessage clientFirstMessage = new ClientFirstMessage(clientFirstMsgBytes); + + byte[] serverFirstMsgBytes = spySaslServer.evaluateResponse(clientFirstMsgBytes); + ServerFirstMessage serverFirstMessage = new ServerFirstMessage(serverFirstMsgBytes); + assertTrue(serverFirstMessage.nonce().startsWith(clientFirstMessage.nonce()), + "Nonce in server message should start with client first message's nonce"); + + byte[] clientFinalMessage = clientFinalMessage(formatter.secureRandomString()); + Mockito.doNothing() + .when(spySaslServer).verifyClientProof(Mockito.any(ScramMessages.ClientFinalMessage.class)); + SaslException saslException = assertThrows(SaslException.class, + () -> spySaslServer.evaluateResponse(clientFinalMessage)); + assertEquals("Invalid client nonce in the final client message.", + saslException.getMessage(), + "Failure message: " + saslException.getMessage()); + } + private byte[] clientFirstMessage(String userName, String authorizationId) { String nonce = formatter.secureRandomString(); String authorizationField = authorizationId != null ? "a=" + authorizationId : ""; String firstMessage = String.format("n,%s,n=%s,r=%s", authorizationField, userName, nonce); return firstMessage.getBytes(StandardCharsets.UTF_8); } + + private byte[] clientFinalMessage(String nonce) { + String channelBinding = randomBytesAsString(); + String proof = randomBytesAsString(); + + String message = String.format("c=%s,r=%s,p=%s", channelBinding, nonce, proof); + return message.getBytes(StandardCharsets.UTF_8); + } + + private String randomBytesAsString() { + return Base64.getEncoder().encodeToString(formatter.secureRandomBytes()); + } } From 4734077f4729a3f1dd86c60a8669b535eb1e8257 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Wed, 11 Sep 2024 10:41:49 -0400 Subject: [PATCH 086/123] KAFKA-17506 KRaftMigrationDriver initialization race (#17147) There is a race condition between KRaftMigrationDriver running its first poll() and being notified by Raft about a leader change. If onControllerChange is called before RecoverMigrationStateFromZKEvent is run, we will end up getting stuck in the INACTIVE state. This patch fixes the race by enqueuing a RecoverMigrationStateFromZKEvent from onControllerChange if the driver has not yet initialized. If another RecoverMigrationStateFromZKEvent was already enqueued, the second one to run will just be ignored. Reviewers: Luke Chen --- .../migration/KRaftMigrationDriver.java | 7 +++-- .../migration/KRaftMigrationDriverTest.java | 26 +++++++++++++++++++ 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java index 61b95c091d..c00c25fd15 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java @@ -359,6 +359,9 @@ public String name() { @Override public void onControllerChange(LeaderAndEpoch newLeaderAndEpoch) { curLeaderAndEpoch = newLeaderAndEpoch; + if (migrationState.equals(MigrationDriverState.UNINITIALIZED)) { + eventQueue.append(new RecoverMigrationStateFromZKEvent()); + } eventQueue.append(new KRaftLeaderEvent(newLeaderAndEpoch)); } @@ -519,8 +522,8 @@ public void run() throws Exception { KRaftMigrationDriver.this.image = image; String metadataType = isSnapshot ? "snapshot" : "delta"; - if (migrationState.equals(MigrationDriverState.INACTIVE)) { - // No need to log anything if this node is not the active controller + if (EnumSet.of(MigrationDriverState.UNINITIALIZED, MigrationDriverState.INACTIVE).contains(migrationState)) { + // No need to log anything if this node is not the active controller or the driver has not initialized completionHandler.accept(null); return; } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java b/metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java index b6e02fd668..dd09a2432e 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/migration/KRaftMigrationDriverTest.java @@ -232,6 +232,32 @@ CompletableFuture enqueueMetadataChangeEventWithFuture( return future; } + @Test + public void testOnControllerChangeWhenUninitialized() throws InterruptedException { + CountingMetadataPropagator metadataPropagator = new CountingMetadataPropagator(); + CapturingMigrationClient.newBuilder().build(); + CapturingMigrationClient migrationClient = CapturingMigrationClient.newBuilder().build(); + MockFaultHandler faultHandler = new MockFaultHandler("testBecomeLeaderUninitialized"); + KRaftMigrationDriver.Builder builder = defaultTestBuilder() + .setZkMigrationClient(migrationClient) + .setPropagator(metadataPropagator) + .setFaultHandler(faultHandler); + try (KRaftMigrationDriver driver = builder.build()) { + // Fake a complete migration with ZK client + migrationClient.setMigrationRecoveryState( + ZkMigrationLeadershipState.EMPTY.withKRaftMetadataOffsetAndEpoch(100, 1)); + + // simulate the Raft layer running before the driver has fully started. + driver.onControllerChange(new LeaderAndEpoch(OptionalInt.of(3000), 1)); + + // start up the driver. this will enqueue a poll event. once run, this will enqueue a recovery event + driver.start(); + + // Even though we contrived a race above, the driver still makes it past initialization. + TestUtils.waitForCondition(() -> driver.migrationState().get(30, TimeUnit.SECONDS).equals(MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM), + "Waiting for KRaftMigrationDriver to enter WAIT_FOR_CONTROLLER_QUORUM state"); + } + } /** * Don't send RPCs to brokers for every metadata change, only when brokers or topics change. * This is a regression test for KAFKA-14668 From 4f0675d5e925bb8f7ad66d14ca7c5d955e777c71 Mon Sep 17 00:00:00 2001 From: Bruno Cadonna Date: Fri, 13 Sep 2024 10:41:45 +0200 Subject: [PATCH 087/123] KAFKA-17489: Do not handle failed tasks as tasks to assign (#17115) Failed tasks discovered when removed from the state updater during assignment or revocation are added to the task registry. From there they are retrieved and handled as normal tasks. This leads to a couple of IllegalStateExceptions because it breaks some invariants that ensure that only good tasks are assigned and processed. This commit solves this bug by distinguish failed from non-failed tasks in the task registry. Reviewer: Lucas Brutschy --- .../processor/internals/TaskManager.java | 10 +- .../streams/processor/internals/Tasks.java | 20 +++ .../processor/internals/TasksRegistry.java | 4 + .../processor/internals/TaskManagerTest.java | 133 +++++++++++++++--- .../processor/internals/TasksTest.java | 44 ++++++ 5 files changed, 185 insertions(+), 26 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index bde092cfe4..d6a0280e06 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -197,7 +197,7 @@ void handleRebalanceComplete() { mainConsumer.pause(mainConsumer.assignment()); } else { // All tasks that are owned by the task manager are ready and do not need to be paused - final Set partitionsNotToPause = tasks.allTasks() + final Set partitionsNotToPause = tasks.allNonFailedTasks() .stream() .flatMap(task -> task.inputPartitions().stream()) .collect(Collectors.toSet()); @@ -524,7 +524,7 @@ private void handleRunningAndSuspendedTasks(final Map> standbyTasksToCreate, final Map> tasksToRecycle, final Set tasksToCloseClean) { - for (final Task task : tasks.allTasks()) { + for (final Task task : tasks.allNonFailedTasks()) { if (!task.isActive()) { throw new IllegalStateException("Standby tasks should only be managed by the state updater, " + "but standby task " + task.id() + " is managed by the stream thread"); @@ -686,7 +686,7 @@ private Task checkIfTaskFailed(final StateUpdater.RemovedTaskResult removedTaskR final Task task = removedTaskResult.task(); if (removedTaskResult.exception().isPresent()) { failedTasks.put(task.id(), removedTaskResult.exception().get()); - tasks.addTask(task); + tasks.addFailedTask(task); return null; } return task; @@ -998,7 +998,7 @@ private void addTasksToStateUpdater() { addTaskToStateUpdater(task); } catch (final RuntimeException e) { // need to add task back to the bookkeeping to be handled by the stream thread - tasks.addTask(task); + tasks.addFailedTask(task); taskExceptions.put(task.id(), e); } } @@ -1030,7 +1030,7 @@ public Map collectExceptionsAndFailedTasksFromStateUpd final RuntimeException exception = exceptionAndTask.exception(); final Task failedTask = exceptionAndTask.task(); // need to add task back to the bookkeeping to be handled by the stream thread - tasks.addTask(failedTask); + tasks.addFailedTask(failedTask); taskExceptions.put(failedTask.id(), exception); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java index e30333aabf..92dd07ba97 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java @@ -58,6 +58,7 @@ class Tasks implements TasksRegistry { private final Map> pendingActiveTasksToCreate = new HashMap<>(); private final Map> pendingStandbyTasksToCreate = new HashMap<>(); private final Set pendingTasksToInit = new HashSet<>(); + private final Set failedTaskIds = new HashSet<>(); // TODO: convert to Stream/StandbyTask when we remove TaskManager#StateMachineTask with mocks private final Map activeTasksPerPartition = new HashMap<>(); @@ -178,6 +179,12 @@ public synchronized void addTask(final Task task) { } } + @Override + public void addFailedTask(final Task task) { + failedTaskIds.add(task.id()); + addTask(task); + } + @Override public synchronized void removeTask(final Task taskToRemove) { final TaskId taskId = taskToRemove.id(); @@ -196,6 +203,7 @@ public synchronized void removeTask(final Task taskToRemove) { throw new IllegalArgumentException("Attempted to remove a standby task that is not owned: " + taskId); } } + failedTaskIds.remove(taskToRemove.id()); } @Override @@ -253,6 +261,7 @@ public synchronized void clear() { activeTasksPerId.clear(); standbyTasksPerId.clear(); activeTasksPerPartition.clear(); + failedTaskIds.clear(); } // TODO: change return type to `StreamTask` @@ -309,6 +318,17 @@ public synchronized Set allTasks() { return union(HashSet::new, new HashSet<>(activeTasksPerId.values()), new HashSet<>(standbyTasksPerId.values())); } + @Override + public synchronized Set allNonFailedTasks() { + final Set nonFailedActiveTasks = activeTasksPerId.values().stream() + .filter(task -> !failedTaskIds.contains(task.id())) + .collect(Collectors.toSet()); + final Set nonFailedStandbyTasks = standbyTasksPerId.values().stream() + .filter(task -> !failedTaskIds.contains(task.id())) + .collect(Collectors.toSet()); + return union(HashSet::new, nonFailedActiveTasks, nonFailedStandbyTasks); + } + @Override public synchronized Set allTaskIds() { return union(HashSet::new, activeTasksPerId.keySet(), standbyTasksPerId.keySet()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java index 64f24702d2..20bee575eb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java @@ -51,6 +51,8 @@ public interface TasksRegistry { void addTask(final Task task); + void addFailedTask(final Task task); + void removeTask(final Task taskToRemove); void replaceActiveWithStandby(final StandbyTask standbyTask); @@ -73,6 +75,8 @@ public interface TasksRegistry { Set allTasks(); + Set allNonFailedTasks(); + Map allTasksPerId(); Set allTaskIds(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 0ae179a914..1dc4cf00b6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -536,7 +536,7 @@ public void shouldCollectFailedTaskFromStateUpdaterAndRethrow() { assertEquals("Encounter unexpected fatal error for task " + failedStandbyTask.id(), exception.getMessage()); assertInstanceOf(RuntimeException.class, exception.getCause()); assertEquals(kaboom.getMessage(), exception.getCause().getMessage()); - verify(tasks).addTask(failedStandbyTask); + verify(tasks).addFailedTask(failedStandbyTask); } @Test @@ -702,7 +702,7 @@ public void shouldMoveReassignedSuspendedActiveTaskToStateUpdater() { .withInputPartitions(taskId03Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(reassignedActiveTask)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(reassignedActiveTask)); taskManager.handleAssignment( mkMap(mkEntry(reassignedActiveTask.id(), reassignedActiveTask.inputPartitions())), @@ -715,6 +715,96 @@ public void shouldMoveReassignedSuspendedActiveTaskToStateUpdater() { verify(standbyTaskCreator).createTasks(Collections.emptyMap()); } + @Test + public void shouldAddFailedActiveTaskToRecycleDuringAssignmentToTaskRegistry() { + final StreamTask failedActiveTaskToRecycle = statefulTask(taskId03, taskId03ChangelogPartitions) + .inState(State.RESTORING) + .withInputPartitions(taskId03Partitions).build(); + final TasksRegistry tasks = mock(TasksRegistry.class); + final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); + when(stateUpdater.getTasks()).thenReturn(mkSet(failedActiveTaskToRecycle)); + final RuntimeException taskException = new RuntimeException("Nobody expects the Spanish inquisition!"); + when(stateUpdater.remove(failedActiveTaskToRecycle.id())) + .thenReturn(CompletableFuture.completedFuture( + new StateUpdater.RemovedTaskResult(failedActiveTaskToRecycle, taskException) + )); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> taskManager.handleAssignment( + Collections.emptyMap(), + mkMap(mkEntry(failedActiveTaskToRecycle.id(), failedActiveTaskToRecycle.inputPartitions())) + ) + ); + + assertEquals("Encounter unexpected fatal error for task " + failedActiveTaskToRecycle.id(), exception.getMessage()); + assertEquals(taskException, exception.getCause()); + verify(tasks).addFailedTask(failedActiveTaskToRecycle); + verify(tasks, never()).addTask(failedActiveTaskToRecycle); + verify(tasks).allNonFailedTasks(); + verify(standbyTaskCreator, never()).createStandbyTaskFromActive(failedActiveTaskToRecycle, taskId03Partitions); + } + + @Test + public void shouldAddFailedStandbyTaskToRecycleDuringAssignmentToTaskRegistry() { + final StandbyTask failedStandbyTaskToRecycle = standbyTask(taskId03, taskId03ChangelogPartitions) + .inState(State.RUNNING) + .withInputPartitions(taskId03Partitions).build(); + final TasksRegistry tasks = mock(TasksRegistry.class); + final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); + when(stateUpdater.getTasks()).thenReturn(mkSet(failedStandbyTaskToRecycle)); + final RuntimeException taskException = new RuntimeException("Nobody expects the Spanish inquisition!"); + when(stateUpdater.remove(failedStandbyTaskToRecycle.id())) + .thenReturn(CompletableFuture.completedFuture( + new StateUpdater.RemovedTaskResult(failedStandbyTaskToRecycle, taskException) + )); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> taskManager.handleAssignment( + mkMap(mkEntry(failedStandbyTaskToRecycle.id(), failedStandbyTaskToRecycle.inputPartitions())), + Collections.emptyMap() + ) + ); + + assertEquals("Encounter unexpected fatal error for task " + failedStandbyTaskToRecycle.id(), exception.getMessage()); + assertEquals(taskException, exception.getCause()); + verify(tasks).addFailedTask(failedStandbyTaskToRecycle); + verify(tasks, never()).addTask(failedStandbyTaskToRecycle); + verify(tasks).allNonFailedTasks(); + verify(activeTaskCreator, never()).createActiveTaskFromStandby(failedStandbyTaskToRecycle, taskId03Partitions, consumer); + } + + @Test + public void shouldAddFailedActiveTasksToReassignWithDifferentInputPartitionsDuringAssignmentToTaskRegistry() { + final StreamTask failedActiveTaskToReassign = statefulTask(taskId03, taskId03ChangelogPartitions) + .inState(State.RESTORING) + .withInputPartitions(taskId03Partitions).build(); + final TasksRegistry tasks = mock(TasksRegistry.class); + final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); + when(stateUpdater.getTasks()).thenReturn(mkSet(failedActiveTaskToReassign)); + final RuntimeException taskException = new RuntimeException("Nobody expects the Spanish inquisition!"); + when(stateUpdater.remove(failedActiveTaskToReassign.id())) + .thenReturn(CompletableFuture.completedFuture( + new StateUpdater.RemovedTaskResult(failedActiveTaskToReassign, taskException) + )); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> taskManager.handleAssignment( + mkMap(mkEntry(failedActiveTaskToReassign.id(), taskId00Partitions)), + Collections.emptyMap() + ) + ); + + assertEquals("Encounter unexpected fatal error for task " + failedActiveTaskToReassign.id(), exception.getMessage()); + assertEquals(taskException, exception.getCause()); + verify(tasks).addFailedTask(failedActiveTaskToReassign); + verify(tasks, never()).addTask(failedActiveTaskToReassign); + verify(tasks).allNonFailedTasks(); + verify(tasks, never()).updateActiveTaskInputPartitions(failedActiveTaskToReassign, taskId00Partitions); + } + @Test public void shouldFirstHandleTasksInStateUpdaterThenSuspendedActiveTasksInTaskRegistry() { final StreamTask reassignedActiveTask1 = statefulTask(taskId03, taskId03ChangelogPartitions) @@ -725,7 +815,7 @@ public void shouldFirstHandleTasksInStateUpdaterThenSuspendedActiveTasksInTaskRe .withInputPartitions(taskId02Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(reassignedActiveTask1)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(reassignedActiveTask1)); when(stateUpdater.getTasks()).thenReturn(mkSet(reassignedActiveTask2)); when(stateUpdater.remove(reassignedActiveTask2.id())) .thenReturn(CompletableFuture.completedFuture(new StateUpdater.RemovedTaskResult(reassignedActiveTask2))); @@ -906,7 +996,7 @@ public void shouldAddRecycledStandbyTaskfromActiveToPendingTasksToInitWithStateU .withInputPartitions(taskId01Partitions) .inState(State.CREATED).build(); final TasksRegistry tasks = mock(TasksRegistry.class); - when(tasks.allTasks()).thenReturn(mkSet(activeTaskToRecycle)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(activeTaskToRecycle)); when(standbyTaskCreator.createStandbyTaskFromActive(activeTaskToRecycle, taskId01Partitions)) .thenReturn(standbyTask); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); @@ -950,7 +1040,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToRecycleIsFoundInTasksRegis .inState(State.RUNNING) .withInputPartitions(taskId03Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); - when(tasks.allTasks()).thenReturn(mkSet(standbyTaskToRecycle)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(standbyTaskToRecycle)); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); final IllegalStateException illegalStateException = assertThrows( @@ -973,7 +1063,7 @@ public void shouldAssignActiveTaskInTasksRegistryToBeClosedCleanlyWithStateUpdat .withInputPartitions(taskId03Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(activeTaskToClose)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(activeTaskToClose)); taskManager.handleAssignment(Collections.emptyMap(), Collections.emptyMap()); @@ -992,7 +1082,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToCloseIsFoundInTasksRegistr .withInputPartitions(taskId03Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(standbyTaskToClose)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(standbyTaskToClose)); final IllegalStateException illegalStateException = assertThrows( IllegalStateException.class, @@ -1012,7 +1102,7 @@ public void shouldAssignActiveTaskInTasksRegistryToUpdateInputPartitionsWithStat final Set newInputPartitions = taskId02Partitions; final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(activeTaskToUpdateInputPartitions)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(activeTaskToUpdateInputPartitions)); when(tasks.updateActiveTaskInputPartitions(activeTaskToUpdateInputPartitions, newInputPartitions)).thenReturn(true); taskManager.handleAssignment( @@ -1032,7 +1122,7 @@ public void shouldResumeActiveRunningTaskInTasksRegistryWithStateUpdaterEnabled( .withInputPartitions(taskId03Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(activeTaskToResume)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(activeTaskToResume)); taskManager.handleAssignment( mkMap(mkEntry(activeTaskToResume.id(), activeTaskToResume.inputPartitions())), @@ -1050,7 +1140,7 @@ public void shouldResumeActiveSuspendedTaskInTasksRegistryAndAddToStateUpdater() .withInputPartitions(taskId03Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(activeTaskToResume)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(activeTaskToResume)); taskManager.handleAssignment( mkMap(mkEntry(activeTaskToResume.id(), activeTaskToResume.inputPartitions())), @@ -1072,7 +1162,7 @@ public void shouldThrowDuringAssignmentIfStandbyTaskToUpdateInputPartitionsIsFou final Set newInputPartitions = taskId03Partitions; final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(standbyTaskToUpdateInputPartitions)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(standbyTaskToUpdateInputPartitions)); final IllegalStateException illegalStateException = assertThrows( IllegalStateException.class, @@ -1097,7 +1187,7 @@ public void shouldAssignMultipleTasksInTasksRegistryWithStateUpdaterEnabled() { .withInputPartitions(taskId02Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(activeTaskToClose)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(activeTaskToClose)); taskManager.handleAssignment( mkMap(mkEntry(activeTaskToCreate.id(), activeTaskToCreate.inputPartitions())), @@ -1173,7 +1263,7 @@ public void shouldRethrowRuntimeExceptionInInitTaskWithStateUpdater() { () -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter) ); verify(stateUpdater, never()).add(task00); - verify(tasks).addTask(task00); + verify(tasks).addFailedTask(task00); assertTrue(streamsException.taskId().isPresent()); assertEquals(task00.id(), streamsException.taskId().get()); assertEquals("Encounter unexpected fatal error for task 0_0", streamsException.getMessage()); @@ -1202,8 +1292,8 @@ public void shouldRethrowTaskCorruptedExceptionFromInitialization() { () -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter) ); - verify(tasks).addTask(statefulTask0); - verify(tasks).addTask(statefulTask1); + verify(tasks).addFailedTask(statefulTask0); + verify(tasks).addFailedTask(statefulTask1); verify(stateUpdater).add(statefulTask2); assertEquals(mkSet(taskId00, taskId01), thrown.corruptedTasks()); assertEquals("Tasks [0_1, 0_0] are corrupted and hence need to be re-initialized", thrown.getMessage()); @@ -1324,19 +1414,20 @@ public void shouldThrowIfRevokingTasksInStateUpdaterFindsFailedTasks() { future1.complete(new StateUpdater.RemovedTaskResult(task1)); final CompletableFuture future2 = new CompletableFuture<>(); when(stateUpdater.remove(task2.id())).thenReturn(future2); - final StreamsException streamsException = new StreamsException("Something happened"); - future2.complete(new StateUpdater.RemovedTaskResult(task2, streamsException)); + final RuntimeException taskException = new RuntimeException("Nobody expects the Spanish inquisition!"); + future2.complete(new StateUpdater.RemovedTaskResult(task2, taskException)); final StreamsException thrownException = assertThrows( StreamsException.class, () -> taskManager.handleRevocation(union(HashSet::new, taskId00Partitions, taskId01Partitions)) ); - assertEquals(thrownException, streamsException); + assertEquals("Encounter unexpected fatal error for task " + task2.id(), thrownException.getMessage()); + assertEquals(thrownException.getCause(), taskException); verify(task1).suspend(); verify(tasks).addTask(task1); verify(task2, never()).suspend(); - verify(tasks).addTask(task2); + verify(tasks).addFailedTask(task2); } @Test @@ -1651,7 +1742,7 @@ public void shouldNotPauseReadyTasksWithStateUpdaterOnRebalanceComplete() { .withInputPartitions(taskId00Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(mkSet(statefulTask0)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(statefulTask0)); final Set assigned = mkSet(t1p0, t1p1); when(consumer.assignment()).thenReturn(assigned); @@ -1701,7 +1792,7 @@ public void shouldReleaseLockForUnassignedTasksAfterRebalanceWithStateUpdater() final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId00, runningStatefulTask))); when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTask, restoringStatefulTask)); - when(tasks.allTasks()).thenReturn(mkSet(runningStatefulTask)); + when(tasks.allNonFailedTasks()).thenReturn(mkSet(runningStatefulTask)); expectLockObtainedFor(taskId00, taskId01, taskId02, taskId03); expectDirectoryNotEmpty(taskId00, taskId01, taskId02, taskId03); makeTaskFolders( diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java index 0f572a548e..0620dcfb00 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java @@ -164,4 +164,48 @@ public void shouldVerifyIfPendingActiveTaskToInitAreDrained() { assertTrue(tasks.hasPendingTasksToInit()); assertTrue(tasks.pendingTasksToInit().containsAll(mkSet(standbyTask1, standbyTask2))); } + + @Test + public void shouldAddFailedTask() { + final StreamTask activeTask1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)).build(); + final StreamTask activeTask2 = statefulTask(TASK_0_1, mkSet(TOPIC_PARTITION_B_1)).build(); + tasks.addTask(activeTask2); + + tasks.addFailedTask(activeTask1); + + assertEquals(activeTask1, tasks.task(TASK_0_0)); + assertEquals(activeTask2, tasks.task(TASK_0_1)); + assertTrue(tasks.allTasks().contains(activeTask1)); + assertTrue(tasks.allTasks().contains(activeTask2)); + assertFalse(tasks.allNonFailedTasks().contains(activeTask1)); + assertTrue(tasks.allNonFailedTasks().contains(activeTask2)); + } + + @Test + public void shouldRemoveFailedTask() { + final StreamTask activeTask1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)) + .inState(State.SUSPENDED).build(); + tasks.addFailedTask(activeTask1); + + tasks.removeTask(activeTask1); + assertFalse(tasks.allNonFailedTasks().contains(activeTask1)); + assertFalse(tasks.allTasks().contains(activeTask1)); + + tasks.addTask(activeTask1); + assertTrue(tasks.allNonFailedTasks().contains(activeTask1)); + } + + @Test + public void shouldClearFailedTask() { + final StreamTask activeTask1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)) + .inState(State.SUSPENDED).build(); + tasks.addFailedTask(activeTask1); + + tasks.clear(); + assertFalse(tasks.allNonFailedTasks().contains(activeTask1)); + assertFalse(tasks.allTasks().contains(activeTask1)); + + tasks.addTask(activeTask1); + assertTrue(tasks.allNonFailedTasks().contains(activeTask1)); + } } From 7d3ba8a0eb85ea67f02bc0ac3af7fc20a150be8a Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Fri, 13 Sep 2024 09:18:24 -0700 Subject: [PATCH 088/123] KAFKA-16468: verify that migrating brokers provide their inter.broker.listener (#17159) When brokers undergoing ZK migration register with the controller, it should verify that they have provided a way to contact them via their inter.broker.listener. Otherwise the migration will fail later on with a more confusing error message. Reviewers: David Arthur --- .../scala/kafka/server/ControllerServer.scala | 1 + .../BrokerRegistrationRequestTest.scala | 8 +++++ .../controller/ClusterControlManager.java | 25 ++++++++++++++-- .../kafka/controller/QuorumController.java | 13 +++++++-- .../controller/ClusterControlManagerTest.java | 29 +++++++++++++++++++ 5 files changed, 72 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 0e4321cffb..0905215f9a 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -266,6 +266,7 @@ class ControllerServer( setDelegationTokenExpiryTimeMs(config.delegationTokenExpiryTimeMs). setDelegationTokenExpiryCheckIntervalMs(config.delegationTokenExpiryCheckIntervalMs). setUncleanLeaderElectionCheckIntervalMs(config.uncleanLeaderElectionCheckIntervalMs). + setInterBrokerListenerName(config.interBrokerListenerName.value()). setEligibleLeaderReplicasEnabled(config.elrEnabled) } controller = controllerBuilder.build() diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala index 18625fd1a6..f3e481ebe3 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala @@ -37,6 +37,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.Timeout import org.junit.jupiter.api.extension.ExtendWith +import java.util import java.util.Collections import java.util.concurrent.{CompletableFuture, TimeUnit, TimeoutException} @@ -122,6 +123,13 @@ class BrokerRegistrationRequestTest { .setIncarnationId(Uuid.randomUuid()) .setIsMigratingZkBroker(zkEpoch.isDefined) .setFeatures(features) + .setListeners(new BrokerRegistrationRequestData.ListenerCollection(util.Arrays.asList( + new BrokerRegistrationRequestData.Listener(). + setName("EXTERNAL"). + setHost("example.com"). + setPort(8082). + setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)) + .iterator())) val resp = sendAndReceive[BrokerRegistrationRequest, BrokerRegistrationResponse]( channelManager, new BrokerRegistrationRequest.Builder(req), 30000) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 77ffd1036b..b0b43f05a4 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -93,6 +93,7 @@ static class Builder { private FeatureControlManager featureControl = null; private boolean zkMigrationEnabled = false; private BrokerUncleanShutdownHandler brokerUncleanShutdownHandler = null; + private String interBrokerListenerName = "PLAINTEXT"; Builder setLogContext(LogContext logContext) { this.logContext = logContext; @@ -139,6 +140,11 @@ Builder setBrokerUncleanShutdownHandler(BrokerUncleanShutdownHandler brokerUncle return this; } + Builder setInterBrokerListenerName(String interBrokerListenerName) { + this.interBrokerListenerName = interBrokerListenerName; + return this; + } + ClusterControlManager build() { if (logContext == null) { logContext = new LogContext(); @@ -166,7 +172,8 @@ ClusterControlManager build() { replicaPlacer, featureControl, zkMigrationEnabled, - brokerUncleanShutdownHandler + brokerUncleanShutdownHandler, + interBrokerListenerName ); } } @@ -260,6 +267,11 @@ boolean check() { private final BrokerUncleanShutdownHandler brokerUncleanShutdownHandler; + /** + * The statically configured inter-broker listener name. + */ + private final String interBrokerListenerName; + /** * Maps controller IDs to controller registrations. */ @@ -279,7 +291,8 @@ private ClusterControlManager( ReplicaPlacer replicaPlacer, FeatureControlManager featureControl, boolean zkMigrationEnabled, - BrokerUncleanShutdownHandler brokerUncleanShutdownHandler + BrokerUncleanShutdownHandler brokerUncleanShutdownHandler, + String interBrokerListenerName ) { this.logContext = logContext; this.clusterId = clusterId; @@ -296,6 +309,7 @@ private ClusterControlManager( this.controllerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0); this.directoryToBroker = new TimelineHashMap<>(snapshotRegistry, 0); this.brokerUncleanShutdownHandler = brokerUncleanShutdownHandler; + this.interBrokerListenerName = interBrokerListenerName; } ReplicaPlacer replicaPlacer() { @@ -377,6 +391,13 @@ public ControllerResult registerBroker( "brokers until the metadata migration is complete."); } + if (request.isMigratingZkBroker()) { + if (request.listeners().find(interBrokerListenerName) == null) { + throw new InvalidRegistrationException("Broker does not have the current inter.broker.listener " + + interBrokerListenerName); + } + } + if (featureControl.metadataVersion().isDirectoryAssignmentSupported()) { if (request.logDirs().isEmpty()) { throw new InvalidRegistrationException("No directories specified in request"); diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 64723b86c0..7445171627 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -225,6 +225,7 @@ public static class Builder { private long delegationTokenExpiryTimeMs; private long delegationTokenExpiryCheckIntervalMs; private long uncleanLeaderElectionCheckIntervalMs = TimeUnit.MINUTES.toMillis(5); + private String interBrokerListenerName = "PLAINTEXT"; public Builder(int nodeId, String clusterId) { this.nodeId = nodeId; @@ -380,6 +381,11 @@ public Builder setUncleanLeaderElectionCheckIntervalMs(long uncleanLeaderElectio return this; } + public Builder setInterBrokerListenerName(String interBrokerListenerName) { + this.interBrokerListenerName = interBrokerListenerName; + return this; + } + public QuorumController build() throws Exception { if (raftClient == null) { throw new IllegalStateException("You must set a raft client."); @@ -437,7 +443,8 @@ public QuorumController build() throws Exception { delegationTokenExpiryTimeMs, delegationTokenExpiryCheckIntervalMs, eligibleLeaderReplicasEnabled, - uncleanLeaderElectionCheckIntervalMs + uncleanLeaderElectionCheckIntervalMs, + interBrokerListenerName ); } catch (Exception e) { Utils.closeQuietly(queue, "event queue"); @@ -1868,7 +1875,8 @@ private QuorumController( long delegationTokenExpiryTimeMs, long delegationTokenExpiryCheckIntervalMs, boolean eligibleLeaderReplicasEnabled, - long uncleanLeaderElectionCheckIntervalMs + long uncleanLeaderElectionCheckIntervalMs, + String interBrokerListenerName ) { this.nonFatalFaultHandler = nonFatalFaultHandler; this.fatalFaultHandler = fatalFaultHandler; @@ -1925,6 +1933,7 @@ private QuorumController( setFeatureControlManager(featureControl). setZkMigrationEnabled(zkMigrationEnabled). setBrokerUncleanShutdownHandler(this::handleUncleanBrokerShutdown). + setInterBrokerListenerName(interBrokerListenerName). build(); this.producerIdControlManager = new ProducerIdControlManager.Builder(). setLogContext(logContext). diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index 0646d4aaa5..786891e695 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -852,4 +852,33 @@ public void testReRegistrationAndBrokerEpoch(boolean newIncarnationId) { clusterControl.brokerRegistrations().get(1).epoch()); } } + + @Test + public void testRegistrationWithIncorrectInterBrokerListenerName() { + ClusterControlManager clusterControl = new ClusterControlManager.Builder(). + setClusterId("pjvUwj3ZTEeSVQmUiH3IJw"). + setFeatureControlManager(new FeatureControlManager.Builder().build()). + setBrokerUncleanShutdownHandler((brokerId, records) -> { }). + setInterBrokerListenerName("INTERNAL"). + setZkMigrationEnabled(true). + build(); + clusterControl.activate(); + assertEquals("Broker does not have the current inter.broker.listener INTERNAL", + assertThrows(InvalidRegistrationException.class, + () -> clusterControl.registerBroker( + new BrokerRegistrationRequestData(). + setBrokerId(1). + setClusterId(clusterControl.clusterId()). + setIncarnationId(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww")). + setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))). + setIsMigratingZkBroker(true). + setListeners(new BrokerRegistrationRequestData.ListenerCollection(Collections.singleton( + new BrokerRegistrationRequestData.Listener(). + setName("PLAINTEXT"). + setHost("example.com"). + setPort(9092). + setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)).iterator())), + 111, + new FinalizedControllerFeatures(Collections.emptyMap(), 100L))).getMessage()); + } } From b43439482d67950cfe4d4f2b1f89c95bf66d1330 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Fri, 13 Sep 2024 16:34:15 -0700 Subject: [PATCH 089/123] KAFKA-17527: Fix NPE for null RecordContext (#17169) Reviewers: Bruno Cadonna --- .../streams/errors/ErrorHandlerContext.java | 21 ++- .../internals/DefaultErrorHandlerContext.java | 2 +- .../internals/RecordCollectorImpl.java | 136 +++++++++++------- .../internals/RecordCollectorTest.java | 92 ++++++++++++ 4 files changed, 193 insertions(+), 58 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java index af67c8f03a..82d3258125 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java @@ -28,7 +28,10 @@ /** - * This interface allows user code to inspect the context of a record that has failed processing. + * This interface allows user code to inspect the context of a record that has failed during processing. + * + *

    {@code ErrorHandlerContext} instances are passed into {@link DeserializationExceptionHandler}, + * {@link ProcessingExceptionHandler}, or {@link ProductionExceptionHandler} dependent on what error occurred. */ public interface ErrorHandlerContext { /** @@ -42,6 +45,8 @@ public interface ErrorHandlerContext { * {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} * (and siblings), that do not always guarantee to provide a valid topic name, as they might be * executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. + * Additionally, when writing into a changelog topic, there is no associated input record, + * and thus no topic name is available. * * @return the topic name */ @@ -58,6 +63,8 @@ public interface ErrorHandlerContext { * {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} * (and siblings), that do not always guarantee to provide a valid partition ID, as they might be * executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. + * Additionally, when writing into a changelog topic, there is no associated input record, + * and thus no partition is available. * * @return the partition ID */ @@ -74,6 +81,8 @@ public interface ErrorHandlerContext { * {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} * (and siblings), that do not always guarantee to provide a valid offset, as they might be * executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. + * Additionally, when writing into a changelog topic, there is no associated input record, + * and thus no offset is available. * * @return the offset */ @@ -90,6 +99,8 @@ public interface ErrorHandlerContext { * {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} * (and siblings), that do not always guarantee to provide valid headers, as they might be * executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. + * Additionally, when writing into a changelog topic, there is no associated input record, + * and thus no headers are available. * * @return the headers */ @@ -110,7 +121,10 @@ public interface ErrorHandlerContext { TaskId taskId(); /** - * Return the current timestamp. + * Return the current timestamp; could be {@code -1} if it is not available. + * + *

    For example, when writing into a changelog topic, there is no associated input record, + * and thus no timestamp is available. * *

    If it is triggered while processing a record streamed from the source processor, * timestamp is defined as the timestamp of the current input record; the timestamp is extracted from @@ -129,8 +143,7 @@ public interface ErrorHandlerContext { * * *

    If it is triggered from a deserialization failure, timestamp is defined as the timestamp of the - * current rawRecord - * {@link org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} + * current rawRecord {@link org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord}. * * @return the timestamp */ diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java index fc44e9c95f..efaa6d57e7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java @@ -35,7 +35,7 @@ public class DefaultErrorHandlerContext implements ErrorHandlerContext { private final TaskId taskId; private final long timestamp; - private ProcessorContext processorContext; + private final ProcessorContext processorContext; public DefaultErrorHandlerContext(final ProcessorContext processorContext, final String topic, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 7525513ea2..b725feb4ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.LogContext; @@ -45,6 +46,7 @@ import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.errors.internals.FailedProcessingException; +import org.apache.kafka.streams.processor.RecordContext; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -251,40 +253,53 @@ public void send(final String topic, final ProducerRecord serializedRecord = new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes, headers); streamsProducer.send(serializedRecord, (metadata, exception) -> { - // if there's already an exception record, skip logging offsets or new exceptions - if (sendException.get() != null) { - return; - } - - if (exception == null) { - final TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); - if (metadata.offset() >= 0L) { - offsets.put(tp, metadata.offset()); - } else { - log.warn("Received offset={} in produce response for {}", metadata.offset(), tp); + try { + // if there's already an exception record, skip logging offsets or new exceptions + if (sendException.get() != null) { + return; } - if (!topic.endsWith("-changelog")) { - // we may not have created a sensor during initialization if the node uses dynamic topic routing, - // as all topics are not known up front, so create the sensor for this topic if absent - final Sensor topicProducedSensor = producedSensorByTopic.computeIfAbsent( - topic, - t -> TopicMetrics.producedSensor( - Thread.currentThread().getName(), - taskId.toString(), - processorNodeId, + if (exception == null) { + final TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); + if (metadata.offset() >= 0L) { + offsets.put(tp, metadata.offset()); + } else { + log.warn("Received offset={} in produce response for {}", metadata.offset(), tp); + } + + if (!topic.endsWith("-changelog")) { + // we may not have created a sensor during initialization if the node uses dynamic topic routing, + // as all topics are not known up front, so create the sensor for this topic if absent + final Sensor topicProducedSensor = producedSensorByTopic.computeIfAbsent( topic, - context.metrics() - ) + t -> TopicMetrics.producedSensor( + Thread.currentThread().getName(), + taskId.toString(), + processorNodeId, + topic, + context.metrics() + ) + ); + final long bytesProduced = producerRecordSizeInBytes(serializedRecord); + topicProducedSensor.record( + bytesProduced, + context.currentSystemTimeMs() + ); + } + } else { + recordSendError( + topic, + exception, + serializedRecord, + context, + processorNodeId ); - final long bytesProduced = producerRecordSizeInBytes(serializedRecord); - topicProducedSensor.record(bytesProduced, context.currentSystemTimeMs()); - } - } else { - recordSendError(topic, exception, serializedRecord, context, processorNodeId); - // KAFKA-7510 only put message key and value in TRACE level log so we don't leak data by default - log.trace("Failed record: (key {} value {} timestamp {}) topic=[{}] partition=[{}]", key, value, timestamp, topic, partition); + // KAFKA-7510 only put message key and value in TRACE level log so we don't leak data by default + log.trace("Failed record: (key {} value {} timestamp {}) topic=[{}] partition=[{}]", key, value, timestamp, topic, partition); + } + } catch (final RuntimeException fatal) { + sendException.set(new StreamsException("Producer.send `Callback` failed", fatal)); } }); } @@ -301,22 +316,17 @@ private void handleException(final ProductionExceptionHandler.Serializati final RuntimeException serializationException) { log.debug(String.format("Error serializing record for topic %s", topic), serializationException); - final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( - null, // only required to pass for DeserializationExceptionHandler - context.recordContext().topic(), - context.recordContext().partition(), - context.recordContext().offset(), - context.recordContext().headers(), - processorNodeId, - taskId, - context.recordContext().timestamp() - ); final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); final ProductionExceptionHandlerResponse response; try { response = Objects.requireNonNull( - productionExceptionHandler.handleSerializationException(errorHandlerContext, record, serializationException, origin), + productionExceptionHandler.handleSerializationException( + errorHandlerContext(context, processorNodeId), + record, + serializationException, + origin + ), "Invalid ProductionExceptionHandler response." ); } catch (final RuntimeException fatalUserException) { @@ -324,7 +334,7 @@ private void handleException(final ProductionExceptionHandler.Serializati String.format( "Production error callback failed after serialization error for record %s: %s", origin.toString().toLowerCase(Locale.ROOT), - errorHandlerContext + errorHandlerContext(context, processorNodeId) ), serializationException ); @@ -351,6 +361,33 @@ private void handleException(final ProductionExceptionHandler.Serializati droppedRecordsSensor.record(); } + private DefaultErrorHandlerContext errorHandlerContext(final InternalProcessorContext context, + final String processorNodeId) { + final RecordContext recordContext = context != null ? context.recordContext() : null; + + return recordContext != null ? + new DefaultErrorHandlerContext( + context, + recordContext.topic(), + recordContext.partition(), + recordContext.offset(), + recordContext.headers(), + processorNodeId, + taskId, + recordContext.timestamp() + ) : + new DefaultErrorHandlerContext( + context, + null, + -1, + -1, + new RecordHeaders(), + processorNodeId, + taskId, + -1L + ); + } + private StreamsException createStreamsExceptionForClassCastException(final ProductionExceptionHandler.SerializationExceptionOrigin origin, final String topic, final KV keyOrValue, @@ -399,21 +436,14 @@ private void recordSendError(final String topic, "`delivery.timeout.ms` to a larger value to wait longer for such scenarios and avoid timeout errors"; sendException.set(new TaskCorruptedException(Collections.singleton(taskId))); } else { - final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( - null, // only required to pass for DeserializationExceptionHandler - context.recordContext().topic(), - context.recordContext().partition(), - context.recordContext().offset(), - context.recordContext().headers(), - processorNodeId, - taskId, - context.recordContext().timestamp() - ); - final ProductionExceptionHandlerResponse response; try { response = Objects.requireNonNull( - productionExceptionHandler.handle(errorHandlerContext, serializedRecord, productionException), + productionExceptionHandler.handle( + errorHandlerContext(context, processorNodeId), + serializedRecord, + productionException + ), "Invalid ProductionExceptionHandler response." ); } catch (final RuntimeException fatalUserException) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index 9ed655332d..e31d247eb7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -1558,6 +1558,98 @@ public void shouldThrowStreamsExceptionWhenKeySerializationFailedAndProductionEx } } + @Test + public void shouldNotFailIfContextIsNotAvailableOnSerializationError() { + try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + streamsProducer, + productionExceptionHandler, + streamsMetrics, + topology + ); + + assertThrows( + StreamsException.class, // should not crash with NullPointerException + () -> collector.send( + topic, + "key", + "val", + null, + 0, + null, + errorSerializer, + stringSerializer, + sinkNodeName, + null // pass `null` context for testing + ) + ); + } + } + + @Test + public void shouldNotFailIfRecordContextIsNotAvailableOnSerializationError() { + try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + streamsProducer, + productionExceptionHandler, + streamsMetrics, + topology + ); + + // RecordContext is null when writing into a changelog topic + context.setRecordContext(null); + assertThrows( + StreamsException.class, // should not crash with NullPointerException + () -> collector.send(topic, "key", "val", null, 0, null, errorSerializer, stringSerializer, sinkNodeName, context) + ); + } + } + + @Test + public void shouldNotFailIfContextIsNotAvailableOnSendError() { + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + getExceptionalStreamsProducerOnSend(new RuntimeException("Kaboom!")), + productionExceptionHandler, + streamsMetrics, + topology + ); + + collector.send( + topic, + "key", + "val", + null, + 0, + null, + stringSerializer, + stringSerializer, + sinkNodeName, + null // pass `null` context for testing + ); + } + + @Test + public void shouldNotFailIfRecordContextIsNotAvailableOnSendError() { + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + getExceptionalStreamsProducerOnSend(new RuntimeException("Kaboom!")), + productionExceptionHandler, + streamsMetrics, + topology + ); + + // RecordContext is null when writing into a changelog topic + context.setRecordContext(null); + collector.send(topic, "key", "val", null, 0, null, stringSerializer, stringSerializer, sinkNodeName, context); + } + @Test public void shouldThrowStreamsExceptionWhenSerializationFailedAndProductionExceptionHandlerReturnsNull() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { From a1a4389c35026805cdccd5aa1a5b5d1ac5cd5205 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Fri, 13 Sep 2024 17:24:48 -0700 Subject: [PATCH 090/123] KAFKA-17543: Enforce that broker.id.generation.enable is not used when migrating to KRaft (#17192) Reviewers: Chia-Ping Tsai , David Arthur --- core/src/main/scala/kafka/server/KafkaConfig.scala | 5 +++++ core/src/main/scala/kafka/server/KafkaServer.scala | 3 --- .../scala/unit/kafka/server/KafkaConfigTest.scala | 11 +++++++++++ .../scala/unit/kafka/server/KafkaServerTest.scala | 1 - 4 files changed, 16 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 1edb310016..ff3ed79cfe 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -852,6 +852,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) throw new ConfigException(s"Missing required configuration `${ZkConfigs.ZK_CONNECT_CONFIG}` which has no default value.") } if (brokerIdGenerationEnable) { + if (migrationEnabled) { + require(brokerId != -1, "broker id generation is incompatible with migration to ZK. Please disable it before enabling migration") + } require(brokerId >= -1 && brokerId <= maxReservedBrokerId, "broker.id must be greater than or equal to -1 and not greater than reserved.broker.max.id") } else { require(brokerId >= 0, "broker.id must be greater than or equal to 0") @@ -969,6 +972,8 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) } else { // ZK-based if (migrationEnabled) { + require(brokerId >= 0, + "broker broker.id.generation.enable is incompatible with migration to ZK. Please disable it before enabling migration") validateQuorumVotersAndQuorumBootstrapServerForMigration() require(controllerListenerNames.nonEmpty, s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must not be empty when running in ZooKeeper migration mode: ${controllerListenerNames.asJava}") diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 6026b566bf..3d70b65276 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -259,9 +259,6 @@ class KafkaServer( /* generate brokerId */ config._brokerId = getOrGenerateBrokerId(initialMetaPropsEnsemble) - // Currently, we are migrating from ZooKeeper to KRaft. If broker.id.generation.enable is set to true, - // we must ensure that the nodeId synchronizes with the broker.id to prevent the nodeId from being -1, - // which would result in a failure during the migration. config._nodeId = config.brokerId logContext = new LogContext(s"[KafkaServer id=${config.brokerId}] ") this.logIdent = logContext.logPrefix diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 3d37cfccd6..6ec2093630 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1884,6 +1884,17 @@ class KafkaConfigTest { assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage) } + @Test + def testMigrationCannotBeEnabledWithBrokerIdGeneration(): Unit = { + val props = TestUtils.createBrokerConfig(-1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort, logDirCount = 2) + props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true") + props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093") + props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") + assertEquals( + "requirement failed: broker id generation is incompatible with migration to ZK. Please disable it before enabling migration", + assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage) + } + @Test def testMigrationEnabledKRaftMode(): Unit = { val props = new Properties() diff --git a/core/src/test/scala/unit/kafka/server/KafkaServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaServerTest.scala index e8cf3ecfa5..ea9256baa5 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaServerTest.scala @@ -201,5 +201,4 @@ class KafkaServerTest extends QuorumTestHarness { val kafkaConfig = KafkaConfig.fromProps(props) TestUtils.createServer(kafkaConfig) } - } From f324ef461f9d9e7fc38324336727e0cc269b95db Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 17 Sep 2024 07:36:08 -0700 Subject: [PATCH 091/123] MINOR: update documentation link to 3.9 (#17216) Reviewers: David Arthur --- docs/documentation.html | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/docs/documentation.html b/docs/documentation.html index a1d5c43b9b..c3dbcf32ca 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -33,7 +33,7 @@

    Documentation

    -

    Kafka 3.8 Documentation

    +

    Kafka 3.9 Documentation

    Prior releases: 0.7.x, 0.8.0, 0.8.1.X, @@ -54,14 +54,15 @@

    Kafka 3.8 Documentation

    2.6.X, 2.7.X, 2.8.X, - 3.0.X. - 3.1.X. - 3.2.X. - 3.3.X. - 3.4.X. - 3.5.X. - 3.6.X. - 3.7.X. + 3.0.X, + 3.1.X, + 3.2.X, + 3.3.X, + 3.4.X, + 3.5.X, + 3.6.X, + 3.7.X, + 3.8.X.

    1. Getting Started

    1.1 Introduction

    From 389a8d8decc05a4d73ab1dce6f158b2d13398450 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 17 Sep 2024 12:05:25 -0700 Subject: [PATCH 092/123] Revert "KAFKA-16803: Change fork, update ShadowJavaPlugin to 8.1.7 (#16295)" (#17218) This reverts commit 391778b8d737f4af074422ffe61bc494b21e6555. Unfortunately that commit re-introduced bug #15127 which prevented the publishing of kafka-clients artifacts to remote maven. As that bug says: The issue triggers only with publishMavenJavaPublicationToMavenRepository due to signing. Generating signed asc files error out for shadowed release artifacts as the module name (clients) differs from the artifact name (kafka-clients). The fix is basically to explicitly define artifact of shadowJar to signing and publish plugin. project.shadow.component(mavenJava) previously outputs the name as client--all.jar though the classifier and archivesBaseName are already defined correctly in :clients and shadowJar construction. Reviewers: David Arthur --- build.gradle | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index 28ff4495e2..f73255bd83 100644 --- a/build.gradle +++ b/build.gradle @@ -44,7 +44,9 @@ plugins { // be dropped from gradle/resources/dependencycheck-suppressions.xml id "com.github.spotbugs" version '5.1.3' apply false id 'org.scoverage' version '8.0.3' apply false - id 'io.github.goooler.shadow' version '8.1.3' apply false + // Updating the shadow plugin version to 8.1.1 causes issue with signing and publishing the shadowed + // artifacts - see https://github.com/johnrengelman/shadow/issues/901 + id 'com.github.johnrengelman.shadow' version '8.1.0' apply false // Spotless 6.13.0 has issue with Java 21 (see https://github.com/diffplug/spotless/pull/1920), and Spotless 6.14.0+ requires JRE 11 // We are going to drop JDK8 support. Hence, the spotless is upgrade to newest version and be applied only if the build env is compatible with JDK 11. // spotless 6.15.0+ has issue in runtime with JDK8 even through we define it with `apply:false`. see https://github.com/diffplug/spotless/issues/2156 for more details @@ -336,7 +338,7 @@ subprojects { if (!shouldPublishWithShadow) { from components.java } else { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.github.johnrengelman.shadow' project.shadow.component(mavenJava) // Fix for avoiding inclusion of runtime dependencies marked as 'shadow' in MANIFEST Class-Path. @@ -2958,7 +2960,7 @@ project(':streams:upgrade-system-tests-38') { project(':jmh-benchmarks') { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.github.johnrengelman.shadow' shadowJar { archiveBaseName = 'kafka-jmh-benchmarks' From c141acb6bf88e549de2459e834657439a2bb5be9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Wed, 18 Sep 2024 14:23:58 -0400 Subject: [PATCH 093/123] KAFKA-17048; Update docs for KIP-853 (#17076) Change the configurations under config/kraft to use controller.quorum.bootstrap.servers instead of controller.quorum.voters. Add comments explaining how to use the older static quorum configuration where appropriate. In docs/ops.html, remove the reference to "tentative timelines for ZooKeeper removal" and "Tiered storage is considered as an early access feature" since they are no longer up-to-date. Add KIP-853 information. In docs/quickstart.html, move the ZK instructions to be after the KRaft instructions. Update the KRaft instructions to use KIP-853. In docs/security.html, add an explanation of --bootstrap-controller and document controller.quorum.bootstrap.servers instead of controller.quorum.voters. Reviewers: Mickael Maison , Alyssa Huang , Colin P. McCabe --- config/kraft/broker.properties | 4 +- config/kraft/controller.properties | 8 +- config/kraft/reconfig-server.properties | 132 ++++++++++++++++++++++++ docs/ops.html | 89 +++++++++++++--- docs/quickstart.html | 35 ++++--- docs/security.html | 14 +-- 6 files changed, 236 insertions(+), 46 deletions(-) create mode 100644 config/kraft/reconfig-server.properties diff --git a/config/kraft/broker.properties b/config/kraft/broker.properties index 2d15997f28..f32d1aae53 100644 --- a/config/kraft/broker.properties +++ b/config/kraft/broker.properties @@ -26,8 +26,8 @@ process.roles=broker # The node id associated with this instance's roles node.id=2 -# The connect string for the controller quorum -controller.quorum.voters=1@localhost:9093 +# Information about the KRaft controller quorum. +controller.quorum.bootstrap.servers=localhost:9093 ############################# Socket Server Settings ############################# diff --git a/config/kraft/controller.properties b/config/kraft/controller.properties index bc4cda107a..8201504468 100644 --- a/config/kraft/controller.properties +++ b/config/kraft/controller.properties @@ -26,13 +26,15 @@ process.roles=controller # The node id associated with this instance's roles node.id=1 -# The connect string for the controller quorum -controller.quorum.voters=1@localhost:9093 +# Information about the KRaft controller quorum. +# Uncomment controller.quorum.voters to use a static controller quorum. +#controller.quorum.voters=1@localhost:9093 +controller.quorum.bootstrap.servers=localhost:9093 ############################# Socket Server Settings ############################# # The address the socket server listens on. -# Note that only the controller listeners are allowed here when `process.roles=controller`, and this listener should be consistent with `controller.quorum.voters` value. +# Note that only the controller listeners are allowed here when `process.roles=controller` # FORMAT: # listeners = listener_name://host_name:port # EXAMPLE: diff --git a/config/kraft/reconfig-server.properties b/config/kraft/reconfig-server.properties new file mode 100644 index 0000000000..a0a25133e8 --- /dev/null +++ b/config/kraft/reconfig-server.properties @@ -0,0 +1,132 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# +# This configuration file is intended for use in KRaft mode, where +# Apache ZooKeeper is not present. +# + +############################# Server Basics ############################# + +# The role of this server. Setting this puts us in KRaft mode +process.roles=broker,controller + +# The node id associated with this instance's roles +node.id=1 + +# List of controller endpoints used connect to the controller cluster +controller.quorum.bootstrap.servers=localhost:9093 + +############################# Socket Server Settings ############################# + +# The address the socket server listens on. +# Combined nodes (i.e. those with `process.roles=broker,controller`) must list the controller listener here at a minimum. +# If the broker listener is not defined, the default listener will use a host name that is equal to the value of java.net.InetAddress.getCanonicalHostName(), +# with PLAINTEXT listener name, and port 9092. +# FORMAT: +# listeners = listener_name://host_name:port +# EXAMPLE: +# listeners = PLAINTEXT://your.host.name:9092 +listeners=PLAINTEXT://:9092,CONTROLLER://:9093 + +# Name of listener used for communication between brokers. +inter.broker.listener.name=PLAINTEXT + +# Listener name, hostname and port the broker or the controller will advertise to clients. +# If not set, it uses the value for "listeners". +advertised.listeners=PLAINTEXT://localhost:9092,CONTROLLER://localhost:9093 + +# A comma-separated list of the names of the listeners used by the controller. +# If no explicit mapping set in `listener.security.protocol.map`, default will be using PLAINTEXT protocol +# This is required if running in KRaft mode. +controller.listener.names=CONTROLLER + +# Maps listener names to security protocols, the default is for them to be the same. See the config documentation for more details +listener.security.protocol.map=CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT,SSL:SSL,SASL_PLAINTEXT:SASL_PLAINTEXT,SASL_SSL:SASL_SSL + +# The number of threads that the server uses for receiving requests from the network and sending responses to the network +num.network.threads=3 + +# The number of threads that the server uses for processing requests, which may include disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=102400 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=102400 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma separated list of directories under which to store log files +log.dirs=/tmp/kraft-combined-logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions=1 + +# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. +# This value is recommended to be increased for installations with data dirs located in RAID array. +num.recovery.threads.per.data.dir=1 + +############################# Internal Topic Settings ############################# +# The replication factor for the group metadata internal topics "__consumer_offsets" and "__transaction_state" +# For anything other than development testing, a value greater than 1 is recommended to ensure availability such as 3. +offsets.topic.replication.factor=1 +transaction.state.log.replication.factor=1 +transaction.state.log.min.isr=1 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to excessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion due to age +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log unless the remaining +# segments drop below log.retention.bytes. Functions independently of log.retention.hours. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=1073741824 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=300000 diff --git a/docs/ops.html b/docs/ops.html index 8f8b0b95a4..c3a7212c9b 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3736,9 +3736,6 @@
    3.x and ZooKeeper Support

    The final 3.x minor release, that supports ZooKeeper mode, will receive critical bug fixes and security fixes for 12 months after its release.

    -
    ZooKeeper and KRaft timeline
    -

    For details and updates on tentative timelines for ZooKeeper removal and planned KRaft feature releases, refer to KIP-833.

    -

    Operationalizing ZooKeeper

    Operationally, we do the following for a healthy ZooKeeper installation:
      @@ -3776,25 +3773,77 @@
      controller.quorum.voters=id1@host1:port1,id2@host2:port2,id3@host3:port3 +
      controller.quorum.bootstrap.servers=host1:port1,host2:port2,host3:port3

      If a Kafka cluster has 3 controllers named controller1, controller2 and controller3, then controller1 may have the following configuration:

      process.roles=controller
       node.id=1
       listeners=CONTROLLER://controller1.example.com:9093
      -controller.quorum.voters=1@controller1.example.com:9093,2@controller2.example.com:9093,3@controller3.example.com:9093
      +controller.quorum.bootstrap.servers=controller1.example.com:9093,controller2.example.com:9093,controller3.example.com:9093 +controller.listener.names=CONTROLLER -

      Every broker and controller must set the controller.quorum.voters property. The node ID supplied in the controller.quorum.voters property must match the corresponding id on the controller servers. For example, on controller1, node.id must be set to 1, and so forth. Each node ID must be unique across all the servers in a particular cluster. No two servers can have the same node ID regardless of their process.roles values. +

      Every broker and controller must set the controller.quorum.bootstrap.servers property. -

      Storage Tool

      +

      Provisioning Nodes

      The kafka-storage.sh random-uuid command can be used to generate a cluster ID for your new cluster. This cluster ID must be used when formatting each server in the cluster with the kafka-storage.sh format command.

      This is different from how Kafka has operated in the past. Previously, Kafka would format blank storage directories automatically, and also generate a new cluster ID automatically. One reason for the change is that auto-formatting can sometimes obscure an error condition. This is particularly important for the metadata log maintained by the controller and broker servers. If a majority of the controllers were able to start with an empty log directory, a leader might be able to be elected with missing committed data.

      +
      Bootstrap a Standalone Controller
      + The recommended method for creating a new KRaft controller cluster is to bootstrap it with one voter and dynamically add the rest of the controllers. Bootstrapping the first controller can be done with the following CLI command: + +
      $ bin/kafka-storage format --cluster-id  --standalone --config controller.properties
      + + This command will 1) create a meta.properties file in metadata.log.dir with a randomly generated directory.id, 2) create a snapshot at 00000000000000000000-0000000000.checkpoint with the necessary control records (KRaftVersionRecord and VotersRecord) to make this Kafka node the only voter for the quorum. + +
      Bootstrap with Multiple Controllers
      + The KRaft cluster metadata partition can also be bootstrapped with more than one voter. This can be done by using the --initial-controllers flag: + +
      cluster-id=$(kafka-storage random-uuid)
      +controller-0-uuid=$(kafka-storage random-uuid)
      +controller-1-uuid=$(kafka-storage random-uuid)
      +controller-2-uuid=$(kafka-storage random-uuid)
      +
      +# In each controller execute
      +kafka-storage format --cluster-id ${cluster-id} \
      +                     --initial-controllers "0@controller-0:1234:${controller-0-uuid},1@controller-1:1234:${controller-1-uuid},2@controller-2:1234:${controller-2-uuid}" \
      +                     --config controller.properties
      + +This command is similar to the standalone version but the snapshot at 00000000000000000000-0000000000.checkpoint will instead contain a VotersRecord that includes information for all of the controllers specified in --initial-controllers. It is important that the value of this flag is the same in all of the controllers with the same cluster id. + +In the replica description 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA, 0 is the replica id, 3Db5QLSqSZieL3rJBUUegA is the replica directory id, controller-0 is the replica's host and 1234 is the replica's port. + +
      Formatting Brokers and New Controllers
      + When provisioning new broker and controller nodes that we want to add to an existing Kafka cluster, use the kafka-storage.sh format command without the --standalone or --initial-controllers flags. + +
      $ bin/kafka-storage format --cluster-id  --config server.properties
      + +

      Controller membership changes

      + +
      Add New Controller
      + If the KRaft Controller cluster already exists, the cluster can be expanded by first provisioning a new controller using the kafka-storage tool and starting the controller. + + After starting the controller, the replication to the new controller can be monitored using the kafka-metadata-quorum describe --replication command. Once the new controller has caught up to the active controller, it can be added to the cluster using the kafka-metadata-quorum add-controller command. + + When using broker endpoints use the --bootstrap-server flag: +
      $ bin/kafka-metadata-quorum --command-config controller.properties --bootstrap-server localhost:9092 add-controller
      + + When using controller endpoints use the --bootstrap-controller flag: +
      $ bin/kafka-metadata-quorum --command-config controller.properties --bootstrap-controller localhost:9092 add-controller
      + +
      Remove Controller
      + If the KRaft Controller cluster already exists, the cluster can be shrunk using the kafka-metadata-quorum remove-controller command. Until KIP-996: Pre-vote has been implemented and released, it is recommended to shutdown the controller that will be removed before running the remove-controller command. + + When using broker endpoints use the --bootstrap-server flag: +
      $ bin/kafka-metadata-quorum --bootstrap-server localhost:9092 remove-controller --controller-id  --controller-directory-id 
      + + When using controller endpoints use the --bootstrap-controller flag: +
      $ bin/kafka-metadata-quorum --bootstrap-controller localhost:9092 remove-controller --controller-id  --controller-directory-id 
      +

      Debugging

      Metadata Quorum Tool
      @@ -3808,8 +3857,12 @@
      < HighWatermark: 10 MaxFollowerLag: 0 MaxFollowerLagTimeMs: -1 -CurrentVoters: [3000,3001,3002] -CurrentObservers: [0,1,2] +CurrentVoters: [{"id": 3000, "directoryId": "ILZ5MPTeRWakmJu99uBJCA", "endpoints": ["CONTROLLER://localhost:9093"]}, + {"id": 3001, "directoryId": "b-DwmhtOheTqZzPoh52kfA", "endpoints": ["CONTROLLER://localhost:9094"]}, + {"id": 3002, "directoryId": "g42deArWBTRM5A1yuVpMCg", "endpoints": ["CONTROLLER://localhost:9095"]}] +CurrentObservers: [{"id": 0, "directoryId": "3Db5QLSqSZieL3rJBUUegA"}, + {"id": 1, "directoryId": "UegA3Db5QLSqSZieL3rJBU"}, + {"id": 2, "directoryId": "L3rJBUUegA3Db5QLSqSZie"}]
      Dump Log Tool
      @@ -3941,7 +3994,7 @@

      Provisioning the KRaft controller quorum

      # Sample KRaft cluster controller.properties listening on 9093
       process.roles=controller
       node.id=3000
      -controller.quorum.voters=3000@localhost:9093
      +controller.quorum.bootstrap.servers=localhost:9093
       controller.listener.names=CONTROLLER
       listeners=CONTROLLER://:9093
       
      @@ -3956,6 +4009,8 @@ 

      Provisioning the KRaft controller quorum

      # Other configs ...
      +

      The new standalone controller in the example configuration above should be formatted using the kafka-storage format --standalonecommand.

      +

      Note: The KRaft cluster node.id values must be different from any existing ZK broker broker.id. In KRaft-mode, the brokers and controllers share the same Node ID namespace.

      @@ -3968,7 +4023,7 @@

      Enter Migration Mode on the Brokers

      • broker.id: Ensure broker.id is set to a non-negative integer even if broker.id.generation.enable is enabled (default is enabled). Additionally, ensure broker.id does not exceed reserved.broker.max.id to avoid failure.
      • -
      • controller.quorum.voters
      • +
      • controller.quorum.bootstrap.servers
      • controller.listener.names
      • The controller.listener.name should also be added to listener.security.property.map
      • zookeeper.metadata.migration.enable
      • @@ -3992,7 +4047,7 @@

        Enter Migration Mode on the Brokers

        zookeeper.connect=localhost:2181 # KRaft controller quorum configuration -controller.quorum.voters=3000@localhost:9093 +controller.quorum.bootstrap.servers=localhost:9093 controller.listener.names=CONTROLLER

        @@ -4040,7 +4095,7 @@

        Migrating brokers to KRaft

        # zookeeper.connect=localhost:2181 # Keep the KRaft controller quorum configuration -controller.quorum.voters=3000@localhost:9093 +controller.quorum.bootstrap.servers=localhost:9093 controller.listener.names=CONTROLLER

        @@ -4061,7 +4116,7 @@

        Finalizing the migration

        # Sample KRaft cluster controller.properties listening on 9093
         process.roles=controller
         node.id=3000
        -controller.quorum.voters=3000@localhost:9093
        +controller.quorum.bootstrap.servers=localhost:9093
         controller.listener.names=CONTROLLER
         listeners=CONTROLLER://:9093
         
        @@ -4133,7 +4188,7 @@ 

        Reverting to ZooKeeper mode During the Migration

      • On each broker, remove the zookeeper.metadata.migration.enable, - controller.listener.names, and controller.quorum.voters + controller.listener.names, and controller.quorum.bootstrap.servers configurations, and replace node.id with broker.id. Then perform a rolling restart of all brokers.
      • @@ -4170,7 +4225,7 @@

        Reverting to ZooKeeper mode During the Migration

      • On each broker, remove the zookeeper.metadata.migration.enable, - controller.listener.names, and controller.quorum.voters + controller.listener.names, and controller.quorum.bootstrap.servers configurations. Then perform a second rolling restart of all brokers.
      • diff --git a/docs/quickstart.html b/docs/quickstart.html index e7dff94004..e1d37fc457 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -50,30 +50,20 @@
        Kafka with KRaft

        Kafka can be run using KRaft mode using local scripts and downloaded files or the docker image. Follow one of the sections below but not both to start the kafka server.

        -
        Using downloaded files
        +
        Using downloaded files

        Generate a Cluster UUID

        $ KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)"

        Format Log Directories

        -
        $ bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties
        +
        $ bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/kraft/reconfig-server.properties

        Start the Kafka Server

        -
        $ bin/kafka-server-start.sh config/kraft/server.properties
        +
        $ bin/kafka-server-start.sh config/kraft/reconfig-server.properties
        -
        Kafka with ZooKeeper
        - -

        Run the following commands in order to start all services in the correct order:

        -
        # Start the ZooKeeper service
        -$ bin/zookeeper-server-start.sh config/zookeeper.properties
        - -

        Open another terminal session and run:

        -
        # Start the Kafka broker service
        -$ bin/kafka-server-start.sh config/server.properties
        - -

        Once all services have successfully launched, you will have a basic Kafka environment running and ready to use.

        +

        Once the Kafka server has successfully launched, you will have a basic Kafka environment running and ready to use.

        -
        Using JVM Based Apache Kafka Docker Image
        +
        Using JVM Based Apache Kafka Docker Image

        Get the Docker image:

        $ docker pull apache/kafka:{{fullDotVersion}}
        @@ -81,7 +71,7 @@
        Using JVM Based Apache Kafka Docker Image

        Start the Kafka Docker container:

        $ docker run -p 9092:9092 apache/kafka:{{fullDotVersion}}
        -
        Using GraalVM Based Native Apache Kafka Docker Image
        +
        Using GraalVM Based Native Apache Kafka Docker Image

        Get the Docker image:

        $ docker pull apache/kafka-native:{{fullDotVersion}}
        @@ -89,7 +79,18 @@
        Using GraalVM Based Native Apache Kafka Docker Image

        Start the Kafka Docker container:

        $ docker run -p 9092:9092 apache/kafka-native:{{fullDotVersion}}
        -

        Once the Kafka server has successfully launched, you will have a basic Kafka environment running and ready to use.

        +
        Kafka with ZooKeeper
        + +

        Run the following commands in order to start all services in the correct order:

        +
        # Start the ZooKeeper service
        +$ bin/zookeeper-server-start.sh config/zookeeper.properties
        + +

        Open another terminal session and run:

        +
        # Start the Kafka broker service
        +$ bin/kafka-server-start.sh config/server.properties
        + +

        Once all services have successfully launched, you will have a basic Kafka environment running and ready to use.

        +
    diff --git a/docs/security.html b/docs/security.html index 6f3f81204a..fa710c226e 100644 --- a/docs/security.html +++ b/docs/security.html @@ -120,7 +120,7 @@

    process.roles=broker listeners=BROKER://localhost:9092 inter.broker.listener.name=BROKER -controller.quorum.voters=0@localhost:9093 +controller.quorum.bootstrap.servers=localhost:9093 controller.listener.names=CONTROLLER listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL @@ -137,15 +137,15 @@

    process.roles=broker,controller listeners=BROKER://localhost:9092,CONTROLLER://localhost:9093 inter.broker.listener.name=BROKER -controller.quorum.voters=0@localhost:9093 +controller.quorum.bootstrap.servers=localhost:9093 controller.listener.names=CONTROLLER listener.security.protocol.map=BROKER:SASL_SSL,CONTROLLER:SASL_SSL -

    It is a requirement for the port defined in controller.quorum.voters to - exactly match one of the exposed controller listeners. For example, here the - CONTROLLER listener is bound to port 9093. The connection string - defined by controller.quorum.voters must then also use port 9093, - as it does here.

    +

    It is a requirement that the host and port defined in controller.quorum.bootstrap.servers + is routed to the exposed controller listeners. For example, here the CONTROLLER + listener is bound to localhost:9093. The connection string defined by + controller.quorum.bootstrap.servers must then also use localhost:9093, as it + does here.

    The controller will accept requests on all listeners defined by controller.listener.names. Typically there would be just one controller listener, but it is possible to have more. From 83091994a6324c19b29f5802489513a943f13664 Mon Sep 17 00:00:00 2001 From: Jakub Scholz Date: Wed, 18 Sep 2024 20:45:25 +0200 Subject: [PATCH 094/123] KAFKA-17543: Improve and clarify the error message about generated broker IDs in migration (#17210) This PR tries to improve the error message when broker.id is set to -1 and ZK migration is enabled. It is not needed to disable the broker.id.generation.enable option. It is sufficient to just not use it (by not setting the broker.id to -1). Reviewers: Chia-Ping Tsai , Luke Chen --- core/src/main/scala/kafka/server/KafkaConfig.scala | 4 ++-- core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index ff3ed79cfe..95a8c768f7 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -853,7 +853,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) } if (brokerIdGenerationEnable) { if (migrationEnabled) { - require(brokerId != -1, "broker id generation is incompatible with migration to ZK. Please disable it before enabling migration") + require(brokerId >= 0, "broker.id generation is incompatible with ZooKeeper migration. Please stop using it before enabling migration (set broker.id to a value greater or equal to 0).") } require(brokerId >= -1 && brokerId <= maxReservedBrokerId, "broker.id must be greater than or equal to -1 and not greater than reserved.broker.max.id") } else { @@ -973,7 +973,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) // ZK-based if (migrationEnabled) { require(brokerId >= 0, - "broker broker.id.generation.enable is incompatible with migration to ZK. Please disable it before enabling migration") + "broker.id generation is incompatible with ZooKeeper migration. Please stop using it before enabling migration (set broker.id to a value greater or equal to 0).") validateQuorumVotersAndQuorumBootstrapServerForMigration() require(controllerListenerNames.nonEmpty, s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must not be empty when running in ZooKeeper migration mode: ${controllerListenerNames.asJava}") diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 6ec2093630..b093112c92 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1891,7 +1891,7 @@ class KafkaConfigTest { props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") assertEquals( - "requirement failed: broker id generation is incompatible with migration to ZK. Please disable it before enabling migration", + "requirement failed: broker.id generation is incompatible with ZooKeeper migration. Please stop using it before enabling migration (set broker.id to a value greater or equal to 0).", assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage) } From 7d14cd6b33a01790e0492b33d3463f404b7321e7 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Tue, 24 Sep 2024 17:36:36 +0800 Subject: [PATCH 095/123] KAFKA-17459 Stablize reassign_partitions_test.py (#17250) This test expects that each partition can receive the record, so using a non-null key helps distribute the records more randomly. Reviewers: Chia-Ping Tsai --- tests/kafkatest/tests/core/reassign_partitions_test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/kafkatest/tests/core/reassign_partitions_test.py b/tests/kafkatest/tests/core/reassign_partitions_test.py index acacd20241..d7c459bfb9 100644 --- a/tests/kafkatest/tests/core/reassign_partitions_test.py +++ b/tests/kafkatest/tests/core/reassign_partitions_test.py @@ -176,7 +176,11 @@ def test_reassign_partitions(self, bounce_brokers, reassign_from_offset_zero, me self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput, - enable_idempotence=True) + enable_idempotence=True, + # This test aims to verify the reassignment without failure, assuming that all partitions have data. + # To avoid the reassignment behavior being affected by the `BuiltInPartitioner` (due to the key not being set), + # we set a key for the message to ensure both even data distribution across all partitions. + repeating_keys=100) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, consumer_timeout_ms=60000, From 333483a16ec6130a70e560e2259c98806e2dd368 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Tue, 24 Sep 2024 21:51:05 +0800 Subject: [PATCH 096/123] MINOR: add a space for kafka.metrics.polling.interval.secs description (#17256) Reviewers: Chia-Ping Tsai --- .../java/org/apache/kafka/server/metrics/MetricConfigs.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java b/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java index 2951cd4ec0..59b63fe913 100644 --- a/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java +++ b/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java @@ -61,7 +61,7 @@ public class MetricConfigs { " the standard MBean convention."; public static final String KAFKA_METRICS_POLLING_INTERVAL_SECONDS_CONFIG = "kafka.metrics.polling.interval.secs"; public static final int KAFKA_METRICS_POLLING_INTERVAL_SECONDS_DEFAULT = 10; - public static final String KAFKA_METRICS_POLLING_INTERVAL_SECONDS_DOC = "The metrics polling interval (in seconds) which can be used in" + + public static final String KAFKA_METRICS_POLLING_INTERVAL_SECONDS_DOC = "The metrics polling interval (in seconds) which can be used in " + KAFKA_METRICS_REPORTER_CLASSES_CONFIG + " implementations."; /** ********* Kafka Client Telemetry Metrics Configuration ***********/ From e36c82d71c07447347181ff5892629db90ee1f14 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Tue, 24 Sep 2024 07:29:11 -0400 Subject: [PATCH 097/123] MINOR: Replace gt and lt char with html encoding (#17235) Reviewers: Chia-Ping Tsai --- docs/ops.html | 44 ++++++++++++++++++++++---------------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index c3a7212c9b..38515f8e8d 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -596,9 +596,9 @@

    MirrorMakerConfig, MirrorConnectorConfig -
  • DefaultTopicFilter for topics, DefaultGroupFilter for consumer groups
  • -
  • Example configuration settings in connect-mirror-maker.properties, KIP-382: MirrorMaker 2.0
  • +
  • MirrorMakerConfig, MirrorConnectorConfig
  • +
  • DefaultTopicFilter for topics, DefaultGroupFilter for consumer groups
  • +
  • Example configuration settings in connect-mirror-maker.properties, KIP-382: MirrorMaker 2.0
  • Configuration File Syntax
    @@ -681,28 +681,28 @@

    us-east.exactly.once.source.support = enabled - +

    For existing MirrorMaker clusters, a two-step upgrade is necessary. Instead of immediately setting the exactly.once.source.support property to enabled, first set it to preparing on all nodes in the cluster. Once this is complete, it can be set to enabled on all nodes in the cluster, in a second round of restarts.

    - +

    In either case, it is also necessary to enable intra-cluster communication between the MirrorMaker nodes, as described in KIP-710. To do this, the dedicated.mode.enable.internal.rest property must be set to true. In addition, many of the REST-related configuration properties available for Kafka Connect can be specified the MirrorMaker config. For example, to enable intra-cluster communication in MirrorMaker cluster with each node listening on port 8080 of their local machine, the following should be added to the MirrorMaker config file:

    dedicated.mode.enable.internal.rest = true
     listeners = http://localhost:8080
    - +

    Note that, if intra-cluster communication is enabled in production environments, it is highly recommended to secure the REST servers brought up by each MirrorMaker node. See the configuration properties for Kafka Connect for information on how this can be accomplished.

    - +

    It is also recommended to filter records from aborted transactions out from replicated data when running MirrorMaker. To do this, ensure that the consumer used to read from source clusters is configured with isolation.level set to read_committed. If replicating data from cluster us-west, this can be done for all replication flows that read from that cluster by adding the following to the MirrorMaker config file:

    @@ -1934,12 +1934,12 @@

    RemoteLogManager Avg Broker Fetch Throttle Time The average time in millis remote fetches was throttled by a broker - kafka.server:type=RemoteLogManager, name=remote-fetch-throttle-time-avg + kafka.server:type=RemoteLogManager, name=remote-fetch-throttle-time-avg RemoteLogManager Max Broker Fetch Throttle Time The max time in millis remote fetches was throttled by a broker - kafka.server:type=RemoteLogManager, name=remote-fetch-throttle-time-max + kafka.server:type=RemoteLogManager, name=remote-fetch-throttle-time-max RemoteLogManager Avg Broker Copy Throttle Time @@ -2055,7 +2055,7 @@
    < Latest Metadata Snapshot Age - The interval in milliseconds since the latest snapshot that the node has generated. + The interval in milliseconds since the latest snapshot that the node has generated. If none have been generated yet, this is approximately the time delta since the process was started. kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs @@ -2160,7 +2160,7 @@
    Counts the number of times this node has seen a new controller elected. A transition to the "no leader" state is not counted here. If the same controller as before becomes active, that still counts. kafka.controller:type=KafkaController,name=NewActiveControllersCount @@ -3723,7 +3723,7 @@

    6.9

    Stable version

    The current stable branch is 3.8. Kafka is regularly updated to include the latest release in the 3.8 series. - +

    ZooKeeper Deprecation

    With the release of Apache Kafka 3.5, Zookeeper is now marked deprecated. Removal of ZooKeeper is planned in the next major release of Apache Kafka (version 4.0), which is scheduled to happen no sooner than April 2024. During the deprecation phase, ZooKeeper is still supported for metadata management of Kafka clusters, @@ -3732,10 +3732,10 @@

    Migration

    Users are recommended to begin planning for migration to KRaft and also begin testing to provide any feedback. Refer to ZooKeeper to KRaft Migration for details on how to perform a live migration from ZooKeeper to KRaft and current limitations.

    - +
    3.x and ZooKeeper Support

    The final 3.x minor release, that supports ZooKeeper mode, will receive critical bug fixes and security fixes for 12 months after its release.

    - +

    Operationalizing ZooKeeper

    Operationally, we do the following for a healthy ZooKeeper installation:
      @@ -3796,7 +3796,7 @@

      Bootstrap a Standalone Controller

      The recommended method for creating a new KRaft controller cluster is to bootstrap it with one voter and dynamically add the rest of the controllers. Bootstrapping the first controller can be done with the following CLI command: -
      $ bin/kafka-storage format --cluster-id  --standalone --config controller.properties
      +
      $ bin/kafka-storage format --cluster-id <cluster-id> --standalone --config controller.properties
      This command will 1) create a meta.properties file in metadata.log.dir with a randomly generated directory.id, 2) create a snapshot at 00000000000000000000-0000000000.checkpoint with the necessary control records (KRaftVersionRecord and VotersRecord) to make this Kafka node the only voter for the quorum. @@ -3820,7 +3820,7 @@
      Formatting Brokers and New Controllers
      When provisioning new broker and controller nodes that we want to add to an existing Kafka cluster, use the kafka-storage.sh format command without the --standalone or --initial-controllers flags. -
      $ bin/kafka-storage format --cluster-id  --config server.properties
      +
      $ bin/kafka-storage format --cluster-id <cluster-id> --config server.properties

      Controller membership changes

      @@ -3839,10 +3839,10 @@
      kafka-metadata-quorum remove-controller command. Until KIP-996: Pre-vote has been implemented and released, it is recommended to shutdown the controller that will be removed before running the remove-controller command. When using broker endpoints use the --bootstrap-server flag: -
      $ bin/kafka-metadata-quorum --bootstrap-server localhost:9092 remove-controller --controller-id  --controller-directory-id 
      +
      $ bin/kafka-metadata-quorum --bootstrap-server localhost:9092 remove-controller --controller-id <id> --controller-directory-id <directory-id>
      When using controller endpoints use the --bootstrap-controller flag: -
      $ bin/kafka-metadata-quorum --bootstrap-controller localhost:9092 remove-controller --controller-id  --controller-directory-id 
      +
      $ bin/kafka-metadata-quorum --bootstrap-controller localhost:9092 remove-controller --controller-id <id> --controller-directory-id <directory-id>

      Debugging

      @@ -4244,7 +4244,7 @@

      Reverting to ZooKeeper mode During the Migration

    • Make sure that on the first cluster roll, zookeeper.metadata.migration.enable remains set to - true. Do not set it to false until the second cluster roll. + true. Do not set it to false until the second cluster roll.
    From 57b098c397c441f645a31ec64b9e346fc0f55b6b Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Wed, 25 Sep 2024 23:16:19 -0700 Subject: [PATCH 098/123] KAFKA-17584: Fix incorrect synonym handling for dynamic log configurations (#17258) Several Kafka log configurations in have synonyms. For example, log retention can be configured either by log.retention.ms, or by log.retention.minutes, or by log.retention.hours. There is also a faculty in Kafka to dynamically change broker configurations without restarting the broker. These dynamically set configurations are stored in the metadata log and override what is in the broker properties file. Unfortunately, these two features interacted poorly; there was a bug where the dynamic log configuration update code ignored synonyms. For example, if you set log.retention.minutes and then reconfigured something unrelated that triggered the LogConfig update path, the retention value that you had configured was overwritten. The reason for this was incorrect handling of synonyms. The code tried to treat the Kafka broker configuration as a bag of key/value entities rather than extracting the correct retention time (or other setting with overrides) from the KafkaConfig object. Reviewers: Luke Chen , Jun Rao , Kamal Chandraprakash, Christo Lolov , Federico Valeri , Rajini Sivaram , amangandhi94 <> --- .../kafka/server/DynamicBrokerConfig.scala | 43 ++++--- .../DynamicBrokerReconfigurationTest.scala | 6 +- .../server/DynamicBrokerConfigTest.scala | 106 +++++++++++++----- 3 files changed, 105 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 23c99e5bcb..2dff381602 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -30,7 +30,7 @@ import kafka.utils.Implicits._ import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.common.Reconfigurable import org.apache.kafka.common.config.internals.BrokerSecurityConfigs -import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SaslConfigs, SslConfigs} +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SaslConfigs, SslConfigs, TopicConfig} import org.apache.kafka.common.metrics.{JmxReporter, Metrics, MetricsReporter} import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} @@ -40,7 +40,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.PasswordEncoder import org.apache.kafka.server.ProcessRole -import org.apache.kafka.server.config.{ConfigType, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals} +import org.apache.kafka.server.config.{ConfigType, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, MetricConfigs} import org.apache.kafka.server.telemetry.ClientTelemetry @@ -662,12 +662,24 @@ trait BrokerReconfigurable { } object DynamicLogConfig { - // Exclude message.format.version for now since we need to check that the version - // is supported on all brokers in the cluster. + /** + * The log configurations that are non-reconfigurable. This set contains the names you + * would use when setting a dynamic configuration on a topic, which are different than the + * corresponding broker configuration names. + * + * For now, message.format.version is not reconfigurable, since we need to check that + * the version is supported on all brokers in the cluster. + */ + val NonReconfigrableLogConfigs: Set[String] = Set(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG) + + /** + * The broker configurations pertaining to logs that are reconfigurable. This set contains + * the names you would use when setting a static or dynamic broker configuration (not topic + * configuration). + */ val ReconfigurableConfigs: Set[String] = - ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.values.asScala.toSet - ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG - val KafkaConfigToLogConfigName: Map[String, String] = - ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) } + ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala. + filterNot(s => NonReconfigrableLogConfigs.contains(s._1)).values.toSet } class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends BrokerReconfigurable with Logging { @@ -732,17 +744,14 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { val originalLogConfig = logManager.currentDefaultConfig val originalUncleanLeaderElectionEnable = originalLogConfig.uncleanLeaderElectionEnable - val newBrokerDefaults = new util.HashMap[String, Object](originalLogConfig.originals) - newConfig.valuesFromThisConfig.forEach { (k, v) => - if (DynamicLogConfig.ReconfigurableConfigs.contains(k)) { - DynamicLogConfig.KafkaConfigToLogConfigName.get(k).foreach { configName => - if (v == null) - newBrokerDefaults.remove(configName) - else - newBrokerDefaults.put(configName, v.asInstanceOf[AnyRef]) - } + val newBrokerDefaults = new util.HashMap[String, Object](newConfig.extractLogConfigMap) + val originalLogConfigMap = originalLogConfig.originals() + DynamicLogConfig.NonReconfigrableLogConfigs.foreach(k => { + Option(originalLogConfigMap.get(k)) match { + case None => newBrokerDefaults.remove(k) + case Some(v) => newBrokerDefaults.put(k, v) } - } + }) logManager.reconfigureDefaultLogConfig(new LogConfig(newBrokerDefaults)) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index acbbd6cfc6..3df10d3cf2 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -63,7 +63,7 @@ import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.{PasswordEncoder, PasswordEncoderConfigs} -import org.apache.kafka.server.config.{ConfigType, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{ConfigType, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs} import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.ShutdownableThread @@ -667,8 +667,10 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup val log = servers.head.logManager.getLog(new TopicPartition(topic, 0)).getOrElse(throw new IllegalStateException("Log not found")) TestUtils.waitUntilTrue(() => log.config.segmentSize == 4000, "Existing topic config using defaults not updated") + val KafkaConfigToLogConfigName: Map[String, String] = + ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) } props.asScala.foreach { case (k, v) => - val logConfigName = DynamicLogConfig.KafkaConfigToLogConfigName(k) + val logConfigName = KafkaConfigToLogConfigName(k) val expectedValue = if (k == ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG) s"[$v]" else v assertEquals(expectedValue, log.config.originals.get(logConfigName).toString, s"Not reconfigured $logConfigName for existing log") diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index e83f5a41a3..bac6149de5 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -19,7 +19,7 @@ package kafka.server import java.{lang, util} import java.util.{Properties, Map => JMap} -import java.util.concurrent.CompletionStage +import java.util.concurrent.{CompletionStage, TimeUnit} import java.util.concurrent.atomic.AtomicReference import kafka.controller.KafkaController import kafka.log.LogManager @@ -57,7 +57,7 @@ class DynamicBrokerConfigTest { @Test def testConfigUpdate(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val props = TestUtils.createBrokerConfig(0, null, port = 8181) val oldKeystore = "oldKs.jks" props.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, oldKeystore) val config = KafkaConfig(props) @@ -102,7 +102,7 @@ class DynamicBrokerConfigTest { @Test def testEnableDefaultUncleanLeaderElection(): Unit = { - val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val origProps = TestUtils.createBrokerConfig(0, null, port = 8181) origProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false") val config = KafkaConfig(origProps) @@ -133,7 +133,7 @@ class DynamicBrokerConfigTest { @Test def testUpdateDynamicThreadPool(): Unit = { - val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val origProps = TestUtils.createBrokerConfig(0, null, port = 8181) origProps.put(ServerConfigs.NUM_IO_THREADS_CONFIG, "4") origProps.put(SocketServerConfigs.NUM_NETWORK_THREADS_CONFIG, "2") origProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") @@ -205,7 +205,7 @@ class DynamicBrokerConfigTest { @nowarn("cat=deprecation") @Test def testConfigUpdateWithSomeInvalidConfigs(): Unit = { - val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val origProps = TestUtils.createBrokerConfig(0, null, port = 8181) origProps.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, "JKS") val config = KafkaConfig(origProps) config.dynamicConfig.initialize(None, None) @@ -227,7 +227,7 @@ class DynamicBrokerConfigTest { @Test def testConfigUpdateWithReconfigurableValidationFailure(): Unit = { - val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val origProps = TestUtils.createBrokerConfig(0, null, port = 8181) origProps.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "100000000") val config = KafkaConfig(origProps) config.dynamicConfig.initialize(None, None) @@ -261,7 +261,7 @@ class DynamicBrokerConfigTest { @Test def testReconfigurableValidation(): Unit = { - val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val origProps = TestUtils.createBrokerConfig(0, null, port = 8181) val config = KafkaConfig(origProps) val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, ServerConfigs.BROKER_ID_CONFIG, "some.prop") val validReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "some.prop") @@ -331,7 +331,7 @@ class DynamicBrokerConfigTest { } private def verifyConfigUpdate(name: String, value: Object, perBrokerConfig: Boolean, expectFailure: Boolean): Unit = { - val configProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val configProps = TestUtils.createBrokerConfig(0, null, port = 8181) configProps.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, "broker.secret") val config = KafkaConfig(configProps) config.dynamicConfig.initialize(None, None) @@ -440,7 +440,7 @@ class DynamicBrokerConfigTest { def testDynamicListenerConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) val oldConfig = KafkaConfig.fromProps(props) - val kafkaServer: KafkaServer = mock(classOf[kafka.server.KafkaServer]) + val kafkaServer: KafkaBroker = mock(classOf[kafka.server.KafkaBroker]) when(kafkaServer.config).thenReturn(oldConfig) props.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://hostname:9092,SASL_PLAINTEXT://hostname:9093") @@ -480,11 +480,11 @@ class DynamicBrokerConfigTest { @Test def testAuthorizerConfig(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val props = TestUtils.createBrokerConfig(0, null, port = 9092) val oldConfig = KafkaConfig.fromProps(props) oldConfig.dynamicConfig.initialize(None, None) - val kafkaServer: KafkaServer = mock(classOf[kafka.server.KafkaServer]) + val kafkaServer: KafkaBroker = mock(classOf[kafka.server.KafkaBroker]) when(kafkaServer.config).thenReturn(oldConfig) when(kafkaServer.kafkaYammerMetrics).thenReturn(KafkaYammerMetrics.INSTANCE) val metrics: Metrics = mock(classOf[Metrics]) @@ -630,7 +630,7 @@ class DynamicBrokerConfigTest { @Test def testImproperConfigsAreRemoved(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + val props = TestUtils.createBrokerConfig(0, null) val config = KafkaConfig(props) config.dynamicConfig.initialize(None, None) @@ -659,7 +659,7 @@ class DynamicBrokerConfigTest { @Test def testUpdateMetricReporters(): Unit = { val brokerId = 0 - val origProps = TestUtils.createBrokerConfig(brokerId, TestUtils.MockZkConnect, port = 8181) + val origProps = TestUtils.createBrokerConfig(brokerId, null, port = 8181) val config = KafkaConfig(origProps) val serverMock = Mockito.mock(classOf[KafkaBroker]) @@ -684,7 +684,7 @@ class DynamicBrokerConfigTest { @nowarn("cat=deprecation") def testUpdateMetricReportersNoJmxReporter(): Unit = { val brokerId = 0 - val origProps = TestUtils.createBrokerConfig(brokerId, TestUtils.MockZkConnect, port = 8181) + val origProps = TestUtils.createBrokerConfig(brokerId, null, port = 8181) origProps.put(MetricConfigs.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false") val config = KafkaConfig(origProps) @@ -711,8 +711,7 @@ class DynamicBrokerConfigTest { @Test def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { - val props = new Properties() - props.put(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") + val props = TestUtils.createBrokerConfig(0, null, port = 8181) props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") val config = new KafkaConfig(props) assertFalse(config.nonInternalValues.containsKey(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) @@ -722,10 +721,10 @@ class DynamicBrokerConfigTest { @Test def testDynamicLogLocalRetentionMsConfig(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val props = TestUtils.createBrokerConfig(0, null, port = 8181) props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, "2592000000") val config = KafkaConfig(props) - val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer])) + val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaBroker])) config.dynamicConfig.initialize(None, None) config.dynamicConfig.addBrokerReconfigurable(dynamicLogConfig) @@ -745,10 +744,10 @@ class DynamicBrokerConfigTest { @Test def testDynamicLogLocalRetentionSizeConfig(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val props = TestUtils.createBrokerConfig(0, null, port = 8181) props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "4294967296") val config = KafkaConfig(props) - val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer])) + val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaBroker])) config.dynamicConfig.initialize(None, None) config.dynamicConfig.addBrokerReconfigurable(dynamicLogConfig) @@ -768,7 +767,7 @@ class DynamicBrokerConfigTest { @Test def testDynamicLogLocalRetentionSkipsOnInvalidConfig(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val props = TestUtils.createBrokerConfig(0, null, port = 8181) props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP, "1000") props.put(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP, "1024") val config = KafkaConfig(props) @@ -794,7 +793,7 @@ class DynamicBrokerConfigTest { @Test def testDynamicRemoteFetchMaxWaitMsConfig(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val props = TestUtils.createBrokerConfig(0, null, port = 8181) val config = KafkaConfig(props) val kafkaBroker = mock(classOf[KafkaBroker]) when(kafkaBroker.config).thenReturn(config) @@ -828,7 +827,7 @@ class DynamicBrokerConfigTest { @Test def testUpdateDynamicRemoteLogManagerConfig(): Unit = { - val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val origProps = TestUtils.createBrokerConfig(0, null, port = 8181) origProps.put(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP, "2") val config = KafkaConfig(origProps) @@ -853,9 +852,9 @@ class DynamicBrokerConfigTest { @Test def testRemoteLogManagerCopyQuotaUpdates(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val props = TestUtils.createBrokerConfig(0, null, port = 9092) val config = KafkaConfig.fromProps(props) - val serverMock: KafkaServer = mock(classOf[KafkaServer]) + val serverMock: KafkaBroker = mock(classOf[KafkaBroker]) val remoteLogManager = mock(classOf[RemoteLogManager]) Mockito.when(serverMock.config).thenReturn(config) @@ -884,9 +883,9 @@ class DynamicBrokerConfigTest { @Test def testRemoteLogManagerFetchQuotaUpdates(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val props = TestUtils.createBrokerConfig(0, null, port = 9092) val config = KafkaConfig.fromProps(props) - val serverMock: KafkaServer = mock(classOf[KafkaServer]) + val serverMock: KafkaBroker = mock(classOf[KafkaBroker]) val remoteLogManager = mock(classOf[RemoteLogManager]) Mockito.when(serverMock.config).thenReturn(config) @@ -919,9 +918,9 @@ class DynamicBrokerConfigTest { val copyQuotaProp = RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP val fetchQuotaProp = RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) + val props = TestUtils.createBrokerConfig(0, null, port = 9092) val config = KafkaConfig.fromProps(props) - val serverMock: KafkaServer = mock(classOf[KafkaServer]) + val serverMock: KafkaBroker = mock(classOf[KafkaBroker]) val remoteLogManager = Mockito.mock(classOf[RemoteLogManager]) Mockito.when(serverMock.config).thenReturn(config) @@ -969,11 +968,11 @@ class DynamicBrokerConfigTest { retentionMs: Long, logLocalRetentionBytes: Long, retentionBytes: Long): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) + val props = TestUtils.createBrokerConfig(0, null, port = 8181) props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, retentionMs.toString) props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, retentionBytes.toString) val config = KafkaConfig(props) - val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaServer])) + val dynamicLogConfig = new DynamicLogConfig(mock(classOf[LogManager]), mock(classOf[KafkaBroker])) config.dynamicConfig.initialize(None, None) config.dynamicConfig.addBrokerReconfigurable(dynamicLogConfig) @@ -985,6 +984,51 @@ class DynamicBrokerConfigTest { // validate per broker config assertThrows(classOf[ConfigException], () => config.dynamicConfig.validate(newProps, perBrokerConfig = true)) } + + class DynamicLogConfigContext(origProps: Properties) { + val config = KafkaConfig(origProps) + val serverMock = Mockito.mock(classOf[BrokerServer]) + val logManagerMock = Mockito.mock(classOf[LogManager]) + + Mockito.when(serverMock.config).thenReturn(config) + Mockito.when(serverMock.logManager).thenReturn(logManagerMock) + Mockito.when(logManagerMock.allLogs).thenReturn(Iterable.empty) + + val currentDefaultLogConfig = new AtomicReference(new LogConfig(new Properties)) + Mockito.when(logManagerMock.currentDefaultConfig).thenAnswer(_ => currentDefaultLogConfig.get()) + Mockito.when(logManagerMock.reconfigureDefaultLogConfig(ArgumentMatchers.any(classOf[LogConfig]))) + .thenAnswer(invocation => currentDefaultLogConfig.set(invocation.getArgument(0))) + + config.dynamicConfig.initialize(None, None) + config.dynamicConfig.addBrokerReconfigurable(new DynamicLogConfig(logManagerMock, serverMock)) + } + + @Test + def testDynamicLogConfigHandlesSynonymsCorrectly(): Unit = { + val origProps = TestUtils.createBrokerConfig(0, null, port = 8181) + origProps.put(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "1") + val ctx = new DynamicLogConfigContext(origProps) + assertEquals(TimeUnit.MINUTES.toMillis(1), ctx.config.logRetentionTimeMillis) + + val props = new Properties() + props.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "12345678") + ctx.config.dynamicConfig.updateDefaultConfig(props) + assertEquals(TimeUnit.MINUTES.toMillis(1), ctx.currentDefaultLogConfig.get().retentionMs) + } + + @Test + def testLogRetentionTimeMinutesIsNotDynamicallyReconfigurable(): Unit = { + val origProps = TestUtils.createBrokerConfig(0, null, port = 8181) + origProps.put(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG, "1") + val ctx = new DynamicLogConfigContext(origProps) + assertEquals(TimeUnit.HOURS.toMillis(1), ctx.config.logRetentionTimeMillis) + + val props = new Properties() + props.put(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG, "3") + ctx.config.dynamicConfig.updateDefaultConfig(props) + assertEquals(TimeUnit.HOURS.toMillis(1), ctx.config.logRetentionTimeMillis) + assertFalse(ctx.currentDefaultLogConfig.get().originals().containsKey(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG)) + } } class TestDynamicThreadPool() extends BrokerReconfigurable { From c2c2dd424b21af53110f1a137127dc80481d4281 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Fri, 6 Sep 2024 13:44:09 -0700 Subject: [PATCH 099/123] KAFKA-16963: Ducktape test for KIP-853 (#17081) Add a ducktape system test for KIP-853 quorum reconfiguration, including adding and removing voters. Reviewers: Colin P. McCabe --- .../org/apache/kafka/raft/LeaderState.java | 2 +- .../services/kafka/config_property.py | 4 + tests/kafkatest/services/kafka/kafka.py | 146 +++++++++++++--- .../services/kafka/templates/kafka.properties | 6 +- .../tests/core/quorum_reconfiguration_test.py | 165 ++++++++++++++++++ 5 files changed, 301 insertions(+), 22 deletions(-) create mode 100644 tests/kafkatest/tests/core/quorum_reconfiguration_test.py diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index 803804858a..ae9be94e13 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -384,7 +384,7 @@ public boolean isResignRequested() { } public boolean isReplicaCaughtUp(ReplicaKey replicaKey, long currentTimeMs) { - // In summary, let's consider a replica caughed up for add voter, if they + // In summary, let's consider a replica caught up for add voter, if they // have fetched within the last hour long anHourInMs = TimeUnit.HOURS.toMillis(1); return Optional.ofNullable(observerStates.get(replicaKey)) diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index bc4708d0d3..28582513ab 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -19,12 +19,16 @@ BROKER_ID = "broker.id" NODE_ID = "node.id" +PROCESS_ROLES = "process.roles" FIRST_BROKER_PORT = 9092 FIRST_CONTROLLER_PORT = FIRST_BROKER_PORT + 500 FIRST_CONTROLLER_ID = 3001 CLUSTER_ID = "I2eXt9rvSnyhct8BYmW6-w" PORT = "port" ADVERTISED_HOSTNAME = "advertised.host.name" +ADVERTISED_LISTENERS = "advertised.listeners" +LISTENERS = "listeners" +CONTROLLER_LISTENER_NAMES = "controller.listener.names" NUM_NETWORK_THREADS = "num.network.threads" NUM_IO_THREADS = "num.io.threads" diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index bd1c9b1e33..547762a534 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -204,7 +204,8 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI controller_num_nodes_override=0, allow_zk_with_kraft=False, quorum_info_provider=None, - use_new_coordinator=None + use_new_coordinator=None, + dynamicRaftQuorum=False ): """ :param context: test context @@ -262,7 +263,7 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI e.g: {1: [["config1", "true"], ["config2", "1000"]], 2: [["config1", "false"], ["config2", "0"]]} :param str extra_kafka_opts: jvm args to add to KAFKA_OPTS variable :param KafkaService isolated_kafka: process.roles=controller for this cluster when not None; ignored when using ZooKeeper - :param int controller_num_nodes_override: the number of nodes to use in the cluster, instead of 5, 3, or 1 based on num_nodes, if positive, not using ZooKeeper, and isolated_kafka is not None; ignored otherwise + :param int controller_num_nodes_override: the number of controller nodes to use in the cluster, instead of 5, 3, or 1 based on num_nodes, if positive, not using ZooKeeper, and isolated_kafka is not None; ignored otherwise :param bool allow_zk_with_kraft: if True, then allow a KRaft broker or controller to also use ZooKeeper :param quorum_info_provider: A function that takes this KafkaService as an argument and returns a ServiceQuorumInfo. If this is None, then the ServiceQuorumInfo is generated from the test context :param use_new_coordinator: When true, use the new implementation of the group coordinator as per KIP-848. If this is None, the default existing group coordinator is used. @@ -298,6 +299,8 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.num_nodes_controller_role = 0 if self.quorum_info.using_kraft: + self.dynamicRaftQuorum = dynamicRaftQuorum + self.first_controller_started = False if self.quorum_info.has_brokers: num_nodes_broker_role = num_nodes if self.quorum_info.has_controllers: @@ -337,7 +340,7 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI listener_security_config=listener_security_config, extra_kafka_opts=extra_kafka_opts, tls_version=tls_version, isolated_kafka=self, allow_zk_with_kraft=self.allow_zk_with_kraft, - server_prop_overrides=server_prop_overrides + server_prop_overrides=server_prop_overrides, dynamicRaftQuorum=self.dynamicRaftQuorum ) self.controller_quorum = self.isolated_controller_quorum @@ -434,15 +437,15 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI kraft_broker_plus_zk_configs = kraft_broker_configs.copy() kraft_broker_plus_zk_configs.update(zk_broker_configs) kraft_broker_plus_zk_configs.pop(config_property.BROKER_ID) - controller_only_configs = { - config_property.NODE_ID: self.idx(node) + config_property.FIRST_CONTROLLER_ID - 1, - } - kraft_controller_plus_zk_configs = controller_only_configs.copy() - kraft_controller_plus_zk_configs.update(zk_broker_configs) - kraft_controller_plus_zk_configs.pop(config_property.BROKER_ID) if node_quorum_info.service_quorum_info.using_zk: node.config = KafkaConfig(**zk_broker_configs) elif not node_quorum_info.has_broker_role: # KRaft controller-only role + controller_only_configs = { + config_property.NODE_ID: self.node_id_as_isolated_controller(node), + } + kraft_controller_plus_zk_configs = controller_only_configs.copy() + kraft_controller_plus_zk_configs.update(zk_broker_configs) + kraft_controller_plus_zk_configs.pop(config_property.BROKER_ID) if self.zk: node.config = KafkaConfig(**kraft_controller_plus_zk_configs) else: @@ -455,6 +458,10 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.combined_nodes_started = 0 self.nodes_to_start = self.nodes + # Does not do any validation to check if this node is part of an isolated controller quorum or not + def node_id_as_isolated_controller(self, node): + return self.idx(node) + config_property.FIRST_CONTROLLER_ID - 1 + def reconfigure_zk_for_migration(self, kraft_quorum): self.configured_for_zk_migration = True self.controller_quorum = kraft_quorum @@ -627,7 +634,7 @@ def start_minikdc_if_necessary(self, add_principals=""): def alive(self, node): return len(self.pids(node)) > 0 - def start(self, add_principals="", nodes_to_skip=[], timeout_sec=60, **kwargs): + def start(self, add_principals="", nodes_to_skip=[], isolated_controllers_to_skip=[], timeout_sec=60, **kwargs): """ Start the Kafka broker and wait until it registers its ID in ZooKeeper Startup will be skipped for any nodes in nodes_to_skip. These nodes can be started later via add_broker @@ -665,7 +672,7 @@ def start(self, add_principals="", nodes_to_skip=[], timeout_sec=60, **kwargs): self._ensure_zk_chroot() if self.isolated_controller_quorum: - self.isolated_controller_quorum.start() + self.isolated_controller_quorum.start(nodes_to_skip=isolated_controllers_to_skip) Service.start(self, **kwargs) @@ -727,7 +734,6 @@ def add_broker(self, node): self.concurrent_start = False self.start_node(node) self.concurrent_start = orig_concurrent_start - wait_until(lambda: self.is_registered(node), 30, 1) def _ensure_zk_chroot(self): self.logger.info("Ensuring zk_chroot %s exists", self.zk_chroot) @@ -867,15 +873,19 @@ def start_node(self, node, timeout_sec=60, **kwargs): self.maybe_setup_broker_scram_credentials(node) if self.quorum_info.using_kraft: - # define controller.quorum.voters text + # define controller.quorum.bootstrap.servrers or controller.quorum.voters text security_protocol_to_use = self.controller_quorum.controller_security_protocol first_node_id = 1 if self.quorum_info.has_brokers_and_controllers else config_property.FIRST_CONTROLLER_ID - self.controller_quorum_voters = ','.join(["%s@%s:%s" % - (self.controller_quorum.idx(node) + first_node_id - 1, - node.account.hostname, - config_property.FIRST_CONTROLLER_PORT + - KafkaService.SECURITY_PROTOCOLS.index(security_protocol_to_use)) - for node in self.controller_quorum.nodes[:self.controller_quorum.num_nodes_controller_role]]) + controller_quorum_bootstrap_servers = ','.join(["%s:%s" % (node.account.hostname, + config_property.FIRST_CONTROLLER_PORT + + KafkaService.SECURITY_PROTOCOLS.index(security_protocol_to_use)) + for node in self.controller_quorum.nodes[:self.controller_quorum.num_nodes_controller_role]]) + if self.dynamicRaftQuorum: + self.controller_quorum_bootstrap_servers = controller_quorum_bootstrap_servers + else: + self.controller_quorum_voters = ','.join(["%s@%s" % (self.controller_quorum.idx(node) + first_node_id - 1, + bootstrap_server) + for bootstrap_server in controller_quorum_bootstrap_servers]) # define controller.listener.names self.controller_listener_names = ','.join(self.controller_listener_name_list(node)) # define sasl.mechanism.controller.protocol to match the isolated quorum if one exists @@ -892,8 +902,13 @@ def start_node(self, node, timeout_sec=60, **kwargs): # format log directories if necessary kafka_storage_script = self.path.script("kafka-storage.sh", node) cmd = "%s format --ignore-formatted --config %s --cluster-id %s" % (kafka_storage_script, KafkaService.CONFIG_FILE, config_property.CLUSTER_ID) + if self.dynamicRaftQuorum: + cmd += " --feature kraft.version=1" + if not self.first_controller_started and self.node_quorum_info.has_controller_role: + cmd += " --standalone" self.logger.info("Running log directory format command...\n%s" % cmd) node.account.ssh(cmd) + self.first_controller_started = True cmd = self.start_cmd(node) self.logger.debug("Attempting to start KafkaService %s on %s with command: %s" %\ @@ -1009,9 +1024,26 @@ def clean_node(self, node): JmxMixin.clean_node(self, node) self.security_config.clean_node(node) node.account.kill_process(self.java_class_name(), - clean_shutdown=False, allow_fail=True) + clean_shutdown=False, allow_fail=True) node.account.ssh("sudo rm -rf -- %s" % KafkaService.PERSISTENT_ROOT, allow_fail=False) + def kafka_metadata_quorum_cmd(self, node, kafka_security_protocol=None, use_controller_bootstrap=False): + if kafka_security_protocol is None: + # it wasn't specified, so use the inter-broker/controller security protocol if it is PLAINTEXT, + # otherwise use the client security protocol + if self.interbroker_security_protocol == SecurityConfig.PLAINTEXT: + security_protocol_to_use = SecurityConfig.PLAINTEXT + else: + security_protocol_to_use = self.security_protocol + else: + security_protocol_to_use = kafka_security_protocol + if use_controller_bootstrap: + bootstrap = "--bootstrap-controller %s" % (self.bootstrap_controllers("CONTROLLER_%s" % security_protocol_to_use)) + else: + bootstrap = "--bootstrap-server %s" % (self.bootstrap_servers(security_protocol_to_use)) + kafka_metadata_script = self.path.script("kafka-metadata-quorum.sh", node) + return "%s %s" % (kafka_metadata_script, bootstrap) + def kafka_topics_cmd_with_optional_security_settings(self, node, force_use_zk_connection, kafka_security_protocol=None, offline_nodes=[]): if self.quorum_info.using_kraft and not self.quorum_info.has_brokers: raise Exception("Must invoke kafka-topics against a broker, not a KRaft controller") @@ -1755,6 +1787,62 @@ def describe_consumer_group(self, group, node=None, command_config=None): self.logger.debug(output) return output + def describe_quorum(self, node=None): + """Run the describe quorum command. + Specifying node is optional, if not specified the command will be run from self.nodes[0] + """ + if node is None: + node = self.nodes[0] + cmd = fix_opts_for_new_jvm(node) + cmd += "%(kafka_metadata_quorum_cmd)s describe --status" % { + 'kafka_metadata_quorum_cmd': self.kafka_metadata_quorum_cmd(node) + } + self.logger.info("Running describe quorum command...\n%s" % cmd) + node.account.ssh(cmd) + + output = "" + for line in node.account.ssh_capture(cmd): + output += line + + return output + + def add_controller(self, controllerId, controller): + """Run the metadata quorum add controller command. This should be run on the node that is being added. + """ + command_config_path = os.path.join(KafkaService.PERSISTENT_ROOT, "controller_command_config.properties") + + configs = f""" +{config_property.NODE_ID}={controllerId} +{config_property.PROCESS_ROLES}=controller +{config_property.METADATA_LOG_DIR}={KafkaService.METADATA_LOG_DIR} +{config_property.ADVERTISED_LISTENERS}={self.advertised_listeners} +{config_property.LISTENERS}={self.listeners} +{config_property.CONTROLLER_LISTENER_NAMES}={self.controller_listener_names}""" + + controller.account.create_file(command_config_path, configs) + cmd = fix_opts_for_new_jvm(controller) + cmd += "%(kafka_metadata_quorum_cmd)s --command-config %(command_config)s add-controller" % { + 'kafka_metadata_quorum_cmd': self.kafka_metadata_quorum_cmd(controller, use_controller_bootstrap=True), + 'command_config': command_config_path + } + self.logger.info("Running add controller command...\n%s" % cmd) + controller.account.ssh(cmd) + + def remove_controller(self, controllerId, directoryId, node=None): + """Run the admin tool remove controller command. + Specifying node is optional, if not specified the command will be run from self.nodes[0] + """ + if node is None: + node = self.nodes[0] + cmd = fix_opts_for_new_jvm(node) + cmd += "%(kafka_metadata_quorum_cmd)s remove-controller -i %(controller_id)s -d %(directory_id)s" % { + 'kafka_metadata_quorum_cmd': self.kafka_metadata_quorum_cmd(node, use_controller_bootstrap=True), + 'controller_id': controllerId, + 'directory_id': directoryId + } + self.logger.info("Running remove controller command...\n%s" % cmd) + node.account.ssh(cmd) + def zk_connect_setting(self): if self.quorum_info.using_kraft and not self.zk: raise Exception("No zookeeper connect string available with KRaft unless ZooKeeper is explicitly enabled") @@ -1769,6 +1857,15 @@ def __bootstrap_servers(self, port, validate=True, offline_nodes=[]): for node in self.nodes if node not in offline_nodes]) + def __bootstrap_controllers(self, port, validate=True, offline_nodes=[]): + if validate and not port.open: + raise ValueError("We are retrieving bootstrap servers for the port: %s which is not currently open. - " % + str(port.port_number)) + + return ','.join([node.account.hostname + ":" + str(port.port_number) + for node in self.controller_quorum.nodes[:self.controller_quorum.num_nodes_controller_role] + if node not in offline_nodes]) + def bootstrap_servers(self, protocol='PLAINTEXT', validate=True, offline_nodes=[]): """Return comma-delimited list of brokers in this cluster formatted as HOSTNAME1:PORT1,HOSTNAME:PORT2,... @@ -1778,6 +1875,15 @@ def bootstrap_servers(self, protocol='PLAINTEXT', validate=True, offline_nodes=[ self.logger.info("Bootstrap client port is: " + str(port_mapping.port_number)) return self.__bootstrap_servers(port_mapping, validate, offline_nodes) + def bootstrap_controllers(self, protocol='CONTROLLER_PLAINTEXT', validate=True, offline_nodes=[]): + """Return comma-delimited list of controllers in this cluster formatted as HOSTNAME1:PORT1,HOSTNAME:PORT2,... + + This is the format expected by many config files. + """ + port_mapping = self.port_mappings[protocol] + self.logger.info("Bootstrap client port is: " + str(port_mapping.port_number)) + return self.__bootstrap_controllers(port_mapping, validate, offline_nodes) + def controller(self): """ Get the controller node """ diff --git a/tests/kafkatest/services/kafka/templates/kafka.properties b/tests/kafkatest/services/kafka/templates/kafka.properties index 65bf389b02..21b60afeb8 100644 --- a/tests/kafkatest/services/kafka/templates/kafka.properties +++ b/tests/kafkatest/services/kafka/templates/kafka.properties @@ -23,8 +23,12 @@ process.roles=controller {% else %} process.roles=broker {% endif %} -# The connect string for the controller quorum +# The connect string for the controller quorum. Only one should be defined +{% if controller_quorum_bootstrap_servers %} +controller.quorum.bootstrap.servers={{ controller_quorum_bootstrap_servers }} +{% else %} controller.quorum.voters={{ controller_quorum_voters }} +{% endif %} controller.listener.names={{ controller_listener_names }} diff --git a/tests/kafkatest/tests/core/quorum_reconfiguration_test.py b/tests/kafkatest/tests/core/quorum_reconfiguration_test.py new file mode 100644 index 0000000000..432b25c168 --- /dev/null +++ b/tests/kafkatest/tests/core/quorum_reconfiguration_test.py @@ -0,0 +1,165 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import json +import re + +from functools import partial + +from ducktape.mark import matrix +from ducktape.mark.resource import cluster + +from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.services.kafka import KafkaService +from kafkatest.services.kafka.quorum import combined_kraft, ServiceQuorumInfo, isolated_kraft +from kafkatest.services.verifiable_producer import VerifiableProducer +from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest +from kafkatest.utils import is_int +from kafkatest.version import DEV_BRANCH + +# +# Test quorum reconfiguration for combined and isolated mode +# +class TestQuorumReconfiguration(ProduceConsumeValidateTest): + def __init__(self, test_context): + super(TestQuorumReconfiguration, self).__init__(test_context=test_context) + + def setUp(self): + self.topic = "test_topic" + self.partitions = 3 + self.replication_factor = 3 + + # Producer and consumer + self.producer_throughput = 1000 + self.num_producers = 1 + self.num_consumers = 1 + + def perform_reconfig(self, active_controller_id, inactive_controller_id, inactive_controller, broker_ids): + # Check describe quorum output shows the controller (first node) is the leader and the only voter + output = self.kafka.describe_quorum() + assert re.search(r"LeaderId:\s*" + str(active_controller_id), output) + assert_nodes_in_output(r"CurrentVoters:.*", output, active_controller_id) + assert_nodes_in_output(r"CurrentObservers:.*", output, *broker_ids) + + # Start second controller + self.kafka.controller_quorum.add_broker(inactive_controller) + output = self.kafka.describe_quorum() + assert re.search(r"LeaderId:\s*" + str(active_controller_id), output) + assert_nodes_in_output(r"CurrentVoters:.*", output, active_controller_id) + assert_nodes_in_output(r"CurrentObservers:.*", output, *broker_ids + [inactive_controller_id]) + + # Add controller to quorum + self.kafka.controller_quorum.add_controller(inactive_controller_id, inactive_controller) + + # Check describe quorum output shows both controllers are voters + output = self.kafka.describe_quorum() + assert re.search(r"LeaderId:\s*" + str(active_controller_id), output) + assert_nodes_in_output(r"CurrentVoters:.*", output, active_controller_id, inactive_controller_id) + assert_nodes_in_output(r"CurrentObservers:.*", output, *broker_ids) + + # Remove leader from quorum + voters = json_from_line(r"CurrentVoters:.*", output) + directory_id = next(voter["directoryId"] for voter in voters if voter["id"] == active_controller_id) + self.kafka.controller_quorum.remove_controller(active_controller_id, directory_id) + + # Check describe quorum output to show second_controller is now the leader + output = self.kafka.describe_quorum() + assert re.search(r"LeaderId:\s*" + str(inactive_controller_id), output) + assert_nodes_in_output(r"CurrentVoters:.*", output, inactive_controller_id) + assert_nodes_in_output(r"CurrentObservers:.*", output, *broker_ids) + + + @cluster(num_nodes=6) + @matrix(metadata_quorum=[combined_kraft]) + def test_combined_mode_reconfig(self, metadata_quorum): + self.kafka = KafkaService(self.test_context, + num_nodes=4, + zk=None, + topics={self.topic: {"partitions": self.partitions, + "replication-factor": self.replication_factor, + 'configs': {"min.insync.replicas": 1}}}, + version=DEV_BRANCH, + controller_num_nodes_override=2, + dynamicRaftQuorum=True) + # Start one out of two controllers (standalone mode) + inactive_controller = self.kafka.nodes[1] + self.kafka.start(nodes_to_skip=[inactive_controller]) + + # Start producer and consumer + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, + self.topic, throughput=self.producer_throughput, + message_validator=is_int, compression_types=["none"], + version=DEV_BRANCH, offline_nodes=[inactive_controller]) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, + self.topic, new_consumer=True, consumer_timeout_ms=30000, + message_validator=is_int, version=DEV_BRANCH) + # Perform reconfigurations + self.run_produce_consume_validate( + core_test_action=lambda: self.perform_reconfig(self.kafka.idx(self.kafka.nodes[0]), + self.kafka.idx(inactive_controller), + inactive_controller, + [self.kafka.idx(node) for node in self.kafka.nodes[2:]])) + + @cluster(num_nodes=7) + @matrix(metadata_quorum=[isolated_kraft]) + def test_isolated_mode_reconfig(self, metadata_quorum): + # Start up KRaft controller in migration mode + remote_quorum = partial(ServiceQuorumInfo, isolated_kraft) + self.kafka = KafkaService(self.test_context, + num_nodes=3, + zk=None, + topics={self.topic: {"partitions": self.partitions, + "replication-factor": self.replication_factor, + 'configs': {"min.insync.replicas": 1}}}, + version=DEV_BRANCH, + controller_num_nodes_override=2, + quorum_info_provider=remote_quorum, + dynamicRaftQuorum=True) + # Start one out of two controllers (standalone mode) + controller_quorum = self.kafka.controller_quorum + inactive_controller = controller_quorum.nodes[1] + self.kafka.start(isolated_controllers_to_skip=[inactive_controller]) + + # Start producer and consumer + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, + self.topic, throughput=self.producer_throughput, + message_validator=is_int, compression_types=["none"], + version=DEV_BRANCH) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, + self.topic, new_consumer=True, consumer_timeout_ms=30000, + message_validator=is_int, version=DEV_BRANCH) + # Perform reconfigurations + self.run_produce_consume_validate( + core_test_action=lambda: self.perform_reconfig(controller_quorum.node_id_as_isolated_controller(self.kafka.controller_quorum.nodes[0]), + controller_quorum.node_id_as_isolated_controller(inactive_controller), + inactive_controller, + [self.kafka.idx(node) for node in self.kafka.nodes])) + +def assert_nodes_in_output(pattern, output, *node_ids): + nodes = json_from_line(pattern, output) + assert len(nodes) == len(node_ids) + + for node in nodes: + assert node["id"] in node_ids + +def json_from_line(pattern, output): + match = re.search(pattern, output) + if not match: + raise Exception("Expected match for pattern %s in describe quorum output" % pattern) + line = match.group(0) + start_index = line.find('[') + end_index = line.rfind(']') + 1 + + return json.loads(line[start_index:end_index]) From 89cb632acd3d895bf057640eb05319ae1c3f4a4d Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Thu, 26 Sep 2024 10:56:19 -0700 Subject: [PATCH 100/123] KAFKA-17608, KAFKA-17604, KAFKA-16963; KRaft controller crashes when active controller is removed (#17146) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This change fixes a few issues. KAFKA-17608; KRaft controller crashes when active controller is removed When a control batch is committed, the quorum controller currently increases the last stable offset but fails to create a snapshot for that offset. This causes an issue if the quorum controller renounces and needs to revert to that offset (which has no snapshot present). Since the control batches are no-ops for the quorum controller, it does not need to update its offsets for control records. We skip handle commit logic for control batches. KAFKA-17604; Describe quorum output missing added voters endpoints Describe quorum output will miss endpoints of voters which were added via AddRaftVoter. This is due to a bug in LeaderState's updateVoterAndObserverStates which will pull replica state from observer states map (which does not include endpoints). The fix is to populate endpoints from the lastVoterSet passed into the method. Reviewers: José Armando García Sancio , Colin P. McCabe , Chia-Ping Tsai --- .../controller/OffsetControlManager.java | 4 + .../kafka/controller/QuorumController.java | 8 +- .../apache/kafka/raft/KafkaRaftClient.java | 2 +- .../org/apache/kafka/raft/LeaderState.java | 9 +- tests/kafkatest/services/kafka/kafka.py | 63 +++++----- .../tests/core/quorum_reconfiguration_test.py | 110 ++++++++++++------ 6 files changed, 123 insertions(+), 73 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java index 9a04301456..e422ad602c 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java @@ -280,6 +280,10 @@ void handleCommitBatch(Batch batch) { this.lastCommittedOffset = batch.lastOffset(); this.lastCommittedEpoch = batch.epoch(); maybeAdvanceLastStableOffset(); + handleCommitBatchMetrics(batch); + } + + void handleCommitBatchMetrics(Batch batch) { metrics.setLastCommittedRecordOffset(batch.lastOffset()); if (!active()) { // On standby controllers, the last applied record offset is equals to the last diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 7445171627..1fe995dddd 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -1065,7 +1065,10 @@ public void handleCommit(BatchReader reader) { int epoch = batch.epoch(); List messages = batch.records(); - if (isActive) { + if (messages.isEmpty()) { + log.debug("Skipping handling commit for batch with no data records with offset {} and epoch {}.", offset, epoch); + offsetControl.handleCommitBatchMetrics(batch); + } else if (isActive) { // If the controller is active, the records were already replayed, // so we don't need to do it here. log.debug("Completing purgatory items up to offset {} and epoch {}.", offset, epoch); @@ -1075,9 +1078,6 @@ public void handleCommit(BatchReader reader) { offsetControl.handleCommitBatch(batch); deferredEventQueue.completeUpTo(offsetControl.lastStableOffset()); deferredUnstableEventQueue.completeUpTo(offsetControl.lastCommittedOffset()); - - // The active controller can delete up to the current committed offset. - snapshotRegistry.deleteSnapshotsUpTo(offsetControl.lastStableOffset()); } else { // If the controller is a standby, replay the records that were // created by the active controller. diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index d75417a77a..4a708cb871 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -2911,7 +2911,7 @@ private long pollResigned(long currentTimeMs) { if (quorum.isVoter()) { transitionToCandidate(currentTimeMs); } else { - // It is posible that the old leader is not a voter in the new voter set. + // It is possible that the old leader is not a voter in the new voter set. // In that case increase the epoch and transition to unattached. The epoch needs // to be increased to avoid FETCH responses with the leader being this replica. transitionToUnattached(quorum.epoch() + 1); diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index ae9be94e13..c09282c87c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -678,6 +678,9 @@ private void updateVoterAndObserverStates(VoterSet lastVoterSet) { // Make sure that the replica key in the replica state matches the voter's state.setReplicaKey(voterNode.voterKey()); + + // Make sure that the listeners are updated + state.updateListeners(voterNode.listeners()); newVoterStates.put(state.replicaKey.id(), state); } voterStates = newVoterStates; @@ -752,8 +755,12 @@ void setReplicaKey(ReplicaKey replicaKey) { this.replicaKey = replicaKey; } + void updateListeners(Endpoints listeners) { + this.listeners = listeners; + } + void clearListeners() { - this.listeners = Endpoints.empty(); + updateListeners(Endpoints.empty()); } boolean matchesKey(ReplicaKey replicaKey) { diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 547762a534..f154e2049c 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -267,6 +267,7 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI :param bool allow_zk_with_kraft: if True, then allow a KRaft broker or controller to also use ZooKeeper :param quorum_info_provider: A function that takes this KafkaService as an argument and returns a ServiceQuorumInfo. If this is None, then the ServiceQuorumInfo is generated from the test context :param use_new_coordinator: When true, use the new implementation of the group coordinator as per KIP-848. If this is None, the default existing group coordinator is used. + :param dynamicRaftQuorum: When true, the quorum uses kraft.version=1, controller_quorum_bootstrap_servers, and bootstraps the first controller using the standalone flag """ self.zk = zk @@ -300,7 +301,8 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI if self.quorum_info.using_kraft: self.dynamicRaftQuorum = dynamicRaftQuorum - self.first_controller_started = False + # Used to ensure not more than one controller bootstraps with the standalone flag + self.standalone_controller_bootstrapped = False if self.quorum_info.has_brokers: num_nodes_broker_role = num_nodes if self.quorum_info.has_controllers: @@ -458,8 +460,12 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.combined_nodes_started = 0 self.nodes_to_start = self.nodes - # Does not do any validation to check if this node is part of an isolated controller quorum or not def node_id_as_isolated_controller(self, node): + """ + Generates the node id for a controller-only node, starting from config_property.FIRST_CONTROLLER_ID so as not + to overlap with broker id numbering. + This method does not do any validation to check this node is actually part of an isolated controller quorum. + """ return self.idx(node) + config_property.FIRST_CONTROLLER_ID - 1 def reconfigure_zk_for_migration(self, kraft_quorum): @@ -755,8 +761,7 @@ def set_protocol_and_port(self, node): for port in self.port_mappings.values(): if port.open: listeners.append(port.listener()) - if not port.name in controller_listener_names: - advertised_listeners.append(port.advertised_listener(node)) + advertised_listeners.append(port.advertised_listener(node)) protocol_map.append(port.listener_security_protocol()) controller_sec_protocol = self.isolated_controller_quorum.controller_security_protocol if self.isolated_controller_quorum \ else self.controller_security_protocol if self.quorum_info.has_brokers_and_controllers and not quorum.NodeQuorumInfo(self.quorum_info, node).has_controller_role \ @@ -873,12 +878,12 @@ def start_node(self, node, timeout_sec=60, **kwargs): self.maybe_setup_broker_scram_credentials(node) if self.quorum_info.using_kraft: - # define controller.quorum.bootstrap.servrers or controller.quorum.voters text + # define controller.quorum.bootstrap.servers or controller.quorum.voters text security_protocol_to_use = self.controller_quorum.controller_security_protocol first_node_id = 1 if self.quorum_info.has_brokers_and_controllers else config_property.FIRST_CONTROLLER_ID - controller_quorum_bootstrap_servers = ','.join(["%s:%s" % (node.account.hostname, - config_property.FIRST_CONTROLLER_PORT + - KafkaService.SECURITY_PROTOCOLS.index(security_protocol_to_use)) + controller_quorum_bootstrap_servers = ','.join(["{}:{}".format(node.account.hostname, + config_property.FIRST_CONTROLLER_PORT + + KafkaService.SECURITY_PROTOCOLS.index(security_protocol_to_use)) for node in self.controller_quorum.nodes[:self.controller_quorum.num_nodes_controller_role]]) if self.dynamicRaftQuorum: self.controller_quorum_bootstrap_servers = controller_quorum_bootstrap_servers @@ -904,11 +909,11 @@ def start_node(self, node, timeout_sec=60, **kwargs): cmd = "%s format --ignore-formatted --config %s --cluster-id %s" % (kafka_storage_script, KafkaService.CONFIG_FILE, config_property.CLUSTER_ID) if self.dynamicRaftQuorum: cmd += " --feature kraft.version=1" - if not self.first_controller_started and self.node_quorum_info.has_controller_role: + if not self.standalone_controller_bootstrapped and self.node_quorum_info.has_controller_role: cmd += " --standalone" + self.standalone_controller_bootstrapped = True self.logger.info("Running log directory format command...\n%s" % cmd) node.account.ssh(cmd) - self.first_controller_started = True cmd = self.start_cmd(node) self.logger.debug("Attempting to start KafkaService %s on %s with command: %s" %\ @@ -1038,11 +1043,12 @@ def kafka_metadata_quorum_cmd(self, node, kafka_security_protocol=None, use_cont else: security_protocol_to_use = kafka_security_protocol if use_controller_bootstrap: - bootstrap = "--bootstrap-controller %s" % (self.bootstrap_controllers("CONTROLLER_%s" % security_protocol_to_use)) + bootstrap = "--bootstrap-controller {}".format( + self.bootstrap_controllers("CONTROLLER_{}".format(security_protocol_to_use))) else: - bootstrap = "--bootstrap-server %s" % (self.bootstrap_servers(security_protocol_to_use)) + bootstrap = "--bootstrap-server {}".format(self.bootstrap_servers(security_protocol_to_use)) kafka_metadata_script = self.path.script("kafka-metadata-quorum.sh", node) - return "%s %s" % (kafka_metadata_script, bootstrap) + return "{} {}".format(kafka_metadata_script, bootstrap) def kafka_topics_cmd_with_optional_security_settings(self, node, force_use_zk_connection, kafka_security_protocol=None, offline_nodes=[]): if self.quorum_info.using_kraft and not self.quorum_info.has_brokers: @@ -1794,10 +1800,8 @@ def describe_quorum(self, node=None): if node is None: node = self.nodes[0] cmd = fix_opts_for_new_jvm(node) - cmd += "%(kafka_metadata_quorum_cmd)s describe --status" % { - 'kafka_metadata_quorum_cmd': self.kafka_metadata_quorum_cmd(node) - } - self.logger.info("Running describe quorum command...\n%s" % cmd) + cmd += f"{self.kafka_metadata_quorum_cmd(node)} describe --status" + self.logger.info(f"Running describe quorum command...\n{cmd}") node.account.ssh(cmd) output = "" @@ -1821,11 +1825,9 @@ def add_controller(self, controllerId, controller): controller.account.create_file(command_config_path, configs) cmd = fix_opts_for_new_jvm(controller) - cmd += "%(kafka_metadata_quorum_cmd)s --command-config %(command_config)s add-controller" % { - 'kafka_metadata_quorum_cmd': self.kafka_metadata_quorum_cmd(controller, use_controller_bootstrap=True), - 'command_config': command_config_path - } - self.logger.info("Running add controller command...\n%s" % cmd) + kafka_metadata_quorum_cmd = self.kafka_metadata_quorum_cmd(controller, use_controller_bootstrap=True) + cmd += f"{kafka_metadata_quorum_cmd} --command-config {command_config_path} add-controller" + self.logger.info(f"Running add controller command...\n{cmd}") controller.account.ssh(cmd) def remove_controller(self, controllerId, directoryId, node=None): @@ -1835,12 +1837,9 @@ def remove_controller(self, controllerId, directoryId, node=None): if node is None: node = self.nodes[0] cmd = fix_opts_for_new_jvm(node) - cmd += "%(kafka_metadata_quorum_cmd)s remove-controller -i %(controller_id)s -d %(directory_id)s" % { - 'kafka_metadata_quorum_cmd': self.kafka_metadata_quorum_cmd(node, use_controller_bootstrap=True), - 'controller_id': controllerId, - 'directory_id': directoryId - } - self.logger.info("Running remove controller command...\n%s" % cmd) + kafka_metadata_quorum_cmd = self.kafka_metadata_quorum_cmd(node, use_controller_bootstrap=True) + cmd += f"{kafka_metadata_quorum_cmd} remove-controller -i {controllerId} -d {directoryId}" + self.logger.info(f"Running remove controller command...\n{cmd}") node.account.ssh(cmd) def zk_connect_setting(self): @@ -1850,8 +1849,8 @@ def zk_connect_setting(self): def __bootstrap_servers(self, port, validate=True, offline_nodes=[]): if validate and not port.open: - raise ValueError("We are retrieving bootstrap servers for the port: %s which is not currently open. - " % - str(port.port_number)) + raise ValueError(f"We are retrieving bootstrap servers for the port: {str(port.port_number)} " + f"which is not currently open.") return ','.join([node.account.hostname + ":" + str(port.port_number) for node in self.nodes @@ -1859,8 +1858,8 @@ def __bootstrap_servers(self, port, validate=True, offline_nodes=[]): def __bootstrap_controllers(self, port, validate=True, offline_nodes=[]): if validate and not port.open: - raise ValueError("We are retrieving bootstrap servers for the port: %s which is not currently open. - " % - str(port.port_number)) + raise ValueError(f"We are retrieving bootstrap controllers for the port: {str(port.port_number)} " + f"which is not currently open.") return ','.join([node.account.hostname + ":" + str(port.port_number) for node in self.controller_quorum.nodes[:self.controller_quorum.num_nodes_controller_role] diff --git a/tests/kafkatest/tests/core/quorum_reconfiguration_test.py b/tests/kafkatest/tests/core/quorum_reconfiguration_test.py index 432b25c168..80cf535021 100644 --- a/tests/kafkatest/tests/core/quorum_reconfiguration_test.py +++ b/tests/kafkatest/tests/core/quorum_reconfiguration_test.py @@ -17,9 +17,13 @@ import re from functools import partial +from typing import List +from ducktape.cluster.cluster import ClusterNode from ducktape.mark import matrix from ducktape.mark.resource import cluster +from ducktape.tests.test import TestContext +from ducktape.utils.util import wait_until from kafkatest.services.console_consumer import ConsoleConsumer from kafkatest.services.kafka import KafkaService @@ -33,7 +37,7 @@ # Test quorum reconfiguration for combined and isolated mode # class TestQuorumReconfiguration(ProduceConsumeValidateTest): - def __init__(self, test_context): + def __init__(self, test_context: TestContext): super(TestQuorumReconfiguration, self).__init__(test_context=test_context) def setUp(self): @@ -46,46 +50,58 @@ def setUp(self): self.num_producers = 1 self.num_consumers = 1 - def perform_reconfig(self, active_controller_id, inactive_controller_id, inactive_controller, broker_ids): + def perform_reconfig(self, + active_controller_id: int, + inactive_controller_id: int, + inactive_controller: ClusterNode, + broker_only_ids: List[int]): + """ + Tests quorum reconfiguration by adding a second controller and then removing the active controller. + + :param active_controller_id: id of the active controller + :param inactive_controller_id: id of the inactive controller + :param inactive_controller: node object of the inactive controller + :param broker_only_ids: broker ids of nodes which have no controller process + """ # Check describe quorum output shows the controller (first node) is the leader and the only voter - output = self.kafka.describe_quorum() - assert re.search(r"LeaderId:\s*" + str(active_controller_id), output) - assert_nodes_in_output(r"CurrentVoters:.*", output, active_controller_id) - assert_nodes_in_output(r"CurrentObservers:.*", output, *broker_ids) - + wait_until(lambda: check_describe_quorum_output(self.kafka.describe_quorum(), + active_controller_id, + [active_controller_id], + broker_only_ids), timeout_sec=5) # Start second controller self.kafka.controller_quorum.add_broker(inactive_controller) - output = self.kafka.describe_quorum() - assert re.search(r"LeaderId:\s*" + str(active_controller_id), output) - assert_nodes_in_output(r"CurrentVoters:.*", output, active_controller_id) - assert_nodes_in_output(r"CurrentObservers:.*", output, *broker_ids + [inactive_controller_id]) - + wait_until(lambda: check_describe_quorum_output(self.kafka.describe_quorum(), + active_controller_id, + [active_controller_id], + broker_only_ids + [inactive_controller_id]), timeout_sec=5) # Add controller to quorum self.kafka.controller_quorum.add_controller(inactive_controller_id, inactive_controller) # Check describe quorum output shows both controllers are voters - output = self.kafka.describe_quorum() - assert re.search(r"LeaderId:\s*" + str(active_controller_id), output) - assert_nodes_in_output(r"CurrentVoters:.*", output, active_controller_id, inactive_controller_id) - assert_nodes_in_output(r"CurrentObservers:.*", output, *broker_ids) - + wait_until(lambda: check_describe_quorum_output(self.kafka.describe_quorum(), + active_controller_id, + [active_controller_id, inactive_controller_id], + broker_only_ids), timeout_sec=5) # Remove leader from quorum - voters = json_from_line(r"CurrentVoters:.*", output) + voters = json_from_line(r"CurrentVoters:.*", self.kafka.describe_quorum()) directory_id = next(voter["directoryId"] for voter in voters if voter["id"] == active_controller_id) self.kafka.controller_quorum.remove_controller(active_controller_id, directory_id) - - # Check describe quorum output to show second_controller is now the leader - output = self.kafka.describe_quorum() - assert re.search(r"LeaderId:\s*" + str(inactive_controller_id), output) - assert_nodes_in_output(r"CurrentVoters:.*", output, inactive_controller_id) - assert_nodes_in_output(r"CurrentObservers:.*", output, *broker_ids) - + # Describe quorum output shows the second controller is now leader, old controller is an observer + wait_until(lambda: check_describe_quorum_output(self.kafka.describe_quorum(), + inactive_controller_id, + [inactive_controller_id], + broker_only_ids + [active_controller_id]), timeout_sec=5) @cluster(num_nodes=6) @matrix(metadata_quorum=[combined_kraft]) def test_combined_mode_reconfig(self, metadata_quorum): + """ + Tests quorum reconfiguration in combined mode with produce & consume validation. + Starts a controller in standalone mode with two other broker nodes, then calls perform_reconfig to add + a second controller and then remove the first controller. + """ self.kafka = KafkaService(self.test_context, - num_nodes=4, + num_nodes=4, # 2 combined, 2 broker-only nodes zk=None, topics={self.topic: {"partitions": self.partitions, "replication-factor": self.replication_factor, @@ -93,7 +109,8 @@ def test_combined_mode_reconfig(self, metadata_quorum): version=DEV_BRANCH, controller_num_nodes_override=2, dynamicRaftQuorum=True) - # Start one out of two controllers (standalone mode) + # Start a controller and the broker-only nodes + # We leave starting the second controller for later in perform_reconfig inactive_controller = self.kafka.nodes[1] self.kafka.start(nodes_to_skip=[inactive_controller]) @@ -115,10 +132,15 @@ def test_combined_mode_reconfig(self, metadata_quorum): @cluster(num_nodes=7) @matrix(metadata_quorum=[isolated_kraft]) def test_isolated_mode_reconfig(self, metadata_quorum): + """ + Tests quorum reconfiguration in isolated mode with produce & consume validation. + Starts a controller in standalone mode with three other broker nodes, then calls perform_reconfig to add + a second controller and then remove the first controller. + """ # Start up KRaft controller in migration mode remote_quorum = partial(ServiceQuorumInfo, isolated_kraft) self.kafka = KafkaService(self.test_context, - num_nodes=3, + num_nodes=3, # 3 broker-only nodes zk=None, topics={self.topic: {"partitions": self.partitions, "replication-factor": self.replication_factor, @@ -127,7 +149,8 @@ def test_isolated_mode_reconfig(self, metadata_quorum): controller_num_nodes_override=2, quorum_info_provider=remote_quorum, dynamicRaftQuorum=True) - # Start one out of two controllers (standalone mode) + # Start a controller and the broker-only nodes + # We leave starting the second controller for later in perform_reconfig controller_quorum = self.kafka.controller_quorum inactive_controller = controller_quorum.nodes[1] self.kafka.start(isolated_controllers_to_skip=[inactive_controller]) @@ -147,17 +170,34 @@ def test_isolated_mode_reconfig(self, metadata_quorum): inactive_controller, [self.kafka.idx(node) for node in self.kafka.nodes])) -def assert_nodes_in_output(pattern, output, *node_ids): +def check_nodes_in_output(pattern: str, output: str, *node_ids: int): nodes = json_from_line(pattern, output) - assert len(nodes) == len(node_ids) + if len(nodes) != len(node_ids): + return False for node in nodes: - assert node["id"] in node_ids - -def json_from_line(pattern, output): + if not node["id"] in node_ids: + return False + return True + +def check_describe_quorum_output(output: str, leader_id: int, voter_ids: List[int], observer_ids: List[int]): + """ + Check that the describe quorum output contains the expected leader, voters, and observers + :param output: Describe quorum output + :param leader_id: Expected leader id + :param voter_ids: Expected voter ids + :param observer_ids: Expected observer ids + :return: + """ + if not re.search(r"LeaderId:\s*" + str(leader_id), output): + return False + return (check_nodes_in_output(r"CurrentVoters:.*", output, *voter_ids) and + check_nodes_in_output(r"CurrentObservers:.*", output, *observer_ids)) + +def json_from_line(pattern: str, output: str): match = re.search(pattern, output) if not match: - raise Exception("Expected match for pattern %s in describe quorum output" % pattern) + raise Exception(f"Expected match for pattern {pattern} in describe quorum output") line = match.group(0) start_index = line.find('[') end_index = line.rfind(']') + 1 From 2cbc5bd3ca22185d9cd357c9db23a2cfb43a0fff Mon Sep 17 00:00:00 2001 From: David Arthur Date: Sat, 28 Sep 2024 09:57:25 -0400 Subject: [PATCH 101/123] KAFKA-17636 Fix missing SCRAM bootstrap records (#17305) Fixes a regression introduced by #16669 which inadvertently stopped processing SCRAM arguments from kafka-storage.sh Reviewers: Colin P. McCabe , Federico Valeri --- .../main/scala/kafka/tools/StorageTool.scala | 2 + .../unit/kafka/tools/StorageToolTest.scala | 51 ++++++++++++++++++- 2 files changed, 51 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 43eb657976..6dc4653961 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -130,6 +130,8 @@ object StorageTool extends Logging { if (namespace.getBoolean("standalone")) { formatter.setInitialVoters(createStandaloneDynamicVoters(config)) } + Option(namespace.getList("add_scram")). + foreach(scramArgs => formatter.setScramArguments(scramArgs.asInstanceOf[util.List[String]])) configToLogDirectories(config).foreach(formatter.addDirectory(_)) formatter.run() } diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 3a8f064b1d..83b21b43fb 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -21,12 +21,14 @@ import java.io.{ByteArrayOutputStream, File, PrintStream} import java.nio.charset.StandardCharsets import java.nio.file.Files import java.util -import java.util.Properties +import java.util.{Optional, Properties} import kafka.server.KafkaConfig import kafka.utils.TestUtils import net.sourceforge.argparse4j.inf.ArgumentParserException +import org.apache.kafka.common.metadata.UserScramCredentialRecord import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.common.Features +import org.apache.kafka.metadata.bootstrap.BootstrapDirectory import org.apache.kafka.metadata.properties.{MetaPropertiesEnsemble, PropertiesUtils} import org.apache.kafka.metadata.storage.FormatterException import org.apache.kafka.raft.QuorumConfig @@ -37,6 +39,7 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource import scala.collection.mutable.ListBuffer +import scala.jdk.CollectionConverters.IterableHasAsScala @Timeout(value = 40) class StorageToolTest { @@ -433,5 +436,49 @@ Found problem: contains("Formatting dynamic metadata voter directory %s".format(availableDirs.head)), "Failed to find content in output: " + stream.toString()) } -} + @Test + def testBootstrapScramRecords(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultDynamicQuorumProperties) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + val stream = new ByteArrayOutputStream() + val arguments = ListBuffer[String]( + "--release-version", "3.9-IV0", + "--add-scram", "SCRAM-SHA-512=[name=alice,password=changeit]", + "--add-scram", "SCRAM-SHA-512=[name=bob,password=changeit]" + ) + + assertEquals(0, runFormatCommand(stream, properties, arguments.toSeq)) + + // Not doing full SCRAM record validation since that's covered elsewhere. + // Just checking that we generate the correct number of records + val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString, Optional.empty).read + val scramRecords = bootstrapMetadata.records().asScala + .filter(apiMessageAndVersion => apiMessageAndVersion.message().isInstanceOf[UserScramCredentialRecord]) + .map(apiMessageAndVersion => apiMessageAndVersion.message().asInstanceOf[UserScramCredentialRecord]) + .toList + assertEquals(2, scramRecords.size) + assertEquals("alice", scramRecords.head.name()) + assertEquals("bob", scramRecords.last.name()) + } + + @Test + def testScramRecordsOldReleaseVersion(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultDynamicQuorumProperties) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + val stream = new ByteArrayOutputStream() + val arguments = ListBuffer[String]( + "--release-version", "3.4", + "--add-scram", "SCRAM-SHA-512=[name=alice,password=changeit]", + "--add-scram", "SCRAM-SHA-512=[name=bob,password=changeit]" + ) + + assertEquals( + "SCRAM is only supported in metadata.version 3.5-IV2 or later.", + assertThrows(classOf[FormatterException], () => runFormatCommand(stream, properties, arguments.toSeq)).getMessage) + } +} From edd77c1e2576b3e379db816530fec578b8ba1637 Mon Sep 17 00:00:00 2001 From: Bill Bejeck Date: Sun, 15 Sep 2024 14:26:06 -0400 Subject: [PATCH 102/123] MINOR: Need to split the controller bootstrap servers on ',' in list comprehenson (#17183) Kafka Streams system tests were failing with this error: Failed to parse host name from entry 3001@d for the configuration controller.quorum.voters. Each entry should be in the form `{id}@{host}:{port}`. The cause is that in kafka.py line 876, we create a delimited string from a list comprehension, but the input is a string itself, so each character gets appended vs. the bootstrap server string of host:port. To fix this, this PR adds split(',') to controller_quorum_bootstrap_servers. Note that this only applies when dynamicRaftQuorum=False Reviewers: Alyssa Huang , Chia-Ping Tsai --- tests/kafkatest/services/kafka/kafka.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index f154e2049c..6987bbd05d 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -890,7 +890,7 @@ def start_node(self, node, timeout_sec=60, **kwargs): else: self.controller_quorum_voters = ','.join(["%s@%s" % (self.controller_quorum.idx(node) + first_node_id - 1, bootstrap_server) - for bootstrap_server in controller_quorum_bootstrap_servers]) + for bootstrap_server in controller_quorum_bootstrap_servers.split(',')]) # define controller.listener.names self.controller_listener_names = ','.join(self.controller_listener_name_list(node)) # define sasl.mechanism.controller.protocol to match the isolated quorum if one exists From 5c95a5da31f87becebab223d0a80d0469c3cecf5 Mon Sep 17 00:00:00 2001 From: Alyssa Huang Date: Mon, 30 Sep 2024 05:51:49 -0700 Subject: [PATCH 103/123] MINOR: Fix kafkatest advertised listeners (#17294) Followup for #17146 Reviewers: Bill Bejeck --- tests/kafkatest/services/kafka/kafka.py | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 6987bbd05d..51eab79db9 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -280,7 +280,8 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.controller_quorum = None # will define below if necessary self.isolated_controller_quorum = None # will define below if necessary self.configured_for_zk_migration = False - + self.dynamicRaftQuorum = False + # Set use_new_coordinator based on context and arguments. default_use_new_coordinator = False @@ -761,7 +762,9 @@ def set_protocol_and_port(self, node): for port in self.port_mappings.values(): if port.open: listeners.append(port.listener()) - advertised_listeners.append(port.advertised_listener(node)) + if (self.dynamicRaftQuorum and quorum.NodeQuorumInfo(self.quorum_info, node).has_controller_role) or \ + port.name not in controller_listener_names: + advertised_listeners.append(port.advertised_listener(node)) protocol_map.append(port.listener_security_protocol()) controller_sec_protocol = self.isolated_controller_quorum.controller_security_protocol if self.isolated_controller_quorum \ else self.controller_security_protocol if self.quorum_info.has_brokers_and_controllers and not quorum.NodeQuorumInfo(self.quorum_info, node).has_controller_role \ @@ -881,16 +884,18 @@ def start_node(self, node, timeout_sec=60, **kwargs): # define controller.quorum.bootstrap.servers or controller.quorum.voters text security_protocol_to_use = self.controller_quorum.controller_security_protocol first_node_id = 1 if self.quorum_info.has_brokers_and_controllers else config_property.FIRST_CONTROLLER_ID - controller_quorum_bootstrap_servers = ','.join(["{}:{}".format(node.account.hostname, - config_property.FIRST_CONTROLLER_PORT + - KafkaService.SECURITY_PROTOCOLS.index(security_protocol_to_use)) - for node in self.controller_quorum.nodes[:self.controller_quorum.num_nodes_controller_role]]) if self.dynamicRaftQuorum: - self.controller_quorum_bootstrap_servers = controller_quorum_bootstrap_servers + self.controller_quorum_bootstrap_servers = ','.join(["{}:{}".format(node.account.hostname, + config_property.FIRST_CONTROLLER_PORT + + KafkaService.SECURITY_PROTOCOLS.index(security_protocol_to_use)) + for node in self.controller_quorum.nodes[:self.controller_quorum.num_nodes_controller_role]]) else: - self.controller_quorum_voters = ','.join(["%s@%s" % (self.controller_quorum.idx(node) + first_node_id - 1, - bootstrap_server) - for bootstrap_server in controller_quorum_bootstrap_servers.split(',')]) + self.controller_quorum_voters = ','.join(["{}@{}:{}".format(self.controller_quorum.idx(node) + + first_node_id - 1, + node.account.hostname, + config_property.FIRST_CONTROLLER_PORT + + KafkaService.SECURITY_PROTOCOLS.index(security_protocol_to_use)) + for node in self.controller_quorum.nodes[:self.controller_quorum.num_nodes_controller_role]]) # define controller.listener.names self.controller_listener_names = ','.join(self.controller_listener_name_list(node)) # define sasl.mechanism.controller.protocol to match the isolated quorum if one exists From 550bf6046056d9ef34a50aa5a0c5b2303d1fe00b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Fri, 4 Oct 2024 10:51:43 -0400 Subject: [PATCH 104/123] KAFKA-16927; Handle expanding leader endpoints (#17363) When a replica restarts in the follower state it is possible for the set of leader endpoints to not match the latest set of leader endpoints. Voters will discover the latest set of leader endpoints through the BEGIN_QUORUM_EPOCH request. This means that KRaft needs to allow for the replica to transition from Follower to Follower when only the set of leader endpoints has changed. Reviewers: Colin P. McCabe , Alyssa Huang --- .../org/apache/kafka/raft/QuorumState.java | 54 ++++++++++++++++--- .../raft/KafkaRaftClientReconfigTest.java | 39 ++++++++++++++ .../kafka/raft/KafkaRaftClientTest.java | 41 ++++++++++++++ .../apache/kafka/raft/QuorumStateTest.java | 34 ++++++++++++ .../kafka/raft/RaftClientTestContext.java | 26 ++++++++- 5 files changed, 185 insertions(+), 9 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 888273723f..0598ce062d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -470,15 +470,53 @@ public void transitionToUnattachedVotedState( */ public void transitionToFollower(int epoch, int leaderId, Endpoints endpoints) { int currentEpoch = state.epoch(); - if (localId.isPresent() && leaderId == localId.getAsInt()) { - throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId + - " and epoch " + epoch + " since it matches the local broker.id " + localId); + if (endpoints.isEmpty()) { + throw new IllegalArgumentException( + String.format( + "Cannot transition to Follower with leader %s and epoch %s without a leader endpoint", + leaderId, + epoch + ) + ); + } else if (localId.isPresent() && leaderId == localId.getAsInt()) { + throw new IllegalStateException( + String.format( + "Cannot transition to Follower with leader %s and epoch %s since it matches the local node.id %s", + leaderId, + epoch, + localId + ) + ); } else if (epoch < currentEpoch) { - throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId + - " and epoch " + epoch + " since the current epoch " + currentEpoch + " is larger"); - } else if (epoch == currentEpoch && (isFollower() || isLeader())) { - throw new IllegalStateException("Cannot transition to Follower with leader " + leaderId + - " and epoch " + epoch + " from state " + state); + throw new IllegalStateException( + String.format( + "Cannot transition to Follower with leader %s and epoch %s since the current epoch %s is larger", + leaderId, + epoch, + currentEpoch + ) + ); + } else if (epoch == currentEpoch) { + if (isFollower() && state.leaderEndpoints().size() >= endpoints.size()) { + throw new IllegalStateException( + String.format( + "Cannot transition to Follower with leader %s, epoch %s and endpoints %s from state %s", + leaderId, + epoch, + endpoints, + state + ) + ); + } else if (isLeader()) { + throw new IllegalStateException( + String.format( + "Cannot transition to Follower with leader %s and epoch %s from state %s", + leaderId, + epoch, + state + ) + ); + } } durableTransitionTo( diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 63e7f046b1..04ab47f34c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -2257,6 +2257,45 @@ void testObserverDiscoversLeaderWithUnknownVoters() throws Exception { assertEquals(-2, fetchRequest.destination().id()); } + @Test + public void testHandleBeginQuorumRequestMoreEndpoints() throws Exception { + ReplicaKey local = replicaKey(randomReplicaId(), true); + ReplicaKey leader = replicaKey(local.id() + 1, true); + int leaderEpoch = 3; + + VoterSet voters = VoterSetTest.voterSet(Stream.of(local, leader)); + + RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withBootstrapSnapshot(Optional.of(voters)) + .withElectedLeader(leaderEpoch, leader.id()) + .withKip853Rpc(true) + .build(); + + context.client.poll(); + + HashMap leaderListenersMap = new HashMap<>(2); + leaderListenersMap.put( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("localhost", 9990 + leader.id()) + ); + leaderListenersMap.put( + ListenerName.normalised("ANOTHER_LISTENER"), + InetSocketAddress.createUnresolved("localhost", 8990 + leader.id()) + ); + Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses(leaderListenersMap); + + context.deliverRequest(context.beginEpochRequest(leaderEpoch, leader.id(), leaderEndpoints)); + context.pollUntilResponse(); + + context.assertElectedLeader(leaderEpoch, leader.id()); + + context.assertSentBeginQuorumEpochResponse( + Errors.NONE, + leaderEpoch, + OptionalInt.of(leader.id()) + ); + } + private static void verifyVotersRecord( VoterSet expectedVoterSet, ByteBuffer recordKey, diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 1477ee0a39..6e5048e9e6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MutableRecordBatch; @@ -55,6 +56,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.OptionalInt; @@ -960,6 +962,45 @@ public void testHandleBeginQuorumRequest(boolean withKip853Rpc) throws Exception ); } + @Test + public void testHandleBeginQuorumRequestMoreEndpoints() throws Exception { + ReplicaKey local = replicaKey(randomReplicaId(), true); + ReplicaKey leader = replicaKey(local.id() + 1, true); + int leaderEpoch = 3; + + VoterSet voters = VoterSetTest.voterSet(Stream.of(local, leader)); + + RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withStaticVoters(voters) + .withElectedLeader(leaderEpoch, leader.id()) + .withKip853Rpc(true) + .build(); + + context.client.poll(); + + HashMap leaderListenersMap = new HashMap<>(2); + leaderListenersMap.put( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("localhost", 9990 + leader.id()) + ); + leaderListenersMap.put( + ListenerName.normalised("ANOTHER_LISTENER"), + InetSocketAddress.createUnresolved("localhost", 8990 + leader.id()) + ); + Endpoints leaderEndpoints = Endpoints.fromInetSocketAddresses(leaderListenersMap); + + context.deliverRequest(context.beginEpochRequest(leaderEpoch, leader.id(), leaderEndpoints)); + context.pollUntilResponse(); + + context.assertElectedLeader(leaderEpoch, leader.id()); + + context.assertSentBeginQuorumEpochResponse( + Errors.NONE, + leaderEpoch, + OptionalInt.of(leader.id()) + ); + } + @ParameterizedTest @ValueSource(booleans = { true, false }) public void testHandleBeginQuorumResponse(boolean withKip853Rpc) throws Exception { diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index fc14f4d9bc..7131701da7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -32,6 +33,7 @@ import java.io.UncheckedIOException; import java.net.InetSocketAddress; import java.util.Collections; +import java.util.HashMap; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; @@ -1211,6 +1213,38 @@ public void testFollowerToFollowerSameEpoch(KRaftVersion kraftVersion) { ); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testFollowerToFollowerSameEpochAndMoreEndpoints(KRaftVersion kraftVersion) { + int node1 = 1; + int node2 = 2; + VoterSet voters = localWithRemoteVoterSet(IntStream.of(node1, node2), kraftVersion); + QuorumState state = initializeEmptyState(voters, kraftVersion); + state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); + state.transitionToFollower( + 8, + node2, + voters.listeners(node2) + ); + + HashMap newNode2ListenersMap = new HashMap<>(2); + newNode2ListenersMap.put( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("localhost", 9990 + node2) + ); + newNode2ListenersMap.put( + ListenerName.normalised("ANOTHER_LISTENER"), + InetSocketAddress.createUnresolved("localhost", 8990 + node2) + ); + Endpoints newNode2Endpoints = Endpoints.fromInetSocketAddresses(newNode2ListenersMap); + + state.transitionToFollower( + 8, + node2, + newNode2Endpoints + ); + } + @ParameterizedTest @EnumSource(value = KRaftVersion.class) public void testFollowerToFollowerHigherEpoch(KRaftVersion kraftVersion) { diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 0f1cfd6f3c..7b1a55c1e0 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -1367,6 +1367,14 @@ BeginQuorumEpochRequestData beginEpochRequest(int epoch, int leaderId) { return beginEpochRequest(clusterId, epoch, leaderId); } + BeginQuorumEpochRequestData beginEpochRequest(int epoch, int leaderId, Endpoints endpoints) { + ReplicaKey localReplicaKey = kip853Rpc ? + ReplicaKey.of(localIdOrThrow(), localDirectoryId) : + ReplicaKey.of(-1, ReplicaKey.NO_DIRECTORY_ID); + + return beginEpochRequest(clusterId, epoch, leaderId, endpoints, localReplicaKey); + } + BeginQuorumEpochRequestData beginEpochRequest(String clusterId, int epoch, int leaderId) { ReplicaKey localReplicaKey = kip853Rpc ? ReplicaKey.of(localIdOrThrow(), localDirectoryId) : @@ -1380,13 +1388,29 @@ BeginQuorumEpochRequestData beginEpochRequest( int epoch, int leaderId, ReplicaKey voterKey + ) { + return beginEpochRequest( + clusterId, + epoch, + leaderId, + startingVoters.listeners(leaderId), + voterKey + ); + } + + BeginQuorumEpochRequestData beginEpochRequest( + String clusterId, + int epoch, + int leaderId, + Endpoints endpoints, + ReplicaKey voterKey ) { return RaftUtil.singletonBeginQuorumEpochRequest( metadataPartition, clusterId, epoch, leaderId, - startingVoters.listeners(leaderId), + endpoints, voterKey ); } From 1d54a7373cba2fb283d4fcabe9b580b77d6824ce Mon Sep 17 00:00:00 2001 From: David Arthur Date: Sun, 4 Aug 2024 15:13:26 -0400 Subject: [PATCH 105/123] KAFKA-17146 Include note to remove migration znode (#16770) When reverting the ZK migration, we must also remove the /migration ZNode in order to allow the migration to be re-attempted in the future. Reviewers: Colin P. McCabe , Chia-Ping Tsai --- docs/ops.html | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/ops.html b/docs/ops.html index 38515f8e8d..5c791dbce5 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4184,7 +4184,10 @@

    Reverting to ZooKeeper mode During the Migration

  • Using zookeeper-shell.sh, run rmr /controller so that one - of the brokers can become the new old-style controller. + of the brokers can become the new old-style controller. Additionally, run + get /migration followed by rmr /migration to clear the + migration state from ZooKeeper. This will allow you to re-attempt the migration + in the future. The data read from "/migration" can be useful for debugging.
  • On each broker, remove the zookeeper.metadata.migration.enable, From 0a70c3a61e7b1961e2d9a5faed801bde0493d0d9 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Mon, 7 Oct 2024 20:00:18 -0700 Subject: [PATCH 106/123] KAFKA-17714 Fix StorageToolTest.scala to compile under Scala 2.12 (#17400) Reviewers: David Arthur , Justine Olshan , Chia-Ping Tsai --- core/src/test/scala/unit/kafka/tools/StorageToolTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 83b21b43fb..7d8ac6860b 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -21,7 +21,7 @@ import java.io.{ByteArrayOutputStream, File, PrintStream} import java.nio.charset.StandardCharsets import java.nio.file.Files import java.util -import java.util.{Optional, Properties} +import java.util.Properties import kafka.server.KafkaConfig import kafka.utils.TestUtils import net.sourceforge.argparse4j.inf.ArgumentParserException @@ -39,7 +39,7 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource import scala.collection.mutable.ListBuffer -import scala.jdk.CollectionConverters.IterableHasAsScala +import scala.jdk.CollectionConverters._ @Timeout(value = 40) class StorageToolTest { @@ -454,7 +454,7 @@ Found problem: // Not doing full SCRAM record validation since that's covered elsewhere. // Just checking that we generate the correct number of records - val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString, Optional.empty).read + val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString, java.util.Optional.empty()).read val scramRecords = bootstrapMetadata.records().asScala .filter(apiMessageAndVersion => apiMessageAndVersion.message().isInstanceOf[UserScramCredentialRecord]) .map(apiMessageAndVersion => apiMessageAndVersion.message().asInstanceOf[UserScramCredentialRecord]) From 8af063a165e2549ea1c2165c90f75aaf80f3bccf Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 8 Oct 2024 15:40:27 -0700 Subject: [PATCH 107/123] KAFKA-17735: release.py must not use home.apache.org (#17421) Previously, Apache Kafka was uploading release candidate (RC) artifacts to users' home directories on home.apache.org. However, since this resource has been decommissioned, we need to follow the standard approach of putting release candidate artifacts into the appropriate subversion directory, at https://dist.apache.org/repos/dist/dev/kafka/. Reviewers: Justine Olshan --- release/README.md | 1 - release/release.py | 12 ++++++--- release/runtime.py | 4 +-- release/sftp.py | 59 ------------------------------------------- release/svn.py | 60 ++++++++++++++++++++++++++++++++++++++++++++ release/templates.py | 24 +++++++++--------- 6 files changed, 82 insertions(+), 78 deletions(-) delete mode 100644 release/sftp.py create mode 100644 release/svn.py diff --git a/release/README.md b/release/README.md index 0222164cd3..1f56f67901 100644 --- a/release/README.md +++ b/release/README.md @@ -8,7 +8,6 @@ This directory contains the tools used to publish a release. * python 3.12 * git * gpg 2.4 -* sftp The full instructions for producing a release are available in https://cwiki.apache.org/confluence/display/KAFKA/Release+Process. diff --git a/release/release.py b/release/release.py index 7710789bba..77d23e66ee 100644 --- a/release/release.py +++ b/release/release.py @@ -70,10 +70,12 @@ import gpg import notes import preferences -import sftp +import svn import templates import textfiles +from svn import SVN_DEV_URL + def get_jdk(version): """ @@ -243,7 +245,6 @@ def prereq(name, soft_check): except Exception as e: fail(f"Pre-requisite not met: {name}. Error: {e}") prereq('Apache Maven CLI (mvn) in PATH', lambda: "Apache Maven" in execute("mvn -v")) - prereq('Apache sftp connection', lambda: sftp.test(apache_id)) prereq("svn CLI in PATH", lambda: "svn" in execute("svn --version")) prereq("Verifying that you have no unstaged git changes", lambda: git.has_unstaged_changes()) prereq("Verifying that you have no staged git changes", lambda: git.has_staged_changes()) @@ -298,7 +299,7 @@ def delete_gitrefs(): git.create_tag(rc_tag) git.switch_branch(starting_branch) -# Note that we don't use tempfile here because mkdtemp causes problems with sftp and being able to determine the absolute path to a file. +# Note that we don't use tempfile here because mkdtemp causes problems with being able to determine the absolute path to a file. # Instead we rely on a fixed path work_dir = os.path.join(repo_dir, ".release_work_dir") clean_up_work_dir = lambda: cmd("Cleaning up work directory", f"rm -rf {work_dir}") @@ -315,6 +316,7 @@ def delete_gitrefs(): git.create_branch(release_version, rc_tag, cwd=kafka_dir) current_year = datetime.datetime.now().year cmd("Verifying the correct year in NOTICE", f"grep {current_year} NOTICE", cwd=kafka_dir) +svn.checkout_svn_dev(work_dir) print("Generating release notes") try: @@ -348,7 +350,9 @@ def delete_gitrefs(): cmd("Listing artifacts to be uploaded:", f"ls -R {artifacts_dir}") cmd("Zipping artifacts", f"tar -czf {artifact_name}.tar.gz {artifact_name}", cwd=work_dir) -sftp.upload_artifacts(apache_id, artifacts_dir) + +confirm_or_fail(f"Going to check in artifacts to svn under {SVN_DEV_URL}/{rc_tag}. OK?") +svn.commit_artifacts(rc_tag, artifacts_dir, work_dir) confirm_or_fail("Going to build and upload mvn artifacts based on these settings:\n" + textfiles.read(global_gradle_props) + '\nOK?') cmd("Building and uploading archives", "./gradlewAll publish", cwd=kafka_dir, env=jdk8_env, shell=True) diff --git a/release/runtime.py b/release/runtime.py index a4cfd11e66..c0d5d67eaf 100644 --- a/release/runtime.py +++ b/release/runtime.py @@ -133,12 +133,12 @@ def cmd(action, cmd_arg, *args, **kwargs): try: output = execute(cmd_arg, *args, stderr=subprocess.STDOUT, **kwargs) print(_prefix("> ", output.strip())) - return + return True except subprocess.CalledProcessError as e: print(e.output.decode("utf-8")) if allow_failure: - return + return False retry = confirm("Retry?") diff --git a/release/sftp.py b/release/sftp.py deleted file mode 100644 index f9ac042cf1..0000000000 --- a/release/sftp.py +++ /dev/null @@ -1,59 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -""" -Auxiliary functions to interact with sftp(1). -""" - -import subprocess - -from runtime import ( - cmd, - confirm_or_fail, - execute, - fail, -) - -REMOTE_DIR = "public_html" - - -def mkdirp(apache_id, dir): - cmd_desc = f"Creating '{dir}' in your Apache home directory" - cmd_str = f"sftp -b - {apache_id}@home.apache.org" - stdin_str = f"mkdir {dir}\n" - cmd(cmd_desc, cmd_str, stdin=stdin_str, allow_failure=True) - - -def upload(apache_id, destination, dir): - cmd_desc = f"Uploading '{dir}' under {REMOTE_DIR} in your Apache home directory, this may take a while." - cmd_str = f"sftp -b - {apache_id}@home.apache.org" - stdin_str = f"cd {destination}\nput -r {dir}\n" - cmd(cmd_desc, cmd_str, stdin=stdin_str) - - -def upload_artifacts(apache_id, dir): - mkdirp(apache_id, REMOTE_DIR) - upload(apache_id, REMOTE_DIR, dir) - confirm_or_fail(f"Are the artifacts present in your Apache home: https://home.apache.org/~{apache_id}/ ?") - - -def test(apache_id): - """ - Test the ability to estalish an sftp session. - """ - execute(f"sftp {apache_id}@home.apache.org", input="bye") - diff --git a/release/svn.py b/release/svn.py new file mode 100644 index 0000000000..4869f79ddf --- /dev/null +++ b/release/svn.py @@ -0,0 +1,60 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Auxiliary functions to interact with svn(1). +""" + +import os +import shutil +import subprocess + +from runtime import cmd + +SVN_DEV_URL="https://dist.apache.org/repos/dist/dev/kafka" + +def delete_old_rc_directory_if_needed(rc_tag, src, work_dir): + svn_dev = os.path.join(work_dir, "svn_dev") + cmd_desc = f"Check if {rc_tag} exists in the subversion repository." + cmd_str = f"svn info --show-item revision {SVN_DEV_URL}/{rc_tag}" + if not cmd(cmd_desc, cmd_str, cwd = svn_dev, allow_failure = True): + print(f"Nothing under {SVN_DEV_URL}/{rc_tag}. Continuing.") + return + cmd_desc = f"Committing the deletion of {SVN_DEV_URL}/{rc_tag} from the svn repo." + cmd_str = f"svn delete -m Remove_{rc_tag} {SVN_DEV_URL}/{rc_tag}" + cmd(cmd_desc, cmd_str, cwd = svn_dev) + +def commit_artifacts(rc_tag, src, work_dir): + delete_old_rc_directory_if_needed(rc_tag, src, work_dir) + svn_dev = os.path.join(work_dir, "svn_dev") + dst = os.path.join(svn_dev, rc_tag) + print(f"Copying {src} to {dst}") + shutil.copytree(src, dst) + cmd_desc = f"Adding {SVN_DEV_URL}/{rc_tag} to the svn repo." + cmd_str = f"svn add ./{rc_tag}" + cmd(cmd_desc, cmd_str, cwd = svn_dev) + cmd_desc = f"Committing the addition of {SVN_DEV_URL}/{rc_tag} to the svn repo. Please wait, this may take a while." + cmd_str = f"svn commit -m Add_{rc_tag}" + cmd(cmd_desc, cmd_str, cwd = svn_dev) + +def checkout_svn_dev(work_dir): + svn_dev = os.path.join(work_dir, "svn_dev") + if os.path.exists(svn_dev): + shutil.rmtree(svn_dev) + cmd_desc = f"Checking out {SVN_DEV_URL} at {svn_dev}" + cmd_str = f"svn checkout --depth empty {SVN_DEV_URL}/ {svn_dev}" + cmd(cmd_desc, cmd_str) diff --git a/release/templates.py b/release/templates.py index 5eb3fc74f9..89826e1097 100644 --- a/release/templates.py +++ b/release/templates.py @@ -167,23 +167,23 @@ def sanity_check_instructions(release_version, rc_tag, apache_id): Some suggested steps: - * Grab the source archive and make sure it compiles: https://home.apache.org/~{apache_id}/kafka-{rc_tag}/kafka-{release_version}-src.tgz - * Grab one of the binary distros and run the quickstarts against them: https://home.apache.org/~{apache_id}/kafka-{rc_tag}/kafka_2.13-{release_version}.tgz - * Extract and verify one of the site docs jars: https://home.apache.org/~{apache_id}/kafka-{rc_tag}/kafka_2.13-{release_version}-site-docs.tgz + * Grab the source archive and make sure it compiles: https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/kafka-{release_version}-src.tgz + * Grab one of the binary distros and run the quickstarts against them: https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/kafka_2.13-{release_version}.tgz + * Extract and verify one of the site docs jars: https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/kafka_2.13-{release_version}-site-docs.tgz * Build a sample against jars in the staging repo: (TODO: Can we get a temporary URL before "closing" the staged artifacts?) * Validate GPG signatures on at least one file: - wget https://home.apache.org/~{apache_id}/kafka-{rc_tag}/kafka-{release_version}-src.tgz && - wget https://home.apache.org/~{apache_id}/kafka-{rc_tag}/kafka-{release_version}-src.tgz.asc && - wget https://home.apache.org/~{apache_id}/kafka-{rc_tag}/kafka-{release_version}-src.tgz.md5 && - wget https://home.apache.org/~{apache_id}/kafka-{rc_tag}/kafka-{release_version}-src.tgz.sha1 && - wget https://home.apache.org/~{apache_id}/kafka-{rc_tag}/kafka-{release_version}-src.tgz.sha512 && + wget https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/kafka-{release_version}-src.tgz && + wget https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/kafka-{release_version}-src.tgz.asc && + wget https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/kafka-{release_version}-src.tgz.md5 && + wget https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/kafka-{release_version}-src.tgz.sha1 && + wget https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/kafka-{release_version}-src.tgz.sha512 && gpg --verify kafka-{release_version}-src.tgz.asc kafka-{release_version}-src.tgz && gpg --print-md md5 kafka-{release_version}-src.tgz | diff - kafka-{release_version}-src.tgz.md5 && gpg --print-md sha1 kafka-{release_version}-src.tgz | diff - kafka-{release_version}-src.tgz.sha1 && gpg --print-md sha512 kafka-{release_version}-src.tgz | diff - kafka-{release_version}-src.tgz.sha512 && rm kafka-{release_version}-src.tgz* && echo "OK" || echo "Failed" - * Validate the javadocs look ok. They are at https://home.apache.org/~{apache_id}/kafka-{rc_tag}/javadoc/ + * Validate the javadocs look ok. They are at https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/javadoc/ ******************************************************************************************************************************************************* """ @@ -201,7 +201,7 @@ def rc_vote_email_text(release_version, rc, rc_tag, dev_branch, docs_version, ap Release notes for the {release_version} release: -https://home.apache.org/~{apache_id}/kafka-{rc_tag}/RELEASE_NOTES.html +https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/RELEASE_NOTES.html *** Please download, test and vote by @@ -210,7 +210,7 @@ def rc_vote_email_text(release_version, rc, rc_tag, dev_branch, docs_version, ap https://kafka.apache.org/KEYS * Release artifacts to be voted upon (source and binary): -https://home.apache.org/~{apache_id}/kafka-{rc_tag}/ +https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/ * Docker release artifacts to be voted upon: @@ -221,7 +221,7 @@ def rc_vote_email_text(release_version, rc, rc_tag, dev_branch, docs_version, ap https://repository.apache.org/content/groups/staging/org/apache/kafka/ * Javadoc: -https://home.apache.org/~{apache_id}/kafka-{rc_tag}/javadoc/ +https://dist.apache.org/repos/dist/dev/kafka/{rc_tag}/javadoc/ * Tag to be voted upon (off {dev_branch} branch) is the {release_version} tag: https://github.com/apache/kafka/releases/tag/{rc_tag} From ab6dafaab63089808d8d746268e126039d7bd3c4 Mon Sep 17 00:00:00 2001 From: Gaurav Narula Date: Thu, 10 Oct 2024 03:13:01 +0530 Subject: [PATCH 108/123] KAFKA-17751; fix pollTimeout calculation in pollFollowerAsVoter (#17434) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit KAFKA-16534 introduced a change to send UpdateVoterRequest every "3 * fetchTimeoutMs" if the voter's configure endpoints are different from the endpoints persisted in the KRaft log. It also introduced a regression where if the voter nodes do not need an update then updateVoterTimer wasn't reset. This resulted in a busy-loop in KafkaRaftClient#poll method resulting in high CPU usage. This PR modifies the conditions in pollFollowerAsVoter to reset updateVoterTimer appropriately. Reviewers: José Armando García Sancio --- .../java/org/apache/kafka/raft/KafkaRaftClient.java | 12 +++++++----- .../kafka/raft/KafkaRaftClientReconfigTest.java | 7 +++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 4a708cb871..51aa5e59f2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -3036,11 +3036,13 @@ private long pollFollowerAsVoter(FollowerState state, long currentTimeMs) { logger.info("Become candidate due to fetch timeout"); transitionToCandidate(currentTimeMs); backoffMs = 0; - } else if (state.hasUpdateVoterPeriodExpired(currentTimeMs) && - partitionState.lastKraftVersion().isReconfigSupported() && - partitionState.lastVoterSet().voterNodeNeedsUpdate(quorum.localVoterNodeOrThrow()) - ) { - backoffMs = maybeSendUpdateVoterRequest(state, currentTimeMs); + } else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) { + if (partitionState.lastKraftVersion().isReconfigSupported() && + partitionState.lastVoterSet().voterNodeNeedsUpdate(quorum.localVoterNodeOrThrow())) { + backoffMs = maybeSendUpdateVoterRequest(state, currentTimeMs); + } else { + backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs); + } state.resetUpdateVoterPeriod(currentTimeMs); } else { backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 04ab47f34c..9e5d68d5e6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -54,6 +54,7 @@ import java.util.List; import java.util.Optional; import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Stream; @@ -61,6 +62,7 @@ import static org.apache.kafka.snapshot.Snapshots.BOOTSTRAP_SNAPSHOT_ID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; public class KafkaRaftClientReconfigTest { @@ -2163,6 +2165,11 @@ void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + // after more than 3 fetch timeouts the update voter period timer should have expired. + // check that the update voter period timer doesn't remain at zero (0) and cause the message queue to get + // called with a zero (0) timeout and result in a busy-loop. + assertNotEquals(OptionalLong.of(0L), context.messageQueue.lastPollTimeoutMs()); } @Test From bf95a3239c6cbd5534ff3338c7b16bbb56a91d55 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Wed, 9 Oct 2024 16:34:26 -0700 Subject: [PATCH 109/123] KAFKA-17753: Update protobuf and commons-io dependencies (#17436) Reviewers: Josep Prat --- LICENSE-binary | 4 ++-- build.gradle | 2 ++ gradle/dependencies.gradle | 4 ++++ 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 817bc3d6e5..0e19528b0f 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -211,7 +211,7 @@ commons-beanutils-1.9.4 commons-cli-1.4 commons-collections-3.2.2 commons-digester-2.1 -commons-io-2.11.0 +commons-io-2.14.0 commons-lang3-3.12.0 commons-logging-1.2 commons-validator-1.7 @@ -335,7 +335,7 @@ BSD 3-Clause jline-3.25.1, see: licenses/jline-BSD-3-clause jsr305-3.0.2, see: licenses/jsr305-BSD-3-clause paranamer-2.8, see: licenses/paranamer-BSD-3-clause -protobuf-java-3.23.4, see: licenses/protobuf-java-BSD-3-clause +protobuf-java-3.25.5, see: licenses/protobuf-java-BSD-3-clause --------------------------------------- Do What The F*ck You Want To Public License diff --git a/build.gradle b/build.gradle index f73255bd83..5b064f4203 100644 --- a/build.gradle +++ b/build.gradle @@ -952,6 +952,7 @@ project(':core') { implementation libs.scalaReflect implementation libs.scalaLogging implementation libs.slf4jApi + implementation libs.commonsIo // ZooKeeper dependency. Do not use, this is going away. implementation(libs.zookeeper) { // Dropwizard Metrics are required by ZooKeeper as of v3.6.0, // but the library should *not* be used in Kafka code @@ -1535,6 +1536,7 @@ project(':clients') { implementation libs.snappy implementation libs.slf4jApi implementation libs.opentelemetryProto + implementation libs.protobuf // libraries which should be added as runtime dependencies in generated pom.xml should be defined here: shadowed libs.zstd diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 8ca49dd674..1b7e354b81 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -90,6 +90,7 @@ versions += [ // gradle/resources/dependencycheck-suppressions.xml checkstyle: "8.36.2", commonsCli: "1.4", + commonsIo: "2.14.0", // ZooKeeper dependency. Do not use, this is going away. commonsValidator: "1.7", dropwizardMetrics: "4.1.12.1", gradle: "8.8", @@ -142,6 +143,7 @@ versions += [ metrics: "2.2.0", netty: "4.1.111.Final", opentelemetryProto: "1.0.0-alpha", + protobuf: "3.25.5", // a dependency of opentelemetryProto pcollections: "4.0.1", reflections: "0.10.2", reload4j: "1.2.25", @@ -179,6 +181,7 @@ libs += [ bcpkix: "org.bouncycastle:bcpkix-jdk18on:$versions.bcpkix", caffeine: "com.github.ben-manes.caffeine:caffeine:$versions.caffeine", commonsCli: "commons-cli:commons-cli:$versions.commonsCli", + commonsIo: "commons-io:commons-io:$versions.commonsIo", commonsValidator: "commons-validator:commons-validator:$versions.commonsValidator", jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson", jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson", @@ -242,6 +245,7 @@ libs += [ nettyTransportNativeEpoll: "io.netty:netty-transport-native-epoll:$versions.netty", pcollections: "org.pcollections:pcollections:$versions.pcollections", opentelemetryProto: "io.opentelemetry.proto:opentelemetry-proto:$versions.opentelemetryProto", + protobuf: "com.google.protobuf:protobuf-java:$versions.protobuf", reflections: "org.reflections:reflections:$versions.reflections", reload4j: "ch.qos.reload4j:reload4j:$versions.reload4j", rocksDBJni: "org.rocksdb:rocksdbjni:$versions.rocksDB", From db4c80a4559ed1f9ed197c237044ef8b32384316 Mon Sep 17 00:00:00 2001 From: Apoorv Mittal Date: Thu, 10 Oct 2024 12:19:05 +0100 Subject: [PATCH 110/123] KAFKA-17731: Removed timed waiting signal for client telemetry close (#17431) Reviewers: Andrew Schofield , Kirk True , Chia-Ping Tsai , Lianet Magrans --- .../internals/AsyncKafkaConsumer.java | 2 +- .../internals/ClassicKafkaConsumer.java | 2 +- .../consumer/internals/ShareConsumerImpl.java | 2 +- .../kafka/clients/producer/KafkaProducer.java | 4 +- .../internals/ClientTelemetryReporter.java | 24 +++-------- .../internals/ClientTelemetrySender.java | 4 +- .../ClientTelemetryReporterTest.java | 42 +++++++++++++++++++ 7 files changed, 55 insertions(+), 25 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index cca3fbc44b..5255ab9f2c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1233,7 +1233,7 @@ private void close(Duration timeout, boolean swallowException) { wakeupTrigger.disableWakeups(); final Timer closeTimer = time.timer(timeout); - clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(timeout.toMillis())); + clientTelemetryReporter.ifPresent(ClientTelemetryReporter::initiateClose); closeTimer.update(); // Prepare shutting down the network thread swallow(log, Level.ERROR, "Failed to release assignment before closing consumer", diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java index b9e73f963a..6ec6ba2cc9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ClassicKafkaConsumer.java @@ -1130,7 +1130,7 @@ private void close(Duration timeout, boolean swallowException) { AtomicReference firstException = new AtomicReference<>(); final Timer closeTimer = createTimerForRequest(timeout); - clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(timeout.toMillis())); + clientTelemetryReporter.ifPresent(ClientTelemetryReporter::initiateClose); closeTimer.update(); // Close objects with a timeout. The timeout is required because the coordinator & the fetcher send requests to // the server in the process of closing which may not respect the overall timeout defined for closing the diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 9977d61fe1..34c5304409 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -828,7 +828,7 @@ private void close(final Duration timeout, final boolean swallowException) { wakeupTrigger.disableWakeups(); final Timer closeTimer = time.timer(timeout); - clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(timeout.toMillis())); + clientTelemetryReporter.ifPresent(ClientTelemetryReporter::initiateClose); closeTimer.update(); // Prepare shutting down the network thread diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index e71f1d5755..ec923101ed 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1392,6 +1392,9 @@ private void close(Duration timeout, boolean swallowException) { } else { // Try to close gracefully. final Timer closeTimer = time.timer(timeout); + clientTelemetryReporter.ifPresent(ClientTelemetryReporter::initiateClose); + closeTimer.update(); + if (this.sender != null) { this.sender.initiateClose(); closeTimer.update(); @@ -1406,7 +1409,6 @@ private void close(Duration timeout, boolean swallowException) { closeTimer.update(); } } - clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(closeTimer.remainingMs())); } } diff --git a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java index 1fbf32011e..78a3900d63 100644 --- a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporter.java @@ -218,10 +218,10 @@ public synchronized void updateMetricsLabels(Map labels) { telemetryProvider.updateLabels(labels); } - public void initiateClose(long timeoutMs) { + public void initiateClose() { log.debug("Initiate close of ClientTelemetryReporter"); try { - clientTelemetrySender.initiateClose(timeoutMs); + clientTelemetrySender.initiateClose(); } catch (Exception exception) { log.error("Failed to initiate close of client telemetry reporter", exception); } @@ -271,7 +271,6 @@ class DefaultClientTelemetrySender implements ClientTelemetrySender { private final ReadWriteLock lock = new ReentrantReadWriteLock(); private final Condition subscriptionLoaded = lock.writeLock().newCondition(); - private final Condition terminalPushInProgress = lock.writeLock().newCondition(); /* Initial state should be subscription needed which should allow issuing first telemetry request of get telemetry subscription. @@ -602,8 +601,8 @@ public void close() { } @Override - public void initiateClose(long timeoutMs) { - log.debug("initiate close for client telemetry, check if terminal push required. Timeout {} ms.", timeoutMs); + public void initiateClose() { + log.debug("initiate close for client telemetry, check if terminal push required."); lock.writeLock().lock(); try { @@ -623,14 +622,7 @@ public void initiateClose(long timeoutMs) { return; } - try { - log.info("About to wait {} ms. for terminal telemetry push to be submitted", timeoutMs); - if (!terminalPushInProgress.await(timeoutMs, TimeUnit.MILLISECONDS)) { - log.info("Wait for terminal telemetry push to be submitted has elapsed; may not have actually sent request"); - } - } catch (InterruptedException e) { - log.warn("Error during client telemetry close", e); - } + log.debug("Updated state to send terminal telemetry push request"); } finally { lock.writeLock().unlock(); } @@ -827,13 +819,9 @@ boolean maybeSetState(ClientTelemetryState newState) { ClientTelemetryState oldState = state; state = oldState.validateTransition(newState); log.debug("Setting telemetry state from {} to {}", oldState, newState); - - if (newState == ClientTelemetryState.TERMINATING_PUSH_IN_PROGRESS) { - terminalPushInProgress.signalAll(); - } return true; } catch (IllegalStateException e) { - log.warn("Error updating client telemetry state, disabled telemetry", e); + log.warn("Error updating client telemetry state, disabled telemetry"); enabled = false; return false; } finally { diff --git a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetrySender.java b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetrySender.java index 17a6da664f..d7fd81364f 100644 --- a/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetrySender.java +++ b/clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetrySender.java @@ -107,8 +107,6 @@ public interface ClientTelemetrySender extends AutoCloseable { /** * Initiates shutdown of this client. This method is called when the enclosing client instance * is being closed. This method should not throw an exception if the client is already closed. - * - * @param timeoutMs The maximum time to wait for the client to close. */ - void initiateClose(long timeoutMs); + void initiateClose(); } diff --git a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java index 7dba97c238..066e9ff74d 100644 --- a/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryReporterTest.java @@ -729,6 +729,48 @@ public void testComputeStaggeredIntervalMs() { assertTrue(timeMs >= 500 && timeMs <= 1500); } + @Test + public void testTelemetryReporterInitiateClose() { + ClientTelemetryReporter.DefaultClientTelemetrySender telemetrySender = (ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter.telemetrySender(); + telemetrySender.updateSubscriptionResult(subscription, time.milliseconds()); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.SUBSCRIPTION_IN_PROGRESS)); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.PUSH_NEEDED)); + + clientTelemetryReporter.initiateClose(); + assertEquals(ClientTelemetryState.TERMINATING_PUSH_NEEDED, ((ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter + .telemetrySender()).state()); + } + + @Test + public void testTelemetryReporterInitiateCloseNoSubscription() { + clientTelemetryReporter.initiateClose(); + assertEquals(ClientTelemetryState.SUBSCRIPTION_NEEDED, ((ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter + .telemetrySender()).state()); + } + + @Test + public void testTelemetryReporterInitiateCloseAlreadyInTerminatedStates() { + ClientTelemetryReporter.DefaultClientTelemetrySender telemetrySender = (ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter.telemetrySender(); + telemetrySender.updateSubscriptionResult(subscription, time.milliseconds()); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.SUBSCRIPTION_IN_PROGRESS)); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.PUSH_NEEDED)); + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.TERMINATING_PUSH_NEEDED)); + + clientTelemetryReporter.initiateClose(); + assertEquals(ClientTelemetryState.TERMINATING_PUSH_NEEDED, ((ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter + .telemetrySender()).state()); + + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.TERMINATING_PUSH_IN_PROGRESS)); + clientTelemetryReporter.initiateClose(); + assertEquals(ClientTelemetryState.TERMINATING_PUSH_IN_PROGRESS, ((ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter + .telemetrySender()).state()); + + assertTrue(telemetrySender.maybeSetState(ClientTelemetryState.TERMINATED)); + clientTelemetryReporter.initiateClose(); + assertEquals(ClientTelemetryState.TERMINATED, ((ClientTelemetryReporter.DefaultClientTelemetrySender) clientTelemetryReporter + .telemetrySender()).state()); + } + @AfterEach public void tearDown() { clientTelemetryReporter.close(); From 4878174b7754e4e48519463065ea389042470bf7 Mon Sep 17 00:00:00 2001 From: PoAn Yang Date: Wed, 31 Jul 2024 03:07:09 +0800 Subject: [PATCH 111/123] KAFKA-16972 Move BrokerTopicMetrics to org.apache.kafka.storage.log.metrics (#16387) Reviewers: Chia-Ping Tsai --- checkstyle/import-control-storage.xml | 5 + .../src/main/scala/kafka/log/UnifiedLog.scala | 3 +- .../kafka/server/KafkaRequestHandler.scala | 310 +------------- .../server/KafkaRequestHandlerTest.scala | 19 +- .../unit/kafka/log/LogValidatorTest.scala | 13 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 5 +- .../unit/kafka/metrics/MetricsTest.scala | 11 +- .../server/AbstractFetcherThreadTest.scala | 2 +- ...FetchRequestDownConversionConfigTest.scala | 7 +- .../kafka/server/ProduceRequestTest.scala | 5 +- .../server/metrics/KafkaMetricsGroup.java | 19 +- .../server/metrics/KafkaMetricsGroupTest.java | 37 ++ .../log/metrics/BrokerTopicMetrics.java | 405 ++++++++++++++++++ 13 files changed, 515 insertions(+), 326 deletions(-) create mode 100644 server-common/src/test/java/org/apache/kafka/server/metrics/KafkaMetricsGroupTest.java create mode 100644 storage/src/main/java/org/apache/kafka/storage/log/metrics/BrokerTopicMetrics.java diff --git a/checkstyle/import-control-storage.xml b/checkstyle/import-control-storage.xml index 445bed2016..18fd7938ad 100644 --- a/checkstyle/import-control-storage.xml +++ b/checkstyle/import-control-storage.xml @@ -83,6 +83,11 @@ + + + + + diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 450f18b4ef..fcbf836606 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -21,7 +21,7 @@ import com.yammer.metrics.core.MetricName import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.LocalLog.nextOption import kafka.log.remote.RemoteLogManager -import kafka.server.{BrokerTopicMetrics, BrokerTopicStats, RequestLocal} +import kafka.server.{BrokerTopicStats, RequestLocal} import kafka.utils._ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic @@ -42,6 +42,7 @@ import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.checkpoint.{LeaderEpochCheckpointFile, PartitionMetadataFile} import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, BatchMetadata, CompletedTxn, FetchDataInfo, FetchIsolation, LastRecord, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogValidator, ProducerAppendInfo, ProducerStateManager, ProducerStateManagerConfig, RollParams, VerificationGuard} +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics import java.io.{File, IOException} import java.nio.file.{Files, Path} diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 209db53171..6ef9f93180 100755 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -21,17 +21,16 @@ import kafka.network.RequestChannel import kafka.utils.{Exit, Logging, Pool} import kafka.server.KafkaRequestHandler.{threadCurrentRequest, threadRequestChannel} -import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit} +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicInteger import com.yammer.metrics.core.Meter import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.utils.{KafkaThread, Time} import org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics import org.apache.kafka.server.metrics.KafkaMetricsGroup +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics -import java.util.Collections import scala.collection.mutable -import scala.jdk.CollectionConverters._ trait ApiRequestHandler { def handle(request: RequestChannel.Request, requestLocal: RequestLocal): Unit @@ -246,284 +245,11 @@ class KafkaRequestHandlerPool( } } -class BrokerTopicMetrics(name: Option[String], remoteStorageEnabled: Boolean = false) { - private val metricsGroup = new KafkaMetricsGroup(this.getClass) - - val tags: java.util.Map[String, String] = name match { - case None => Collections.emptyMap() - case Some(topic) => Map("topic" -> topic).asJava - } - - case class MeterWrapper(metricType: String, eventType: String) { - @volatile private var lazyMeter: Meter = _ - private val meterLock = new Object - - def meter(): Meter = { - var meter = lazyMeter - if (meter == null) { - meterLock synchronized { - meter = lazyMeter - if (meter == null) { - meter = metricsGroup.newMeter(metricType, eventType, TimeUnit.SECONDS, tags) - lazyMeter = meter - } - } - } - meter - } - - def close(): Unit = meterLock synchronized { - if (lazyMeter != null) { - metricsGroup.removeMetric(metricType, tags) - lazyMeter = null - } - } - - if (tags.isEmpty) // greedily initialize the general topic metrics - meter() - } - - case class GaugeWrapper(metricType: String) { - // The map to store: - // - per-partition value for topic-level metrics. The key will be the partition number - // - per-topic value for broker-level metrics. The key will be the topic name - private val metricValues = new ConcurrentHashMap[String, Long]() - - def setValue(key: String, value: Long): Unit = { - newGaugeIfNeed() - metricValues.put(key, value) - } - - def removeKey(key: String): Unit = { - newGaugeIfNeed() - metricValues.remove(key) - } - - // metricsGroup uses ConcurrentMap to store gauges, so we don't need to use synchronized block here - def close(): Unit = { - metricsGroup.removeMetric(metricType, tags) - metricValues.clear() - } - - def value(): Long = metricValues.values().stream().mapToLong(v => v).sum() - - // metricsGroup uses ConcurrentMap to store gauges, so we don't need to use synchronized block here - private def newGaugeIfNeed(): Unit = { - metricsGroup.newGauge(metricType, () => value(), tags) - } - - newGaugeIfNeed() - } - - // an internal map for "lazy initialization" of certain metrics - private val metricTypeMap = new Pool[String, MeterWrapper]() - private val metricGaugeTypeMap = new Pool[String, GaugeWrapper]() - metricTypeMap.putAll(Map( - BrokerTopicStats.MessagesInPerSec -> MeterWrapper(BrokerTopicStats.MessagesInPerSec, "messages"), - BrokerTopicStats.BytesInPerSec -> MeterWrapper(BrokerTopicStats.BytesInPerSec, "bytes"), - BrokerTopicStats.BytesOutPerSec -> MeterWrapper(BrokerTopicStats.BytesOutPerSec, "bytes"), - BrokerTopicStats.BytesRejectedPerSec -> MeterWrapper(BrokerTopicStats.BytesRejectedPerSec, "bytes"), - BrokerTopicStats.FailedProduceRequestsPerSec -> MeterWrapper(BrokerTopicStats.FailedProduceRequestsPerSec, "requests"), - BrokerTopicStats.FailedFetchRequestsPerSec -> MeterWrapper(BrokerTopicStats.FailedFetchRequestsPerSec, "requests"), - BrokerTopicStats.TotalProduceRequestsPerSec -> MeterWrapper(BrokerTopicStats.TotalProduceRequestsPerSec, "requests"), - BrokerTopicStats.TotalFetchRequestsPerSec -> MeterWrapper(BrokerTopicStats.TotalFetchRequestsPerSec, "requests"), - BrokerTopicStats.FetchMessageConversionsPerSec -> MeterWrapper(BrokerTopicStats.FetchMessageConversionsPerSec, "requests"), - BrokerTopicStats.ProduceMessageConversionsPerSec -> MeterWrapper(BrokerTopicStats.ProduceMessageConversionsPerSec, "requests"), - BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec -> MeterWrapper(BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec, "requests"), - BrokerTopicStats.InvalidMagicNumberRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidMagicNumberRecordsPerSec, "requests"), - BrokerTopicStats.InvalidMessageCrcRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidMessageCrcRecordsPerSec, "requests"), - BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec, "requests") - ).asJava) - - if (name.isEmpty) { - metricTypeMap.put(BrokerTopicStats.ReplicationBytesInPerSec, MeterWrapper(BrokerTopicStats.ReplicationBytesInPerSec, "bytes")) - metricTypeMap.put(BrokerTopicStats.ReplicationBytesOutPerSec, MeterWrapper(BrokerTopicStats.ReplicationBytesOutPerSec, "bytes")) - metricTypeMap.put(BrokerTopicStats.ReassignmentBytesInPerSec, MeterWrapper(BrokerTopicStats.ReassignmentBytesInPerSec, "bytes")) - metricTypeMap.put(BrokerTopicStats.ReassignmentBytesOutPerSec, MeterWrapper(BrokerTopicStats.ReassignmentBytesOutPerSec, "bytes")) - } - - if (remoteStorageEnabled) { - metricTypeMap.putAll(Map( - RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName, "bytes"), - RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName, "bytes"), - RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName, "requests"), - RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName, "requests") - ).asJava) - - metricGaugeTypeMap.putAll(Map( - RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName), - RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName), - RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName), - RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName), - RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName), - RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName), - RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName -> GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName) - ).asJava) - } - - // used for testing only - def metricMap: Map[String, MeterWrapper] = metricTypeMap.toMap - - def metricGaugeMap: Map[String, GaugeWrapper] = metricGaugeTypeMap.toMap - - def messagesInRate: Meter = metricTypeMap.get(BrokerTopicStats.MessagesInPerSec).meter() - - def bytesInRate: Meter = metricTypeMap.get(BrokerTopicStats.BytesInPerSec).meter() - - def bytesOutRate: Meter = metricTypeMap.get(BrokerTopicStats.BytesOutPerSec).meter() - - def bytesRejectedRate: Meter = metricTypeMap.get(BrokerTopicStats.BytesRejectedPerSec).meter() - - private[server] def replicationBytesInRate: Option[Meter] = - if (name.isEmpty) Some(metricTypeMap.get(BrokerTopicStats.ReplicationBytesInPerSec).meter()) - else None - - private[server] def replicationBytesOutRate: Option[Meter] = - if (name.isEmpty) Some(metricTypeMap.get(BrokerTopicStats.ReplicationBytesOutPerSec).meter()) - else None - - private[server] def reassignmentBytesInPerSec: Option[Meter] = - if (name.isEmpty) Some(metricTypeMap.get(BrokerTopicStats.ReassignmentBytesInPerSec).meter()) - else None - - private[server] def reassignmentBytesOutPerSec: Option[Meter] = - if (name.isEmpty) Some(metricTypeMap.get(BrokerTopicStats.ReassignmentBytesOutPerSec).meter()) - else None - - def failedProduceRequestRate: Meter = metricTypeMap.get(BrokerTopicStats.FailedProduceRequestsPerSec).meter() - - def failedFetchRequestRate: Meter = metricTypeMap.get(BrokerTopicStats.FailedFetchRequestsPerSec).meter() - - def totalProduceRequestRate: Meter = metricTypeMap.get(BrokerTopicStats.TotalProduceRequestsPerSec).meter() - - def totalFetchRequestRate: Meter = metricTypeMap.get(BrokerTopicStats.TotalFetchRequestsPerSec).meter() - - def fetchMessageConversionsRate: Meter = metricTypeMap.get(BrokerTopicStats.FetchMessageConversionsPerSec).meter() - - def produceMessageConversionsRate: Meter = metricTypeMap.get(BrokerTopicStats.ProduceMessageConversionsPerSec).meter() - - def noKeyCompactedTopicRecordsPerSec: Meter = metricTypeMap.get(BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec).meter() - - def invalidMagicNumberRecordsPerSec: Meter = metricTypeMap.get(BrokerTopicStats.InvalidMagicNumberRecordsPerSec).meter() - - def invalidMessageCrcRecordsPerSec: Meter = metricTypeMap.get(BrokerTopicStats.InvalidMessageCrcRecordsPerSec).meter() - - def invalidOffsetOrSequenceRecordsPerSec: Meter = metricTypeMap.get(BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec).meter() - - def remoteCopyLagBytesAggrMetric(): GaugeWrapper = { - metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName) - } - - // Visible for testing - def remoteCopyLagBytes: Long = remoteCopyLagBytesAggrMetric().value() - - def remoteCopyLagSegmentsAggrMetric(): GaugeWrapper = { - metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName) - } - - // Visible for testing - def remoteCopyLagSegments: Long = remoteCopyLagSegmentsAggrMetric().value() - - def remoteLogMetadataCountAggrMetric(): GaugeWrapper = { - metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName) - } - - def remoteLogMetadataCount: Long = remoteLogMetadataCountAggrMetric().value() - - def remoteLogSizeBytesAggrMetric(): GaugeWrapper = { - metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName) - } - - def remoteLogSizeBytes: Long = remoteLogSizeBytesAggrMetric().value() - - def remoteLogSizeComputationTimeAggrMetric(): GaugeWrapper = { - metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName) - } - - def remoteLogSizeComputationTime: Long = remoteLogSizeComputationTimeAggrMetric().value() - - def remoteDeleteLagBytesAggrMetric(): GaugeWrapper = { - metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName) - } - - // Visible for testing - def remoteDeleteLagBytes: Long = remoteDeleteLagBytesAggrMetric().value() - - def remoteDeleteLagSegmentsAggrMetric(): GaugeWrapper = { - metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName) - } - - // Visible for testing - def remoteDeleteLagSegments: Long = remoteDeleteLagSegmentsAggrMetric().value() - - def remoteCopyBytesRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName).meter() - - def remoteFetchBytesRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName).meter() - - def remoteFetchRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName).meter() - - def remoteCopyRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName).meter() - - def remoteDeleteRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName).meter() - - def buildRemoteLogAuxStateRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName).meter() - - def failedRemoteFetchRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName).meter() - - def failedRemoteCopyRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName).meter() - - def failedRemoteDeleteRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName).meter() - - def failedBuildRemoteLogAuxStateRate: Meter = metricTypeMap.get(RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName).meter() - - def closeMetric(metricType: String): Unit = { - val meter = metricTypeMap.get(metricType) - if (meter != null) - meter.close() - val gauge = metricGaugeTypeMap.get(metricType) - if (gauge != null) - gauge.close() - } - - def close(): Unit = { - metricTypeMap.values.foreach(_.close()) - metricGaugeTypeMap.values.foreach(_.close()) - } -} - -object BrokerTopicStats { - val MessagesInPerSec = "MessagesInPerSec" - val BytesInPerSec = "BytesInPerSec" - val BytesOutPerSec = "BytesOutPerSec" - val BytesRejectedPerSec = "BytesRejectedPerSec" - val ReplicationBytesInPerSec = "ReplicationBytesInPerSec" - val ReplicationBytesOutPerSec = "ReplicationBytesOutPerSec" - val FailedProduceRequestsPerSec = "FailedProduceRequestsPerSec" - val FailedFetchRequestsPerSec = "FailedFetchRequestsPerSec" - val TotalProduceRequestsPerSec = "TotalProduceRequestsPerSec" - val TotalFetchRequestsPerSec = "TotalFetchRequestsPerSec" - val FetchMessageConversionsPerSec = "FetchMessageConversionsPerSec" - val ProduceMessageConversionsPerSec = "ProduceMessageConversionsPerSec" - val ReassignmentBytesInPerSec = "ReassignmentBytesInPerSec" - val ReassignmentBytesOutPerSec = "ReassignmentBytesOutPerSec" - // These following topics are for LogValidator for better debugging on failed records - val NoKeyCompactedTopicRecordsPerSec = "NoKeyCompactedTopicRecordsPerSec" - val InvalidMagicNumberRecordsPerSec = "InvalidMagicNumberRecordsPerSec" - val InvalidMessageCrcRecordsPerSec = "InvalidMessageCrcRecordsPerSec" - val InvalidOffsetOrSequenceRecordsPerSec = "InvalidOffsetOrSequenceRecordsPerSec" -} - class BrokerTopicStats(remoteStorageEnabled: Boolean = false) extends Logging { - private val valueFactory = (k: String) => new BrokerTopicMetrics(Some(k), remoteStorageEnabled) + private val valueFactory = (k: String) => new BrokerTopicMetrics(k, remoteStorageEnabled) private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory)) - val allTopicsStats = new BrokerTopicMetrics(None, remoteStorageEnabled) + val allTopicsStats = new BrokerTopicMetrics(remoteStorageEnabled) def isTopicStatsExisted(topic: String): Boolean = stats.contains(topic) @@ -532,25 +258,25 @@ class BrokerTopicStats(remoteStorageEnabled: Boolean = false) extends Logging { stats.getAndMaybePut(topic) def updateReplicationBytesIn(value: Long): Unit = { - allTopicsStats.replicationBytesInRate.foreach { metric => + allTopicsStats.replicationBytesInRate.ifPresent { metric => metric.mark(value) } } private def updateReplicationBytesOut(value: Long): Unit = { - allTopicsStats.replicationBytesOutRate.foreach { metric => + allTopicsStats.replicationBytesOutRate.ifPresent { metric => metric.mark(value) } } def updateReassignmentBytesIn(value: Long): Unit = { - allTopicsStats.reassignmentBytesInPerSec.foreach { metric => + allTopicsStats.reassignmentBytesInPerSec.ifPresent { metric => metric.mark(value) } } private def updateReassignmentBytesOut(value: Long): Unit = { - allTopicsStats.reassignmentBytesOutPerSec.foreach { metric => + allTopicsStats.reassignmentBytesOutPerSec.ifPresent { metric => metric.mark(value) } } @@ -559,14 +285,14 @@ class BrokerTopicStats(remoteStorageEnabled: Boolean = false) extends Logging { def removeOldLeaderMetrics(topic: String): Unit = { val topicMetrics = topicStats(topic) if (topicMetrics != null) { - topicMetrics.closeMetric(BrokerTopicStats.MessagesInPerSec) - topicMetrics.closeMetric(BrokerTopicStats.BytesInPerSec) - topicMetrics.closeMetric(BrokerTopicStats.BytesRejectedPerSec) - topicMetrics.closeMetric(BrokerTopicStats.FailedProduceRequestsPerSec) - topicMetrics.closeMetric(BrokerTopicStats.TotalProduceRequestsPerSec) - topicMetrics.closeMetric(BrokerTopicStats.ProduceMessageConversionsPerSec) - topicMetrics.closeMetric(BrokerTopicStats.ReplicationBytesOutPerSec) - topicMetrics.closeMetric(BrokerTopicStats.ReassignmentBytesOutPerSec) + topicMetrics.closeMetric(BrokerTopicMetrics.MESSAGE_IN_PER_SEC) + topicMetrics.closeMetric(BrokerTopicMetrics.BYTES_IN_PER_SEC) + topicMetrics.closeMetric(BrokerTopicMetrics.BYTES_REJECTED_PER_SEC) + topicMetrics.closeMetric(BrokerTopicMetrics.FAILED_PRODUCE_REQUESTS_PER_SEC) + topicMetrics.closeMetric(BrokerTopicMetrics.TOTAL_PRODUCE_REQUESTS_PER_SEC) + topicMetrics.closeMetric(BrokerTopicMetrics.PRODUCE_MESSAGE_CONVERSIONS_PER_SEC) + topicMetrics.closeMetric(BrokerTopicMetrics.REPLICATION_BYTES_OUT_PER_SEC) + topicMetrics.closeMetric(BrokerTopicMetrics.REASSIGNMENT_BYTES_OUT_PER_SEC) topicMetrics.closeMetric(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName) topicMetrics.closeMetric(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName) topicMetrics.closeMetric(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName) @@ -592,8 +318,8 @@ class BrokerTopicStats(remoteStorageEnabled: Boolean = false) extends Logging { def removeOldFollowerMetrics(topic: String): Unit = { val topicMetrics = topicStats(topic) if (topicMetrics != null) { - topicMetrics.closeMetric(BrokerTopicStats.ReplicationBytesInPerSec) - topicMetrics.closeMetric(BrokerTopicStats.ReassignmentBytesInPerSec) + topicMetrics.closeMetric(BrokerTopicMetrics.REPLICATION_BYTES_IN_PER_SEC) + topicMetrics.closeMetric(BrokerTopicMetrics.REASSIGNMENT_BYTES_IN_PER_SEC) } } diff --git a/core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala b/core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala index a71e8cc6a0..adcd8a13ef 100644 --- a/core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala +++ b/core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.{BufferSupplier, MockTime, Time} import org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest @@ -210,19 +211,19 @@ class KafkaRequestHandlerTest { RemoteStorageMetrics.brokerTopicStatsMetrics.forEach(metric => { if (systemRemoteStorageEnabled) { if (!gaugeMetrics.contains(metric.getName)) { - assertTrue(brokerTopicMetrics.metricMap.contains(metric.getName), "the metric is missing: " + metric.getName) + assertTrue(brokerTopicMetrics.metricMapKeySet().contains(metric.getName), "the metric is missing: " + metric.getName) } else { - assertFalse(brokerTopicMetrics.metricMap.contains(metric.getName), "the metric should not appear: " + metric.getName) + assertFalse(brokerTopicMetrics.metricMapKeySet().contains(metric.getName), "the metric should not appear: " + metric.getName) } } else { - assertFalse(brokerTopicMetrics.metricMap.contains(metric.getName)) + assertFalse(brokerTopicMetrics.metricMapKeySet().contains(metric.getName)) } }) gaugeMetrics.foreach(metricName => { if (systemRemoteStorageEnabled) { - assertTrue(brokerTopicMetrics.metricGaugeMap.contains(metricName), "The metric is missing:" + metricName) + assertTrue(brokerTopicMetrics.metricGaugeMap.containsKey(metricName), "The metric is missing:" + metricName) } else { - assertFalse(brokerTopicMetrics.metricGaugeMap.contains(metricName), "The metric should appear:" + metricName) + assertFalse(brokerTopicMetrics.metricGaugeMap.containsKey(metricName), "The metric should appear:" + metricName) } }) } @@ -241,7 +242,7 @@ class KafkaRequestHandlerTest { def setupBrokerTopicMetrics(systemRemoteStorageEnabled: Boolean = true): BrokerTopicMetrics = { val topic = "topic" - new BrokerTopicMetrics(Option.apply(topic), systemRemoteStorageEnabled) + new BrokerTopicMetrics(topic, systemRemoteStorageEnabled) } @ParameterizedTest @@ -259,8 +260,8 @@ class KafkaRequestHandlerTest { brokerTopicStats.recordRemoteCopyLagBytes(topic2, 0, 100) assertEquals(600, brokerTopicStats.allTopicsStats.remoteCopyLagBytes) } else { - assertEquals(None, brokerTopicMetrics.metricGaugeMap.get(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName)) - assertEquals(None, brokerTopicStats.allTopicsStats.metricGaugeMap.get(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName)) + assertEquals(null, brokerTopicMetrics.metricGaugeMap.get(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName)) + assertEquals(null, brokerTopicStats.allTopicsStats.metricGaugeMap.get(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName)) } } @@ -599,7 +600,7 @@ class KafkaRequestHandlerTest { brokerTopicStats.recordRemoteLogSizeBytes(topic2, 0, 100) assertEquals(600, brokerTopicStats.allTopicsStats.remoteLogSizeBytes) } else { - assertEquals(None, brokerTopicMetrics.metricGaugeMap.get(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName)) + assertEquals(null, brokerTopicMetrics.metricGaugeMap.get(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName)) } } diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala index dceee266f6..ea2349059b 100644 --- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala @@ -30,6 +30,7 @@ import org.apache.kafka.storage.internals.log.LogValidator.ValidationResult import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AppendOrigin, LogValidator, RecordValidationException} +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics import org.apache.kafka.test.TestUtils import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -110,8 +111,8 @@ class LogValidatorTest { assertThrows(classOf[RecordValidationException], () => validateMessages(recordsWithInvalidInnerMagic(batchMagic, recordMagic, compression), batchMagic, compression.`type`(), compression) ) - assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}")), 1) - assertTrue(meterCount(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}") > 0) + assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicMetrics.INVALID_MAGIC_NUMBER_RECORDS_PER_SEC}")), 1) + assertTrue(meterCount(s"${BrokerTopicMetrics.INVALID_MAGIC_NUMBER_RECORDS_PER_SEC}") > 0) } private def validateMessages(records: MemoryRecords, @@ -732,8 +733,8 @@ class LogValidatorTest { ) ) - assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidMessageCrcRecordsPerSec}")), 1) - assertTrue(meterCount(s"${BrokerTopicStats.InvalidMessageCrcRecordsPerSec}") > 0) + assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicMetrics.INVALID_MESSAGE_CRC_RECORDS_PER_SEC}")), 1) + assertTrue(meterCount(s"${BrokerTopicMetrics.INVALID_MESSAGE_CRC_RECORDS_PER_SEC}") > 0) } @@ -1421,8 +1422,8 @@ class LogValidatorTest { ).validateMessagesAndAssignOffsets( PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier )) - assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec}")), 1) - assertTrue(meterCount(s"${BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec}") > 0) + assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicMetrics.INVALID_OFFSET_OR_SEQUENCE_RECORDS_PER_SEC}")), 1) + assertTrue(meterCount(s"${BrokerTopicMetrics.INVALID_OFFSET_OR_SEQUENCE_RECORDS_PER_SEC}") > 0) } @Test diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 6d7614d9b5..448f017a2d 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -40,6 +40,7 @@ import org.apache.kafka.storage.internals.checkpoint.{LeaderEpochCheckpointFile, import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, EpochEntry, FetchIsolation, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig, RecordValidationException, VerificationGuard} import org.apache.kafka.storage.internals.utils.Throttler +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -1859,8 +1860,8 @@ class UnifiedLogTest { assertTrue(e.recordErrors.get(0).message.startsWith(errorMsgPrefix)) // check if metric for NoKeyCompactedTopicRecordsPerSec is logged - assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec}")), 1) - assertTrue(TestUtils.meterCount(s"${BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec}") > 0) + assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicMetrics.NO_KEY_COMPACTED_TOPIC_RECORDS_PER_SEC}")), 1) + assertTrue(TestUtils.meterCount(s"${BrokerTopicMetrics.NO_KEY_COMPACTED_TOPIC_RECORDS_PER_SEC}") > 0) // the following should succeed without any InvalidMessageException log.appendAsLeader(messageSetWithKeyedMessage, leaderEpoch = 0) diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index e2355dfe11..3f6c5f0b4e 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -35,6 +35,7 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.metadata.migration.ZkMigrationState import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector} +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics import org.junit.jupiter.api.Timeout import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -147,7 +148,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { // The broker metrics for all topics should be greedily registered assertTrue(topicMetrics(None).nonEmpty, "General topic metrics don't exist") - assertEquals(brokers.head.brokerTopicStats.allTopicsStats.metricMap.size, topicMetrics(None).size) + assertEquals(brokers.head.brokerTopicStats.allTopicsStats.metricMapKeySet().size, topicMetrics(None).size) assertEquals(0, brokers.head.brokerTopicStats.allTopicsStats.metricGaugeMap.size) // topic metrics should be lazily registered assertTrue(topicMetricGroups(topic).isEmpty, "Topic metrics aren't lazily registered") @@ -169,10 +170,10 @@ class MetricsTest extends KafkaServerTestHarness with Logging { @ValueSource(strings = Array("zk", "kraft")) def testBrokerTopicMetricsBytesInOut(quorum: String): Unit = { val topic = "test-bytes-in-out" - val replicationBytesIn = BrokerTopicStats.ReplicationBytesInPerSec - val replicationBytesOut = BrokerTopicStats.ReplicationBytesOutPerSec - val bytesIn = s"${BrokerTopicStats.BytesInPerSec},topic=$topic" - val bytesOut = s"${BrokerTopicStats.BytesOutPerSec},topic=$topic" + val replicationBytesIn = BrokerTopicMetrics.REPLICATION_BYTES_IN_PER_SEC + val replicationBytesOut = BrokerTopicMetrics.REPLICATION_BYTES_OUT_PER_SEC + val bytesIn = s"${BrokerTopicMetrics.BYTES_IN_PER_SEC},topic=$topic" + val bytesOut = s"${BrokerTopicMetrics.BYTES_OUT_PER_SEC},topic=$topic" val topicConfig = new Properties topicConfig.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2") diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index 2f5116c424..b98c1ddfd0 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -68,7 +68,7 @@ class AbstractFetcherThreadTest { fetcher.start() - val brokerTopicStatsMetrics = fetcher.brokerTopicStats.allTopicsStats.metricMap.keySet + val brokerTopicStatsMetrics = fetcher.brokerTopicStats.allTopicsStats.metricMapKeySet().asScala val fetcherMetrics = Set(FetcherMetrics.BytesPerSec, FetcherMetrics.RequestsPerSec, FetcherMetrics.ConsumerLag) // wait until all fetcher metrics are present diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala index 84d744479d..a05a7c3c71 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestDownConversionConfigTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer import org.apache.kafka.server.config.ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -168,7 +169,7 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest { val fetchRequest = "request=Fetch" val fetchTemporaryMemoryBytesMetricName = s"$TemporaryMemoryBytes,$fetchRequest" val fetchMessageConversionsTimeMsMetricName = s"$MessageConversionsTimeMs,$fetchRequest" - val initialFetchMessageConversionsPerSec = TestUtils.metersCount(BrokerTopicStats.FetchMessageConversionsPerSec) + val initialFetchMessageConversionsPerSec = TestUtils.metersCount(BrokerTopicMetrics.FETCH_MESSAGE_CONVERSIONS_PER_SEC) val initialFetchMessageConversionsTimeMs = TestUtils.metersCount(fetchMessageConversionsTimeMsMetricName) val initialFetchTemporaryMemoryBytes = TestUtils.metersCount(fetchTemporaryMemoryBytesMetricName) val topicWithDownConversionEnabled = "foo" @@ -224,9 +225,9 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest { } def verifyMetrics(): Unit = { - TestUtils.waitUntilTrue(() => TestUtils.metersCount(BrokerTopicStats.FetchMessageConversionsPerSec) > initialFetchMessageConversionsPerSec, + TestUtils.waitUntilTrue(() => TestUtils.metersCount(BrokerTopicMetrics.FETCH_MESSAGE_CONVERSIONS_PER_SEC) > initialFetchMessageConversionsPerSec, s"The `FetchMessageConversionsPerSec` metric count is not incremented after 5 seconds. " + - s"init: $initialFetchMessageConversionsPerSec final: ${TestUtils.metersCount(BrokerTopicStats.FetchMessageConversionsPerSec)}", 5000) + s"init: $initialFetchMessageConversionsPerSec final: ${TestUtils.metersCount(BrokerTopicMetrics.FETCH_MESSAGE_CONVERSIONS_PER_SEC)}", 5000) TestUtils.waitUntilTrue(() => TestUtils.metersCount(fetchMessageConversionsTimeMsMetricName) > initialFetchMessageConversionsTimeMs, s"The `MessageConversionsTimeMs` in fetch request metric count is not incremented after 5 seconds. " + diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index fea41b7611..086d0d0b29 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -29,6 +29,7 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.record.BrokerCompressionType +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} @@ -204,8 +205,8 @@ class ProduceRequestTest extends BaseRequestTest { assertEquals(Errors.CORRUPT_MESSAGE, Errors.forCode(partitionProduceResponse.errorCode)) assertEquals(-1, partitionProduceResponse.baseOffset) assertEquals(-1, partitionProduceResponse.logAppendTimeMs) - assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidMessageCrcRecordsPerSec}")), 1) - assertTrue(TestUtils.meterCount(s"${BrokerTopicStats.InvalidMessageCrcRecordsPerSec}") > 0) + assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicMetrics.INVALID_MESSAGE_CRC_RECORDS_PER_SEC}")), 1) + assertTrue(TestUtils.meterCount(s"${BrokerTopicMetrics.INVALID_MESSAGE_CRC_RECORDS_PER_SEC}") > 0) } @ParameterizedTest diff --git a/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaMetricsGroup.java b/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaMetricsGroup.java index f876ea76c1..5c5607db73 100644 --- a/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaMetricsGroup.java +++ b/server-common/src/main/java/org/apache/kafka/server/metrics/KafkaMetricsGroup.java @@ -33,10 +33,21 @@ import java.util.stream.Collectors; public class KafkaMetricsGroup { - private final Class klass; + private final String pkg; + private final String simpleName; public KafkaMetricsGroup(Class klass) { - this.klass = klass; + this(klass.getPackage() == null ? "" : klass.getPackage().getName(), klass.getSimpleName().replaceAll("\\$$", "")); + } + + /** + * This constructor allows caller to build metrics name with custom package and class name. This is useful to keep metrics + * compatibility in migrating scala code, since the difference of either package or class name will impact the mbean name and + * that will break the backward compatibility. + */ + public KafkaMetricsGroup(String packageName, String simpleName) { + this.pkg = packageName; + this.simpleName = simpleName; } /** @@ -47,9 +58,7 @@ public KafkaMetricsGroup(Class klass) { * @return Sanitized metric name object. */ public MetricName metricName(String name, Map tags) { - String pkg = klass.getPackage() == null ? "" : klass.getPackage().getName(); - String simpleName = klass.getSimpleName().replaceAll("\\$$", ""); - return explicitMetricName(pkg, simpleName, name, tags); + return explicitMetricName(this.pkg, this.simpleName, name, tags); } public static MetricName explicitMetricName(String group, String typeName, diff --git a/server-common/src/test/java/org/apache/kafka/server/metrics/KafkaMetricsGroupTest.java b/server-common/src/test/java/org/apache/kafka/server/metrics/KafkaMetricsGroupTest.java new file mode 100644 index 0000000000..08b5b00fb2 --- /dev/null +++ b/server-common/src/test/java/org/apache/kafka/server/metrics/KafkaMetricsGroupTest.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.metrics; + +import com.yammer.metrics.core.MetricName; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class KafkaMetricsGroupTest { + @Test + public void testConstructorWithPackageAndSimpleName() { + String packageName = "testPackage"; + String simpleName = "testSimple"; + KafkaMetricsGroup group = new KafkaMetricsGroup(packageName, simpleName); + MetricName metricName = group.metricName("metric-name", Collections.emptyMap()); + assertEquals(packageName, metricName.getGroup()); + assertEquals(simpleName, metricName.getType()); + } +} diff --git a/storage/src/main/java/org/apache/kafka/storage/log/metrics/BrokerTopicMetrics.java b/storage/src/main/java/org/apache/kafka/storage/log/metrics/BrokerTopicMetrics.java new file mode 100644 index 0000000000..c638beb1e9 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/storage/log/metrics/BrokerTopicMetrics.java @@ -0,0 +1,405 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.storage.log.metrics; + +import org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics; +import org.apache.kafka.server.metrics.KafkaMetricsGroup; + +import com.yammer.metrics.core.Meter; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class BrokerTopicMetrics { + public static final String MESSAGE_IN_PER_SEC = "MessagesInPerSec"; + public static final String BYTES_IN_PER_SEC = "BytesInPerSec"; + public static final String BYTES_OUT_PER_SEC = "BytesOutPerSec"; + public static final String BYTES_REJECTED_PER_SEC = "BytesRejectedPerSec"; + public static final String REPLICATION_BYTES_IN_PER_SEC = "ReplicationBytesInPerSec"; + public static final String REPLICATION_BYTES_OUT_PER_SEC = "ReplicationBytesOutPerSec"; + public static final String FAILED_PRODUCE_REQUESTS_PER_SEC = "FailedProduceRequestsPerSec"; + public static final String FAILED_FETCH_REQUESTS_PER_SEC = "FailedFetchRequestsPerSec"; + public static final String TOTAL_PRODUCE_REQUESTS_PER_SEC = "TotalProduceRequestsPerSec"; + public static final String TOTAL_FETCH_REQUESTS_PER_SEC = "TotalFetchRequestsPerSec"; + public static final String FETCH_MESSAGE_CONVERSIONS_PER_SEC = "FetchMessageConversionsPerSec"; + public static final String PRODUCE_MESSAGE_CONVERSIONS_PER_SEC = "ProduceMessageConversionsPerSec"; + public static final String REASSIGNMENT_BYTES_IN_PER_SEC = "ReassignmentBytesInPerSec"; + public static final String REASSIGNMENT_BYTES_OUT_PER_SEC = "ReassignmentBytesOutPerSec"; + // These following topics are for LogValidator for better debugging on failed records + public static final String NO_KEY_COMPACTED_TOPIC_RECORDS_PER_SEC = "NoKeyCompactedTopicRecordsPerSec"; + public static final String INVALID_MAGIC_NUMBER_RECORDS_PER_SEC = "InvalidMagicNumberRecordsPerSec"; + public static final String INVALID_MESSAGE_CRC_RECORDS_PER_SEC = "InvalidMessageCrcRecordsPerSec"; + public static final String INVALID_OFFSET_OR_SEQUENCE_RECORDS_PER_SEC = "InvalidOffsetOrSequenceRecordsPerSec"; + + // KAFKA-16972: BrokerTopicMetrics is migrated from "kafka.server" package. + // For backward compatibility, we keep the old package name as metric group name. + private final KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "BrokerTopicMetrics"); + private final Map tags; + private final Map metricTypeMap = new java.util.HashMap<>(); + private final Map metricGaugeTypeMap = new java.util.HashMap<>(); + + public BrokerTopicMetrics(boolean remoteStorageEnabled) { + this(Optional.empty(), remoteStorageEnabled); + } + + public BrokerTopicMetrics(String name, boolean remoteStorageEnabled) { + this(Optional.of(name), remoteStorageEnabled); + } + + private BrokerTopicMetrics(Optional name, boolean remoteStorageEnabled) { + this.tags = name.map(s -> Collections.singletonMap("topic", s)).orElse(Collections.emptyMap()); + + metricTypeMap.put(MESSAGE_IN_PER_SEC, new MeterWrapper(MESSAGE_IN_PER_SEC, "messages")); + metricTypeMap.put(BYTES_IN_PER_SEC, new MeterWrapper(BYTES_IN_PER_SEC, "bytes")); + metricTypeMap.put(BYTES_OUT_PER_SEC, new MeterWrapper(BYTES_OUT_PER_SEC, "bytes")); + metricTypeMap.put(BYTES_REJECTED_PER_SEC, new MeterWrapper(BYTES_REJECTED_PER_SEC, "bytes")); + metricTypeMap.put(FAILED_PRODUCE_REQUESTS_PER_SEC, new MeterWrapper(FAILED_PRODUCE_REQUESTS_PER_SEC, "requests")); + metricTypeMap.put(FAILED_FETCH_REQUESTS_PER_SEC, new MeterWrapper(FAILED_FETCH_REQUESTS_PER_SEC, "requests")); + metricTypeMap.put(TOTAL_PRODUCE_REQUESTS_PER_SEC, new MeterWrapper(TOTAL_PRODUCE_REQUESTS_PER_SEC, "requests")); + metricTypeMap.put(TOTAL_FETCH_REQUESTS_PER_SEC, new MeterWrapper(TOTAL_FETCH_REQUESTS_PER_SEC, "requests")); + metricTypeMap.put(FETCH_MESSAGE_CONVERSIONS_PER_SEC, new MeterWrapper(FETCH_MESSAGE_CONVERSIONS_PER_SEC, "requests")); + metricTypeMap.put(PRODUCE_MESSAGE_CONVERSIONS_PER_SEC, new MeterWrapper(PRODUCE_MESSAGE_CONVERSIONS_PER_SEC, "requests")); + metricTypeMap.put(NO_KEY_COMPACTED_TOPIC_RECORDS_PER_SEC, new MeterWrapper(NO_KEY_COMPACTED_TOPIC_RECORDS_PER_SEC, "requests")); + metricTypeMap.put(INVALID_MAGIC_NUMBER_RECORDS_PER_SEC, new MeterWrapper(INVALID_MAGIC_NUMBER_RECORDS_PER_SEC, "requests")); + metricTypeMap.put(INVALID_MESSAGE_CRC_RECORDS_PER_SEC, new MeterWrapper(INVALID_MESSAGE_CRC_RECORDS_PER_SEC, "requests")); + metricTypeMap.put(INVALID_OFFSET_OR_SEQUENCE_RECORDS_PER_SEC, new MeterWrapper(INVALID_OFFSET_OR_SEQUENCE_RECORDS_PER_SEC, "requests")); + + if (!name.isPresent()) { + metricTypeMap.put(REPLICATION_BYTES_IN_PER_SEC, new MeterWrapper(REPLICATION_BYTES_IN_PER_SEC, "bytes")); + metricTypeMap.put(REPLICATION_BYTES_OUT_PER_SEC, new MeterWrapper(REPLICATION_BYTES_OUT_PER_SEC, "bytes")); + metricTypeMap.put(REASSIGNMENT_BYTES_IN_PER_SEC, new MeterWrapper(REASSIGNMENT_BYTES_IN_PER_SEC, "bytes")); + metricTypeMap.put(REASSIGNMENT_BYTES_OUT_PER_SEC, new MeterWrapper(REASSIGNMENT_BYTES_OUT_PER_SEC, "bytes")); + } + + if (remoteStorageEnabled) { + metricTypeMap.put(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName(), "bytes")); + metricTypeMap.put(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName(), "bytes")); + metricTypeMap.put(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName(), "requests")); + metricTypeMap.put(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName(), "requests")); + metricTypeMap.put(RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName(), "requests")); + metricTypeMap.put(RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName(), "requests")); + metricTypeMap.put(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName(), "requests")); + metricTypeMap.put(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName(), "requests")); + metricTypeMap.put(RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName(), "requests")); + metricTypeMap.put(RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName(), new MeterWrapper(RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName(), "requests")); + + metricGaugeTypeMap.put(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName(), new GaugeWrapper(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName())); + metricGaugeTypeMap.put(RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName(), new GaugeWrapper(RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName())); + metricGaugeTypeMap.put(RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName(), new GaugeWrapper(RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName())); + metricGaugeTypeMap.put(RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName(), new GaugeWrapper(RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName())); + metricGaugeTypeMap.put(RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName(), new GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName())); + metricGaugeTypeMap.put(RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName(), new GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName())); + metricGaugeTypeMap.put(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName(), new GaugeWrapper(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName())); + } + } + + public void closeMetric(String metricName) { + MeterWrapper mw = metricTypeMap.get(metricName); + if (mw != null) mw.close(); + GaugeWrapper mg = metricGaugeTypeMap.get(metricName); + if (mg != null) mg.close(); + } + + public void close() { + metricTypeMap.values().forEach(MeterWrapper::close); + metricGaugeTypeMap.values().forEach(GaugeWrapper::close); + } + + // used for testing only + public Set metricMapKeySet() { + return metricTypeMap.keySet(); + } + + public Map metricGaugeMap() { + return metricGaugeTypeMap; + } + + public Meter messagesInRate() { + return metricTypeMap.get(MESSAGE_IN_PER_SEC).meter(); + } + + public Meter bytesInRate() { + return metricTypeMap.get(BYTES_IN_PER_SEC).meter(); + } + + public Meter bytesOutRate() { + return metricTypeMap.get(BYTES_OUT_PER_SEC).meter(); + } + + public Meter bytesRejectedRate() { + return metricTypeMap.get(BYTES_REJECTED_PER_SEC).meter(); + } + + public Optional replicationBytesInRate() { + if (tags.isEmpty()) { + return Optional.of(metricTypeMap.get(REPLICATION_BYTES_IN_PER_SEC).meter()); + } else { + return Optional.empty(); + } + } + + public Optional replicationBytesOutRate() { + if (tags.isEmpty()) { + return Optional.of(metricTypeMap.get(REPLICATION_BYTES_OUT_PER_SEC).meter()); + } else { + return Optional.empty(); + } + } + + public Optional reassignmentBytesInPerSec() { + if (tags.isEmpty()) { + return Optional.of(metricTypeMap.get(REASSIGNMENT_BYTES_IN_PER_SEC).meter()); + } else { + return Optional.empty(); + } + } + + public Optional reassignmentBytesOutPerSec() { + if (tags.isEmpty()) { + return Optional.of(metricTypeMap.get(REASSIGNMENT_BYTES_OUT_PER_SEC).meter()); + } else { + return Optional.empty(); + } + } + + public Meter failedProduceRequestRate() { + return metricTypeMap.get(FAILED_PRODUCE_REQUESTS_PER_SEC).meter(); + } + + public Meter failedFetchRequestRate() { + return metricTypeMap.get(FAILED_FETCH_REQUESTS_PER_SEC).meter(); + } + + public Meter totalProduceRequestRate() { + return metricTypeMap.get(TOTAL_PRODUCE_REQUESTS_PER_SEC).meter(); + } + + public Meter totalFetchRequestRate() { + return metricTypeMap.get(TOTAL_FETCH_REQUESTS_PER_SEC).meter(); + } + + public Meter fetchMessageConversionsRate() { + return metricTypeMap.get(FETCH_MESSAGE_CONVERSIONS_PER_SEC).meter(); + } + + public Meter produceMessageConversionsRate() { + return metricTypeMap.get(PRODUCE_MESSAGE_CONVERSIONS_PER_SEC).meter(); + } + + public Meter noKeyCompactedTopicRecordsPerSec() { + return metricTypeMap.get(NO_KEY_COMPACTED_TOPIC_RECORDS_PER_SEC).meter(); + } + + public Meter invalidMagicNumberRecordsPerSec() { + return metricTypeMap.get(INVALID_MAGIC_NUMBER_RECORDS_PER_SEC).meter(); + } + + public Meter invalidMessageCrcRecordsPerSec() { + return metricTypeMap.get(INVALID_MESSAGE_CRC_RECORDS_PER_SEC).meter(); + } + + public Meter invalidOffsetOrSequenceRecordsPerSec() { + return metricTypeMap.get(INVALID_OFFSET_OR_SEQUENCE_RECORDS_PER_SEC).meter(); + } + + public GaugeWrapper remoteCopyLagBytesAggrMetric() { + return metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_LAG_BYTES_METRIC.getName()); + } + + // Visible for testing + public long remoteCopyLagBytes() { + return remoteCopyLagBytesAggrMetric().value(); + } + + public GaugeWrapper remoteCopyLagSegmentsAggrMetric() { + return metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_LAG_SEGMENTS_METRIC.getName()); + } + + // Visible for testing + public long remoteCopyLagSegments() { + return remoteCopyLagSegmentsAggrMetric().value(); + } + + public GaugeWrapper remoteLogMetadataCountAggrMetric() { + return metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_LOG_METADATA_COUNT_METRIC.getName()); + } + + public long remoteLogMetadataCount() { + return remoteLogMetadataCountAggrMetric().value(); + } + + public GaugeWrapper remoteLogSizeBytesAggrMetric() { + return metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_LOG_SIZE_BYTES_METRIC.getName()); + } + + public long remoteLogSizeBytes() { + return remoteLogSizeBytesAggrMetric().value(); + } + + public GaugeWrapper remoteLogSizeComputationTimeAggrMetric() { + return metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_LOG_SIZE_COMPUTATION_TIME_METRIC.getName()); + } + + public long remoteLogSizeComputationTime() { + return remoteLogSizeComputationTimeAggrMetric().value(); + } + + public GaugeWrapper remoteDeleteLagBytesAggrMetric() { + return metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_DELETE_LAG_BYTES_METRIC.getName()); + } + + // Visible for testing + public long remoteDeleteLagBytes() { + return remoteDeleteLagBytesAggrMetric().value(); + } + + public GaugeWrapper remoteDeleteLagSegmentsAggrMetric() { + return metricGaugeTypeMap.get(RemoteStorageMetrics.REMOTE_DELETE_LAG_SEGMENTS_METRIC.getName()); + } + + // Visible for testing + public long remoteDeleteLagSegments() { + return remoteDeleteLagSegmentsAggrMetric().value(); + } + + public Meter remoteCopyBytesRate() { + return metricTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName()).meter(); + } + + public Meter remoteFetchBytesRate() { + return metricTypeMap.get(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName()).meter(); + } + + public Meter remoteFetchRequestRate() { + return metricTypeMap.get(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName()).meter(); + } + + public Meter remoteCopyRequestRate() { + return metricTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName()).meter(); + } + + public Meter remoteDeleteRequestRate() { + return metricTypeMap.get(RemoteStorageMetrics.REMOTE_DELETE_REQUESTS_PER_SEC_METRIC.getName()).meter(); + } + + public Meter buildRemoteLogAuxStateRequestRate() { + return metricTypeMap.get(RemoteStorageMetrics.BUILD_REMOTE_LOG_AUX_STATE_REQUESTS_PER_SEC_METRIC.getName()).meter(); + } + + public Meter failedRemoteFetchRequestRate() { + return metricTypeMap.get(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName()).meter(); + } + + public Meter failedRemoteCopyRequestRate() { + return metricTypeMap.get(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName()).meter(); + } + + public Meter failedRemoteDeleteRequestRate() { + return metricTypeMap.get(RemoteStorageMetrics.FAILED_REMOTE_DELETE_PER_SEC_METRIC.getName()).meter(); + } + + public Meter failedBuildRemoteLogAuxStateRate() { + return metricTypeMap.get(RemoteStorageMetrics.FAILED_BUILD_REMOTE_LOG_AUX_STATE_PER_SEC_METRIC.getName()).meter(); + } + + private class MeterWrapper { + private final String metricType; + private final String eventType; + private volatile Meter lazyMeter; + private final Lock meterLock = new ReentrantLock(); + + public MeterWrapper(String metricType, String eventType) { + this.metricType = metricType; + this.eventType = eventType; + if (tags.isEmpty()) { + meter(); // greedily initialize the general topic metrics + } + } + + public Meter meter() { + Meter meter = lazyMeter; + if (meter == null) { + meterLock.lock(); + try { + meter = lazyMeter; + if (meter == null) { + meter = metricsGroup.newMeter(metricType, eventType, TimeUnit.SECONDS, tags); + lazyMeter = meter; + } + } finally { + meterLock.unlock(); + } + } + return meter; + } + + public void close() { + meterLock.lock(); + try { + if (lazyMeter != null) { + metricsGroup.removeMetric(metricType, tags); + lazyMeter = null; + } + } finally { + meterLock.unlock(); + } + } + } + + public class GaugeWrapper { + // The map to store: + // - per-partition value for topic-level metrics. The key will be the partition number + // - per-topic value for broker-level metrics. The key will be the topic name + private final ConcurrentHashMap metricValues = new ConcurrentHashMap<>(); + private final String metricType; + + public GaugeWrapper(String metricType) { + this.metricType = metricType; + newGaugeIfNeed(); + } + + public void setValue(String key, long value) { + newGaugeIfNeed(); + metricValues.put(key, value); + } + + public void removeKey(String key) { + newGaugeIfNeed(); + metricValues.remove(key); + } + + public void close() { + metricsGroup.removeMetric(metricType, tags); + metricValues.clear(); + } + + public long value() { + return metricValues.values().stream().mapToLong(v -> v).sum(); + } + + // metricsGroup uses ConcurrentMap to store gauges, so we don't need to use synchronized block here + private void newGaugeIfNeed() { + metricsGroup.newGauge(metricType, () -> metricValues.values().stream().mapToLong(v -> v).sum(), tags); + } + } +} From 44f15cc22cfc8540cdcacb1af2aa1a117bceff44 Mon Sep 17 00:00:00 2001 From: Mickael Maison Date: Thu, 10 Oct 2024 09:20:14 -0700 Subject: [PATCH 112/123] KAFKA-17749: Fix Throttler metrics name Reviewers: Colin P. McCabe --- docs/upgrade.html | 3 +++ .../org/apache/kafka/storage/internals/utils/Throttler.java | 3 ++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index faa4862154..478d9f0488 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -41,6 +41,9 @@
    Notable changes in 3 See KIP-956 for more details.
  • +
  • In 3.8.0 the kafka.utils.Thottler metric was accidentally renamed to org.apache.kafka.storage.internals.utils.Throttler. + This change has been reverted and the metric is now named kafka.utils.Thottler again. +
  • Upgrading to 3.8.0 from any version 0.8.x through 3.7.x

    diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/utils/Throttler.java b/storage/src/main/java/org/apache/kafka/storage/internals/utils/Throttler.java index b4e7fc0c10..01d8e585d2 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/utils/Throttler.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/utils/Throttler.java @@ -62,7 +62,8 @@ public Throttler(double desiredRatePerSec, Time time) { this.desiredRatePerSec = desiredRatePerSec; this.checkIntervalNs = TimeUnit.MILLISECONDS.toNanos(checkIntervalMs); - this.meter = new KafkaMetricsGroup(Throttler.class).newMeter(metricName, units, TimeUnit.SECONDS); + // For compatibility - this metrics group was previously defined within a Scala class named `kafka.utils.Throttler` + this.meter = new KafkaMetricsGroup("kafka.utils", "Throttler").newMeter(metricName, units, TimeUnit.SECONDS); this.time = time; this.periodStartNs = time.nanoseconds(); } From 8c3c6c3841dd7491b1e2e782e19ec40535500110 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 10 Oct 2024 16:20:49 -0400 Subject: [PATCH 113/123] KAFKA-17193: Pin all external GitHub Actions to the specific git hash (#16960) (#17461) Co-authored-by: Mickael Maison Reviewers: Chia-Ping Tsai , Colin P. McCabe --- .github/workflows/docker_build_and_test.yml | 2 +- .github/workflows/docker_official_image_build_and_test.yml | 2 +- .github/workflows/docker_promote.yml | 6 +++--- .github/workflows/docker_rc_release.yml | 6 +++--- .github/workflows/docker_scan.yml | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/workflows/docker_build_and_test.yml b/.github/workflows/docker_build_and_test.yml index dc6214633f..2618a717c7 100644 --- a/.github/workflows/docker_build_and_test.yml +++ b/.github/workflows/docker_build_and_test.yml @@ -46,7 +46,7 @@ jobs: run: | python docker_build_test.py kafka/test -tag=test -type=${{ github.event.inputs.image_type }} -u=${{ github.event.inputs.kafka_url }} - name: Run CVE scan - uses: aquasecurity/trivy-action@master + uses: aquasecurity/trivy-action@6e7b7d1fd3e4fef0c5fa8cce1229c54b2c9bd0d8 # v0.24.0 with: image-ref: 'kafka/test:test' format: 'table' diff --git a/.github/workflows/docker_official_image_build_and_test.yml b/.github/workflows/docker_official_image_build_and_test.yml index c3219bd8aa..1db476de53 100644 --- a/.github/workflows/docker_official_image_build_and_test.yml +++ b/.github/workflows/docker_official_image_build_and_test.yml @@ -45,7 +45,7 @@ jobs: run: | python docker_official_image_build_test.py kafka/test -tag=test -type=${{ github.event.inputs.image_type }} -v=${{ github.event.inputs.kafka_version }} - name: Run CVE scan - uses: aquasecurity/trivy-action@master + uses: aquasecurity/trivy-action@6e7b7d1fd3e4fef0c5fa8cce1229c54b2c9bd0d8 # v0.24.0 with: image-ref: 'kafka/test:test' format: 'table' diff --git a/.github/workflows/docker_promote.yml b/.github/workflows/docker_promote.yml index 04872f9d59..d22a8458c9 100644 --- a/.github/workflows/docker_promote.yml +++ b/.github/workflows/docker_promote.yml @@ -31,11 +31,11 @@ jobs: runs-on: ubuntu-latest steps: - name: Set up QEMU - uses: docker/setup-qemu-action@v3 + uses: docker/setup-qemu-action@49b3bc8e6bdd4a60e6116a5414239cba5943d3cf # v3.2.0 - name: Set up Docker Buildx - uses: docker/setup-buildx-action@v3 + uses: docker/setup-buildx-action@988b5a0280414f521da01fcc63a27aeeb4b104db # v3.6.1 - name: Login to Docker Hub - uses: docker/login-action@v3 + uses: docker/login-action@9780b0c442fbb1117ed29e0efdff1e18412f7567 # v3.3.0 with: username: ${{ secrets.DOCKERHUB_USER }} password: ${{ secrets.DOCKERHUB_TOKEN }} diff --git a/.github/workflows/docker_rc_release.yml b/.github/workflows/docker_rc_release.yml index 3a06064d62..dbca7fe23c 100644 --- a/.github/workflows/docker_rc_release.yml +++ b/.github/workflows/docker_rc_release.yml @@ -47,11 +47,11 @@ jobs: python -m pip install --upgrade pip pip install -r docker/requirements.txt - name: Set up QEMU - uses: docker/setup-qemu-action@v3 + uses: docker/setup-qemu-action@49b3bc8e6bdd4a60e6116a5414239cba5943d3cf # v3.2.0 - name: Set up Docker Buildx - uses: docker/setup-buildx-action@v3 + uses: docker/setup-buildx-action@988b5a0280414f521da01fcc63a27aeeb4b104db # v3.6.1 - name: Login to Docker Hub - uses: docker/login-action@v3 + uses: docker/login-action@9780b0c442fbb1117ed29e0efdff1e18412f7567 # v3.3.0 with: username: ${{ secrets.DOCKERHUB_USER }} password: ${{ secrets.DOCKERHUB_TOKEN }} diff --git a/.github/workflows/docker_scan.yml b/.github/workflows/docker_scan.yml index 2134ef7eef..b7efaa4ff9 100644 --- a/.github/workflows/docker_scan.yml +++ b/.github/workflows/docker_scan.yml @@ -29,7 +29,7 @@ jobs: supported_image_tag: ['latest', '3.7.0'] steps: - name: Run CVE scan - uses: aquasecurity/trivy-action@master + uses: aquasecurity/trivy-action@6e7b7d1fd3e4fef0c5fa8cce1229c54b2c9bd0d8 # v0.24.0 if: always() with: image-ref: apache/kafka:${{ matrix.supported_image_tag }} From 51253e2bf4134c1232ead4ce3666949669d0dea6 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Tue, 15 Oct 2024 00:15:59 +0800 Subject: [PATCH 114/123] KAFKA-17520 align the low bound of ducktape version (#17481) Reviewers: Colin Patrick McCabe , Chia-Ping Tsai --- tests/docker/Dockerfile | 3 ++- tests/setup.py | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index bca82534cd..57b71bf4fd 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -63,7 +63,8 @@ LABEL ducker.creator=$ducker_creator # we have to install git since it is included in openjdk:8 but not openjdk:11 RUN apt update && apt install -y sudo git netcat iptables rsync unzip wget curl jq coreutils openssh-server net-tools vim python3-pip python3-dev libffi-dev libssl-dev cmake pkg-config libfuse-dev iperf traceroute iproute2 iputils-ping && apt-get -y clean RUN python3 -m pip install -U pip==21.1.1; -RUN pip3 install --upgrade cffi virtualenv pyasn1 boto3 pycrypto pywinrm ipaddress enum34 debugpy && pip3 install --upgrade "ducktape>0.8" +# NOTE: ducktape 0.11.4 requires python3.9 +RUN pip3 install --upgrade cffi virtualenv pyasn1 boto3 pycrypto pywinrm ipaddress enum34 debugpy && pip3 install --upgrade "ducktape==0.11.4" COPY --from=build-native-image /build/kafka-binary/ /opt/kafka-binary/ # Set up ssh diff --git a/tests/setup.py b/tests/setup.py index f60870dcaa..fd3b973867 100644 --- a/tests/setup.py +++ b/tests/setup.py @@ -51,7 +51,7 @@ def run_tests(self): license="apache2.0", packages=find_packages(), include_package_data=True, - install_requires=["ducktape==0.8.14", "requests==2.24.0"], + install_requires=["ducktape==0.11.4", "requests==2.31.0"], tests_require=["pytest", "mock"], cmdclass={'test': PyTest}, zip_safe=False From 796ce2121bbb9cd96eb175f2098c3fc2d381c513 Mon Sep 17 00:00:00 2001 From: "Colin P. McCabe" Date: Mon, 14 Oct 2024 17:21:03 -0700 Subject: [PATCH 115/123] KAFKA-17788: During ZK migration, always include control.plane.listener.name in advertisedBrokerListeners During ZK migration, always include control.plane.listener.name in advertisedBrokerListeners, to be bug-compatible with earlier Apache Kafka versions that ignored this misconfiguration. (Just as before, control.plane.listener.name is not supported in KRaft mode itself.) Reviewers: Luke Chen --- .../main/scala/kafka/server/KafkaConfig.scala | 13 ++++++-- .../scala/unit/kafka/KafkaConfigTest.scala | 30 +++++++++++++++++++ 2 files changed, 41 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 95a8c768f7..7992f833bc 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -758,8 +758,17 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) } def effectiveAdvertisedBrokerListeners: Seq[EndPoint] = { - // Only expose broker listeners - advertisedListeners.filterNot(l => controllerListenerNames.contains(l.listenerName.value())) + advertisedListeners.filter(l => { + if (!controllerListenerNames.contains(l.listenerName.value())) { + true + } else if (migrationEnabled && Some(l.listenerName.value()).equals(controlPlaneListener.map(_.listenerName.value()))) { + // KAFKA-17788: during ZK migration, always include control.plane.listener.name + // in advertisedBrokerListeners. + true + } else { + false + } + }) } // Use advertised listeners if defined, fallback to listeners otherwise diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index fec4eed488..4abfa04938 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -32,6 +32,8 @@ import org.apache.kafka.server.config.{KRaftConfigs, ZkConfigs} import org.apache.kafka.server.config.ReplicationConfigs import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ @@ -420,4 +422,32 @@ class KafkaConfigTest { file.getAbsolutePath } finally writer.close() } + + @ParameterizedTest + @ValueSource(booleans = Array(false, true)) + def testIncludeControlPlaneListenerNameInAdvertisedBrokerListeners(migration: Boolean): Unit = { + val properties = new Properties() + if (migration) { + properties.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true") + } else { + properties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") + } + properties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") + properties.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9092") + properties.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:9093,INTERNAL://:9092") + properties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") + properties.setProperty(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG, "CONTROLLER") + properties.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "CONTROLLER://localhost:9093,INTERNAL://localhost:9092") + properties.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:SSL,INTERNAL:PLAINTEXT") + properties.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "INTERNAL") + properties.setProperty("zookeeper.connect", "localhost:2181") + if (migration) { + val config = KafkaConfig.fromProps(properties) + assertEquals(Seq("CONTROLLER", "INTERNAL"), config.effectiveAdvertisedBrokerListeners.map(_.listenerName.value())) + assertEquals(Seq("CONTROLLER"), config.effectiveAdvertisedControllerListeners.map(_.listenerName.value())) + } else { + assertEquals("requirement failed: control.plane.listener.name is not supported in KRaft mode.", + assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(properties)).getMessage) + } + } } From abd4bf08aba782da93c960bf6f21fadd1b24db62 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 15 Oct 2024 14:36:16 -0700 Subject: [PATCH 116/123] KAFKA-17790: Document that control.plane.listener should be removed before ZK migration is finished (#17501) Reviewers: Luke Chen --- docs/ops.html | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ops.html b/docs/ops.html index 5c791dbce5..bdf3109e6a 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -4069,7 +4069,8 @@

    Migrating brokers to KRaft

    To migrate the brokers to KRaft, they simply need to be reconfigured as KRaft brokers and restarted. Using the above broker configuration as an example, we would replace the broker.id with node.id and add process.roles=broker. It is important that the broker maintain the same Broker/Node ID when it is restarted. - The zookeeper configurations should be removed at this point. + The zookeeper configurations should be removed at this point. Finally, if you have set + control.plane.listener.name. please remove it before restarting in KRaft mode.

    From de9a7199dfbcfbc63334772f7b4556826fbdf4ce Mon Sep 17 00:00:00 2001 From: Josep Prat Date: Thu, 17 Oct 2024 08:24:11 +0200 Subject: [PATCH 117/123] KAFKA-17810 upgrade Jetty because of CVE-2024-8184 (#17517) Reviewers: Chia-Ping Tsai --- LICENSE-binary | 20 ++++++++++---------- gradle/dependencies.gradle | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 0e19528b0f..0977c8a186 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -229,16 +229,16 @@ jackson-module-scala_2.13-2.16.2 jackson-module-scala_2.12-2.16.2 jakarta.validation-api-2.0.2 javassist-3.29.2-GA -jetty-client-9.4.54.v20240208 -jetty-continuation-9.4.54.v20240208 -jetty-http-9.4.54.v20240208 -jetty-io-9.4.54.v20240208 -jetty-security-9.4.54.v20240208 -jetty-server-9.4.54.v20240208 -jetty-servlet-9.4.54.v20240208 -jetty-servlets-9.4.54.v20240208 -jetty-util-9.4.54.v20240208 -jetty-util-ajax-9.4.54.v20240208 +jetty-client-9.4.56.v20240826 +jetty-continuation-9.4.56.v20240826 +jetty-http-9.4.56.v20240826 +jetty-io-9.4.56.v20240826 +jetty-security-9.4.56.v20240826 +jetty-server-9.4.56.v20240826 +jetty-servlet-9.4.56.v20240826 +jetty-servlets-9.4.56.v20240826 +jetty-util-9.4.56.v20240826 +jetty-util-ajax-9.4.56.v20240826 jose4j-0.9.4 lz4-java-1.8.0 maven-artifact-3.9.6 diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 1b7e354b81..2f29c95e1e 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -99,7 +99,7 @@ versions += [ jackson: "2.16.2", jacoco: "0.8.10", javassist: "3.29.2-GA", - jetty: "9.4.54.v20240208", + jetty: "9.4.56.v20240826", jersey: "2.39.1", jline: "3.25.1", jmh: "1.37", From 7842e25d32131e0e1362592fde86a46278e9dcff Mon Sep 17 00:00:00 2001 From: Federico Valeri Date: Mon, 21 Oct 2024 19:39:11 +0200 Subject: [PATCH 118/123] KAFKA-17031: Make RLM thread pool configurations public and fix default handling (#17499) According to KIP-950, remote.log.manager.thread.pool.size should be marked as deprecated and replaced by two new configurations: remote.log.manager.copier.thread.pool.size and remote.log.manager.expiration.thread.pool.size. Fix default handling so that -1 works as expected. Reviewers: Luke Chen , Gaurav Narula , Satish Duggana , Colin P. McCabe --- .../unit/kafka/server/KafkaConfigTest.scala | 2 + .../storage/RemoteLogManagerConfig.java | 40 ++++++++++++------- .../storage/RemoteLogManagerConfigTest.java | 15 +++++-- 3 files changed, 39 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index b093112c92..541080aee8 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1106,6 +1106,8 @@ class KafkaConfigTest { case RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP => // ignore string case RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case RemoteLogManagerConfig.REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) + case RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -2) + case RemoteLogManagerConfig.REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -2) case RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_RETRY_BACK_OFF_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_RETRY_BACK_OFF_MAX_MS_PROP => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java index 9f73af4a7c..bf0aba8e85 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfig.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; import java.util.Collections; import java.util.Map; @@ -93,20 +94,29 @@ public final class RemoteLogManagerConfig { public static final long DEFAULT_REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES = 1024 * 1024 * 1024L; public static final String REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP = "remote.log.manager.thread.pool.size"; - public static final String REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in scheduling tasks to copy " + + public static final String REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_DOC = "Deprecated. Size of the thread pool used in scheduling tasks to copy " + "segments, fetch remote log indexes and clean up remote log segments."; public static final int DEFAULT_REMOTE_LOG_MANAGER_THREAD_POOL_SIZE = 10; + private static final String REMOTE_LOG_MANAGER_THREAD_POOL_FALLBACK = "The default value of -1 means that this will be set to the configured value of " + + REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP + ", if available; otherwise, it defaults to " + DEFAULT_REMOTE_LOG_MANAGER_THREAD_POOL_SIZE + "."; + private static final ConfigDef.Validator REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_VALIDATOR = ConfigDef.LambdaValidator.with( + (name, value) -> { + if ((int) value < -1 || (int) value == 0) throw new ConfigException(name, value, "Value can be -1 or greater than 0"); + }, + () -> REMOTE_LOG_MANAGER_THREAD_POOL_FALLBACK + ); + public static final String REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP = "remote.log.manager.copier.thread.pool.size"; - public static final String REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in " + - "scheduling tasks to copy segments."; - public static final int DEFAULT_REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE = 10; + public static final String REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in scheduling tasks " + + "to copy segments. " + REMOTE_LOG_MANAGER_THREAD_POOL_FALLBACK; + public static final int DEFAULT_REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE = -1; public static final String REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP = "remote.log.manager.expiration.thread.pool.size"; - public static final String REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in" + - " scheduling tasks to clean up remote log segments."; - public static final int DEFAULT_REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE = 10; - + public static final String REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in scheduling tasks " + + "to clean up remote log segments. " + REMOTE_LOG_MANAGER_THREAD_POOL_FALLBACK; + public static final int DEFAULT_REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE = -1; + public static final String REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP = "remote.log.manager.task.interval.ms"; public static final String REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_DOC = "Interval at which remote log manager runs the scheduled tasks like copy " + "segments, and clean up remote log segments."; @@ -253,16 +263,16 @@ public static ConfigDef configDef() { atLeast(1), MEDIUM, REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_DOC) - .defineInternal(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP, + .define(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP, INT, DEFAULT_REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE, - atLeast(1), + REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_VALIDATOR, MEDIUM, REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_DOC) - .defineInternal(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP, + .define(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP, INT, DEFAULT_REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE, - atLeast(1), + REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_VALIDATOR, MEDIUM, REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_DOC) .define(REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, @@ -385,11 +395,13 @@ public int remoteLogManagerThreadPoolSize() { } public int remoteLogManagerCopierThreadPoolSize() { - return config.getInt(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP); + int size = config.getInt(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP); + return size == -1 ? remoteLogManagerThreadPoolSize() : size; } public int remoteLogManagerExpirationThreadPoolSize() { - return config.getInt(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP); + int size = config.getInt(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP); + return size == -1 ? remoteLogManagerThreadPoolSize() : size; } public long remoteLogManagerTaskIntervalMs() { diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfigTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfigTest.java index cb28f71a45..7ce0c46a51 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfigTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerConfigTest.java @@ -29,7 +29,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; public class RemoteLogManagerConfigTest { - @Test public void testValidConfigs() { String rsmPrefix = "__custom.rsm."; @@ -56,6 +55,16 @@ public void testDefaultConfigs() { assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM, remoteLogManagerConfigEmptyConfig.remoteLogManagerCopyNumQuotaSamples()); } + @Test + public void testThreadPoolDefaults() { + // Even with empty properties, RemoteLogManagerConfig has default values + Map emptyProps = new HashMap<>(); + RemoteLogManagerConfig remoteLogManagerConfigEmptyConfig = new RLMTestConfig(emptyProps).remoteLogManagerConfig(); + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_THREAD_POOL_SIZE, remoteLogManagerConfigEmptyConfig.remoteLogManagerThreadPoolSize()); + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_THREAD_POOL_SIZE, remoteLogManagerConfigEmptyConfig.remoteLogManagerCopierThreadPoolSize()); + assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_THREAD_POOL_SIZE, remoteLogManagerConfigEmptyConfig.remoteLogManagerExpirationThreadPoolSize()); + } + @Test public void testValidateEmptyStringConfig() { // Test with a empty string props should throw ConfigException @@ -65,7 +74,6 @@ public void testValidateEmptyStringConfig() { } private Map getRLMProps(String rsmPrefix, String rlmmPrefix) { - Map props = new HashMap<>(); props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, true); props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, @@ -108,7 +116,6 @@ private Map getRLMProps(String rsmPrefix, String rlmmPrefix) { } private static class RLMTestConfig extends AbstractConfig { - private final RemoteLogManagerConfig rlmConfig; public RLMTestConfig(Map originals) { @@ -120,4 +127,4 @@ public RemoteLogManagerConfig remoteLogManagerConfig() { return rlmConfig; } } -} \ No newline at end of file +} From c821449fb7c1e2c0f398eeb5cfc4a8abc210c0a4 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Mon, 21 Oct 2024 10:06:41 -0700 Subject: [PATCH 119/123] KAFKA-17794: Add some formatting safeguards for KIP-853 (#17504) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit KIP-853 adds support for dynamic KRaft quorums. This means that the quorum topology is no longer statically determined by the controller.quorum.voters configuration. Instead, it is contained in the storage directories of each controller and broker. Users of dynamic quorums must format at least one controller storage directory with either the --initial-controllers or --standalone flags. If they fail to do this, no quorum can be established. This PR changes the storage tool to warn about the case where a KIP-853 flag has not been supplied to format a KIP-853 controller. (Note that broker storage directories can continue to be formatted without a KIP-853 flag.) There are cases where we don't want to specify initial voters when formatting a controller. One example is where we format a single controller with --standalone, and then dynamically add 4 more controllers with no initial topology. In this case, we want the 4 later controllers to grab the quorum topology from the initial one. To support this case, this PR adds the --no-initial-controllers flag. Reviewers: José Armando García Sancio , Federico Valeri --- .../main/scala/kafka/tools/StorageTool.scala | 41 +++++++++---- .../unit/kafka/tools/StorageToolTest.scala | 61 +++++++++++++++++-- docs/ops.html | 4 +- .../kafka/metadata/storage/Formatter.java | 6 +- .../kafka/metadata/storage/FormatterTest.java | 6 +- tests/kafkatest/services/kafka/kafka.py | 9 ++- 6 files changed, 101 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 6dc4653961..6215e35d38 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package kafka.tools import kafka.server.KafkaConfig @@ -31,7 +30,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.metadata.storage.{Formatter, FormatterException} -import org.apache.kafka.raft.DynamicVoters +import org.apache.kafka.raft.{DynamicVoters, QuorumConfig} import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.config.ReplicationConfigs @@ -126,9 +125,20 @@ object StorageTool extends Logging { foreach(v => formatter.setReleaseVersion(MetadataVersion.fromVersionString(v.toString))) } Option(namespace.getString("initial_controllers")). - foreach(v => formatter.setInitialVoters(DynamicVoters.parse(v))) + foreach(v => formatter.setInitialControllers(DynamicVoters.parse(v))) if (namespace.getBoolean("standalone")) { - formatter.setInitialVoters(createStandaloneDynamicVoters(config)) + formatter.setInitialControllers(createStandaloneDynamicVoters(config)) + } + if (!namespace.getBoolean("no_initial_controllers")) { + if (config.processRoles.contains(ProcessRole.ControllerRole)) { + if (config.quorumVoters.isEmpty) { + if (!formatter.initialVoters().isPresent()) { + throw new TerseFailure("Because " + QuorumConfig.QUORUM_VOTERS_CONFIG + + " is not set on this controller, you must specify one of the following: " + + "--standalone, --initial-controllers, or --no-initial-controllers."); + } + } + } } Option(namespace.getList("add_scram")). foreach(scramArgs => formatter.setScramArguments(scramArgs.asInstanceOf[util.List[String]])) @@ -140,7 +150,7 @@ object StorageTool extends Logging { config: KafkaConfig ): DynamicVoters = { if (!config.processRoles.contains(ProcessRole.ControllerRole)) { - throw new TerseFailure("You cannot use --standalone on a broker node.") + throw new TerseFailure("You can only use --standalone on a controller.") } if (config.effectiveAdvertisedControllerListeners.isEmpty) { throw new RuntimeException("No controller listeners found.") @@ -191,13 +201,20 @@ object StorageTool extends Logging { help("The setting to use for a specific feature, in feature=level format. For example: `kraft.version=1`."). action(append()) val reconfigurableQuorumOptions = formatParser.addMutuallyExclusiveGroup() - reconfigurableQuorumOptions.addArgument("--standalone", "-s"). - help("Used to initialize a single-node quorum controller quorum."). - action(storeTrue()) - reconfigurableQuorumOptions.addArgument("--initial-controllers", "-I"). - help("The initial controllers, as a comma-separated list of id@hostname:port:directory. The same values must be used to format all nodes. For example:\n" + - "0@example.com:8082:JEXY6aqzQY-32P5TStzaFg,1@example.com:8083:MvDxzVmcRsaTz33bUuRU6A,2@example.com:8084:07R5amHmR32VDA6jHkGbTA\n"). - action(store()) + reconfigurableQuorumOptions.addArgument("--standalone", "-s") + .help("Used to initialize a controller as a single-node dynamic quorum.") + .action(storeTrue()) + + reconfigurableQuorumOptions.addArgument("--no-initial-controllers", "-N") + .help("Used to initialize a server without a dynamic quorum topology.") + .action(storeTrue()) + + reconfigurableQuorumOptions.addArgument("--initial-controllers", "-I") + .help("Used to initialize a server with a specific dynamic quorum topology. The argument " + + "is a comma-separated list of id@hostname:port:directory. The same values must be used to " + + "format all nodes. For example:\n0@example.com:8082:JEXY6aqzQY-32P5TStzaFg,1@example.com:8083:" + + "MvDxzVmcRsaTz33bUuRU6A,2@example.com:8084:07R5amHmR32VDA6jHkGbTA\n") + .action(store()) parser.parseArgs(args) } diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 7d8ac6860b..42ee07c50e 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -177,8 +177,9 @@ Found problem: defaultDynamicQuorumProperties.setProperty("process.roles", "controller") defaultDynamicQuorumProperties.setProperty("node.id", "0") defaultDynamicQuorumProperties.setProperty("controller.listener.names", "CONTROLLER") - defaultDynamicQuorumProperties.setProperty("controller.quorum.voters", "0@localhost:9093") - defaultDynamicQuorumProperties.setProperty("listeners", "CONTROLLER://127.0.0.1:9093") + defaultDynamicQuorumProperties.setProperty("controller.quorum.bootstrap.servers", "localhost:9093") + defaultDynamicQuorumProperties.setProperty("listeners", "CONTROLLER://:9093") + defaultDynamicQuorumProperties.setProperty("advertised.listeners", "CONTROLLER://127.0.0.1:9093") defaultDynamicQuorumProperties.setProperty(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") defaultDynamicQuorumProperties.setProperty(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG , "true") @@ -378,7 +379,7 @@ Found problem: properties.setProperty("log.dirs", availableDirs.mkString(",")) val stream = new ByteArrayOutputStream() val arguments = ListBuffer[String]("--release-version", "3.9-IV0", "--standalone") - assertEquals("You cannot use --standalone on a broker node.", + assertEquals("You can only use --standalone on a controller.", assertThrows(classOf[TerseFailure], () => runFormatCommand(stream, properties, arguments.toSeq)).getMessage) } @@ -437,11 +438,61 @@ Found problem: "Failed to find content in output: " + stream.toString()) } + @ParameterizedTest + @ValueSource(strings = Array("controller", "broker,controller")) + def testFormatWithoutStaticQuorumFailsWithoutInitialControllersOnController(processRoles: String): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultDynamicQuorumProperties) + if (processRoles.contains("broker")) { + properties.setProperty("listeners", "PLAINTEXT://:9092,CONTROLLER://:9093") + properties.setProperty("advertised.listeners", "PLAINTEXT://127.0.0.1:9092,CONTROLLER://127.0.0.1:9093") + } + properties.setProperty("process.roles", processRoles) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + assertEquals("Because controller.quorum.voters is not set on this controller, you must " + + "specify one of the following: --standalone, --initial-controllers, or " + + "--no-initial-controllers.", + assertThrows(classOf[TerseFailure], + () => runFormatCommand(new ByteArrayOutputStream(), properties, + Seq("--release-version", "3.9-IV0"))).getMessage) + } + @Test - def testBootstrapScramRecords(): Unit = { + def testFormatWithNoInitialControllersSucceedsOnController(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultDynamicQuorumProperties) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + val stream = new ByteArrayOutputStream() + assertEquals(0, runFormatCommand(stream, properties, + Seq("--no-initial-controllers", "--release-version", "3.9-IV0"))) + assertTrue(stream.toString(). + contains("Formatting metadata directory %s".format(availableDirs.head)), + "Failed to find content in output: " + stream.toString()) + } + + @Test + def testFormatWithoutStaticQuorumSucceedsWithoutInitialControllersOnBroker(): Unit = { val availableDirs = Seq(TestUtils.tempDir()) val properties = new Properties() properties.putAll(defaultDynamicQuorumProperties) + properties.setProperty("listeners", "PLAINTEXT://:9092") + properties.setProperty("advertised.listeners", "PLAINTEXT://127.0.0.1:9092") + properties.setProperty("process.roles", "broker") + properties.setProperty("log.dirs", availableDirs.mkString(",")) + val stream = new ByteArrayOutputStream() + assertEquals(0, runFormatCommand(stream, properties, Seq("--release-version", "3.9-IV0"))) + assertTrue(stream.toString(). + contains("Formatting metadata directory %s".format(availableDirs.head)), + "Failed to find content in output: " + stream.toString()) + } + + @Test + def testBootstrapScramRecords(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultStaticQuorumProperties) properties.setProperty("log.dirs", availableDirs.mkString(",")) val stream = new ByteArrayOutputStream() val arguments = ListBuffer[String]( @@ -468,7 +519,7 @@ Found problem: def testScramRecordsOldReleaseVersion(): Unit = { val availableDirs = Seq(TestUtils.tempDir()) val properties = new Properties() - properties.putAll(defaultDynamicQuorumProperties) + properties.putAll(defaultStaticQuorumProperties) properties.setProperty("log.dirs", availableDirs.mkString(",")) val stream = new ByteArrayOutputStream() val arguments = ListBuffer[String]( diff --git a/docs/ops.html b/docs/ops.html index bdf3109e6a..405137f7b7 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3818,9 +3818,9 @@

    In the replica description 0@controller-0:1234:3Db5QLSqSZieL3rJBUUegA, 0 is the replica id, 3Db5QLSqSZieL3rJBUUegA is the replica directory id, controller-0 is the replica's host and 1234 is the replica's port.
    Formatting Brokers and New Controllers
    - When provisioning new broker and controller nodes that we want to add to an existing Kafka cluster, use the kafka-storage.sh format command without the --standalone or --initial-controllers flags. + When provisioning new broker and controller nodes that we want to add to an existing Kafka cluster, use the kafka-storage.sh format command with the --no-initial-controllers flag. -
    $ bin/kafka-storage format --cluster-id <cluster-id> --config server.properties
    +
    $ bin/kafka-storage.sh format --cluster-id <cluster-id> --config server.properties --no-initial-controllers

    Controller membership changes

    diff --git a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java index 72995fb753..847285c744 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java @@ -202,11 +202,15 @@ public Formatter setMetadataLogDirectory(String metadataLogDirectory) { return this; } - public Formatter setInitialVoters(DynamicVoters initialControllers) { + public Formatter setInitialControllers(DynamicVoters initialControllers) { this.initialControllers = Optional.of(initialControllers); return this; } + public Optional initialVoters() { + return initialControllers; + } + boolean hasDynamicQuorum() { if (initialControllers.isPresent()) { return true; diff --git a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java index 6dc6aa7e66..c5c32484ad 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java @@ -372,7 +372,7 @@ public void testFormatWithInitialVoters(boolean specifyKRaftVersion) throws Exce formatter1.formatter.setFeatureLevel("kraft.version", (short) 1); } formatter1.formatter.setUnstableFeatureVersionsEnabled(true); - formatter1.formatter.setInitialVoters(DynamicVoters. + formatter1.formatter.setInitialControllers(DynamicVoters. parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); formatter1.formatter.run(); assertEquals(Arrays.asList( @@ -403,7 +403,7 @@ public void testFormatWithInitialVotersFailsWithOlderKraftVersion() throws Excep FormatterContext formatter1 = testEnv.newFormatter(); formatter1.formatter.setFeatureLevel("kraft.version", (short) 0); formatter1.formatter.setUnstableFeatureVersionsEnabled(true); - formatter1.formatter.setInitialVoters(DynamicVoters. + formatter1.formatter.setInitialControllers(DynamicVoters. parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); assertTrue(formatter1.formatter.hasDynamicQuorum()); assertEquals("Cannot set kraft.version to 0 if KIP-853 configuration is present. " + @@ -433,7 +433,7 @@ public void testFormatWithInitialVotersFailsWithOlderMetadataVersion() throws Ex FormatterContext formatter1 = testEnv.newFormatter(); formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_8_IV0); formatter1.formatter.setFeatureLevel("kraft.version", (short) 1); - formatter1.formatter.setInitialVoters(DynamicVoters. + formatter1.formatter.setInitialControllers(DynamicVoters. parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); formatter1.formatter.setUnstableFeatureVersionsEnabled(true); assertEquals("kraft.version could not be set to 1 because it depends on " + diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 51eab79db9..a74667cc83 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -914,9 +914,12 @@ def start_node(self, node, timeout_sec=60, **kwargs): cmd = "%s format --ignore-formatted --config %s --cluster-id %s" % (kafka_storage_script, KafkaService.CONFIG_FILE, config_property.CLUSTER_ID) if self.dynamicRaftQuorum: cmd += " --feature kraft.version=1" - if not self.standalone_controller_bootstrapped and self.node_quorum_info.has_controller_role: - cmd += " --standalone" - self.standalone_controller_bootstrapped = True + if self.node_quorum_info.has_controller_role: + if self.standalone_controller_bootstrapped: + cmd += " --no-initial-controllers" + else: + cmd += " --standalone" + self.standalone_controller_bootstrapped = True self.logger.info("Running log directory format command...\n%s" % cmd) node.account.ssh(cmd) From 398b4c4fa1a5678be28a3fb9196ba4553f356291 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Fri, 25 Oct 2024 12:33:22 -0700 Subject: [PATCH 120/123] KAFKA-17868: Do not ignore --feature flag in kafka-storage.sh (#17597) Reviewers: Chia-Ping Tsai , Justine Olshan --- .../main/scala/kafka/tools/StorageTool.scala | 28 +++++++++++ .../unit/kafka/tools/StorageToolTest.scala | 46 +++++++++++++++++++ 2 files changed, 74 insertions(+) diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 6215e35d38..0ce6471023 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -124,6 +124,10 @@ object StorageTool extends Logging { case None => Option(config.originals.get(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)). foreach(v => formatter.setReleaseVersion(MetadataVersion.fromVersionString(v.toString))) } + Option(namespace.getList[String]("feature")).foreach( + featureNamesAndLevels(_).foreach { + kv => formatter.setFeatureLevel(kv._1, kv._2) + }) Option(namespace.getString("initial_controllers")). foreach(v => formatter.setInitialControllers(DynamicVoters.parse(v))) if (namespace.getBoolean("standalone")) { @@ -311,4 +315,28 @@ object StorageTool extends Logging { } } } + + def parseNameAndLevel(input: String): (String, java.lang.Short) = { + val equalsIndex = input.indexOf("=") + if (equalsIndex < 0) + throw new RuntimeException("Can't parse feature=level string " + input + ": equals sign not found.") + val name = input.substring(0, equalsIndex).trim + val levelString = input.substring(equalsIndex + 1).trim + try { + (name, levelString.toShort) + } catch { + case _: Throwable => + throw new RuntimeException("Can't parse feature=level string " + input + ": " + "unable to parse " + levelString + " as a short.") + } + } + + def featureNamesAndLevels(features: java.util.List[String]): Map[String, java.lang.Short] = { + val scalaFeatures = new mutable.ArrayBuffer[String] + features.forEach(scalaFeatures += _) + scalaFeatures.map { (feature: String) => + // Ensure the feature exists + val nameAndLevel = parseNameAndLevel(feature) + (nameAndLevel._1, nameAndLevel._2) + }.toMap + } } diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 42ee07c50e..139d752c77 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -329,6 +329,31 @@ Found problem: "Failed to find content in output: " + stream.toString()) } + @Test + def testFormatWithInvalidFeature(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultStaticQuorumProperties) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + assertEquals("Unsupported feature: non.existent.feature. Supported features are: " + + "group.version, kraft.version, transaction.version", + assertThrows(classOf[FormatterException], () => + runFormatCommand(new ByteArrayOutputStream(), properties, + Seq("--feature", "non.existent.feature=20"))).getMessage) + } + + @Test + def testFormatWithInvalidKRaftVersionLevel(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultDynamicQuorumProperties) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + assertEquals("No feature:kraft.version with feature level 999", + assertThrows(classOf[IllegalArgumentException], () => + runFormatCommand(new ByteArrayOutputStream(), properties, + Seq("--feature", "kraft.version=999", "--standalone"))).getMessage) + } + @Test def testFormatWithReleaseVersionAndKRaftVersion(): Unit = { val availableDirs = Seq(TestUtils.tempDir()) @@ -532,4 +557,25 @@ Found problem: "SCRAM is only supported in metadata.version 3.5-IV2 or later.", assertThrows(classOf[FormatterException], () => runFormatCommand(stream, properties, arguments.toSeq)).getMessage) } + + @Test + def testParseNameAndLevel(): Unit = { + assertEquals(("foo.bar", 56.toShort), StorageTool.parseNameAndLevel("foo.bar=56")) + } + + @Test + def testParseNameAndLevelWithNoEquals(): Unit = { + assertEquals("Can't parse feature=level string kraft.version5: equals sign not found.", + assertThrows(classOf[RuntimeException], + () => StorageTool.parseNameAndLevel("kraft.version5")). + getMessage) + } + + @Test + def testParseNameAndLevelWithNoNumber(): Unit = { + assertEquals("Can't parse feature=level string kraft.version=foo: unable to parse foo as a short.", + assertThrows(classOf[RuntimeException], + () => StorageTool.parseNameAndLevel("kraft.version=foo")). + getMessage) + } } From 2854533f42cf105501af1bc4cac40c70c9ffed27 Mon Sep 17 00:00:00 2001 From: Robin Han Date: Thu, 31 Oct 2024 12:02:17 +0800 Subject: [PATCH 121/123] fix(merge): fix compile error Signed-off-by: Robin Han --- .../main/java/kafka/automq/StorageUtil.java | 2 +- .../server/BrokerTopicPartitionMetrics.java | 26 +++++++++++++++++++ .../kafka/server/KafkaRequestHandler.scala | 12 ++++----- .../main/scala/kafka/tools/StorageTool.scala | 24 +++++------------ .../unit/kafka/server/KafkaApisTest.scala | 6 ++++- .../controller/ClusterControlManager.java | 4 ++- .../kafka/controller/QuorumController.java | 2 +- 7 files changed, 48 insertions(+), 28 deletions(-) create mode 100644 core/src/main/scala/kafka/server/BrokerTopicPartitionMetrics.java diff --git a/core/src/main/java/kafka/automq/StorageUtil.java b/core/src/main/java/kafka/automq/StorageUtil.java index dec89ff40e..02a14f924c 100644 --- a/core/src/main/java/kafka/automq/StorageUtil.java +++ b/core/src/main/java/kafka/automq/StorageUtil.java @@ -36,7 +36,7 @@ public static void formatStorage(Properties serverProps) { String metadataLog = new KafkaConfig(serverProps, false).metadataLogDir(); if (!Files.exists(Paths.get(metadataLog, "meta.properties"))) { String configFilePath = persistConfig(serverProps, metadataLog); - StorageTool.execute(new String[] {"format", "-t", clusterId, "-c=" + configFilePath}); + StorageTool.execute(new String[] {"format", "-t", clusterId, "-c=" + configFilePath}, System.out); } else { persistConfig(serverProps, metadataLog); } diff --git a/core/src/main/scala/kafka/server/BrokerTopicPartitionMetrics.java b/core/src/main/scala/kafka/server/BrokerTopicPartitionMetrics.java new file mode 100644 index 0000000000..5b711d5b51 --- /dev/null +++ b/core/src/main/scala/kafka/server/BrokerTopicPartitionMetrics.java @@ -0,0 +1,26 @@ +/* + * Copyright 2024, AutoMQ HK Limited. + * + * The use of this file is governed by the Business Source License, + * as detailed in the file "/LICENSE.S3Stream" included in this repository. + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ + +package kafka.server; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics; + +import java.util.Map; + +public class BrokerTopicPartitionMetrics extends BrokerTopicMetrics { + final Map tags; + + public BrokerTopicPartitionMetrics(TopicPartition topicPartition) { + super(topicPartition.topic(), false); + this.tags = Map.of("topic", topicPartition.topic(), "partition", String.valueOf(topicPartition.partition())); + } +} diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 15fa0eec20..41338e6418 100755 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -32,7 +32,6 @@ import org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics -import java.util.Collections import scala.collection.Set import scala.collection.mutable @@ -259,6 +258,11 @@ class BrokerTopicStats(remoteStorageEnabled: Boolean = false) extends Logging { private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory)) val allTopicsStats = new BrokerTopicMetrics(remoteStorageEnabled) + // AutoMQ inject start + private val partitionValueFactory = (k: TopicPartition) => new BrokerTopicPartitionMetrics(k) + private val partitionStats = new Pool[TopicPartition, BrokerTopicPartitionMetrics](Some(partitionValueFactory)) + // AutoMQ inject end + def isTopicStatsExisted(topic: String): Boolean = stats.contains(topic) @@ -512,9 +516,3 @@ class BrokerTopicStats(remoteStorageEnabled: Boolean = false) extends Logging { info("Broker and topic stats closed") } } - -// AutoMQ inject start -class BrokerTopicPartitionMetrics(tp: TopicPartition) extends BrokerTopicMetrics(Some(tp.topic()), false) { - override lazy val tags: java.util.Map[String, String] = Map("topic" -> tp.topic(), "partition" -> tp.partition().toString).asJava -} -// AutoMQ inject end diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 90e962d6a8..a27270d4fe 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -31,7 +31,6 @@ import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.metadata.storage.{Formatter, FormatterException} import org.apache.kafka.raft.{DynamicVoters, QuorumConfig} -import org.apache.kafka.server.common.automq.AutoMQVersion import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.config.ReplicationConfigs @@ -63,8 +62,8 @@ object StorageTool extends Logging { * @return The exit code */ def execute( - args: Array[String], - printStream: PrintStream + args: Array[String], + printStream: PrintStream ): Int = { val namespace = try { parseArguments(args) @@ -170,9 +169,9 @@ object StorageTool extends Logging { } def parseArguments(args: Array[String]): Namespace = { - val parser = ArgumentParsers - .newArgumentParser("kafka-storage", true, "-", "@") - .description("The Kafka storage tool.") + val parser = ArgumentParsers. + newArgumentParser("kafka-storage", /* defaultHelp */ true, /* prefixChars */ "-", /* fromFilePrefix */ "@"). + description("The Kafka storage tool.") val subparsers = parser.addSubparsers().dest("command") @@ -180,22 +179,13 @@ object StorageTool extends Logging { help("Get information about the Kafka log directories on this node.") val formatParser = subparsers.addParser("format"). help("Format the Kafka log directories on this node.") - val autoFormatParser = subparsers.addParser("auto-format"). - help("Auto format the Kafka log directories on this node. ") subparsers.addParser("random-uuid").help("Print a random UUID.") - List(infoParser, formatParser, autoFormatParser).foreach(parser => { + List(infoParser, formatParser).foreach(parser => { parser.addArgument("--config", "-c"). action(store()). required(true). help("The Kafka configuration file to use.") }) - configureFormatParser(formatParser) - configureFormatParser(autoFormatParser) - - parser.parseArgsOrFail(args) - } - - private def configureFormatParser(formatParser: Subparser): Unit = { formatParser.addArgument("--cluster-id", "-t"). action(store()). required(true). @@ -349,4 +339,4 @@ object StorageTool extends Logging { (nameAndLevel._1, nameAndLevel._2) }.toMap } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 1df9cebca9..f9df9778cb 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -130,7 +130,11 @@ class KafkaApisTest extends Logging { protected val clientRequestQuotaManager: ClientRequestQuotaManager = mock(classOf[ClientRequestQuotaManager]) protected val clientControllerQuotaManager: ControllerMutationQuotaManager = mock(classOf[ControllerMutationQuotaManager]) protected val replicaQuotaManager: ReplicationQuotaManager = mock(classOf[ReplicationQuotaManager]) - protected val quotas = QuotaManagers(clientQuotaManager, clientQuotaManager, clientRequestQuotaManager, + // AutoMQ inject start + protected val brokerQuotaManager: BrokerQuotaManager = mock(classOf[BrokerQuotaManager]) + // AutoMQ inject end + + protected val quotas = QuotaManagers(clientQuotaManager, clientQuotaManager, clientRequestQuotaManager, brokerQuotaManager, clientControllerQuotaManager, replicaQuotaManager, replicaQuotaManager, replicaQuotaManager, None) protected val fetchManager: FetchManager = mock(classOf[FetchManager]) protected val clientMetricsManager: ClientMetricsManager = mock(classOf[ClientMetricsManager]) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 128b5a9840..2078201725 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -185,7 +185,9 @@ ClusterControlManager build() { featureControl, zkMigrationEnabled, brokerUncleanShutdownHandler, - quorumVoters + // AutoMQ inject start + quorumVoters, + // AutoMQ inject end interBrokerListenerName ); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index a9265856f9..b978fdea7f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -2110,7 +2110,7 @@ private QuorumController( // AutoMQ inject start StreamClient streamClient, List quorumVoters, - Function extension + Function extension, // AutoMQ inject end long uncleanLeaderElectionCheckIntervalMs, From 7e759baf40eff20c47434e7078a298e883a473b2 Mon Sep 17 00:00:00 2001 From: Robin Han Date: Thu, 31 Oct 2024 16:04:47 +0800 Subject: [PATCH 122/123] fix(merge): fix automq version Signed-off-by: Robin Han --- .../apache/kafka/server/common/Features.java | 6 ++++ .../server/common/automq/AutoMQVersion.java | 30 +++++++++++++++---- 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/server-common/src/main/java/org/apache/kafka/server/common/Features.java b/server-common/src/main/java/org/apache/kafka/server/common/Features.java index 15269e5183..adc27e1ba2 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/Features.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/Features.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.kafka.server.common.automq.AutoMQVersion; /** * This is enum for the various features implemented for Kafka clusters. @@ -41,6 +42,11 @@ public enum Features { * * See {@link TestFeatureVersion} as an example. See {@link FeatureVersion} when implementing a new feature. */ + + // AutoMQ inject start + AUTOMQ_VERSION(AutoMQVersion.FEATURE_NAME, AutoMQVersion.values()), + // AutoMQ inject end + TEST_VERSION("test.feature.version", TestFeatureVersion.values()), KRAFT_VERSION("kraft.version", KRaftVersion.values()), TRANSACTION_VERSION("transaction.version", TransactionVersion.values()); diff --git a/server-common/src/main/java/org/apache/kafka/server/common/automq/AutoMQVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/automq/AutoMQVersion.java index 3f0c01d3cf..fa1d0c513f 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/automq/AutoMQVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/automq/AutoMQVersion.java @@ -12,28 +12,33 @@ package org.apache.kafka.server.common.automq; import com.automq.stream.Version; +import java.util.Map; +import org.apache.kafka.server.common.FeatureVersion; +import org.apache.kafka.server.common.MetadataVersion; -public enum AutoMQVersion { +public enum AutoMQVersion implements FeatureVersion { - V0((short) 1), + V0((short) 1, MetadataVersion.IBP_3_4_IV0), // Support reassignment v1: elect leader after partition open in the new broker // Support stream tags - V1((short) 2), + V1((short) 2, MetadataVersion.IBP_3_7_IV0), // Support composite object // Support object bucket index // Support huge cluster // Support node registration - V2((short) 3); + V2((short) 3, MetadataVersion.IBP_3_8_IV0); public static final String FEATURE_NAME = "automq.version"; public static final AutoMQVersion LATEST = V2; private final short level; private final Version s3streamVersion; + private final MetadataVersion metadataVersion; - AutoMQVersion(short level) { + AutoMQVersion(short level, MetadataVersion metadataVersion) { this.level = level; s3streamVersion = mapS3StreamVersion(level); + this.metadataVersion = metadataVersion; } public static AutoMQVersion from(short level) { @@ -53,6 +58,21 @@ public short featureLevel() { return level; } + @Override + public String featureName() { + return FEATURE_NAME; + } + + @Override + public MetadataVersion bootstrapMetadataVersion() { + return metadataVersion; + } + + @Override + public Map dependencies() { + return Map.of(); + } + public boolean isReassignmentV1Supported() { return isAtLeast(V1); } From 0e84ac7de200a32793e7f4ced76557629c95f677 Mon Sep 17 00:00:00 2001 From: Robin Han Date: Thu, 31 Oct 2024 17:03:23 +0800 Subject: [PATCH 123/123] fix(merge): fix unit test Signed-off-by: Robin Han --- .../server/BrokerTopicPartitionMetrics.java | 6 ++---- .../log/streamaspect/ElasticUnifiedLogTest.scala | 14 ++++++++++++++ .../org/apache/kafka/server/common/Features.java | 2 +- .../server/common/automq/AutoMQVersion.java | 6 ++++-- .../storage/log/metrics/BrokerTopicMetrics.java | 16 +++++++++++++++- 5 files changed, 36 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerTopicPartitionMetrics.java b/core/src/main/scala/kafka/server/BrokerTopicPartitionMetrics.java index 5b711d5b51..888c641c4a 100644 --- a/core/src/main/scala/kafka/server/BrokerTopicPartitionMetrics.java +++ b/core/src/main/scala/kafka/server/BrokerTopicPartitionMetrics.java @@ -15,12 +15,10 @@ import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics; import java.util.Map; +import java.util.Optional; public class BrokerTopicPartitionMetrics extends BrokerTopicMetrics { - final Map tags; - public BrokerTopicPartitionMetrics(TopicPartition topicPartition) { - super(topicPartition.topic(), false); - this.tags = Map.of("topic", topicPartition.topic(), "partition", String.valueOf(topicPartition.partition())); + super(Optional.of(topicPartition.topic()), Map.of("partition", String.valueOf(topicPartition.partition())), false); } } diff --git a/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala b/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala index 1da0985897..ddf7747fcc 100644 --- a/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala +++ b/core/src/test/scala/kafka/log/streamaspect/ElasticUnifiedLogTest.scala @@ -584,4 +584,18 @@ class ElasticUnifiedLogTest extends UnifiedLogTest { override def testConvertToOffsetMetadataDoesNotThrowOffsetOutOfRangeError(): Unit = { // AutoMQ embedded tiered storage in S3Stream } + + override def testRetentionOnLocalLogDeletionWhenRemoteLogCopyEnabledAndDefaultLocalRetentionBytes(): Unit = { + // AutoMQ embedded tiered storage in S3Stream + } + + + override def testRetentionOnLocalLogDeletionWhenRemoteLogCopyDisabled(): Unit = { + // AutoMQ embedded tiered storage in S3Stream + } + + override def testRetentionOnLocalLogDeletionWhenRemoteLogCopyEnabledAndDefaultLocalRetentionMs(): Unit = { + // AutoMQ embedded tiered storage in S3Stream + } + } diff --git a/server-common/src/main/java/org/apache/kafka/server/common/Features.java b/server-common/src/main/java/org/apache/kafka/server/common/Features.java index adc27e1ba2..35be50ff90 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/Features.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/Features.java @@ -17,13 +17,13 @@ package org.apache.kafka.server.common; import org.apache.kafka.common.feature.SupportedVersionRange; +import org.apache.kafka.server.common.automq.AutoMQVersion; import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.apache.kafka.server.common.automq.AutoMQVersion; /** * This is enum for the various features implemented for Kafka clusters. diff --git a/server-common/src/main/java/org/apache/kafka/server/common/automq/AutoMQVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/automq/AutoMQVersion.java index fa1d0c513f..2a411bace2 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/automq/AutoMQVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/automq/AutoMQVersion.java @@ -11,11 +11,13 @@ package org.apache.kafka.server.common.automq; -import com.automq.stream.Version; -import java.util.Map; import org.apache.kafka.server.common.FeatureVersion; import org.apache.kafka.server.common.MetadataVersion; +import com.automq.stream.Version; + +import java.util.Map; + public enum AutoMQVersion implements FeatureVersion { V0((short) 1, MetadataVersion.IBP_3_4_IV0), diff --git a/storage/src/main/java/org/apache/kafka/storage/log/metrics/BrokerTopicMetrics.java b/storage/src/main/java/org/apache/kafka/storage/log/metrics/BrokerTopicMetrics.java index c638beb1e9..a6b876467b 100644 --- a/storage/src/main/java/org/apache/kafka/storage/log/metrics/BrokerTopicMetrics.java +++ b/storage/src/main/java/org/apache/kafka/storage/log/metrics/BrokerTopicMetrics.java @@ -22,6 +22,7 @@ import com.yammer.metrics.core.Meter; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -66,8 +67,21 @@ public BrokerTopicMetrics(String name, boolean remoteStorageEnabled) { this(Optional.of(name), remoteStorageEnabled); } + // AutoMQ inject start private BrokerTopicMetrics(Optional name, boolean remoteStorageEnabled) { - this.tags = name.map(s -> Collections.singletonMap("topic", s)).orElse(Collections.emptyMap()); + this(name, Collections.emptyMap(), remoteStorageEnabled); + } + + protected BrokerTopicMetrics(Optional name, Map tagsAddon, boolean remoteStorageEnabled) { + Map newTags = Collections.emptyMap(); + if (!(tagsAddon.isEmpty() && name.isEmpty())) { + newTags = new HashMap<>(tagsAddon); + if (name.isPresent()) { + newTags.put("topic", name.get()); + } + } + this.tags = newTags; + // AutoMQ inject start metricTypeMap.put(MESSAGE_IN_PER_SEC, new MeterWrapper(MESSAGE_IN_PER_SEC, "messages")); metricTypeMap.put(BYTES_IN_PER_SEC, new MeterWrapper(BYTES_IN_PER_SEC, "bytes"));