From 9cfac392a74c698f607c043bd8eca9455b7bfbee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Mon, 27 May 2024 15:21:38 +0200 Subject: [PATCH 01/11] KAFKA-16448: Add ProcessingExceptionHandler interface and implementations Co-authored-by: Dabz Co-authored-by: sebastienviale --- .../streams/errors/ErrorHandlerContext.java | 133 ++++++++++++++++++ .../errors/ErrorHandlerContextImpl.java | 90 ++++++++++++ .../errors/ProcessingExceptionHandler.java | 56 ++++++++ ...cessingLogAndContinueExceptionHandler.java | 46 ++++++ .../ProcessingLogAndFailExceptionHandler.java | 42 ++++++ 5 files changed, 367 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java 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 new file mode 100644 index 0000000000000..5f28b711f2e1c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java @@ -0,0 +1,133 @@ +/* + * 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.common.header.Headers; +import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; +import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.TaskId; + +/** + * This interface allows user code to inspect the context of a record that has failed processing. + */ +public interface ErrorHandlerContext { + /** + * Return the topic name of the current input record; could be {@code null} if it is not + * available. + * + *

For example, 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, the record won't have an associated topic. + * Another example is + * {@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. + * + * @return the topic name + */ + String topic(); + + /** + * Return the partition id of the current input record; could be {@code -1} if it is not + * available. + * + *

For example, 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, the record won't have an associated partition id. + * Another example is + * {@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. + * + * @return the partition id + */ + int partition(); + + /** + * Return the offset of the current input record; could be {@code -1} if it is not + * available. + * + *

For example, 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, the record won't have an associated offset. + * Another example is + * {@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. + * + * @return the offset + */ + long offset(); + + /** + * Return the headers of the current source record; could be an empty header if it is not + * available. + * + *

For example, 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, the record might not have any associated headers. + * Another example is + * {@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. + * + * @return the headers + */ + 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 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 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. + * + * @return the processor node id + */ + String processorNodeId(); + + /** + * Return the task id. + * + * @return the task id + */ + TaskId taskId(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java new file mode 100644 index 0000000000000..c801799a6cdad --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java @@ -0,0 +1,90 @@ +/* + * 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.common.header.Headers; +import org.apache.kafka.streams.processor.TaskId; + +public class ErrorHandlerContextImpl implements ErrorHandlerContext { + private final String topic; + 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; + + public ErrorHandlerContextImpl(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; + } + + + @Override + public String topic() { + return this.topic; + } + + @Override + public int partition() { + return this.partition; + } + + @Override + public long offset() { + return this.offset; + } + + @Override + public Headers headers() { + return this.headers; + } + + @Override + public byte[] sourceRawKey() { + return this.sourceRawKey; + } + + @Override + public byte[] sourceRawValue() { + return this.sourceRawValue; + } + + @Override + public String processorNodeId() { + return this.processorNodeId; + } + + @Override + public TaskId taskId() { + return this.taskId; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java new file mode 100644 index 0000000000000..7cbc450b0cf1b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -0,0 +1,56 @@ +/* + * 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.common.Configurable; +import org.apache.kafka.streams.processor.api.Record; + +/** + * An interface that allows user code to inspect a record that has failed processing + */ +public interface ProcessingExceptionHandler extends Configurable { + /** + * Inspect a record and the exception received + * + * @param context processing context metadata + * @param record record where the exception occurred + * @param exception the actual exception + */ + ProcessingHandlerResponse handle(ErrorHandlerContext context, Record record, Exception exception); + + enum ProcessingHandlerResponse { + /* continue with processing */ + CONTINUE(1, "CONTINUE"), + /* fail the processing and stop */ + FAIL(2, "FAIL"); + + /** + * the permanent and immutable name of processing exception response + */ + public final String name; + + /** + * the permanent and immutable id of processing exception response + */ + public final int id; + + ProcessingHandlerResponse(final int id, final String name) { + this.id = id; + this.name = name; + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java new file mode 100644 index 0000000000000..3d12814cc3e85 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java @@ -0,0 +1,46 @@ +/* + * 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.streams.processor.api.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * Processing exception handler that logs a processing exception and then + * signals the processing pipeline to continue processing more records. + */ +public class ProcessingLogAndContinueExceptionHandler implements ProcessingExceptionHandler { + private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndContinueExceptionHandler.class); + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + log.warn("Exception caught during message processing, " + + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", + context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), + exception); + + return ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // ignore + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java new file mode 100644 index 0000000000000..42d4a45b46c36 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java @@ -0,0 +1,42 @@ +/* + * 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.streams.processor.api.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +public class ProcessingLogAndFailExceptionHandler implements ProcessingExceptionHandler { + private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndFailExceptionHandler.class); + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + log.warn("Exception caught during message processing, " + + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", + context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), + exception); + + return ProcessingHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + // ignore + } +} From b957861a3628171f89cb4131f8c7e2c8163e2e7c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Mon, 27 May 2024 17:09:41 +0200 Subject: [PATCH 02/11] KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration Co-authored-by: Dabz Co-authored-by: sebastienviale --- .../org/apache/kafka/streams/StreamsConfig.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) 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 70e4cba9625ac..a2200b89098b7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -39,6 +39,8 @@ import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingLogAndFailExceptionHandler; import org.apache.kafka.streams.errors.ProductionExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.internals.StreamsConfigUtils; @@ -553,6 +555,11 @@ public class StreamsConfig extends AbstractConfig { public static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.production.exception.handler"; private static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProductionExceptionHandler interface."; + /** {@code processing.exception.handler} */ + @SuppressWarnings("WeakerAccess") + public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG = "processing.exception.handler"; + public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProcessingExceptionHandler interface."; + /** {@code default.dsl.store} */ @Deprecated @SuppressWarnings("WeakerAccess") @@ -926,6 +933,11 @@ public class StreamsConfig extends AbstractConfig { DefaultProductionExceptionHandler.class.getName(), Importance.MEDIUM, DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) + .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + ProcessingLogAndFailExceptionHandler.class.getName(), + Importance.MEDIUM, + PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, Type.CLASS, FailOnInvalidTimestamp.class.getName(), @@ -1915,6 +1927,11 @@ public ProductionExceptionHandler defaultProductionExceptionHandler() { return getConfiguredInstance(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, ProductionExceptionHandler.class); } + @SuppressWarnings("WeakerAccess") + public ProcessingExceptionHandler processingExceptionHandler() { + return getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); + } + /** * Override any client properties in the original configs with overrides * From 9eef86dea2a35cab2a4b518bec99360c13fcb034 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Mon, 27 May 2024 15:21:38 +0200 Subject: [PATCH 03/11] KAFKA-16448: Add ProcessingExceptionHandler interface and implementations Co-authored-by: Dabz Co-authored-by: sebastienviale --- .../streams/errors/ErrorHandlerContext.java | 133 ++++++++++++++++++ ...AndContinueProcessingExceptionHandler.java | 46 ++++++ .../LogAndFailProcessingExceptionHandler.java | 46 ++++++ .../errors/ProcessingExceptionHandler.java | 56 ++++++++ .../internals/DefaultErrorHandlerContext.java | 90 ++++++++++++ 5 files changed, 371 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java 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 new file mode 100644 index 0000000000000..0c50547549027 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java @@ -0,0 +1,133 @@ +/* + * 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.common.header.Headers; +import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; +import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.TaskId; + +/** + * This interface allows user code to inspect the context of a record that has failed processing. + */ +public interface ErrorHandlerContext { + /** + * Return the topic name of the current input record; could be {@code null} if it is not + * available. + * + *

For example, 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, the record won't have an associated topic. + * Another example is + * {@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. + * + * @return the topic name + */ + String topic(); + + /** + * Return the partition ID of the current input record; could be {@code -1} if it is not + * available. + * + *

For example, 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, the record won't have an associated partition ID. + * Another example is + * {@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. + * + * @return the partition ID + */ + int partition(); + + /** + * Return the offset of the current input record; could be {@code -1} if it is not + * available. + * + *

For example, 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, the record won't have an associated offset. + * Another example is + * {@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. + * + * @return the offset + */ + long offset(); + + /** + * Return the headers of the current source record; could be an empty header if it is not + * available. + * + *

For example, 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, the record might not have any associated headers. + * Another example is + * {@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. + * + * @return the headers + */ + 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. + * + * @return the processor node ID + */ + String processorNodeId(); + + /** + * Return the task ID. + * + * @return the task ID + */ + TaskId taskId(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java new file mode 100644 index 0000000000000..bcfe9e37cedbf --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -0,0 +1,46 @@ +/* + * 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.streams.processor.api.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * Processing exception handler that logs a processing exception and then + * signals the processing pipeline to continue processing more records. + */ +public class LogAndContinueProcessingExceptionHandler implements ProcessingExceptionHandler { + private static final Logger log = LoggerFactory.getLogger(LogAndContinueProcessingExceptionHandler.class); + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + log.warn("Exception caught during message processing, " + + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", + context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), + exception); + + return ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // ignore + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java new file mode 100644 index 0000000000000..d271199c099c0 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -0,0 +1,46 @@ +/* + * 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.streams.processor.api.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * Processing exception handler that logs a processing exception and then + * signals the processing pipeline to stop processing more records and fail. + */ +public class LogAndFailProcessingExceptionHandler implements ProcessingExceptionHandler { + private static final Logger log = LoggerFactory.getLogger(LogAndFailProcessingExceptionHandler.class); + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + log.warn("Exception caught during message processing, " + + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", + context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), + exception); + + return ProcessingHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + // ignore + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java new file mode 100644 index 0000000000000..33b2596be1227 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -0,0 +1,56 @@ +/* + * 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.common.Configurable; +import org.apache.kafka.streams.processor.api.Record; + +/** + * An interface that allows user code to inspect a record that has failed processing + */ +public interface ProcessingExceptionHandler extends Configurable { + /** + * Inspect a record and the exception received + * + * @param context processing context metadata + * @param record record where the exception occurred + * @param exception the actual exception + */ + ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception); + + enum ProcessingHandlerResponse { + /* continue with processing */ + CONTINUE(1, "CONTINUE"), + /* fail the processing and stop */ + FAIL(2, "FAIL"); + + /** + * the permanent and immutable name of processing exception response + */ + public final String name; + + /** + * the permanent and immutable id of processing exception response + */ + public final int id; + + ProcessingHandlerResponse(final int id, final String name) { + this.id = id; + this.name = name; + } + } +} 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 new file mode 100644 index 0000000000000..4723f247db603 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java @@ -0,0 +1,90 @@ +/* + * 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.internals; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.streams.errors.ErrorHandlerContext; +import org.apache.kafka.streams.processor.TaskId; + +public class DefaultErrorHandlerContext implements ErrorHandlerContext { + private final String topic; + 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; + + 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; + } + + @Override + public String topic() { + return this.topic; + } + + @Override + public int partition() { + return this.partition; + } + + @Override + public long offset() { + return this.offset; + } + + @Override + public Headers headers() { + return this.headers; + } + + @Override + public byte[] sourceRawKey() { + return this.sourceRawKey; + } + + @Override + public byte[] sourceRawValue() { + return this.sourceRawValue; + } + + @Override + public String processorNodeId() { + return this.processorNodeId; + } + + @Override + public TaskId taskId() { + return this.taskId; + } +} From 3cb797b73a83e430ac6454852543058e6ef2dd2e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Tue, 28 May 2024 19:28:41 +0200 Subject: [PATCH 04/11] KAFKA-16448: Fix indentation Co-authored-by: Dabz Co-authored-by: sebastienviale --- .../java/org/apache/kafka/streams/StreamsConfig.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) 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 a2200b89098b7..b6a736ec5b7fd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -934,10 +934,10 @@ public class StreamsConfig extends AbstractConfig { Importance.MEDIUM, DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - ProcessingLogAndFailExceptionHandler.class.getName(), - Importance.MEDIUM, - PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) + Type.CLASS, + ProcessingLogAndFailExceptionHandler.class.getName(), + Importance.MEDIUM, + PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, Type.CLASS, FailOnInvalidTimestamp.class.getName(), @@ -1927,7 +1927,6 @@ public ProductionExceptionHandler defaultProductionExceptionHandler() { return getConfiguredInstance(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, ProductionExceptionHandler.class); } - @SuppressWarnings("WeakerAccess") public ProcessingExceptionHandler processingExceptionHandler() { return getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); } From c1c0d5c56ca64c5b6f046c3d4db4bccd79704d66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Tue, 28 May 2024 19:40:46 +0200 Subject: [PATCH 05/11] KAFKA-16448: Remove unecessary classes Co-authored-by: Dabz Co-authored-by: sebastienviale --- .../apache/kafka/streams/StreamsConfig.java | 4 +- .../errors/ErrorHandlerContextImpl.java | 90 ------------------- ...cessingLogAndContinueExceptionHandler.java | 46 ---------- .../ProcessingLogAndFailExceptionHandler.java | 42 --------- 4 files changed, 2 insertions(+), 180 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java 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 b6a736ec5b7fd..7e8c09deca777 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -39,8 +39,8 @@ import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; 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.ProcessingLogAndFailExceptionHandler; import org.apache.kafka.streams.errors.ProductionExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.internals.StreamsConfigUtils; @@ -935,7 +935,7 @@ public class StreamsConfig extends AbstractConfig { DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, Type.CLASS, - ProcessingLogAndFailExceptionHandler.class.getName(), + LogAndFailProcessingExceptionHandler.class.getName(), Importance.MEDIUM, PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java deleted file mode 100644 index c801799a6cdad..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java +++ /dev/null @@ -1,90 +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.common.header.Headers; -import org.apache.kafka.streams.processor.TaskId; - -public class ErrorHandlerContextImpl implements ErrorHandlerContext { - private final String topic; - 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; - - public ErrorHandlerContextImpl(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; - } - - - @Override - public String topic() { - return this.topic; - } - - @Override - public int partition() { - return this.partition; - } - - @Override - public long offset() { - return this.offset; - } - - @Override - public Headers headers() { - return this.headers; - } - - @Override - public byte[] sourceRawKey() { - return this.sourceRawKey; - } - - @Override - public byte[] sourceRawValue() { - return this.sourceRawValue; - } - - @Override - public String processorNodeId() { - return this.processorNodeId; - } - - @Override - public TaskId taskId() { - return this.taskId; - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java deleted file mode 100644 index 3d12814cc3e85..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java +++ /dev/null @@ -1,46 +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.streams.processor.api.Record; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; - -/** - * Processing exception handler that logs a processing exception and then - * signals the processing pipeline to continue processing more records. - */ -public class ProcessingLogAndContinueExceptionHandler implements ProcessingExceptionHandler { - private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndContinueExceptionHandler.class); - - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - log.warn("Exception caught during message processing, " + - "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", - context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), - exception); - - return ProcessingHandlerResponse.CONTINUE; - } - - @Override - public void configure(final Map configs) { - // ignore - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java deleted file mode 100644 index 42d4a45b46c36..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java +++ /dev/null @@ -1,42 +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.streams.processor.api.Record; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; - -public class ProcessingLogAndFailExceptionHandler implements ProcessingExceptionHandler { - private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndFailExceptionHandler.class); - - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - log.warn("Exception caught during message processing, " + - "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", - context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), - exception); - - return ProcessingHandlerResponse.FAIL; - } - - @Override - public void configure(final Map configs) { - // ignore - } -} From bea1fb80e9ebc32a8137efe724c2f2a0ef8adc56 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 29 May 2024 10:15:59 +0200 Subject: [PATCH 06/11] KAFKA-16448: add Streams configuration test for ProcessingExceptionHandler Co-authored-by: Dabz Co-authored-by: loicgreffier --- .../apache/kafka/streams/StreamsConfigTest.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index a264de083b47f..a688e7c0e5440 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -37,6 +37,7 @@ import org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor; import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.streams.state.BuiltInDslStoreSuppliers; +import org.bouncycastle.util.io.Streams; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -1579,6 +1580,21 @@ public void shouldDisableMetricCollectionOnMainConsumerOnly() { ); } + @Test + public void shouldGetDefaultValueProcessingExceptionHandler() { + final StreamsConfig streamsConfig = new StreamsConfig(props); + + assertEquals("org.apache.kafka.streams.errors.LogAndFailProcessingExceptionHandler", streamsConfig.processingExceptionHandler().getClass().getName()); + } + + @Test + public void shouldOverrideDefaultProcessingExceptionHandler() { + props.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, "org.apache.kafka.streams.errors.LogAndContinueProcessingExceptionHandler"); + final StreamsConfig streamsConfig = new StreamsConfig(props); + + assertEquals("org.apache.kafka.streams.errors.LogAndContinueProcessingExceptionHandler", streamsConfig.processingExceptionHandler().getClass().getName()); + } + static class MisconfiguredSerde implements Serde { @Override public void configure(final Map configs, final boolean isKey) { From b7058d22490bd4a88526d1b79979de102eaad97f Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 29 May 2024 10:15:59 +0200 Subject: [PATCH 07/11] KAFKA-16448: add Streams configuration test for ProcessingExceptionHandler Co-authored-by: Dabz Co-authored-by: loicgreffier --- .../apache/kafka/streams/StreamsConfigTest.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index a264de083b47f..1e284348e6fa9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -1579,6 +1579,21 @@ public void shouldDisableMetricCollectionOnMainConsumerOnly() { ); } + @Test + public void shouldGetDefaultValueProcessingExceptionHandler() { + final StreamsConfig streamsConfig = new StreamsConfig(props); + + assertEquals("org.apache.kafka.streams.errors.LogAndFailProcessingExceptionHandler", streamsConfig.processingExceptionHandler().getClass().getName()); + } + + @Test + public void shouldOverrideDefaultProcessingExceptionHandler() { + props.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, "org.apache.kafka.streams.errors.LogAndContinueProcessingExceptionHandler"); + final StreamsConfig streamsConfig = new StreamsConfig(props); + + assertEquals("org.apache.kafka.streams.errors.LogAndContinueProcessingExceptionHandler", streamsConfig.processingExceptionHandler().getClass().getName()); + } + static class MisconfiguredSerde implements Serde { @Override public void configure(final Map configs, final boolean isKey) { From 3c7de0cf3184378117a7f198043b2c3ea82ff758 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 29 May 2024 11:10:28 +0200 Subject: [PATCH 08/11] KAFKA-16448: add Streams configuration test for invalid ProcessingExceptionHandler Co-authored-by: Dabz Co-authored-by: loicgreffier --- .../java/org/apache/kafka/streams/StreamsConfigTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 1e284348e6fa9..9c0c4d56517c0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -1594,6 +1594,12 @@ public void shouldOverrideDefaultProcessingExceptionHandler() { assertEquals("org.apache.kafka.streams.errors.LogAndContinueProcessingExceptionHandler", streamsConfig.processingExceptionHandler().getClass().getName()); } + @Test + public void testInvalidProcessingExceptionHandler() { + props.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, "org.apache.kafka.streams.errors.InvalidProcessingExceptionHandler"); + assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + } + static class MisconfiguredSerde implements Serde { @Override public void configure(final Map configs, final boolean isKey) { From 5e7a65692ae9a3e185a39be9a226c4e8c0731a85 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 29 May 2024 13:04:16 +0200 Subject: [PATCH 09/11] KAFKA-16448: add error message test in case of invalid ProcessingExceptionHandler Co-authored-by: Dabz Co-authored-by: loicgreffier --- .../java/org/apache/kafka/streams/StreamsConfigTest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 9c0c4d56517c0..6215dd0d9471d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -1597,7 +1597,13 @@ public void shouldOverrideDefaultProcessingExceptionHandler() { @Test public void testInvalidProcessingExceptionHandler() { props.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, "org.apache.kafka.streams.errors.InvalidProcessingExceptionHandler"); - assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + final Exception exception = assertThrows(ConfigException.class, () -> new StreamsConfig(props)); + + assertThat( + exception.getMessage(), + containsString("Invalid value org.apache.kafka.streams.errors.InvalidProcessingExceptionHandler " + + "for configuration processing.exception.handler: Class org.apache.kafka.streams.errors.InvalidProcessingExceptionHandler could not be found.") + ); } static class MisconfiguredSerde implements Serde { From 60bba3bff094159ce8df1cbf02815c2380abe1b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Mon, 27 May 2024 15:21:38 +0200 Subject: [PATCH 10/11] KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration Co-authored-by: Dabz Co-authored-by: sebastienviale --- .github/workflows/docker_build_and_test.yml | 1 + .../docker_official_image_build_and_test.yml | 66 + .github/workflows/docker_promote.yml | 4 +- .github/workflows/docker_rc_release.yml | 3 +- .../prepare_docker_official_image_source.yml | 52 + Jenkinsfile | 10 +- build.gradle | 222 +++- checkstyle/import-control.xml | 5 + checkstyle/suppressions.xml | 10 +- .../internals/AsyncKafkaConsumer.java | 193 +-- .../consumer/internals/CompletedFetch.java | 47 +- .../internals/ConsumerNetworkThread.java | 48 +- .../internals/HeartbeatRequestManager.java | 10 +- .../internals/MembershipManagerImpl.java | 1 + .../events/AbstractTopicMetadataEvent.java | 5 +- .../events/AllTopicsMetadataEvent.java | 6 +- .../events/ApplicationEventHandler.java | 12 +- .../events/ApplicationEventProcessor.java | 18 +- .../internals/events/CommitEvent.java | 6 - .../events/CompletableApplicationEvent.java | 13 +- .../events/CompletableBackgroundEvent.java | 14 +- .../internals/events/CompletableEvent.java | 103 ++ .../events/CompletableEventReaper.java | 157 +++ ...rRebalanceListenerCallbackNeededEvent.java | 2 +- .../internals/events/EventProcessor.java | 115 +- .../events/FetchCommittedOffsetsEvent.java | 5 +- .../internals/events/LeaveOnCloseEvent.java | 6 +- .../internals/events/ListOffsetsEvent.java | 5 +- .../internals/events/ResetPositionsEvent.java | 6 +- .../internals/events/SyncCommitEvent.java | 5 +- .../internals/events/TopicMetadataEvent.java | 6 +- .../internals/events/UnsubscribeEvent.java | 6 +- .../events/ValidatePositionsEvent.java | 6 +- .../RecordDeserializationException.java | 76 +- .../internals/AsyncKafkaConsumerTest.java | 138 +- .../internals/CompletedFetchTest.java | 32 +- .../internals/ConsumerNetworkThreadTest.java | 75 +- .../internals/ConsumerTestBuilder.java | 28 - .../HeartbeatRequestManagerTest.java | 16 +- .../events/ApplicationEventProcessorTest.java | 6 +- .../events/CompletableEventReaperTest.java | 200 +++ .../org/apache/kafka/connect/data/Values.java | 1161 ++++++++++------- .../apache/kafka/connect/data/ValuesTest.java | 248 +++- .../kafka/connect/mirror/Checkpoint.java | 14 + .../kafka/connect/mirror/CheckpointStore.java | 203 +++ .../mirror/MirrorCheckpointConfig.java | 1 + .../connect/mirror/MirrorCheckpointTask.java | 51 +- .../kafka/connect/mirror/OffsetSyncStore.java | 22 +- .../connect/mirror/CheckpointStoreTest.java | 97 ++ .../mirror/MirrorCheckpointTaskTest.java | 125 +- .../connect/mirror/OffsetSyncStoreTest.java | 108 +- .../kafka/connect/util/KafkaBasedLog.java | 12 +- .../util/clusters/EmbeddedKafkaCluster.java | 5 +- .../kafka/log/remote/RemoteLogManager.java | 14 +- .../ReplicaAlterLogDirsTierStateMachine.java | 41 - .../ReplicaFetcherTierStateMachine.java | 280 ---- .../java/kafka/server/TierStateMachine.java | 237 +++- .../main/scala/kafka/cluster/Partition.scala | 2 +- .../group/GroupMetadataManager.scala | 4 +- .../transaction/TransactionStateManager.scala | 4 +- core/src/main/scala/kafka/log/LocalLog.scala | 19 +- .../src/main/scala/kafka/log/LogCleaner.scala | 13 +- .../src/main/scala/kafka/log/UnifiedLog.scala | 21 +- .../kafka/metrics/KafkaMetricsConfig.scala | 8 +- .../scala/kafka/network/SocketServer.scala | 8 +- .../scala/kafka/raft/KafkaMetadataLog.scala | 2 +- .../main/scala/kafka/raft/RaftManager.scala | 13 +- .../main/scala/kafka/serializer/Decoder.scala | 7 +- .../kafka/server/BrokerLifecycleManager.scala | 42 +- .../scala/kafka/server/BrokerServer.scala | 19 +- .../scala/kafka/server/ControllerServer.scala | 2 +- .../scala/kafka/server/DelayedFetch.scala | 14 +- .../kafka/server/DynamicBrokerConfig.scala | 18 +- .../main/scala/kafka/server/KafkaConfig.scala | 273 ++-- .../main/scala/kafka/server/KafkaServer.scala | 26 +- .../scala/kafka/server/MetadataCache.scala | 3 +- .../NodeToControllerChannelManager.scala | 29 +- .../server/ReplicaAlterLogDirsThread.scala | 2 +- .../kafka/server/ReplicaFetcherThread.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 35 +- .../server/metadata/ZkMetadataCache.scala | 9 +- .../scala/kafka/tools/DumpLogSegments.scala | 54 +- .../main/scala/kafka/tools/StorageTool.scala | 4 +- .../main/scala/kafka/utils/CoreUtils.scala | 16 +- .../main/scala/kafka/utils/Throttler.scala | 111 -- .../log/remote/RemoteLogManagerTest.java | 124 +- .../junit/RaftClusterInvocationContext.java | 124 +- .../kafka/testkit/KafkaClusterTestKit.java | 6 +- .../admin/BrokerApiVersionsCommandTest.scala | 5 +- .../AbstractAuthorizerIntegrationTest.scala | 11 +- .../kafka/api/AbstractConsumerTest.scala | 5 +- ...minClientWithPoliciesIntegrationTest.scala | 6 +- .../kafka/api/BaseAdminIntegrationTest.scala | 23 +- .../integration/kafka/api/BaseQuotaTest.scala | 6 +- .../kafka/api/ConsumerTopicCreationTest.scala | 7 +- .../kafka/api/CustomQuotaCallbackTest.scala | 6 +- ...gationTokenEndToEndAuthorizationTest.scala | 7 +- .../DescribeAuthorizedOperationsTest.scala | 6 +- .../kafka/api/EndToEndAuthorizationTest.scala | 10 +- .../api/GroupAuthorizerIntegrationTest.scala | 9 +- .../api/PlaintextAdminIntegrationTest.scala | 118 +- .../api/ProducerFailureHandlingTest.scala | 6 +- .../kafka/api/ProducerIdExpirationTest.scala | 4 +- .../api/SaslSslAdminIntegrationTest.scala | 172 ++- .../kafka/api/SslAdminIntegrationTest.scala | 58 +- .../kafka/api/TransactionsBounceTest.scala | 6 +- .../api/TransactionsExpirationTest.scala | 4 +- .../kafka/api/TransactionsTest.scala | 5 +- .../TransactionsWithMaxInFlightOneTest.scala | 4 +- .../DynamicNumNetworkThreadsTest.scala | 20 +- .../kafka/server/DelayedFetchTest.scala | 66 +- .../DynamicBrokerReconfigurationTest.scala | 16 +- .../kafka/server/KRaftClusterTest.scala | 38 + .../kafka/server/QuorumTestHarness.scala | 4 +- .../kafka/raft/KafkaMetadataLogTest.scala | 18 + .../unit/kafka/admin/AclCommandTest.scala | 3 +- .../unit/kafka/admin/DeleteTopicTest.scala | 3 +- .../ControllerChannelManagerTest.scala | 4 +- .../group/GroupMetadataManagerTest.scala | 128 ++ ...tricsDuringTopicCreationDeletionTest.scala | 4 +- .../scala/unit/kafka/log/LocalLogTest.scala | 37 + ...gCleanerParameterizedIntegrationTest.scala | 3 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 5 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 2 +- .../scala/unit/kafka/log/LogSegmentTest.scala | 32 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 47 +- .../unit/kafka/network/SocketServerTest.scala | 4 +- .../unit/kafka/raft/RaftManagerTest.scala | 4 +- .../DelegationTokenManagerTest.scala | 8 +- .../server/AbstractFetcherManagerTest.scala | 5 +- .../server/AbstractMetadataRequestTest.scala | 4 +- ...mCredentialsRequestNotAuthorizedTest.scala | 6 +- ...AlterUserScramCredentialsRequestTest.scala | 10 +- .../server/AutoTopicCreationManagerTest.scala | 3 +- .../kafka/server/BaseFetchRequestTest.scala | 3 +- .../unit/kafka/server/BaseRequestTest.scala | 3 +- .../server/BrokerLifecycleManagerTest.scala | 6 +- .../kafka/server/ControllerApisTest.scala | 4 +- .../server/ControllerMutationQuotaTest.scala | 4 +- .../server/DelegationTokenRequestsTest.scala | 9 +- ...opicsRequestWithDeletionDisabledTest.scala | 4 +- .../server/DescribeClusterRequestTest.scala | 4 +- ...mCredentialsRequestNotAuthorizedTest.scala | 8 +- ...cribeUserScramCredentialsRequestTest.scala | 10 +- .../server/DynamicBrokerConfigTest.scala | 34 +- .../server/FetchRequestMaxBytesTest.scala | 3 +- .../unit/kafka/server/KafkaApisTest.scala | 10 +- .../unit/kafka/server/KafkaConfigTest.scala | 144 +- .../KafkaMetricReporterClusterIdTest.scala | 7 +- .../server/KafkaMetricsReporterTest.scala | 5 +- .../unit/kafka/server/LogDirFailureTest.scala | 30 +- .../kafka/server/MockTierStateMachine.scala | 9 +- .../kafka/server/ReplicaManagerTest.scala | 86 +- .../unit/kafka/server/RequestQuotaTest.scala | 10 +- .../server/ServerGenerateBrokerIdTest.scala | 9 +- .../scala/unit/kafka/server/ServerTest.scala | 5 +- ...eTest.scala => TierStateMachineTest.scala} | 26 +- ...opicIdWithOldInterBrokerProtocolTest.scala | 4 +- .../kafka/tools/DumpLogSegmentsTest.scala | 32 +- .../unit/kafka/tools/StorageToolTest.scala | 18 +- .../unit/kafka/utils/CoreUtilsTest.scala | 13 - .../scala/unit/kafka/utils/TestUtils.scala | 49 +- .../unit/kafka/utils/ThrottlerTest.scala | 108 -- docker/docker_official_image_build_test.py | 87 ++ docker/docker_official_images/.gitkeep | 0 .../extract_docker_official_image_artifact.py | 77 ++ docker/generate_kafka_pr_template.py | 92 ++ .../prepare_docker_official_image_source.py | 68 + docs/streams/quickstart.html | 4 - docs/upgrade.html | 20 + gradle/dependencies.gradle | 8 +- gradle/spotbugs-exclude.xml | 15 +- .../group/CoordinatorRecordHelpers.java | 4 +- .../group/GroupCoordinatorConfig.java | 6 +- .../group/GroupCoordinatorService.java | 7 +- .../group/GroupCoordinatorShard.java | 5 +- .../group/GroupMetadataManager.java | 440 +++++-- .../group/assignor/AssignmentMemberSpec.java | 9 +- .../ConsumerGroupPartitionAssignor.java | 29 + .../GeneralUniformAssignmentBuilder.java | 4 +- .../coordinator/group/assignor/GroupSpec.java | 42 + ...AssignmentSpec.java => GroupSpecImpl.java} | 52 +- .../OptimizedUniformAssignmentBuilder.java | 64 +- .../group/assignor/PartitionAssignor.java | 6 +- .../assignor/PartitionAssignorException.java | 2 +- .../group/assignor/RangeAssignor.java | 16 +- .../group/assignor/UniformAssignor.java | 14 +- .../group/consumer/ConsumerGroup.java | 123 +- .../group/consumer/ConsumerGroupMember.java | 21 +- .../consumer/TargetAssignmentBuilder.java | 66 +- .../coordinator/group/consumer/TopicIds.java | 176 +++ .../group/runtime/CoordinatorRuntime.java | 29 +- .../group/runtime/CoordinatorTimer.java | 12 + .../runtime/MultiThreadedEventProcessor.java | 3 - .../coordinator/group/AssignmentTestUtil.java | 30 + .../group/GroupCoordinatorConfigTest.java | 4 +- .../group/GroupCoordinatorServiceTest.java | 9 +- .../group/GroupMetadataManagerTest.java | 925 +++++++++++-- .../GroupMetadataManagerTestContext.java | 32 +- .../group/MockCoordinatorTimer.java | 13 + .../group/MockPartitionAssignor.java | 8 +- .../group/NoOpPartitionAssignor.java | 10 +- .../GeneralUniformAssignmentBuilderTest.java | 142 +- .../group/assignor/GroupSpecImplTest.java | 84 ++ ...OptimizedUniformAssignmentBuilderTest.java | 194 ++- .../group/assignor/RangeAssignorTest.java | 249 ++-- .../consumer/ConsumerGroupMemberTest.java | 13 +- .../group/consumer/ConsumerGroupTest.java | 169 +++ .../consumer/TargetAssignmentBuilderTest.java | 42 +- .../group/consumer/TopicIdsTest.java | 190 +++ .../group/runtime/CoordinatorRuntimeTest.java | 103 ++ .../MultiThreadedEventProcessorTest.java | 9 +- .../jmh/assignor/AssignorBenchmarkUtils.java | 77 ++ .../assignor/ServerSideAssignorBenchmark.java | 60 +- .../TargetAssignmentBuilderBenchmark.java | 40 +- .../kafka/jmh/connect/ValuesBenchmark.java | 297 +++++ .../ReplicaFetcherThreadBenchmark.java | 2 +- .../metadata/MetadataRequestBenchmark.java | 6 +- .../kafka/jmh/server/CheckpointBench.java | 2 +- .../jmh/server/PartitionCreationBench.java | 2 +- .../log4jappender/KafkaLog4jAppender.java | 6 +- .../log4jappender/MockKafkaLog4jAppender.java | 1 + .../apache/kafka/raft/KafkaRaftClient.java | 5 + .../org/apache/kafka/raft/QuorumState.java | 1 + .../apache/kafka/raft/internals/VoterSet.java | 14 + .../kafka/server/config/ServerLogConfigs.java | 5 + .../org/apache/kafka/server/util/Csv.java | 16 + .../org/apache/kafka/server/util/CsvTest.java | 22 +- .../apache/kafka/server/config/Defaults.java | 60 - .../config/DelegationTokenManagerConfigs.java | 41 + .../kafka/server/config/ServerConfigs.java | 139 ++ .../storage/internals/log/FetchParams.java | 6 + .../internals/log/LogOffsetMetadata.java | 16 +- .../storage/internals/log/LogSegment.java | 18 +- .../storage/internals/utils/Throttler.java | 102 ++ .../internals/log/LogOffsetMetadataTest.java | 82 ++ .../internals/utils/ThrottlerTest.java | 115 ++ .../storage/TieredStorageTestBuilder.java | 17 + .../storage/TieredStorageTestContext.java | 17 +- .../storage/TieredStorageTestHarness.java | 2 +- .../storage/actions/AlterLogDirAction.java | 80 ++ .../actions/EraseBrokerStorageAction.java | 16 +- .../storage/actions/ExpectLeaderAction.java | 10 +- .../storage/integration/AlterLogDirTest.java | 68 + ...FromLeaderWithCorruptedCheckpointTest.java | 84 ++ .../TransactionsWithTieredStoreTest.java | 2 +- .../storage/utils/BrokerLocalStorage.java | 54 +- .../storage/utils/TieredStorageTestUtils.java | 8 + .../apache/kafka/streams/StreamsConfig.java | 26 + .../streams/errors/ErrorHandlerContext.java | 133 ++ ...AndContinueProcessingExceptionHandler.java | 46 + .../LogAndFailProcessingExceptionHandler.java | 46 + .../errors/ProcessingExceptionHandler.java | 56 + .../internals/DefaultErrorHandlerContext.java | 90 ++ .../kstream/internals/KStreamImplJoin.java | 6 +- .../kstream/internals/KStreamKStreamJoin.java | 160 ++- .../internals/KStreamKStreamJoinLeftSide.java | 71 + .../KStreamKStreamJoinRightSide.java | 70 + .../assignment/ApplicationState.java | 15 +- .../assignment/KafkaStreamsAssignment.java | 39 +- .../assignment/KafkaStreamsState.java | 5 + .../processor/assignment/TaskAssignor.java | 6 + .../processor/assignment/TaskInfo.java | 56 + .../assignment/TaskTopicPartition.java | 56 + .../assignors/StickyTaskAssignor.java | 466 +++++++ .../internals/InternalTopologyBuilder.java | 9 + .../internals/StreamsPartitionAssignor.java | 276 +++- .../assignment/ApplicationStateImpl.java | 36 +- .../assignment/AssignorConfiguration.java | 19 + .../BalanceSubtopologyGraphConstructor.java | 38 +- .../internals/assignment/ClientState.java | 12 + .../internals/assignment/DefaultTaskInfo.java | 62 + .../assignment/DefaultTaskTopicPartition.java | 99 ++ .../assignment/KafkaStreamsStateImpl.java | 10 +- .../MinTrafficGraphConstructor.java | 4 +- .../assignment/RackAwareGraphConstructor.java | 4 +- .../RackAwareGraphConstructorFactory.java | 13 +- .../assignment/RackAwareTaskAssignor.java | 38 +- .../internals/assignment/RackUtils.java | 157 +++ .../state/internals/LeftOrRightValue.java | 15 - .../state/internals/MeteredIterator.java | 30 + .../state/internals/MeteredKeyValueStore.java | 53 +- ...MeteredMultiVersionedKeyQueryIterator.java | 36 +- .../state/internals/MeteredSessionStore.java | 55 +- .../MeteredTimestampedKeyValueStore.java | 19 +- .../MeteredVersionedKeyValueStore.java | 9 +- .../state/internals/MeteredWindowStore.java | 55 +- .../internals/MeteredWindowStoreIterator.java | 38 +- .../MeteredWindowedKeyValueIterator.java | 38 +- .../internals/TimestampedKeyAndJoinSide.java | 22 +- ...TimestampedKeyAndJoinSideDeserializer.java | 5 +- .../internals/metrics/StateStoreMetrics.java | 47 +- .../kafka/streams/StreamsConfigTest.java | 35 + ...ailabilityTaskAssignorIntegrationTest.java | 8 +- .../utils/EmbeddedKafkaCluster.java | 6 +- .../integration/utils/KafkaEmbedded.java | 7 +- .../internals/KStreamKStreamJoinTest.java | 3 +- .../assignment/KafkaStreamsStateTest.java | 4 +- .../RackAwareGraphConstructorTest.java | 4 +- .../internals/MeteredKeyValueStoreTest.java | 78 +- .../internals/MeteredSessionStoreTest.java | 76 +- .../MeteredTimestampedKeyValueStoreTest.java | 77 +- .../MeteredVersionedKeyValueStoreTest.java | 84 +- .../internals/MeteredWindowStoreTest.java | 76 +- ...mestampedKeyAndJoinSideSerializerTest.java | 6 +- .../consumer/ConsoleConsumerOptions.java | 21 +- .../consumer/ConsoleConsumerOptionsTest.java | 27 + .../org/apache/kafka/tools/api/Decoder.java | 25 + .../kafka/tools/api/DefaultDecoder.java | 27 + .../kafka/tools/api/IntegerDecoder.java | 29 + .../apache/kafka/tools/api/LongDecoder.java | 29 + .../apache/kafka/tools/api/StringDecoder.java | 29 + 312 files changed, 11808 insertions(+), 3590 deletions(-) create mode 100644 .github/workflows/docker_official_image_build_and_test.yml create mode 100644 .github/workflows/prepare_docker_official_image_source.yml create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java create mode 100644 connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java create mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java delete mode 100644 core/src/main/java/kafka/server/ReplicaAlterLogDirsTierStateMachine.java delete mode 100644 core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java delete mode 100644 core/src/main/scala/kafka/utils/Throttler.scala rename core/src/test/scala/unit/kafka/server/{ReplicaFetcherTierStateMachineTest.scala => TierStateMachineTest.scala} (93%) delete mode 100755 core/src/test/scala/unit/kafka/utils/ThrottlerTest.scala create mode 100644 docker/docker_official_image_build_test.py create mode 100644 docker/docker_official_images/.gitkeep create mode 100644 docker/extract_docker_official_image_artifact.py create mode 100644 docker/generate_kafka_pr_template.py create mode 100644 docker/prepare_docker_official_image_source.py create mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ConsumerGroupPartitionAssignor.java create mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupSpec.java rename group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/{AssignmentSpec.java => GroupSpecImpl.java} (55%) create mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/TopicIds.java create mode 100644 group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/GroupSpecImplTest.java create mode 100644 group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/TopicIdsTest.java create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/AssignorBenchmarkUtils.java create mode 100644 jmh-benchmarks/src/main/java/org/apache/kafka/jmh/connect/ValuesBenchmark.java delete mode 100644 server/src/main/java/org/apache/kafka/server/config/Defaults.java create mode 100644 server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java create mode 100644 server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java create mode 100644 storage/src/main/java/org/apache/kafka/storage/internals/utils/Throttler.java create mode 100644 storage/src/test/java/org/apache/kafka/storage/internals/log/LogOffsetMetadataTest.java create mode 100644 storage/src/test/java/org/apache/kafka/storage/internals/utils/ThrottlerTest.java create mode 100644 storage/src/test/java/org/apache/kafka/tiered/storage/actions/AlterLogDirAction.java create mode 100644 storage/src/test/java/org/apache/kafka/tiered/storage/integration/AlterLogDirTest.java create mode 100644 storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/internals/DefaultErrorHandlerContext.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinLeftSide.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinRightSide.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskInfo.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/assignment/TaskTopicPartition.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskInfo.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/DefaultTaskTopicPartition.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/RackUtils.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredIterator.java create mode 100644 tools/tools-api/src/main/java/org/apache/kafka/tools/api/Decoder.java create mode 100644 tools/tools-api/src/main/java/org/apache/kafka/tools/api/DefaultDecoder.java create mode 100644 tools/tools-api/src/main/java/org/apache/kafka/tools/api/IntegerDecoder.java create mode 100644 tools/tools-api/src/main/java/org/apache/kafka/tools/api/LongDecoder.java create mode 100644 tools/tools-api/src/main/java/org/apache/kafka/tools/api/StringDecoder.java diff --git a/.github/workflows/docker_build_and_test.yml b/.github/workflows/docker_build_and_test.yml index 695c08672fd87..1d30aa85ea25d 100644 --- a/.github/workflows/docker_build_and_test.yml +++ b/.github/workflows/docker_build_and_test.yml @@ -23,6 +23,7 @@ on: description: Docker image type to build and test options: - "jvm" + - "native" kafka_url: description: Kafka url to be used to build the docker image required: true diff --git a/.github/workflows/docker_official_image_build_and_test.yml b/.github/workflows/docker_official_image_build_and_test.yml new file mode 100644 index 0000000000000..a315cd0e0d2ac --- /dev/null +++ b/.github/workflows/docker_official_image_build_and_test.yml @@ -0,0 +1,66 @@ +# 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. + +name: Docker Official Image Build Test + +on: + workflow_dispatch: + inputs: + image_type: + type: choice + description: Docker image type to build and test + options: + - "jvm" + kafka_version: + description: Kafka version for the docker official image. This should be >=3.7.0 + required: true + +jobs: + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Set up Python 3.10 + uses: actions/setup-python@v3 + with: + python-version: "3.10" + - name: Install dependencies + run: | + python -m pip install --upgrade pip + pip install -r docker/requirements.txt + - name: Build image and run tests + working-directory: ./docker + 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 + with: + image-ref: 'kafka/test:test' + format: 'table' + severity: 'CRITICAL,HIGH' + output: scan_report_${{ github.event.inputs.image_type }}.txt + exit-code: '1' + - name: Upload test report + if: always() + uses: actions/upload-artifact@v3 + with: + name: report_${{ github.event.inputs.image_type }}.html + path: docker/test/report_${{ github.event.inputs.image_type }}.html + - name: Upload CVE scan report + if: always() + uses: actions/upload-artifact@v3 + with: + name: scan_report_${{ github.event.inputs.image_type }}.txt + path: scan_report_${{ github.event.inputs.image_type }}.txt diff --git a/.github/workflows/docker_promote.yml b/.github/workflows/docker_promote.yml index 3449265877b8a..04872f9d59d3b 100644 --- a/.github/workflows/docker_promote.yml +++ b/.github/workflows/docker_promote.yml @@ -19,10 +19,10 @@ on: workflow_dispatch: inputs: rc_docker_image: - description: RC docker image that needs to be promoted (Example:- apache/kafka:3.6.0-rc0) + description: RC docker image that needs to be promoted (Example:- apache/kafka:3.8.0-rc0 (OR) apache/kafka-native:3.8.0-rc0) required: true promoted_docker_image: - description: Docker image name of the promoted image (Example:- apache/kafka:3.6.0) + description: Docker image name of the promoted image (Example:- apache/kafka:3.8.0 (OR) apache/kafka-native:3.8.0) required: true jobs: diff --git a/.github/workflows/docker_rc_release.yml b/.github/workflows/docker_rc_release.yml index c7082dcac910c..22dd924b51b61 100644 --- a/.github/workflows/docker_rc_release.yml +++ b/.github/workflows/docker_rc_release.yml @@ -23,8 +23,9 @@ on: description: Docker image type to be built and pushed options: - "jvm" + - "native" rc_docker_image: - description: RC docker image that needs to be built and pushed to Dockerhub (Example:- apache/kafka:3.6.0-rc0) + description: RC docker image that needs to be built and pushed to Dockerhub (Example:- apache/kafka:3.8.0-rc0 (OR) apache/kafka-native:3.8.0-rc0) required: true kafka_url: description: Kafka url to be used to build the docker image diff --git a/.github/workflows/prepare_docker_official_image_source.yml b/.github/workflows/prepare_docker_official_image_source.yml new file mode 100644 index 0000000000000..45491045836f6 --- /dev/null +++ b/.github/workflows/prepare_docker_official_image_source.yml @@ -0,0 +1,52 @@ +# 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. + +name: Docker Prepare Docker Official Image Source + +on: + workflow_dispatch: + inputs: + image_type: + type: choice + description: Docker image type to build and test + options: + - "jvm" + kafka_version: + description: Kafka version for the docker official image. This should be >=3.7.0 + required: true + +jobs: + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Set up Python 3.10 + uses: actions/setup-python@v3 + with: + python-version: "3.10" + - name: Install dependencies + run: | + python -m pip install --upgrade pip + pip install -r docker/requirements.txt + - name: Build Docker Official Image Artifact + working-directory: ./docker + run: | + python prepare_docker_official_image_source.py -type=${{ github.event.inputs.image_type }} -v=${{ github.event.inputs.kafka_version }} + - name: Upload Docker Official Image Artifact + if: success() + uses: actions/upload-artifact@v4 + with: + name: ${{ github.event.inputs.kafka_version }} + path: docker/docker_official_images/${{ github.event.inputs.kafka_version }} diff --git a/Jenkinsfile b/Jenkinsfile index 50b7f6a298e71..0a795637ff638 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -36,6 +36,12 @@ def doTest(env, target = "test") { junit '**/build/test-results/**/TEST-*.xml' } +def runTestOnDevBranch(env) { + if (!isChangeRequest(env)) { + doTest(env) + } +} + def doStreamsArchetype() { echo 'Verify that Kafka Streams archetype compiles' @@ -132,7 +138,7 @@ pipeline { } steps { doValidation() - doTest(env) + runTestOnDevBranch(env) echo 'Skipping Kafka Streams archetype test for Java 11' } } @@ -151,7 +157,7 @@ pipeline { } steps { doValidation() - doTest(env) + runTestOnDevBranch(env) echo 'Skipping Kafka Streams archetype test for Java 17' } } diff --git a/build.gradle b/build.gradle index 031389f62c6e9..8d9326d9bc833 100644 --- a/build.gradle +++ b/build.gradle @@ -233,7 +233,8 @@ if (repo != null) { '**/generated/**', 'clients/src/test/resources/serializedData/*', 'docker/test/fixtures/secrets/*', - 'docker/examples/fixtures/secrets/*' + 'docker/examples/fixtures/secrets/*', + 'docker/docker_official_images/.gitkeep' ]) } } else { @@ -356,7 +357,7 @@ subprojects { artifact task } - artifactId = archivesBaseName + artifactId = base.archivesName.get() pom { name = 'Apache Kafka' url = 'https://kafka.apache.org' @@ -741,8 +742,8 @@ subprojects { } test.dependsOn('spotbugsMain') - tasks.withType(com.github.spotbugs.snom.SpotBugsTask) { - reports { + tasks.withType(com.github.spotbugs.snom.SpotBugsTask).configureEach { + reports.configure { // Continue supporting `xmlFindBugsReport` for compatibility xml.enabled(project.hasProperty('xmlSpotBugsReport') || project.hasProperty('xmlFindBugsReport')) html.enabled(!project.hasProperty('xmlSpotBugsReport') && !project.hasProperty('xmlFindBugsReport')) @@ -850,7 +851,9 @@ tasks.create(name: "jarConnect", dependsOn: connectPkgs.collect { it + ":jar" }) tasks.create(name: "testConnect", dependsOn: connectPkgs.collect { it + ":test" }) {} project(':server') { - archivesBaseName = "kafka-server" + base { + archivesName = "kafka-server" + } dependencies { implementation project(':clients') @@ -922,7 +925,10 @@ project(':core') { } if (userEnableTestCoverage) apply plugin: "org.scoverage" - archivesBaseName = "kafka_${versions.baseScala}" + + base { + archivesName = "kafka_${versions.baseScala}" + } configurations { generator @@ -1250,7 +1256,9 @@ project(':core') { } project(':metadata') { - archivesBaseName = "kafka-metadata" + base { + archivesName = "kafka-metadata" + } configurations { generator @@ -1317,7 +1325,9 @@ project(':metadata') { } project(':group-coordinator') { - archivesBaseName = "kafka-group-coordinator" + base { + archivesName = "kafka-group-coordinator" + } configurations { generator @@ -1384,7 +1394,9 @@ project(':group-coordinator') { } project(':transaction-coordinator') { - archivesBaseName = "kafka-transaction-coordinator" + base { + archivesName = "kafka-transaction-coordinator" + } sourceSets { main { @@ -1405,7 +1417,9 @@ project(':transaction-coordinator') { } project(':examples') { - archivesBaseName = "kafka-examples" + base { + archivesName = "kafka-examples" + } dependencies { implementation project(':clients') @@ -1435,7 +1449,9 @@ project(':generator') { } project(':clients') { - archivesBaseName = "kafka-clients" + base { + archivesName = "kafka-clients" + } configurations { generator @@ -1610,7 +1626,9 @@ project(':clients') { } project(':raft') { - archivesBaseName = "kafka-raft" + base { + archivesName = "kafka-raft" + } configurations { generator @@ -1706,7 +1724,9 @@ project(':raft') { } project(':server-common') { - archivesBaseName = "kafka-server-common" + base { + archivesName = "kafka-server-common" + } dependencies { api project(':clients') @@ -1764,7 +1784,9 @@ project(':server-common') { } project(':storage:storage-api') { - archivesBaseName = "kafka-storage-api" + base { + archivesName = "kafka-storage-api" + } dependencies { implementation project(':clients') @@ -1832,7 +1854,9 @@ project(':storage:storage-api') { } project(':storage') { - archivesBaseName = "kafka-storage" + base { + archivesName = "kafka-storage" + } configurations { generator @@ -1954,7 +1978,9 @@ project(':storage') { } project(':tools:tools-api') { - archivesBaseName = "kafka-tools-api" + base { + archivesName = "kafka-tools-api" + } dependencies { implementation project(':clients') @@ -2009,7 +2035,10 @@ project(':tools:tools-api') { } project(':tools') { - archivesBaseName = "kafka-tools" + base { + archivesName = "kafka-tools" + } + dependencies { implementation project(':clients') implementation project(':storage') @@ -2073,7 +2102,9 @@ project(':tools') { } project(':trogdor') { - archivesBaseName = "trogdor" + base { + archivesName = "trogdor" + } dependencies { implementation project(':clients') @@ -2123,7 +2154,9 @@ project(':trogdor') { } project(':shell') { - archivesBaseName = "kafka-shell" + base { + archivesName = "kafka-shell" + } dependencies { implementation libs.argparse4j @@ -2173,7 +2206,10 @@ project(':shell') { } project(':streams') { - archivesBaseName = "kafka-streams" + base { + archivesName = "kafka-streams" + } + ext.buildStreamsVersionFileName = "kafka-streams-version.properties" configurations { @@ -2335,7 +2371,11 @@ project(':streams') { project(':streams:streams-scala') { apply plugin: 'scala' - archivesBaseName = "kafka-streams-scala_${versions.baseScala}" + + base { + archivesName = "kafka-streams-scala_${versions.baseScala}" + } + dependencies { api project(':streams') @@ -2397,7 +2437,9 @@ project(':streams:streams-scala') { } project(':streams:test-utils') { - archivesBaseName = "kafka-streams-test-utils" + base { + archivesName = "kafka-streams-test-utils" + } dependencies { api project(':streams') @@ -2432,7 +2474,9 @@ project(':streams:test-utils') { } project(':streams:examples') { - archivesBaseName = "kafka-streams-examples" + base { + archivesName = "kafka-streams-examples" + } dependencies { // this dependency should be removed after we unify data API @@ -2469,7 +2513,9 @@ project(':streams:examples') { } project(':streams:upgrade-system-tests-0100') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0100" + base { + archivesName = "kafka-streams-upgrade-system-tests-0100" + } dependencies { testImplementation(libs.kafkaStreams_0100) { @@ -2485,7 +2531,9 @@ project(':streams:upgrade-system-tests-0100') { } project(':streams:upgrade-system-tests-0101') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0101" + base { + archivesName = "kafka-streams-upgrade-system-tests-0101" + } dependencies { testImplementation(libs.kafkaStreams_0101) { @@ -2501,7 +2549,9 @@ project(':streams:upgrade-system-tests-0101') { } project(':streams:upgrade-system-tests-0102') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0102" + base { + archivesName = "kafka-streams-upgrade-system-tests-0102" + } dependencies { testImplementation libs.kafkaStreams_0102 @@ -2514,7 +2564,9 @@ project(':streams:upgrade-system-tests-0102') { } project(':streams:upgrade-system-tests-0110') { - archivesBaseName = "kafka-streams-upgrade-system-tests-0110" + base{ + archivesName = "kafka-streams-upgrade-system-tests-0110" + } dependencies { testImplementation libs.kafkaStreams_0110 @@ -2527,7 +2579,9 @@ project(':streams:upgrade-system-tests-0110') { } project(':streams:upgrade-system-tests-10') { - archivesBaseName = "kafka-streams-upgrade-system-tests-10" + base { + archivesName = "kafka-streams-upgrade-system-tests-10" + } dependencies { testImplementation libs.kafkaStreams_10 @@ -2540,7 +2594,9 @@ project(':streams:upgrade-system-tests-10') { } project(':streams:upgrade-system-tests-11') { - archivesBaseName = "kafka-streams-upgrade-system-tests-11" + base { + archivesName = "kafka-streams-upgrade-system-tests-11" + } dependencies { testImplementation libs.kafkaStreams_11 @@ -2553,7 +2609,9 @@ project(':streams:upgrade-system-tests-11') { } project(':streams:upgrade-system-tests-20') { - archivesBaseName = "kafka-streams-upgrade-system-tests-20" + base { + archivesName = "kafka-streams-upgrade-system-tests-20" + } dependencies { testImplementation libs.kafkaStreams_20 @@ -2566,7 +2624,9 @@ project(':streams:upgrade-system-tests-20') { } project(':streams:upgrade-system-tests-21') { - archivesBaseName = "kafka-streams-upgrade-system-tests-21" + base { + archivesName = "kafka-streams-upgrade-system-tests-21" + } dependencies { testImplementation libs.kafkaStreams_21 @@ -2579,7 +2639,9 @@ project(':streams:upgrade-system-tests-21') { } project(':streams:upgrade-system-tests-22') { - archivesBaseName = "kafka-streams-upgrade-system-tests-22" + base { + archivesName = "kafka-streams-upgrade-system-tests-22" + } dependencies { testImplementation libs.kafkaStreams_22 @@ -2592,7 +2654,9 @@ project(':streams:upgrade-system-tests-22') { } project(':streams:upgrade-system-tests-23') { - archivesBaseName = "kafka-streams-upgrade-system-tests-23" + base { + archivesName = "kafka-streams-upgrade-system-tests-23" + } dependencies { testImplementation libs.kafkaStreams_23 @@ -2605,7 +2669,9 @@ project(':streams:upgrade-system-tests-23') { } project(':streams:upgrade-system-tests-24') { - archivesBaseName = "kafka-streams-upgrade-system-tests-24" + base { + archivesName = "kafka-streams-upgrade-system-tests-24" + } dependencies { testImplementation libs.kafkaStreams_24 @@ -2618,7 +2684,9 @@ project(':streams:upgrade-system-tests-24') { } project(':streams:upgrade-system-tests-25') { - archivesBaseName = "kafka-streams-upgrade-system-tests-25" + base { + archivesName = "kafka-streams-upgrade-system-tests-25" + } dependencies { testImplementation libs.kafkaStreams_25 @@ -2631,7 +2699,9 @@ project(':streams:upgrade-system-tests-25') { } project(':streams:upgrade-system-tests-26') { - archivesBaseName = "kafka-streams-upgrade-system-tests-26" + base { + archivesName = "kafka-streams-upgrade-system-tests-26" + } dependencies { testImplementation libs.kafkaStreams_26 @@ -2644,7 +2714,9 @@ project(':streams:upgrade-system-tests-26') { } project(':streams:upgrade-system-tests-27') { - archivesBaseName = "kafka-streams-upgrade-system-tests-27" + base { + archivesName = "kafka-streams-upgrade-system-tests-27" + } dependencies { testImplementation libs.kafkaStreams_27 @@ -2657,7 +2729,9 @@ project(':streams:upgrade-system-tests-27') { } project(':streams:upgrade-system-tests-28') { - archivesBaseName = "kafka-streams-upgrade-system-tests-28" + base { + archivesName = "kafka-streams-upgrade-system-tests-28" + } dependencies { testImplementation libs.kafkaStreams_28 @@ -2670,7 +2744,9 @@ project(':streams:upgrade-system-tests-28') { } project(':streams:upgrade-system-tests-30') { - archivesBaseName = "kafka-streams-upgrade-system-tests-30" + base { + archivesName = "kafka-streams-upgrade-system-tests-30" + } dependencies { testImplementation libs.kafkaStreams_30 @@ -2683,7 +2759,9 @@ project(':streams:upgrade-system-tests-30') { } project(':streams:upgrade-system-tests-31') { - archivesBaseName = "kafka-streams-upgrade-system-tests-31" + base { + archivesName = "kafka-streams-upgrade-system-tests-31" + } dependencies { testImplementation libs.kafkaStreams_31 @@ -2696,7 +2774,9 @@ project(':streams:upgrade-system-tests-31') { } project(':streams:upgrade-system-tests-32') { - archivesBaseName = "kafka-streams-upgrade-system-tests-32" + base { + archivesName = "kafka-streams-upgrade-system-tests-32" + } dependencies { testImplementation libs.kafkaStreams_32 @@ -2709,7 +2789,9 @@ project(':streams:upgrade-system-tests-32') { } project(':streams:upgrade-system-tests-33') { - archivesBaseName = "kafka-streams-upgrade-system-tests-33" + base { + archivesName = "kafka-streams-upgrade-system-tests-33" + } dependencies { testImplementation libs.kafkaStreams_33 @@ -2722,7 +2804,9 @@ project(':streams:upgrade-system-tests-33') { } project(':streams:upgrade-system-tests-34') { - archivesBaseName = "kafka-streams-upgrade-system-tests-34" + base { + archivesName = "kafka-streams-upgrade-system-tests-34" + } dependencies { testImplementation libs.kafkaStreams_34 @@ -2735,7 +2819,9 @@ project(':streams:upgrade-system-tests-34') { } project(':streams:upgrade-system-tests-35') { - archivesBaseName = "kafka-streams-upgrade-system-tests-35" + base { + archivesName = "kafka-streams-upgrade-system-tests-35" + } dependencies { testImplementation libs.kafkaStreams_35 @@ -2748,7 +2834,9 @@ project(':streams:upgrade-system-tests-35') { } project(':streams:upgrade-system-tests-36') { - archivesBaseName = "kafka-streams-upgrade-system-tests-36" + base { + archivesName = "kafka-streams-upgrade-system-tests-36" + } dependencies { testImplementation libs.kafkaStreams_36 @@ -2761,7 +2849,9 @@ project(':streams:upgrade-system-tests-36') { } project(':streams:upgrade-system-tests-37') { - archivesBaseName = "kafka-streams-upgrade-system-tests-37" + base { + archivesName = "kafka-streams-upgrade-system-tests-37" + } dependencies { testImplementation libs.kafkaStreams_37 @@ -2846,7 +2936,9 @@ project(':jmh-benchmarks') { } project(':log4j-appender') { - archivesBaseName = "kafka-log4j-appender" + base { + archivesName = "kafka-log4j-appender" + } dependencies { implementation project(':clients') @@ -2865,7 +2957,9 @@ project(':log4j-appender') { } project(':connect:api') { - archivesBaseName = "connect-api" + base { + archivesName = "connect-api" + } dependencies { api project(':clients') @@ -2900,7 +2994,9 @@ project(':connect:api') { } project(':connect:transforms') { - archivesBaseName = "connect-transforms" + base { + archivesName = "connect-transforms" + } dependencies { api project(':connect:api') @@ -2936,7 +3032,9 @@ project(':connect:transforms') { } project(':connect:json') { - archivesBaseName = "connect-json" + base { + archivesName = "connect-json" + } dependencies { api project(':connect:api') @@ -2980,7 +3078,9 @@ project(':connect:runtime') { swagger } - archivesBaseName = "connect-runtime" + base { + archivesName = "connect-runtime" + } dependencies { // connect-runtime is used in tests, use `api` for modules below for backwards compatibility even though @@ -3122,7 +3222,9 @@ project(':connect:runtime') { } project(':connect:file') { - archivesBaseName = "connect-file" + base { + archivesName = "connect-file" + } dependencies { implementation project(':connect:api') @@ -3162,7 +3264,9 @@ project(':connect:file') { } project(':connect:basic-auth-extension') { - archivesBaseName = "connect-basic-auth-extension" + base { + archivesName = "connect-basic-auth-extension" + } dependencies { implementation project(':connect:api') @@ -3202,7 +3306,9 @@ project(':connect:basic-auth-extension') { } project(':connect:mirror') { - archivesBaseName = "connect-mirror" + base { + archivesName = "connect-mirror" + } dependencies { implementation project(':connect:api') @@ -3290,7 +3396,9 @@ project(':connect:mirror') { } project(':connect:mirror-client') { - archivesBaseName = "connect-mirror-client" + base { + archivesName = "connect-mirror-client" + } dependencies { implementation project(':clients') @@ -3325,7 +3433,9 @@ project(':connect:mirror-client') { } project(':connect:test-plugins') { - archivesBaseName = "connect-test-plugins" + base { + archivesName = "connect-test-plugins" + } dependencies { api project(':connect:api') diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index ed0f34e4fffad..ab6177961f51e 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -211,6 +211,11 @@ + + + + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index a8bc6351db3fa..eaeb29d3243a2 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -99,7 +99,7 @@ files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest|KafkaRaftClientTest).java"/> + files="(ConsumerCoordinator|BufferPool|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|PluginUtils|MiniTrogdorCluster|TasksRequest|KafkaProducer|AbstractStickyAssignor|KafkaRaftClient|Authorizer|FetchSessionHandler|RecordAccumulator|Shell).java"/> @@ -143,7 +143,7 @@ + files="(DistributedHerder|DistributedConfig|KafkaConfigBackingStore|IncrementalCooperativeAssignor).java"/> + files="(JsonConverter|ConnectHeaders).java"/> + files="(KafkaConfigBackingStore|ConnectMetricsRegistry).java"/> @@ -347,7 +347,7 @@ + files="(GroupMetadataManager|GroupMetadataManagerTest).java"/> implements ConsumerDelegate { *
  • {@link ConsumerRebalanceListener} callbacks that are to be executed on the application thread
  • * */ - private class BackgroundEventProcessor extends EventProcessor { + private class BackgroundEventProcessor implements EventProcessor { - private final ApplicationEventHandler applicationEventHandler; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; - public BackgroundEventProcessor(final LogContext logContext, - final BlockingQueue backgroundEventQueue, - final ApplicationEventHandler applicationEventHandler, - final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) { - super(logContext, backgroundEventQueue); - this.applicationEventHandler = applicationEventHandler; + public BackgroundEventProcessor(final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker) { this.rebalanceListenerInvoker = rebalanceListenerInvoker; } - /** - * Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}. - * It is possible that {@link ErrorEvent an error} - * could occur when processing the events. In such cases, the processor will take a reference to the first - * error, continue to process the remaining events, and then throw the first error that occurred. - */ - @Override - public boolean process() { - AtomicReference firstError = new AtomicReference<>(); - - ProcessHandler processHandler = (event, error) -> { - if (error.isPresent()) { - KafkaException e = error.get(); - - if (!firstError.compareAndSet(null, e)) { - log.warn("An error occurred when processing the event: {}", e.getMessage(), e); - } - } - }; - - boolean hadEvents = process(processHandler); - - if (firstError.get() != null) - throw firstError.get(); - - return hadEvents; - } - @Override public void process(final BackgroundEvent event) { switch (event.type()) { @@ -247,7 +217,9 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { private final KafkaConsumerMetrics kafkaConsumerMetrics; private Logger log; private final String clientId; + private final BlockingQueue backgroundEventQueue; private final BackgroundEventProcessor backgroundEventProcessor; + private final CompletableEventReaper backgroundEventReaper; private final Deserializers deserializers; /** @@ -294,6 +266,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { valueDeserializer, Time.SYSTEM, ApplicationEventHandler::new, + CompletableEventReaper::new, FetchCollector::new, ConsumerMetadata::new, new LinkedBlockingQueue<>() @@ -306,6 +279,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { final Deserializer valueDeserializer, final Time time, final ApplicationEventHandlerFactory applicationEventHandlerFactory, + final CompletableEventReaperFactory backgroundEventReaperFactory, final FetchCollectorFactory fetchCollectorFactory, final ConsumerMetadataFactory metadataFactory, final LinkedBlockingQueue backgroundEventQueue) { @@ -317,6 +291,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); LogContext logContext = createLogContext(config, groupRebalanceConfig); + this.backgroundEventQueue = backgroundEventQueue; this.log = logContext.logger(getClass()); log.debug("Initializing the Kafka consumer"); @@ -378,12 +353,12 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { ); final Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier(logContext, metadata, - applicationEventQueue, requestManagersSupplier); this.applicationEventHandler = applicationEventHandlerFactory.build( logContext, time, applicationEventQueue, + new CompletableEventReaper(logContext), applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); @@ -395,11 +370,9 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { new RebalanceCallbackMetricsManager(metrics) ); this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, rebalanceListenerInvoker ); + this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext); this.assignors = ConsumerPartitionAssignor.getAssignorInstances( config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) @@ -444,6 +417,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { Time time, ApplicationEventHandler applicationEventHandler, BlockingQueue backgroundEventQueue, + CompletableEventReaper backgroundEventReaper, ConsumerRebalanceListenerInvoker rebalanceListenerInvoker, Metrics metrics, SubscriptionState subscriptions, @@ -461,12 +435,9 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; this.interceptors = Objects.requireNonNull(interceptors); this.time = time; - this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker - ); + this.backgroundEventQueue = backgroundEventQueue; + this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); + this.backgroundEventReaper = backgroundEventReaper; this.metrics = metrics; this.groupMetadata.set(initializeGroupMetadata(groupId, Optional.empty())); this.metadata = metadata; @@ -526,7 +497,7 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { this.groupMetadata.set(initializeGroupMetadata(config, groupRebalanceConfig)); BlockingQueue applicationEventQueue = new LinkedBlockingQueue<>(); - BlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + this.backgroundEventQueue = new LinkedBlockingQueue<>(); BackgroundEventHandler backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue); ConsumerRebalanceListenerInvoker rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker( logContext, @@ -563,21 +534,17 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) { Supplier applicationEventProcessorSupplier = ApplicationEventProcessor.supplier( logContext, metadata, - applicationEventQueue, requestManagersSupplier ); this.applicationEventHandler = new ApplicationEventHandler(logContext, time, applicationEventQueue, + new CompletableEventReaper(logContext), applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); - this.backgroundEventProcessor = new BackgroundEventProcessor( - logContext, - backgroundEventQueue, - applicationEventHandler, - rebalanceListenerInvoker - ); + this.backgroundEventProcessor = new BackgroundEventProcessor(rebalanceListenerInvoker); + this.backgroundEventReaper = new CompletableEventReaper(logContext); } // auxiliary interface for testing @@ -587,6 +554,7 @@ ApplicationEventHandler build( final LogContext logContext, final Time time, final BlockingQueue applicationEventQueue, + final CompletableEventReaper applicationEventReaper, final Supplier applicationEventProcessorSupplier, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier @@ -594,6 +562,13 @@ ApplicationEventHandler build( } + // auxiliary interface for testing + interface CompletableEventReaperFactory { + + CompletableEventReaper build(final LogContext logContext); + + } + // auxiliary interface for testing interface FetchCollectorFactory { @@ -939,14 +914,12 @@ public Map committed(final Set committedOffsets = applicationEventHandler.addAndGet(event, - timer); + final Map committedOffsets = applicationEventHandler.addAndGet(event); committedOffsets.forEach(this::updateLastSeenEpochIfNewer); return committedOffsets; } catch (TimeoutException e) { @@ -992,12 +965,11 @@ public List partitionsFor(String topic, Duration timeout) { throw new TimeoutException(); } - final Timer timer = time.timer(timeout); - final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, timer); + final TopicMetadataEvent topicMetadataEvent = new TopicMetadataEvent(topic, calculateDeadlineMs(time, timeout)); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { Map> topicMetadata = - applicationEventHandler.addAndGet(topicMetadataEvent, timer); + applicationEventHandler.addAndGet(topicMetadataEvent); return topicMetadata.getOrDefault(topic, Collections.emptyList()); } finally { @@ -1021,11 +993,10 @@ public Map> listTopics(Duration timeout) { throw new TimeoutException(); } - final Timer timer = time.timer(timeout); - final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(timer); + final AllTopicsMetadataEvent topicMetadataEvent = new AllTopicsMetadataEvent(calculateDeadlineMs(time, timeout)); wakeupTrigger.setActiveTask(topicMetadataEvent.future()); try { - return applicationEventHandler.addAndGet(topicMetadataEvent, timer); + return applicationEventHandler.addAndGet(topicMetadataEvent); } finally { wakeupTrigger.clearTask(); } @@ -1093,10 +1064,9 @@ public Map offsetsForTimes(Map offsetsForTimes(Map beginningOrEndOffset(Collection timestampToSearch = partitions .stream() .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); - Timer timer = time.timer(timeout); ListOffsetsEvent listOffsetsEvent = new ListOffsetsEvent( timestampToSearch, - timer, + calculateDeadlineMs(time, timeout), false); // If timeout is set to zero return empty immediately; otherwise try to get the results @@ -1167,9 +1136,7 @@ private Map beginningOrEndOffset(Collection offsetAndTimestampMap; - offsetAndTimestampMap = applicationEventHandler.addAndGet( - listOffsetsEvent, - timer); + offsetAndTimestampMap = applicationEventHandler.addAndGet(listOffsetsEvent); return offsetAndTimestampMap.entrySet() .stream() .collect(Collectors.toMap( @@ -1269,6 +1236,12 @@ private void close(Duration timeout, boolean swallowException) { if (applicationEventHandler != null) closeQuietly(() -> applicationEventHandler.close(Duration.ofMillis(closeTimer.remainingMs())), "Failed shutting down network thread", firstException); closeTimer.update(); + + // close() can be called from inside one of the constructors. In that case, it's possible that neither + // the reaper nor the background event queue were constructed, so check them first to avoid NPE. + if (backgroundEventReaper != null && backgroundEventQueue != null) + backgroundEventReaper.reap(backgroundEventQueue); + closeQuietly(interceptors, "consumer interceptors", firstException); closeQuietly(kafkaConsumerMetrics, "kafka consumer metrics", firstException); closeQuietly(metrics, "consumer metrics", firstException); @@ -1295,21 +1268,21 @@ private void close(Duration timeout, boolean swallowException) { void prepareShutdown(final Timer timer, final AtomicReference firstException) { if (!groupMetadata.get().isPresent()) return; - maybeAutoCommitSync(autoCommitEnabled, timer); + + if (autoCommitEnabled) + autoCommitSync(timer); + applicationEventHandler.add(new CommitOnCloseEvent()); completeQuietly( () -> { maybeRevokePartitions(); - applicationEventHandler.addAndGet(new LeaveOnCloseEvent(timer), timer); + applicationEventHandler.addAndGet(new LeaveOnCloseEvent(calculateDeadlineMs(timer))); }, "Failed to send leaveGroup heartbeat with a timeout(ms)=" + timer.timeoutMs(), firstException); } // Visible for testing - void maybeAutoCommitSync(final boolean shouldAutoCommit, - final Timer timer) { - if (!shouldAutoCommit) - return; + void autoCommitSync(final Timer timer) { Map allConsumed = subscriptions.allConsumed(); log.debug("Sending synchronous auto-commit of offsets {} on closing", allConsumed); try { @@ -1376,10 +1349,10 @@ public void commitSync(Map offsets, Duration acquireAndEnsureOpen(); long commitStart = time.nanoseconds(); try { - Timer requestTimer = time.timer(timeout.toMillis()); - SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, requestTimer); + SyncCommitEvent syncCommitEvent = new SyncCommitEvent(offsets, calculateDeadlineMs(time, timeout)); CompletableFuture commitFuture = commit(syncCommitEvent); + Timer requestTimer = time.timer(timeout.toMillis()); awaitPendingAsyncCommitsAndExecuteCommitCallbacks(requestTimer, true); wakeupTrigger.setActiveTask(commitFuture); @@ -1523,12 +1496,12 @@ public void unsubscribe() { fetchBuffer.retainAll(Collections.emptySet()); if (groupMetadata.get().isPresent()) { Timer timer = time.timer(Long.MAX_VALUE); - UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(timer); + UnsubscribeEvent unsubscribeEvent = new UnsubscribeEvent(calculateDeadlineMs(timer)); applicationEventHandler.add(unsubscribeEvent); log.info("Unsubscribing all topics or patterns and assigned partitions"); try { - processBackgroundEvents(backgroundEventProcessor, unsubscribeEvent.future(), timer); + processBackgroundEvents(unsubscribeEvent.future(), timer); log.info("Unsubscribed all topics or patterns and assigned partitions"); } catch (TimeoutException e) { log.error("Failed while waiting for the unsubscribe event to complete"); @@ -1637,7 +1610,7 @@ private boolean updateFetchPositions(final Timer timer) { // Validate positions using the partition leader end offsets, to detect if any partition // has been truncated due to a leader change. This will trigger an OffsetForLeaderEpoch // request, retrieve the partition end offsets, and validate the current position against it. - applicationEventHandler.addAndGet(new ValidatePositionsEvent(timer), timer); + applicationEventHandler.addAndGet(new ValidatePositionsEvent(calculateDeadlineMs(timer))); cachedSubscriptionHasAllFetchPositions = subscriptions.hasAllFetchPositions(); if (cachedSubscriptionHasAllFetchPositions) return true; @@ -1660,7 +1633,7 @@ private boolean updateFetchPositions(final Timer timer) { // which are awaiting reset. This will trigger a ListOffset request, retrieve the // partition offsets according to the strategy (ex. earliest, latest), and update the // positions. - applicationEventHandler.addAndGet(new ResetPositionsEvent(timer), timer); + applicationEventHandler.addAndGet(new ResetPositionsEvent(calculateDeadlineMs(timer))); return true; } catch (TimeoutException e) { return false; @@ -1693,9 +1666,9 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer timer) { final FetchCommittedOffsetsEvent event = new FetchCommittedOffsetsEvent( initializingPartitions, - timer); + calculateDeadlineMs(timer)); wakeupTrigger.setActiveTask(event.future()); - final Map offsets = applicationEventHandler.addAndGet(event, timer); + final Map offsets = applicationEventHandler.addAndGet(event); refreshCommittedOffsets(offsets, metadata, subscriptions); return true; } catch (TimeoutException e) { @@ -1722,7 +1695,7 @@ public boolean updateAssignmentMetadataIfNeeded(Timer timer) { maybeThrowFencedInstanceException(); offsetCommitCallbackInvoker.executeCallbacks(); maybeUpdateSubscriptionMetadata(); - backgroundEventProcessor.process(); + processBackgroundEvents(); return updateFetchPositions(timer); } @@ -1848,6 +1821,40 @@ private void subscribeInternal(Collection topics, Optional firstError = new AtomicReference<>(); + + LinkedList events = new LinkedList<>(); + backgroundEventQueue.drainTo(events); + + for (BackgroundEvent event : events) { + try { + if (event instanceof CompletableEvent) + backgroundEventReaper.add((CompletableEvent) event); + + backgroundEventProcessor.process(event); + } catch (Throwable t) { + KafkaException e = ConsumerUtils.maybeWrapAsKafkaException(t); + + if (!firstError.compareAndSet(null, e)) + log.warn("An error occurred when processing the background event: {}", e.getMessage(), e); + } + } + + backgroundEventReaper.reap(time.milliseconds()); + + if (firstError.get() != null) + throw firstError.get(); + + return !events.isEmpty(); + } + /** * This method can be used by cases where the caller has an event that needs to both block for completion but * also process background events. For some events, in order to fully process the associated logic, the @@ -1870,28 +1877,26 @@ private void subscribeInternal(Collection topics, Optional T processBackgroundEvents(EventProcessor eventProcessor, - Future future, - Timer timer) { + T processBackgroundEvents(Future future, Timer timer) { do { - boolean hadEvents = eventProcessor.process(); + boolean hadEvents = processBackgroundEvents(); try { if (future.isDone()) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java index 8959345bffdcf..5244af9c827d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.RecordDeserializationException; +import org.apache.kafka.common.errors.RecordDeserializationException.DeserializationExceptionOrigin; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; @@ -311,25 +312,39 @@ ConsumerRecord parseRecord(Deserializers deserializers, Optional leaderEpoch, TimestampType timestampType, Record record) { + ByteBuffer keyBytes = record.key(); + ByteBuffer valueBytes = record.value(); + Headers headers = new RecordHeaders(record.headers()); + K key; + V value; try { - long offset = record.offset(); - long timestamp = record.timestamp(); - Headers headers = new RecordHeaders(record.headers()); - ByteBuffer keyBytes = record.key(); - K key = keyBytes == null ? null : deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes); - ByteBuffer valueBytes = record.value(); - V value = valueBytes == null ? null : deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes); - return new ConsumerRecord<>(partition.topic(), partition.partition(), offset, - timestamp, timestampType, - keyBytes == null ? ConsumerRecord.NULL_SIZE : keyBytes.remaining(), - valueBytes == null ? ConsumerRecord.NULL_SIZE : valueBytes.remaining(), - key, value, headers, leaderEpoch); + key = keyBytes == null ? null : deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes); } catch (RuntimeException e) { - log.error("Deserializers with error: {}", deserializers); - throw new RecordDeserializationException(partition, record.offset(), - "Error deserializing key/value for partition " + partition + - " at offset " + record.offset() + ". If needed, please seek past the record to continue consumption.", e); + log.error("Key Deserializers with error: {}", deserializers); + throw newRecordDeserializationException(DeserializationExceptionOrigin.KEY, partition, timestampType, record, e, headers); } + try { + value = valueBytes == null ? null : deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes); + } catch (RuntimeException e) { + log.error("Value Deserializers with error: {}", deserializers); + throw newRecordDeserializationException(DeserializationExceptionOrigin.VALUE, partition, timestampType, record, e, headers); + } + return new ConsumerRecord<>(partition.topic(), partition.partition(), record.offset(), + record.timestamp(), timestampType, + keyBytes == null ? ConsumerRecord.NULL_SIZE : keyBytes.remaining(), + valueBytes == null ? ConsumerRecord.NULL_SIZE : valueBytes.remaining(), + key, value, headers, leaderEpoch); + } + + private static RecordDeserializationException newRecordDeserializationException(DeserializationExceptionOrigin origin, + TopicPartition partition, + TimestampType timestampType, + Record record, + RuntimeException e, + Headers headers) { + return new RecordDeserializationException(origin, partition, record.offset(), record.timestamp(), timestampType, record.key(), record.value(), headers, + "Error deserializing " + origin.name() + " for partition " + partition + " at offset " + record.offset() + + ". If needed, please seek past the record to continue consumption.", e); } private Optional maybeLeaderEpoch(int leaderEpoch) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index aa352cd68a22e..adee6594603bb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -20,6 +20,8 @@ import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventProcessor; import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.common.internals.IdempotentCloser; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.KafkaThread; @@ -31,9 +33,11 @@ import java.io.Closeable; import java.time.Duration; import java.util.Collection; +import java.util.LinkedList; import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.BlockingQueue; import java.util.function.Supplier; import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.DEFAULT_CLOSE_TIMEOUT_MS; @@ -50,6 +54,8 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { private static final String BACKGROUND_THREAD_NAME = "consumer_background_thread"; private final Time time; private final Logger log; + private final BlockingQueue applicationEventQueue; + private final CompletableEventReaper applicationEventReaper; private final Supplier applicationEventProcessorSupplier; private final Supplier networkClientDelegateSupplier; private final Supplier requestManagersSupplier; @@ -63,12 +69,16 @@ public class ConsumerNetworkThread extends KafkaThread implements Closeable { public ConsumerNetworkThread(LogContext logContext, Time time, + BlockingQueue applicationEventQueue, + CompletableEventReaper applicationEventReaper, Supplier applicationEventProcessorSupplier, Supplier networkClientDelegateSupplier, Supplier requestManagersSupplier) { super(BACKGROUND_THREAD_NAME, true); this.time = time; this.log = logContext.logger(getClass()); + this.applicationEventQueue = applicationEventQueue; + this.applicationEventReaper = applicationEventReaper; this.applicationEventProcessorSupplier = applicationEventProcessorSupplier; this.networkClientDelegateSupplier = networkClientDelegateSupplier; this.requestManagersSupplier = requestManagersSupplier; @@ -125,10 +135,7 @@ void initializeResources() { * */ void runOnce() { - // Process the events—if any—that were produced by the application thread. It is possible that when processing - // an event generates an error. In such cases, the processor will log an exception, but we do not want those - // errors to be propagated to the caller. - applicationEventProcessor.process(); + processApplicationEvents(); final long currentTimeMs = time.milliseconds(); final long pollWaitTimeMs = requestManagers.entries().stream() @@ -144,6 +151,36 @@ void runOnce() { .map(Optional::get) .map(rm -> rm.maximumTimeToWait(currentTimeMs)) .reduce(Long.MAX_VALUE, Math::min); + + reapExpiredApplicationEvents(currentTimeMs); + } + + /** + * Process the events—if any—that were produced by the application thread. + */ + private void processApplicationEvents() { + LinkedList events = new LinkedList<>(); + applicationEventQueue.drainTo(events); + + for (ApplicationEvent event : events) { + try { + if (event instanceof CompletableEvent) + applicationEventReaper.add((CompletableEvent) event); + + applicationEventProcessor.process(event); + } catch (Throwable t) { + log.warn("Error processing event {}", t.getMessage(), t); + } + } + } + + /** + * "Complete" any events that have expired. This cleanup step should only be called after the network I/O + * thread has made at least one call to {@link NetworkClientDelegate#poll(long, long) poll} so that each event + * is given least one attempt to satisfy any network requests before checking if a timeout has expired. + */ + private void reapExpiredApplicationEvents(long currentTimeMs) { + applicationEventReaper.reap(currentTimeMs); } /** @@ -273,9 +310,10 @@ void cleanup() { log.error("Unexpected error during shutdown. Proceed with closing.", e); } finally { sendUnsentRequests(timer); + applicationEventReaper.reap(applicationEventQueue); + closeQuietly(requestManagers, "request managers"); closeQuietly(networkClientDelegate, "network client delegate"); - closeQuietly(applicationEventProcessor, "application event processor"); log.debug("Closed the consumer network thread"); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java index f65a0642a4300..a956ef3a93960 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java @@ -383,7 +383,15 @@ private void onErrorResponse(final ConsumerGroupHeartbeatResponse response, case UNRELEASED_INSTANCE_ID: logger.error("GroupHeartbeatRequest failed due to unreleased instance id {}: {}", membershipManager.groupInstanceId().orElse("null"), errorMessage); - handleFatalFailure(Errors.UNRELEASED_INSTANCE_ID.exception(errorMessage)); + handleFatalFailure(error.exception(errorMessage)); + break; + + case FENCED_INSTANCE_ID: + logger.error("GroupHeartbeatRequest failed due to fenced instance id {}: {}. " + + "This is expected in the case that the member was removed from the group " + + "by an admin client, and another member joined using the same group instance id.", + membershipManager.groupInstanceId().orElse("null"), errorMessage); + handleFatalFailure(error.exception(errorMessage)); break; case INVALID_REQUEST: diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java index 8e93de5a24c4c..76a550ad71985 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java @@ -1337,6 +1337,7 @@ private CompletableFuture enqueueConsumerRebalanceListenerCallback(Consume Set partitions) { SortedSet sortedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); sortedPartitions.addAll(partitions); + CompletableBackgroundEvent event = new ConsumerRebalanceListenerCallbackNeededEvent(methodName, sortedPartitions); backgroundEventHandler.add(event); log.debug("The event to trigger the {} method execution was enqueued successfully", methodName.fullyQualifiedMethodName()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java index 3347002cc6fea..9621e34ef5b94 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AbstractTopicMetadataEvent.java @@ -17,14 +17,13 @@ package org.apache.kafka.clients.consumer.internals.events; import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.utils.Timer; import java.util.List; import java.util.Map; public abstract class AbstractTopicMetadataEvent extends CompletableApplicationEvent>> { - protected AbstractTopicMetadataEvent(final Type type, final Timer timer) { - super(type, timer); + protected AbstractTopicMetadataEvent(final Type type, final long deadlineMs) { + super(type, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java index bda18e642105b..8fe1702c85bd6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AllTopicsMetadataEvent.java @@ -16,11 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - public class AllTopicsMetadataEvent extends AbstractTopicMetadataEvent { - public AllTopicsMetadataEvent(final Timer timer) { - super(Type.ALL_TOPICS_METADATA, timer); + public AllTopicsMetadataEvent(final long deadlineMs) { + super(Type.ALL_TOPICS_METADATA, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java index cb2616fda478c..1e082e11978be 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.internals.IdempotentCloser; 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.common.utils.Utils; import org.slf4j.Logger; @@ -32,7 +31,6 @@ import java.util.Objects; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import java.util.function.Supplier; /** @@ -49,6 +47,7 @@ public class ApplicationEventHandler implements Closeable { public ApplicationEventHandler(final LogContext logContext, final Time time, final BlockingQueue applicationEventQueue, + final CompletableEventReaper applicationEventReaper, final Supplier applicationEventProcessorSupplier, final Supplier networkClientDelegateSupplier, final Supplier requestManagersSupplier) { @@ -56,6 +55,8 @@ public ApplicationEventHandler(final LogContext logContext, this.applicationEventQueue = applicationEventQueue; this.networkThread = new ConsumerNetworkThread(logContext, time, + applicationEventQueue, + applicationEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier); @@ -99,17 +100,16 @@ public long maximumTimeToWait() { * *

    * - * See {@link ConsumerUtils#getResult(Future, Timer)} and {@link Future#get(long, TimeUnit)} for more details. + * See {@link ConsumerUtils#getResult(Future)} for more details. * * @param event A {@link CompletableApplicationEvent} created by the polling thread * @return Value that is the result of the event * @param Type of return value of the event */ - public T addAndGet(final CompletableApplicationEvent event, final Timer timer) { + public T addAndGet(final CompletableApplicationEvent event) { Objects.requireNonNull(event, "CompletableApplicationEvent provided to addAndGet must be non-null"); - Objects.requireNonNull(timer, "Timer provided to addAndGet must be non-null"); add(event); - return ConsumerUtils.getResult(event.future(), timer); + return ConsumerUtils.getResult(event.future()); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java index d5cb1c04b38f4..7ee0c09d40df9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java @@ -33,7 +33,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; import java.util.function.Supplier; @@ -42,31 +41,20 @@ * An {@link EventProcessor} that is created and executes in the {@link ConsumerNetworkThread network thread} * which processes {@link ApplicationEvent application events} generated by the application thread. */ -public class ApplicationEventProcessor extends EventProcessor { +public class ApplicationEventProcessor implements EventProcessor { private final Logger log; private final ConsumerMetadata metadata; private final RequestManagers requestManagers; public ApplicationEventProcessor(final LogContext logContext, - final BlockingQueue applicationEventQueue, final RequestManagers requestManagers, final ConsumerMetadata metadata) { - super(logContext, applicationEventQueue); this.log = logContext.logger(ApplicationEventProcessor.class); this.requestManagers = requestManagers; this.metadata = metadata; } - /** - * Process the events—if any—that were produced by the application thread. It is possible that when processing - * an event generates an error. In such cases, the processor will log an exception, but we do not want those - * errors to be propagated to the caller. - */ - public boolean process() { - return process((event, error) -> error.ifPresent(e -> log.warn("Error processing event {}", e.getMessage(), e))); - } - @SuppressWarnings({"CyclomaticComplexity"}) @Override public void process(ApplicationEvent event) { @@ -273,7 +261,7 @@ private void process(final ConsumerRebalanceListenerCallbackCompletedEvent event manager.consumerRebalanceListenerCallbackCompleted(event); } - private void process(final CommitOnCloseEvent event) { + private void process(@SuppressWarnings("unused") final CommitOnCloseEvent event) { if (!requestManagers.commitRequestManager.isPresent()) return; log.debug("Signal CommitRequestManager closing"); @@ -309,7 +297,6 @@ private void process(final LeaveOnCloseEvent event) { */ public static Supplier supplier(final LogContext logContext, final ConsumerMetadata metadata, - final BlockingQueue applicationEventQueue, final Supplier requestManagersSupplier) { return new CachedSupplier() { @Override @@ -317,7 +304,6 @@ protected ApplicationEventProcessor create() { RequestManagers requestManagers = requestManagersSupplier.get(); return new ApplicationEventProcessor( logContext, - applicationEventQueue, requestManagers, metadata ); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java index 1da7b84039ab8..dc863b0ee659a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CommitEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -30,11 +29,6 @@ public abstract class CommitEvent extends CompletableApplicationEvent { */ private final Map offsets; - protected CommitEvent(final Type type, final Map offsets, final Timer timer) { - super(type, timer); - this.offsets = validate(offsets); - } - protected CommitEvent(final Type type, final Map offsets, final long deadlineMs) { super(type, deadlineMs); this.offsets = validate(offsets); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java index dae9e9f1017ba..dffac12902177 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableApplicationEvent.java @@ -16,9 +16,6 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - -import java.util.Objects; import java.util.concurrent.CompletableFuture; /** @@ -32,13 +29,9 @@ public abstract class CompletableApplicationEvent extends ApplicationEvent im private final CompletableFuture future; private final long deadlineMs; - protected CompletableApplicationEvent(final Type type, final Timer timer) { - super(type); - this.future = new CompletableFuture<>(); - Objects.requireNonNull(timer); - this.deadlineMs = timer.remainingMs() + timer.currentTimeMs(); - } - + /** + * Note: the {@code deadlineMs} is the future time of expiration, not a timeout. + */ protected CompletableApplicationEvent(final Type type, final long deadlineMs) { super(type); this.future = new CompletableFuture<>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java index 1a58515a5cbce..d02010496e545 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableBackgroundEvent.java @@ -27,10 +27,15 @@ public abstract class CompletableBackgroundEvent extends BackgroundEvent implements CompletableEvent { private final CompletableFuture future; + private final long deadlineMs; - protected CompletableBackgroundEvent(final Type type) { + /** + * Note: the {@code deadlineMs} is the future time of expiration, not a timeout. + */ + protected CompletableBackgroundEvent(final Type type, final long deadlineMs) { super(type); this.future = new CompletableFuture<>(); + this.deadlineMs = deadlineMs; } @Override @@ -38,8 +43,13 @@ public CompletableFuture future() { return future; } + @Override + public long deadlineMs() { + return deadlineMs; + } + @Override protected String toStringBase() { - return super.toStringBase() + ", future=" + future; + return super.toStringBase() + ", future=" + future + ", deadlineMs=" + deadlineMs; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java index 97559d8cb9be2..20231b0f99a10 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEvent.java @@ -16,9 +16,112 @@ */ package org.apache.kafka.clients.consumer.internals.events; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; + +import java.time.Duration; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; + +import static java.util.Objects.requireNonNull; +/** + * {@code CompletableEvent} is an interface that is used by both {@link CompletableApplicationEvent} and + * {@link CompletableBackgroundEvent} for common processing and logic. A {@code CompletableEvent} is one that + * allows the caller to get the {@link #future() future} related to the event and the event's + * {@link #deadlineMs() expiration timestamp}. + * + * @param Return type for the event when completed + */ public interface CompletableEvent { + /** + * Returns the {@link CompletableFuture future} associated with this event. Any event will have some related + * logic that is executed on its behalf. The event can complete in one of the following ways: + * + *

      + *
    • + * Success: when the logic for the event completes successfully, the data generated by that event + * (if applicable) is passed to {@link CompletableFuture#complete(Object)}. In the case where the generic + * bound type is specified as {@link Void}, {@code null} is provided.
    • + *
    • + * Error: when the the event logic generates an error, the error is passed to + * {@link CompletableFuture#completeExceptionally(Throwable)}. + *
    • + *
    • + * Timeout: when the time spent executing the event logic exceeds the {@link #deadlineMs() deadline}, an + * instance of {@link TimeoutException} should be created and passed to + * {@link CompletableFuture#completeExceptionally(Throwable)}. This also occurs when an event remains + * incomplete when the consumer closes. + *
    • + *
    + * + * @return Future on which the caller may block or query for completion + * + * @see CompletableEventReaper + */ CompletableFuture future(); + + /** + * This is the deadline that represents the absolute wall clock time by which any event-specific execution should + * complete. This is not a timeout value. After this time has passed, + * {@link CompletableFuture#completeExceptionally(Throwable)} will be invoked with an instance of + * {@link TimeoutException}. + * + * @return Absolute time for event to be completed + * + * @see CompletableEventReaper + */ + long deadlineMs(); + + /** + * Calculate the deadline timestamp based on {@link Timer#currentTimeMs()} and {@link Timer#remainingMs()}. + * + * @param timer Timer + * + * @return Absolute time by which event should be completed + */ + static long calculateDeadlineMs(final Timer timer) { + requireNonNull(timer); + return calculateDeadlineMs(timer.currentTimeMs(), timer.remainingMs()); + } + + /** + * Calculate the deadline timestamp based on {@link Timer#currentTimeMs()} and {@link Duration#toMillis()}. + * + * @param time Time + * @param duration Duration + * + * @return Absolute time by which event should be completed + */ + static long calculateDeadlineMs(final Time time, final Duration duration) { + return calculateDeadlineMs(requireNonNull(time).milliseconds(), requireNonNull(duration).toMillis()); + } + + /** + * Calculate the deadline timestamp based on {@link Timer#currentTimeMs()} and timeout. + * + * @param time Time + * @param timeoutMs Timeout, in milliseconds + * + * @return Absolute time by which event should be completed + */ + static long calculateDeadlineMs(final Time time, final long timeoutMs) { + return calculateDeadlineMs(requireNonNull(time).milliseconds(), timeoutMs); + } + + /** + * Calculate the deadline timestamp based on the current time and timeout. + * + * @param currentTimeMs Current time, in milliseconds + * @param timeoutMs Timeout, in milliseconds + * + * @return Absolute time by which event should be completed + */ + static long calculateDeadlineMs(final long currentTimeMs, final long timeoutMs) { + if (currentTimeMs > Long.MAX_VALUE - timeoutMs) + return Long.MAX_VALUE; + else + return currentTimeMs + timeoutMs; + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java new file mode 100644 index 0000000000000..545a03df8b30d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaper.java @@ -0,0 +1,157 @@ +/* + * 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.clients.consumer.internals.events; + +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + +/** + * {@code CompletableEventReaper} is responsible for tracking {@link CompletableEvent time-bound events} and removing + * any that exceed their {@link CompletableEvent#deadlineMs() deadline} (unless they've already completed). This + * mechanism is used by the {@link AsyncKafkaConsumer} to enforce the timeout provided by the user in its API + * calls (e.g. {@link AsyncKafkaConsumer#commitSync(Duration)}). + */ +public class CompletableEventReaper { + + private final Logger log; + + /** + * List of tracked events that are candidates for expiration. + */ + private final List> tracked; + + public CompletableEventReaper(LogContext logContext) { + this.log = logContext.logger(CompletableEventReaper.class); + this.tracked = new ArrayList<>(); + } + + /** + * Adds a new {@link CompletableEvent event} to track for later completion/expiration. + * + * @param event Event to track + */ + public void add(CompletableEvent event) { + tracked.add(Objects.requireNonNull(event, "Event to track must be non-null")); + } + + /** + * This method performs a two-step process to "complete" {@link CompletableEvent events} that have either expired + * or completed normally: + * + *
      + *
    1. + * For each tracked event which has exceeded its {@link CompletableEvent#deadlineMs() deadline}, an + * instance of {@link TimeoutException} is created and passed to + * {@link CompletableFuture#completeExceptionally(Throwable)}. + *
    2. + *
    3. + * For each tracked event of which its {@link CompletableEvent#future() future} is already in the + * {@link CompletableFuture#isDone() done} state, it will be removed from the list of tracked events. + *
    4. + *
    + * + *

    + * + * This method should be called at regular intervals, based upon the needs of the resource that owns the reaper. + * + * @param currentTimeMs Current time with which to compare against the + * {@link CompletableEvent#deadlineMs() expiration time} + */ + public void reap(long currentTimeMs) { + Consumer> expireEvent = event -> { + long pastDueMs = currentTimeMs - event.deadlineMs(); + TimeoutException error = new TimeoutException(String.format("%s was %s ms past its expiration of %s", event.getClass().getSimpleName(), pastDueMs, event.deadlineMs())); + + if (event.future().completeExceptionally(error)) { + log.debug("Event {} completed exceptionally since its expiration of {} passed {} ms ago", event, event.deadlineMs(), pastDueMs); + } else { + log.trace("Event {} not completed exceptionally since it was previously completed", event); + } + }; + + // First, complete (exceptionally) any events that have passed their deadline AND aren't already complete. + tracked.stream() + .filter(e -> !e.future().isDone()) + .filter(e -> currentTimeMs >= e.deadlineMs()) + .forEach(expireEvent); + // Second, remove any events that are already complete, just to make sure we don't hold references. This will + // include any events that finished successfully as well as any events we just completed exceptionally above. + tracked.removeIf(e -> e.future().isDone()); + } + + /** + * It is possible for the {@link AsyncKafkaConsumer#close() consumer to close} before completing the processing of + * all the events in the queue. In this case, we need to + * {@link CompletableFuture#completeExceptionally(Throwable) expire} any remaining events. + * + *

    + * + * Check each of the {@link #add(CompletableEvent) previously-added} {@link CompletableEvent completable events}, + * and for any that are incomplete, expire them. Also check the core event queue for any incomplete events and + * likewise expire them. + * + *

    + * + * Note: because this is called in the context of {@link AsyncKafkaConsumer#close() closing consumer}, + * don't take the deadline into consideration, just close it regardless. + * + * @param events Events from a queue that have not yet been tracked that also need to be reviewed + */ + public void reap(Collection events) { + Objects.requireNonNull(events, "Event queue to reap must be non-null"); + + Consumer> expireEvent = event -> { + TimeoutException error = new TimeoutException(String.format("%s could not be completed before the consumer closed", event.getClass().getSimpleName())); + + if (event.future().completeExceptionally(error)) { + log.debug("Event {} completed exceptionally since the consumer is closing", event); + } else { + log.trace("Event {} not completed exceptionally since it was completed prior to the consumer closing", event); + } + }; + + tracked.stream() + .filter(e -> !e.future().isDone()) + .forEach(expireEvent); + tracked.clear(); + + events.stream() + .filter(e -> e instanceof CompletableEvent) + .map(e -> (CompletableEvent) e) + .filter(e -> !e.future().isDone()) + .forEach(expireEvent); + events.clear(); + } + + public int size() { + return tracked.size(); + } + + public boolean contains(CompletableEvent event) { + return event != null && tracked.contains(event); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java index 6ce833580c88d..ecb9eedab22c7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java @@ -39,7 +39,7 @@ public class ConsumerRebalanceListenerCallbackNeededEvent extends CompletableBac public ConsumerRebalanceListenerCallbackNeededEvent(final ConsumerRebalanceListenerMethodName methodName, final SortedSet partitions) { - super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED); + super(Type.CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, Long.MAX_VALUE); this.methodName = Objects.requireNonNull(methodName); this.partitions = Collections.unmodifiableSortedSet(partitions); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java index 21916034b37b6..1c0bb0305989e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/EventProcessor.java @@ -16,111 +16,26 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.clients.consumer.internals.ConsumerUtils; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.internals.IdempotentCloser; -import org.apache.kafka.common.utils.LogContext; -import org.slf4j.Logger; - -import java.io.Closeable; -import java.util.LinkedList; -import java.util.List; -import java.util.Objects; -import java.util.Optional; import java.util.concurrent.BlockingQueue; /** - * An {@link EventProcessor} is the means by which events produced by thread A are - * processed by thread B. By definition, threads A and B run in parallel to - * each other, so a mechanism is needed with which to receive and process the events from the other thread. That - * communication channel is formed around {@link BlockingQueue a shared queue} into which thread A - * enqueues events and thread B reads and processes those events. + * An {@code EventProcessor} is the means by which events are processed, the meaning of which is left + * intentionally loose. This is in large part to keep the {@code EventProcessor} focused on what it means to process + * the events, and not linking itself too closely with the rest of the surrounding application. + * + *

    + * + * The {@code EventProcessor} is envisaged as a stateless service that acts as a conduit, receiving an event and + * dispatching to another block of code to process. The semantic meaning of each event is different, so the + * {@code EventProcessor} will need to interact with other parts of the system that maintain state. The + * implementation should not be concerned with the mechanism by which an event arrived for processing. While the + * events are shuffled around the consumer subsystem by means of {@link BlockingQueue shared queues}, it should + * be considered an anti-pattern to need to know how it arrived or what happens after its is processed. */ -public abstract class EventProcessor implements Closeable { - - private final Logger log; - private final BlockingQueue eventQueue; - private final IdempotentCloser closer; - - protected EventProcessor(final LogContext logContext, final BlockingQueue eventQueue) { - this.log = logContext.logger(EventProcessor.class); - this.eventQueue = eventQueue; - this.closer = new IdempotentCloser(); - } - - public abstract boolean process(); - - protected abstract void process(T event); - - @Override - public void close() { - closer.close(this::closeInternal, () -> log.warn("The event processor was already closed")); - } - - protected interface ProcessHandler { - - void onProcess(T event, Optional error); - } - - /** - * Drains all available events from the queue, and then processes them in order. If any errors are thrown while - * processing the individual events, these are submitted to the given {@link ProcessHandler}. - */ - protected boolean process(ProcessHandler processHandler) { - closer.assertOpen("The processor was previously closed, so no further processing can occur"); - - List events = drain(); - - if (events.isEmpty()) - return false; - - for (T event : events) { - try { - Objects.requireNonNull(event, "Attempted to process a null event"); - process(event); - processHandler.onProcess(event, Optional.empty()); - } catch (Throwable t) { - KafkaException error = ConsumerUtils.maybeWrapAsKafkaException(t); - processHandler.onProcess(event, Optional.of(error)); - } - } - - return true; - } - - /** - * It is possible for the consumer to close before complete processing all the events in the queue. In - * this case, we need to throw an exception to notify the user the consumer is closed. - */ - private void closeInternal() { - log.trace("Closing event processor"); - List incompleteEvents = drain(); - - if (incompleteEvents.isEmpty()) - return; - - KafkaException exception = new KafkaException("The consumer is closed"); - - // Check each of the events and if it has a Future that is incomplete, complete it exceptionally. - incompleteEvents - .stream() - .filter(e -> e instanceof CompletableEvent) - .map(e -> ((CompletableEvent) e).future()) - .filter(f -> !f.isDone()) - .forEach(f -> { - log.debug("Completing {} with exception {}", f, exception.getMessage()); - f.completeExceptionally(exception); - }); - - log.debug("Discarding {} events because the consumer is closing", incompleteEvents.size()); - } +public interface EventProcessor { /** - * Moves all the events from the queue to the returned list. + * Process an event that is received. */ - private List drain() { - LinkedList events = new LinkedList<>(); - eventQueue.drainTo(events); - return events; - } + void process(T event); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java index 980a8f1104261..785736791a7c7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/FetchCommittedOffsetsEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.Map; @@ -31,8 +30,8 @@ public class FetchCommittedOffsetsEvent extends CompletableApplicationEvent partitions; - public FetchCommittedOffsetsEvent(final Set partitions, final Timer timer) { - super(Type.FETCH_COMMITTED_OFFSETS, timer); + public FetchCommittedOffsetsEvent(final Set partitions, final long deadlineMs) { + super(Type.FETCH_COMMITTED_OFFSETS, deadlineMs); this.partitions = Collections.unmodifiableSet(partitions); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java index e77b4dfb2893c..647265a1500c8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/LeaveOnCloseEvent.java @@ -16,11 +16,9 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - public class LeaveOnCloseEvent extends CompletableApplicationEvent { - public LeaveOnCloseEvent(final Timer timer) { - super(Type.LEAVE_ON_CLOSE, timer); + public LeaveOnCloseEvent(final long deadlineMs) { + super(Type.LEAVE_ON_CLOSE, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java index 3df4719a7b065..8ae2f1ea57612 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ListOffsetsEvent.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.internals.OffsetAndTimestampInternal; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Collections; import java.util.HashMap; @@ -38,9 +37,9 @@ public class ListOffsetsEvent extends CompletableApplicationEvent timestampToSearch, - Timer timer, + long deadlineMs, boolean requireTimestamps) { - super(Type.LIST_OFFSETS, timer); + super(Type.LIST_OFFSETS, deadlineMs); this.timestampsToSearch = Collections.unmodifiableMap(timestampToSearch); this.requireTimestamps = requireTimestamps; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java index 65893b62ecaa5..86dbb80c0f0ac 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ResetPositionsEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Event for resetting offsets for all assigned partitions that require it. This is an * asynchronous event that generates ListOffsets requests, and completes by updating in-memory @@ -26,7 +24,7 @@ */ public class ResetPositionsEvent extends CompletableApplicationEvent { - public ResetPositionsEvent(final Timer timer) { - super(Type.RESET_POSITIONS, timer); + public ResetPositionsEvent(final long deadlineMs) { + super(Type.RESET_POSITIONS, deadlineMs); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java index 87945616ea71b..7dc7a023a8d01 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/SyncCommitEvent.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Timer; import java.util.Map; @@ -28,7 +27,7 @@ */ public class SyncCommitEvent extends CommitEvent { - public SyncCommitEvent(final Map offsets, final Timer timer) { - super(Type.COMMIT_SYNC, offsets, timer); + public SyncCommitEvent(final Map offsets, final long deadlineMs) { + super(Type.COMMIT_SYNC, offsets, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java index 33e1270ce6040..9758ae0efa0f0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/TopicMetadataEvent.java @@ -16,16 +16,14 @@ */ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - import java.util.Objects; public class TopicMetadataEvent extends AbstractTopicMetadataEvent { private final String topic; - public TopicMetadataEvent(final String topic, final Timer timer) { - super(Type.TOPIC_METADATA, timer); + public TopicMetadataEvent(final String topic, final long deadlineMs) { + super(Type.TOPIC_METADATA, deadlineMs); this.topic = Objects.requireNonNull(topic); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java index 0b988370014a5..327feaa22f69b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/UnsubscribeEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Application event triggered when a user calls the unsubscribe API. This will make the consumer * release all its assignments and send a heartbeat request to leave the consumer group. @@ -28,8 +26,8 @@ */ public class UnsubscribeEvent extends CompletableApplicationEvent { - public UnsubscribeEvent(final Timer timer) { - super(Type.UNSUBSCRIBE, timer); + public UnsubscribeEvent(final long deadlineMs) { + super(Type.UNSUBSCRIBE, deadlineMs); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java index 21e7f3cf6eba1..a93ff9859a58e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ValidatePositionsEvent.java @@ -17,8 +17,6 @@ package org.apache.kafka.clients.consumer.internals.events; -import org.apache.kafka.common.utils.Timer; - /** * Event for validating offsets for all assigned partitions for which a leader change has been * detected. This is an asynchronous event that generates OffsetForLeaderEpoch requests, and @@ -26,7 +24,7 @@ */ public class ValidatePositionsEvent extends CompletableApplicationEvent { - public ValidatePositionsEvent(final Timer timer) { - super(Type.VALIDATE_POSITIONS, timer); + public ValidatePositionsEvent(final long deadlineMs) { + super(Type.VALIDATE_POSITIONS, deadlineMs); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RecordDeserializationException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordDeserializationException.java index a15df6c7ff52c..aee57c47d28de 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RecordDeserializationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordDeserializationException.java @@ -16,7 +16,12 @@ */ package org.apache.kafka.common.errors; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.record.TimestampType; + +import java.nio.ByteBuffer; /** * This exception is raised for any error that occurs while deserializing records received by the consumer using @@ -24,14 +29,61 @@ */ public class RecordDeserializationException extends SerializationException { - private static final long serialVersionUID = 1L; + private static final long serialVersionUID = 2L; + + public enum DeserializationExceptionOrigin { + KEY, + VALUE + } + + private final DeserializationExceptionOrigin origin; private final TopicPartition partition; private final long offset; + private final TimestampType timestampType; + private final long timestamp; + private final ByteBuffer keyBuffer; + private final ByteBuffer valueBuffer; + private final Headers headers; - public RecordDeserializationException(TopicPartition partition, long offset, String message, Throwable cause) { + @Deprecated + public RecordDeserializationException(TopicPartition partition, + long offset, + String message, + Throwable cause) { super(message, cause); + this.origin = null; this.partition = partition; this.offset = offset; + this.timestampType = TimestampType.NO_TIMESTAMP_TYPE; + this.timestamp = ConsumerRecord.NO_TIMESTAMP; + this.keyBuffer = null; + this.valueBuffer = null; + this.headers = null; + } + + public RecordDeserializationException(DeserializationExceptionOrigin origin, + TopicPartition partition, + long offset, + long timestamp, + TimestampType timestampType, + ByteBuffer keyBuffer, + ByteBuffer valueBuffer, + Headers headers, + String message, + Throwable cause) { + super(message, cause); + this.origin = origin; + this.offset = offset; + this.timestampType = timestampType; + this.timestamp = timestamp; + this.partition = partition; + this.keyBuffer = keyBuffer; + this.valueBuffer = valueBuffer; + this.headers = headers; + } + + public DeserializationExceptionOrigin origin() { + return origin; } public TopicPartition topicPartition() { @@ -41,4 +93,24 @@ public TopicPartition topicPartition() { public long offset() { return offset; } + + public TimestampType timestampType() { + return timestampType; + } + + public long timestamp() { + return timestamp; + } + + public ByteBuffer keyBuffer() { + return keyBuffer; + } + + public ByteBuffer valueBuffer() { + return valueBuffer; + } + + public Headers headers() { + return headers; + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index b9f64445dcfb3..eab9f3e5ac12d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -39,6 +39,7 @@ import org.apache.kafka.clients.consumer.internals.events.CommitOnCloseEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableBackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ConsumerRebalanceListenerCallbackNeededEvent; import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.clients.consumer.internals.events.EventProcessor; @@ -150,6 +151,7 @@ public class AsyncKafkaConsumerTest { private final ApplicationEventHandler applicationEventHandler = mock(ApplicationEventHandler.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); private final LinkedBlockingQueue backgroundEventQueue = new LinkedBlockingQueue<>(); + private final CompletableEventReaper backgroundEventReaper = mock(CompletableEventReaper.class); @AfterEach public void resetAll() { @@ -190,7 +192,8 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { new StringDeserializer(), new StringDeserializer(), time, - (a, b, c, d, e, f) -> applicationEventHandler, + (a, b, c, d, e, f, g) -> applicationEventHandler, + a -> backgroundEventReaper, (a, b, c, d, e, f, g) -> fetchCollector, (a, b, c, d) -> metadata, backgroundEventQueue @@ -218,6 +221,7 @@ private AsyncKafkaConsumer newConsumer( time, applicationEventHandler, backgroundEventQueue, + backgroundEventReaper, rebalanceListenerInvoker, new Metrics(), subscriptions, @@ -318,6 +322,7 @@ private static Stream commitExceptionSupplier() { @Test public void testCommitAsyncWithFencedException() { consumer = newConsumer(); + completeCommitSyncApplicationEventSuccessfully(); final Map offsets = mockTopicPartitionOffset(); MockCommitCallback callback = new MockCommitCallback(); @@ -339,7 +344,7 @@ public void testCommitted() { completeFetchedCommittedOffsetApplicationEventSuccessfully(topicPartitionOffsets); assertEquals(topicPartitionOffsets, consumer.committed(topicPartitionOffsets.keySet(), Duration.ofMillis(1000))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); final Metric metric = consumer.metrics() .get(consumer.metricsRegistry().metricName("committed-time-ns-total", "consumer-metrics")); assertTrue((double) metric.metricValue() > 0); @@ -361,7 +366,7 @@ public void testCommittedLeaderEpochUpdate() { verify(metadata).updateLastSeenEpochIfNewer(t0, 2); verify(metadata).updateLastSeenEpochIfNewer(t2, 3); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), any()); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); } @Test @@ -369,7 +374,7 @@ public void testCommittedExceptionThrown() { consumer = newConsumer(); Map offsets = mockTopicPartitionOffset(); when(applicationEventHandler.addAndGet( - any(FetchCommittedOffsetsEvent.class), any())).thenAnswer(invocation -> { + any(FetchCommittedOffsetsEvent.class))).thenAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); assertInstanceOf(FetchCommittedOffsetsEvent.class, event); throw new KafkaException("Test exception"); @@ -387,6 +392,7 @@ public void testWakeupBeforeCallingPoll() { doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + completeCommitSyncApplicationEventSuccessfully(); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -408,6 +414,7 @@ public void testWakeupAfterEmptyFetch() { }).doAnswer(invocation -> Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + completeCommitSyncApplicationEventSuccessfully(); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -431,6 +438,7 @@ public void testWakeupAfterNonEmptyFetch() { }).when(fetchCollector).collectFetch(Mockito.any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + completeCommitSyncApplicationEventSuccessfully(); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -486,6 +494,7 @@ public void testClearWakeupTriggerAfterPoll() { .when(fetchCollector).collectFetch(any(FetchBuffer.class)); Map offsets = mkMap(mkEntry(tp, new OffsetAndMetadata(1))); completeFetchedCommittedOffsetApplicationEventSuccessfully(offsets); + completeCommitSyncApplicationEventSuccessfully(); doReturn(LeaderAndEpoch.noLeaderOrEpoch()).when(metadata).currentLeader(any()); consumer.assign(singleton(tp)); @@ -558,6 +567,7 @@ public void testCommitAsyncLeaderEpochUpdate() { singletonList(new RoundRobinAssignor()), "group-id", "client-id"); + completeCommitSyncApplicationEventSuccessfully(); final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); HashMap topicPartitionOffsets = new HashMap<>(); @@ -759,9 +769,9 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() { @Test public void testVerifyApplicationEventOnShutdown() { consumer = newConsumer(); - doReturn(null).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(null).when(applicationEventHandler).addAndGet(any()); consumer.close(); - verify(applicationEventHandler).addAndGet(any(LeaveOnCloseEvent.class), any()); + verify(applicationEventHandler).addAndGet(any(LeaveOnCloseEvent.class)); verify(applicationEventHandler).add(any(CommitOnCloseEvent.class)); } @@ -804,7 +814,7 @@ public void testFailedPartitionRevocationOnClose() { subscriptions.assignFromSubscribed(singleton(tp)); doThrow(new KafkaException()).when(listener).onPartitionsRevoked(eq(singleton(tp))); assertThrows(KafkaException.class, () -> consumer.close(Duration.ZERO)); - verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseEvent.class), any()); + verify(applicationEventHandler, never()).addAndGet(any(LeaveOnCloseEvent.class)); verify(listener).onPartitionsRevoked(eq(singleton(tp))); assertEquals(emptySet(), subscriptions.assignedPartitions()); } @@ -827,6 +837,7 @@ public void testCompleteQuietly() { @Test public void testAutoCommitSyncEnabled() { + completeCommitSyncApplicationEventSuccessfully(); SubscriptionState subscriptions = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE); consumer = newConsumer( mock(FetchBuffer.class), @@ -839,7 +850,7 @@ public void testAutoCommitSyncEnabled() { consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(true, time.timer(100)); + consumer.autoCommitSync(time.timer(100)); verify(applicationEventHandler).add(any(SyncCommitEvent.class)); } @@ -857,7 +868,6 @@ public void testAutoCommitSyncDisabled() { consumer.subscribe(singleton("topic"), mock(ConsumerRebalanceListener.class)); subscriptions.assignFromSubscribed(singleton(new TopicPartition("topic", 0))); subscriptions.seek(new TopicPartition("topic", 0), 100); - consumer.maybeAutoCommitSync(false, time.timer(100)); verify(applicationEventHandler, never()).add(any(SyncCommitEvent.class)); } @@ -936,8 +946,9 @@ public void testBeginningOffsets() { consumer = newConsumer(); Map expectedOffsets = mockOffsetAndTimestamp(); - when(applicationEventHandler.addAndGet(any(ListOffsetsEvent.class), any())).thenAnswer(invocation -> { - Timer timer = invocation.getArgument(1); + when(applicationEventHandler.addAndGet(any(ListOffsetsEvent.class))).thenAnswer(invocation -> { + ListOffsetsEvent event = invocation.getArgument(0); + Timer timer = time.timer(event.deadlineMs() - time.milliseconds()); if (timer.remainingMs() == 0) { fail("Timer duration should not be zero."); } @@ -950,7 +961,7 @@ public void testBeginningOffsets() { assertTrue(result.containsKey(key)); assertEquals(value.offset(), result.get(key)); }); - verify(applicationEventHandler).addAndGet(any(ListOffsetsEvent.class), any(Timer.class)); + verify(applicationEventHandler).addAndGet(any(ListOffsetsEvent.class)); } @Test @@ -960,26 +971,23 @@ public void testBeginningOffsetsThrowsKafkaExceptionForUnderlyingExecutionFailur Throwable eventProcessingFailure = new KafkaException("Unexpected failure " + "processing List Offsets event"); doThrow(eventProcessingFailure).when(applicationEventHandler).addAndGet( - any(ListOffsetsEvent.class), - any()); + any(ListOffsetsEvent.class)); Throwable consumerError = assertThrows(KafkaException.class, () -> consumer.beginningOffsets(partitions, Duration.ofMillis(1))); assertEquals(eventProcessingFailure, consumerError); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class)); } @Test public void testBeginningOffsetsTimeoutOnEventProcessingTimeout() { consumer = newConsumer(); - doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any(), any()); + doThrow(new TimeoutException()).when(applicationEventHandler).addAndGet(any()); assertThrows(TimeoutException.class, () -> consumer.beginningOffsets( Collections.singletonList(new TopicPartition("t1", 0)), Duration.ofMillis(1))); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class)); } @Test @@ -1014,15 +1022,14 @@ public void testOffsetsForTimes() { Map expectedResult = mockOffsetAndTimestamp(); Map timestampToSearch = mockTimestampToSearch(); - doReturn(expectedResult).when(applicationEventHandler).addAndGet(any(), any()); + doReturn(expectedResult).when(applicationEventHandler).addAndGet(any()); Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ofMillis(1))); expectedResult.forEach((key, value) -> { OffsetAndTimestamp expected = value.buildOffsetAndTimestamp(); assertEquals(expected, result.get(key)); }); - verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class)); } // This test ensures same behaviour as the current consumer when offsetsForTimes is called @@ -1049,8 +1056,7 @@ public void testOffsetsForTimesWithZeroTimeout() { Map result = assertDoesNotThrow(() -> consumer.offsetsForTimes(timestampToSearch, Duration.ZERO)); assertEquals(expectedResult, result); - verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class), - ArgumentMatchers.isA(Timer.class)); + verify(applicationEventHandler, never()).addAndGet(ArgumentMatchers.isA(ListOffsetsEvent.class)); } @Test @@ -1059,13 +1065,12 @@ public void testWakeupCommitted() { final Map offsets = mockTopicPartitionOffset(); doAnswer(invocation -> { CompletableApplicationEvent event = invocation.getArgument(0); - Timer timer = invocation.getArgument(1); assertInstanceOf(FetchCommittedOffsetsEvent.class, event); assertTrue(event.future().isCompletedExceptionally()); - return ConsumerUtils.getResult(event.future(), timer); + return ConsumerUtils.getResult(event.future()); }) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class)); consumer.wakeup(); assertThrows(WakeupException.class, () -> consumer.committed(offsets.keySet())); @@ -1216,6 +1221,7 @@ public void testNoInterceptorCommitAsyncFailed() { @Test public void testRefreshCommittedOffsetsSuccess() { consumer = newConsumer(); + completeCommitSyncApplicationEventSuccessfully(); TopicPartition partition = new TopicPartition("t1", 1); Set partitions = Collections.singleton(partition); Map committedOffsets = Collections.singletonMap(partition, new OffsetAndMetadata(10L)); @@ -1661,20 +1667,20 @@ private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean co consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class)); if (committedOffsetsEnabled) { // Verify there was an FetchCommittedOffsets event and no ResetPositions event verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class)); } else { // Verify there was not any FetchCommittedOffsets event but there should be a ResetPositions verify(applicationEventHandler, never()) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class)); } } @@ -1689,11 +1695,11 @@ private void testRefreshCommittedOffsetsSuccess(Set partitions, consumer.poll(Duration.ZERO); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ValidatePositionsEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(FetchCommittedOffsetsEvent.class)); verify(applicationEventHandler, atLeast(1)) - .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class), ArgumentMatchers.isA(Timer.class)); + .addAndGet(ArgumentMatchers.isA(ResetPositionsEvent.class)); } @Test @@ -1730,7 +1736,7 @@ public void testLongPollWaitIsLimited() { } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} takes a bit of time to complete, but does within the timeout. */ @Test @@ -1756,16 +1762,14 @@ public void testProcessBackgroundEventsWithInitialDelay() throws Exception { return null; }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { - consumer.processBackgroundEvents(processor, future, timer); + consumer.processBackgroundEvents(future, timer); - // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. - assertEquals(800, timer.remainingMs()); - } + // 800 is the 1000 ms timeout (above) minus the 200 ms delay for the two incremental timeouts/retries. + assertEquals(800, timer.remainingMs()); } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} is already complete when invoked, so it doesn't have to wait. */ @Test @@ -1776,17 +1780,15 @@ public void testProcessBackgroundEventsWithoutDelay() { // Create a future that is already completed. CompletableFuture future = CompletableFuture.completedFuture(null); - try (EventProcessor processor = mock(EventProcessor.class)) { - consumer.processBackgroundEvents(processor, future, timer); + consumer.processBackgroundEvents(future, timer); - // Because we didn't need to perform a timed get, we should still have every last millisecond - // of our initial timeout. - assertEquals(1000, timer.remainingMs()); - } + // Because we didn't need to perform a timed get, we should still have every last millisecond + // of our initial timeout. + assertEquals(1000, timer.remainingMs()); } /** - * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(EventProcessor, Future, Timer) processBackgroundEvents} + * Tests {@link AsyncKafkaConsumer#processBackgroundEvents(Future, Timer) processBackgroundEvents} * handles the case where the {@link Future} does not complete within the timeout. */ @Test @@ -1801,12 +1803,10 @@ public void testProcessBackgroundEventsTimesOut() throws Exception { throw new java.util.concurrent.TimeoutException("Intentional timeout"); }).when(future).get(any(Long.class), any(TimeUnit.class)); - try (EventProcessor processor = mock(EventProcessor.class)) { - assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(processor, future, timer)); + assertThrows(TimeoutException.class, () -> consumer.processBackgroundEvents(future, timer)); - // Because we forced our mocked future to continuously time out, we should have no time remaining. - assertEquals(0, timer.remainingMs()); - } + // Because we forced our mocked future to continuously time out, we should have no time remaining. + assertEquals(0, timer.remainingMs()); } /** @@ -1835,7 +1835,31 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { } assertDoesNotThrow(() -> consumer.poll(Duration.ZERO)); } - + + @Test + void testReaperInvokedInClose() { + consumer = newConsumer(); + consumer.close(); + verify(backgroundEventReaper).reap(backgroundEventQueue); + } + + @Test + void testReaperInvokedInUnsubscribe() { + consumer = newConsumer(); + completeUnsubscribeApplicationEventSuccessfully(); + consumer.unsubscribe(); + verify(backgroundEventReaper).reap(time.milliseconds()); + } + + @Test + void testReaperInvokedInPoll() { + consumer = newConsumer(); + doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); + consumer.subscribe(Collections.singletonList("topic")); + consumer.poll(Duration.ZERO); + verify(backgroundEventReaper).reap(time.milliseconds()); + } + private Map mockTopicPartitionOffset() { final TopicPartition t0 = new TopicPartition("t0", 2); final TopicPartition t1 = new TopicPartition("t0", 3); @@ -1898,13 +1922,13 @@ private void completeCommitSyncApplicationEventSuccessfully() { private void completeFetchedCommittedOffsetApplicationEventSuccessfully(final Map committedOffsets) { doReturn(committedOffsets) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class)); } private void completeFetchedCommittedOffsetApplicationEventExceptionally(Exception ex) { doThrow(ex) .when(applicationEventHandler) - .addAndGet(any(FetchCommittedOffsetsEvent.class), any(Timer.class)); + .addAndGet(any(FetchCommittedOffsetsEvent.class)); } private void completeUnsubscribeApplicationEventSuccessfully() { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java index 803c2f14cb275..5b437c2fb5e0e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CompletedFetchTest.java @@ -23,6 +23,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.RecordDeserializationException; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ApiKeys; @@ -41,6 +43,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; @@ -48,7 +51,9 @@ import java.util.List; import java.util.UUID; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; public class CompletedFetchTest { @@ -161,6 +166,10 @@ public void testCorruptedMessage() { final UUIDSerializer serializer = new UUIDSerializer()) { builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID()))); builder.append(0L, "key".getBytes(), "value".getBytes()); + builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID()))); + Headers headers = new RecordHeaders(); + headers.add("hkey", "hvalue".getBytes()); + builder.append(10L, serializer.serialize("key", UUID.randomUUID()), "otherValue".getBytes(), headers.toArray()); Records records = builder.build(); FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData() @@ -176,8 +185,29 @@ public void testCorruptedMessage() { completedFetch.fetchRecords(fetchConfig, deserializers, 10); - assertThrows(RecordDeserializationException.class, + RecordDeserializationException thrown = assertThrows(RecordDeserializationException.class, () -> completedFetch.fetchRecords(fetchConfig, deserializers, 10)); + assertEquals(RecordDeserializationException.DeserializationExceptionOrigin.KEY, thrown.origin()); + assertEquals(1, thrown.offset()); + assertEquals(TOPIC_NAME, thrown.topicPartition().topic()); + assertEquals(0, thrown.topicPartition().partition()); + assertEquals(0, thrown.timestamp()); + assertArrayEquals("key".getBytes(), Utils.toNullableArray(thrown.keyBuffer())); + assertArrayEquals("value".getBytes(), Utils.toNullableArray(thrown.valueBuffer())); + assertEquals(0, thrown.headers().toArray().length); + + CompletedFetch completedFetch2 = newCompletedFetch(2, partitionData); + completedFetch2.fetchRecords(fetchConfig, deserializers, 10); + RecordDeserializationException valueThrown = assertThrows(RecordDeserializationException.class, + () -> completedFetch2.fetchRecords(fetchConfig, deserializers, 10)); + assertEquals(RecordDeserializationException.DeserializationExceptionOrigin.VALUE, valueThrown.origin()); + assertEquals(3, valueThrown.offset()); + assertEquals(TOPIC_NAME, valueThrown.topicPartition().topic()); + assertEquals(0, valueThrown.topicPartition().partition()); + assertEquals(10L, valueThrown.timestamp()); + assertNotNull(valueThrown.keyBuffer()); + assertArrayEquals("otherValue".getBytes(), Utils.toNullableArray(valueThrown.valueBuffer())); + assertEquals(headers, valueThrown.headers()); } } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java index 0e68b5df95051..8c3f97dd64379 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -23,6 +23,8 @@ import org.apache.kafka.clients.consumer.internals.events.AssignmentChangeEvent; import org.apache.kafka.clients.consumer.internals.events.AsyncCommitEvent; import org.apache.kafka.clients.consumer.internals.events.CompletableApplicationEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEvent; +import org.apache.kafka.clients.consumer.internals.events.CompletableEventReaper; import org.apache.kafka.clients.consumer.internals.events.ListOffsetsEvent; import org.apache.kafka.clients.consumer.internals.events.NewTopicsMetadataUpdateRequestEvent; import org.apache.kafka.clients.consumer.internals.events.PollEvent; @@ -32,6 +34,7 @@ import org.apache.kafka.clients.consumer.internals.events.ValidatePositionsEvent; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.FindCoordinatorRequestData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FindCoordinatorRequest; @@ -41,7 +44,6 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -55,6 +57,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.Map; import java.util.Optional; import java.util.concurrent.BlockingQueue; @@ -62,6 +65,8 @@ import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_HEARTBEAT_INTERVAL_MS; import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DEFAULT_REQUEST_TIMEOUT_MS; +import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.createDefaultGroupInformation; +import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -70,7 +75,9 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -78,7 +85,7 @@ public class ConsumerNetworkThreadTest { - private ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder testBuilder; + private ConsumerTestBuilder testBuilder; private Time time; private ConsumerMetadata metadata; private NetworkClientDelegate networkClient; @@ -88,11 +95,12 @@ public class ConsumerNetworkThreadTest { private CommitRequestManager commitRequestManager; private CoordinatorRequestManager coordinatorRequestManager; private ConsumerNetworkThread consumerNetworkThread; + private final CompletableEventReaper applicationEventReaper = mock(CompletableEventReaper.class); private MockClient client; @BeforeEach public void setup() { - testBuilder = new ConsumerTestBuilder.ConsumerNetworkThreadTestBuilder(); + testBuilder = new ConsumerTestBuilder(createDefaultGroupInformation()); time = testBuilder.time; metadata = testBuilder.metadata; networkClient = testBuilder.networkClientDelegate; @@ -102,14 +110,24 @@ public void setup() { commitRequestManager = testBuilder.commitRequestManager.orElseThrow(IllegalStateException::new); offsetsRequestManager = testBuilder.offsetsRequestManager; coordinatorRequestManager = testBuilder.coordinatorRequestManager.orElseThrow(IllegalStateException::new); - consumerNetworkThread = testBuilder.consumerNetworkThread; + consumerNetworkThread = new ConsumerNetworkThread( + testBuilder.logContext, + time, + testBuilder.applicationEventQueue, + applicationEventReaper, + () -> applicationEventProcessor, + () -> testBuilder.networkClientDelegate, + () -> testBuilder.requestManagers + ); consumerNetworkThread.initializeResources(); } @AfterEach public void tearDown() { - if (testBuilder != null) + if (testBuilder != null) { testBuilder.close(); + consumerNetworkThread.close(Duration.ZERO); + } } @Test @@ -157,8 +175,7 @@ public void testAsyncCommitEvent() { @Test public void testSyncCommitEvent() { - Timer timer = time.timer(100); - ApplicationEvent e = new SyncCommitEvent(new HashMap<>(), timer); + ApplicationEvent e = new SyncCommitEvent(new HashMap<>(), calculateDeadlineMs(time, 100)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(SyncCommitEvent.class)); @@ -168,8 +185,7 @@ public void testSyncCommitEvent() { @ValueSource(booleans = {true, false}) public void testListOffsetsEventIsProcessed(boolean requireTimestamp) { Map timestamps = Collections.singletonMap(new TopicPartition("topic1", 1), 5L); - Timer timer = time.timer(100); - ApplicationEvent e = new ListOffsetsEvent(timestamps, timer, requireTimestamp); + ApplicationEvent e = new ListOffsetsEvent(timestamps, calculateDeadlineMs(time, 100), requireTimestamp); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ListOffsetsEvent.class)); @@ -178,8 +194,7 @@ public void testListOffsetsEventIsProcessed(boolean requireTimestamp) { @Test public void testResetPositionsEventIsProcessed() { - Timer timer = time.timer(100); - ResetPositionsEvent e = new ResetPositionsEvent(timer); + ResetPositionsEvent e = new ResetPositionsEvent(calculateDeadlineMs(time, 100)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ResetPositionsEvent.class)); @@ -190,8 +205,7 @@ public void testResetPositionsEventIsProcessed() { public void testResetPositionsProcessFailureIsIgnored() { doThrow(new NullPointerException()).when(offsetsRequestManager).resetPositionsIfNeeded(); - Timer timer = time.timer(100); - ResetPositionsEvent event = new ResetPositionsEvent(timer); + ResetPositionsEvent event = new ResetPositionsEvent(calculateDeadlineMs(time, 100)); applicationEventsQueue.add(event); assertDoesNotThrow(() -> consumerNetworkThread.runOnce()); @@ -200,8 +214,7 @@ public void testResetPositionsProcessFailureIsIgnored() { @Test public void testValidatePositionsEventIsProcessed() { - Timer timer = time.timer(100); - ValidatePositionsEvent e = new ValidatePositionsEvent(timer); + ValidatePositionsEvent e = new ValidatePositionsEvent(calculateDeadlineMs(time, 100)); applicationEventsQueue.add(e); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(ValidatePositionsEvent.class)); @@ -226,8 +239,7 @@ public void testAssignmentChangeEvent() { @Test void testFetchTopicMetadata() { - Timer timer = time.timer(Long.MAX_VALUE); - applicationEventsQueue.add(new TopicMetadataEvent("topic", timer)); + applicationEventsQueue.add(new TopicMetadataEvent("topic", Long.MAX_VALUE)); consumerNetworkThread.runOnce(); verify(applicationEventProcessor).process(any(TopicMetadataEvent.class)); } @@ -282,6 +294,22 @@ void testEnsureMetadataUpdateOnPoll() { @Test void testEnsureEventsAreCompleted() { + // Mimic the logic of CompletableEventReaper.reap(Collection): + doAnswer(__ -> { + Iterator i = applicationEventsQueue.iterator(); + + while (i.hasNext()) { + ApplicationEvent event = i.next(); + + if (event instanceof CompletableEvent) + ((CompletableEvent) event).future().completeExceptionally(new TimeoutException()); + + i.remove(); + } + + return null; + }).when(applicationEventReaper).reap(any(Collection.class)); + Node node = metadata.fetch().nodes().get(0); coordinatorRequestManager.markCoordinatorUnknown("test", time.milliseconds()); client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "group-id", node)); @@ -294,12 +322,23 @@ void testEnsureEventsAreCompleted() { applicationEventsQueue.add(event2); assertFalse(future.isDone()); assertFalse(applicationEventsQueue.isEmpty()); - consumerNetworkThread.cleanup(); assertTrue(future.isCompletedExceptionally()); assertTrue(applicationEventsQueue.isEmpty()); } + @Test + void testCleanupInvokesReaper() { + consumerNetworkThread.cleanup(); + verify(applicationEventReaper).reap(applicationEventsQueue); + } + + @Test + void testRunOnceInvokesReaper() { + consumerNetworkThread.runOnce(); + verify(applicationEventReaper).reap(any(Long.class)); + } + private void prepareOffsetCommitRequest(final Map expectedOffsets, final Errors error, final boolean disconnected) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java index 3095b0c6ed91f..9f6fd4a764b0a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerTestBuilder.java @@ -38,7 +38,6 @@ import org.apache.kafka.common.utils.Timer; import java.io.Closeable; -import java.time.Duration; import java.util.HashMap; import java.util.Optional; import java.util.Properties; @@ -270,7 +269,6 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA ); this.applicationEventProcessor = spy(new ApplicationEventProcessor( logContext, - applicationEventQueue, requestManagers, metadata ) @@ -287,32 +285,6 @@ public ConsumerTestBuilder(Optional groupInfo, boolean enableA @Override public void close() { closeQuietly(requestManagers, RequestManagers.class.getSimpleName()); - closeQuietly(applicationEventProcessor, ApplicationEventProcessor.class.getSimpleName()); - } - - public static class ConsumerNetworkThreadTestBuilder extends ConsumerTestBuilder { - - final ConsumerNetworkThread consumerNetworkThread; - - public ConsumerNetworkThreadTestBuilder() { - this(createDefaultGroupInformation()); - } - - public ConsumerNetworkThreadTestBuilder(Optional groupInfo) { - super(groupInfo); - this.consumerNetworkThread = new ConsumerNetworkThread( - logContext, - time, - () -> applicationEventProcessor, - () -> networkClientDelegate, - () -> requestManagers - ); - } - - @Override - public void close() { - consumerNetworkThread.close(Duration.ZERO); - } } public static class GroupInformation { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java index f7a73dbae8a9b..8334fb23605ae 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager.HeartbeatState; import org.apache.kafka.clients.consumer.internals.MembershipManager.LocalAssignment; import org.apache.kafka.clients.consumer.internals.events.BackgroundEventHandler; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.Uuid; @@ -51,6 +52,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; import java.util.Arrays; import java.util.Collection; @@ -483,8 +485,7 @@ public void testHeartbeatResponseOnErrorHandling(final Errors error, final boole break; default: if (isFatal) { - // The memberStateManager should have stopped heartbeat at this point - ensureFatalError(); + ensureFatalError(error); } else { verify(backgroundEventHandler, never()).add(any()); assertNextHeartbeatTiming(0); @@ -781,9 +782,15 @@ private void mockStableMember() { assertEquals(MemberState.STABLE, membershipManager.state()); } - private void ensureFatalError() { + private void ensureFatalError(Errors expectedError) { verify(membershipManager).transitionToFatal(); - verify(backgroundEventHandler).add(any()); + + final ArgumentCaptor errorEventArgumentCaptor = ArgumentCaptor.forClass(ErrorEvent.class); + verify(backgroundEventHandler).add(errorEventArgumentCaptor.capture()); + ErrorEvent errorEvent = errorEventArgumentCaptor.getValue(); + assertInstanceOf(expectedError.exception().getClass(), errorEvent.error(), + "The fatal error propagated to the app thread does not match the error received in the heartbeat response."); + ensureHeartbeatStopped(); } @@ -808,6 +815,7 @@ private static Collection errorProvider() { Arguments.of(Errors.UNSUPPORTED_ASSIGNOR, true), Arguments.of(Errors.UNSUPPORTED_VERSION, true), Arguments.of(Errors.UNRELEASED_INSTANCE_ID, true), + Arguments.of(Errors.FENCED_INSTANCE_ID, true), Arguments.of(Errors.GROUP_MAX_SIZE_REACHED, true)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index fdcb0cdc39953..451743ae2ad83 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -39,6 +38,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -77,7 +77,6 @@ public void setup() { ); processor = new ApplicationEventProcessor( new LogContext(), - applicationEventQueue, requestManagers, metadata ); @@ -93,8 +92,7 @@ public void testPrepClosingCommitEvents() { @Test public void testPrepClosingLeaveGroupEvent() { - Timer timer = time.timer(100); - LeaveOnCloseEvent event = new LeaveOnCloseEvent(timer); + LeaveOnCloseEvent event = new LeaveOnCloseEvent(calculateDeadlineMs(time, 100)); when(heartbeatRequestManager.membershipManager()).thenReturn(membershipManager); when(membershipManager.leaveGroup()).thenReturn(CompletableFuture.completedFuture(null)); processor.process(event); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java new file mode 100644 index 0000000000000..460b7368fb9e6 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/CompletableEventReaperTest.java @@ -0,0 +1,200 @@ +/* + * 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.clients.consumer.internals.events; + +import org.apache.kafka.clients.consumer.internals.ConsumerUtils; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collection; + +import static org.apache.kafka.clients.consumer.internals.events.CompletableEvent.calculateDeadlineMs; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class CompletableEventReaperTest { + + private final LogContext logContext = new LogContext(); + private final Time time = new MockTime(); + private final CompletableEventReaper reaper = new CompletableEventReaper(logContext); + + @Test + public void testExpired() { + // Add a new event to the reaper. + long timeoutMs = 100; + UnsubscribeEvent event = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + reaper.add(event); + + // Without any time passing, we check the reaper and verify that the event is not done amd is still + // being tracked. + reaper.reap(time.milliseconds()); + assertFalse(event.future().isDone()); + assertEquals(1, reaper.size()); + + // Sleep for at least 1 ms. *more* than the timeout so that the event is considered expired. + time.sleep(timeoutMs + 1); + + // However, until we actually invoke the reaper, the event isn't complete and is still being tracked. + assertFalse(event.future().isDone()); + assertEquals(1, reaper.size()); + + // Call the reaper and validate that the event is now "done" (expired), the correct exception type is + // thrown, and the event is no longer tracked. + reaper.reap(time.milliseconds()); + assertTrue(event.future().isDone()); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event.future())); + assertEquals(0, reaper.size()); + } + + @Test + public void testCompleted() { + // Add a new event to the reaper. + long timeoutMs = 100; + UnsubscribeEvent event = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + reaper.add(event); + + // Without any time passing, we check the reaper and verify that the event is not done amd is still + // being tracked. + reaper.reap(time.milliseconds()); + assertFalse(event.future().isDone()); + assertEquals(1, reaper.size()); + + // We'll cause the event to be completed normally. Note that because we haven't called the reaper, the + // event is still being tracked. + event.future().complete(null); + assertTrue(event.future().isDone()); + assertEquals(1, reaper.size()); + + // To ensure we don't accidentally expire an event that completed normally, sleep past the timeout. + time.sleep(timeoutMs + 1); + + // Call the reaper and validate that the event is not considered expired, but is still no longer tracked. + reaper.reap(time.milliseconds()); + assertTrue(event.future().isDone()); + assertNull(ConsumerUtils.getResult(event.future())); + assertEquals(0, reaper.size()); + } + + @Test + public void testCompletedAndExpired() { + // Add two events to the reaper. One event will be completed, the other we will let expire. + long timeoutMs = 100; + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + reaper.add(event1); + reaper.add(event2); + + // Without any time passing, we check the reaper and verify that the event is not done amd is still + // being tracked. + reaper.reap(time.milliseconds()); + assertFalse(event1.future().isDone()); + assertFalse(event2.future().isDone()); + assertEquals(2, reaper.size()); + + // We'll cause the first event to be completed normally, but then sleep past the timer deadline. + event1.future().complete(null); + assertTrue(event1.future().isDone()); + + time.sleep(timeoutMs + 1); + + // Though the first event is completed, it's still being tracked, along with the second expired event. + assertEquals(2, reaper.size()); + + // Validate that the first (completed) event is not expired, but the second one is expired. In either case, + // both should be completed and neither should be tracked anymore. + reaper.reap(time.milliseconds()); + assertTrue(event1.future().isDone()); + assertTrue(event2.future().isDone()); + assertNull(ConsumerUtils.getResult(event1.future())); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); + assertEquals(0, reaper.size()); + } + + @Test + public void testIncompleteQueue() { + long timeoutMs = 100; + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + + Collection> queue = new ArrayList<>(); + queue.add(event1); + queue.add(event2); + + // Complete one of our events, just to make sure it isn't inadvertently canceled. + event1.future().complete(null); + + // In this test, our events aren't tracked in the reaper, just in the queue. + assertEquals(0, reaper.size()); + assertEquals(2, queue.size()); + + // Go ahead and reap the incomplete from the queue. + reaper.reap(queue); + + // The first event was completed, so we didn't expire it in the reaper. + assertTrue(event1.future().isDone()); + assertFalse(event1.future().isCancelled()); + assertNull(ConsumerUtils.getResult(event1.future())); + + // The second event was incomplete, so it was expired. + assertTrue(event2.future().isCompletedExceptionally()); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); + + // Because the events aren't tracked in the reaper *and* the queue is cleared as part of the + // cancellation process, our data structures should both be the same as above. + assertEquals(0, reaper.size()); + assertEquals(0, queue.size()); + } + + @Test + public void testIncompleteTracked() { + // This queue is just here to test the case where the queue is empty. + Collection> queue = new ArrayList<>(); + + // Add two events for the reaper to track. + long timeoutMs = 100; + UnsubscribeEvent event1 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + UnsubscribeEvent event2 = new UnsubscribeEvent(calculateDeadlineMs(time.milliseconds(), timeoutMs)); + reaper.add(event1); + reaper.add(event2); + + // Complete one of our events, just to make sure it isn't inadvertently canceled. + event1.future().complete(null); + + // In this test, our events are tracked exclusively in the reaper, not the queue. + assertEquals(2, reaper.size()); + + // Go ahead and reap the incomplete events. Both sets should be zero after that. + reaper.reap(queue); + assertEquals(0, reaper.size()); + assertEquals(0, queue.size()); + + // The first event was completed, so we didn't cancel it in the reaper. + assertTrue(event1.future().isDone()); + assertNull(ConsumerUtils.getResult(event1.future())); + + // The second event was incomplete, so it was canceled. + assertTrue(event2.future().isCompletedExceptionally()); + assertThrows(TimeoutException.class, () -> ConsumerUtils.getResult(event2.future())); + } +} diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java index 7b78c64af0ca7..e144b7c69b6c7 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Values.java @@ -22,25 +22,24 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; import java.math.BigDecimal; +import java.math.BigInteger; +import java.math.RoundingMode; import java.nio.ByteBuffer; import java.text.CharacterIterator; import java.text.DateFormat; -import java.text.ParseException; +import java.text.ParsePosition; import java.text.SimpleDateFormat; import java.text.StringCharacterIterator; import java.util.ArrayList; -import java.util.Arrays; import java.util.Base64; import java.util.Calendar; -import java.util.Collections; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; -import java.util.Set; import java.util.TimeZone; import java.util.regex.Pattern; @@ -61,42 +60,16 @@ */ public class Values { - private static final Logger LOG = LoggerFactory.getLogger(Values.class); - private static final TimeZone UTC = TimeZone.getTimeZone("UTC"); private static final SchemaAndValue NULL_SCHEMA_AND_VALUE = new SchemaAndValue(null, null); - private static final SchemaAndValue TRUE_SCHEMA_AND_VALUE = new SchemaAndValue(Schema.BOOLEAN_SCHEMA, Boolean.TRUE); - private static final SchemaAndValue FALSE_SCHEMA_AND_VALUE = new SchemaAndValue(Schema.BOOLEAN_SCHEMA, Boolean.FALSE); private static final Schema ARRAY_SELECTOR_SCHEMA = SchemaBuilder.array(Schema.STRING_SCHEMA).build(); private static final Schema MAP_SELECTOR_SCHEMA = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA).build(); private static final Schema STRUCT_SELECTOR_SCHEMA = SchemaBuilder.struct().build(); - private static final String TRUE_LITERAL = Boolean.TRUE.toString(); - private static final String FALSE_LITERAL = Boolean.FALSE.toString(); private static final long MILLIS_PER_DAY = 24 * 60 * 60 * 1000; private static final String NULL_VALUE = "null"; static final String ISO_8601_DATE_FORMAT_PATTERN = "yyyy-MM-dd"; static final String ISO_8601_TIME_FORMAT_PATTERN = "HH:mm:ss.SSS'Z'"; static final String ISO_8601_TIMESTAMP_FORMAT_PATTERN = ISO_8601_DATE_FORMAT_PATTERN + "'T'" + ISO_8601_TIME_FORMAT_PATTERN; - private static final Set TEMPORAL_LOGICAL_TYPE_NAMES = - Collections.unmodifiableSet( - new HashSet<>( - Arrays.asList(Time.LOGICAL_NAME, - Timestamp.LOGICAL_NAME, - Date.LOGICAL_NAME - ) - ) - ); - - private static final String QUOTE_DELIMITER = "\""; - private static final String COMMA_DELIMITER = ","; - private static final String ENTRY_DELIMITER = ":"; - private static final String ARRAY_BEGIN_DELIMITER = "["; - private static final String ARRAY_END_DELIMITER = "]"; - private static final String MAP_BEGIN_DELIMITER = "{"; - private static final String MAP_END_DELIMITER = "}"; - private static final int ISO_8601_DATE_LENGTH = ISO_8601_DATE_FORMAT_PATTERN.length(); - private static final int ISO_8601_TIME_LENGTH = ISO_8601_TIME_FORMAT_PATTERN.length() - 2; // subtract single quotes - private static final int ISO_8601_TIMESTAMP_LENGTH = ISO_8601_TIMESTAMP_FORMAT_PATTERN.length() - 4; // subtract single quotes private static final Pattern TWO_BACKSLASHES = Pattern.compile("\\\\"); @@ -112,7 +85,17 @@ public class Values { * @throws DataException if the value could not be converted to a boolean */ public static Boolean convertToBoolean(Schema schema, Object value) throws DataException { - return (Boolean) convertTo(Schema.OPTIONAL_BOOLEAN_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Boolean) { + return (Boolean) value; + } else if (value instanceof String) { + SchemaAndValue parsed = parseString(value.toString()); + if (parsed.value() instanceof Boolean) { + return (Boolean) parsed.value(); + } + } + return asLong(value, schema, null) == 0L ? Boolean.FALSE : Boolean.TRUE; } /** @@ -125,7 +108,12 @@ public static Boolean convertToBoolean(Schema schema, Object value) throws DataE * @throws DataException if the value could not be converted to a byte */ public static Byte convertToByte(Schema schema, Object value) throws DataException { - return (Byte) convertTo(Schema.OPTIONAL_INT8_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Byte) { + return (Byte) value; + } + return (byte) asLong(value, schema, null); } /** @@ -138,7 +126,12 @@ public static Byte convertToByte(Schema schema, Object value) throws DataExcepti * @throws DataException if the value could not be converted to a short */ public static Short convertToShort(Schema schema, Object value) throws DataException { - return (Short) convertTo(Schema.OPTIONAL_INT16_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Short) { + return (Short) value; + } + return (short) asLong(value, schema, null); } /** @@ -151,7 +144,12 @@ public static Short convertToShort(Schema schema, Object value) throws DataExcep * @throws DataException if the value could not be converted to an integer */ public static Integer convertToInteger(Schema schema, Object value) throws DataException { - return (Integer) convertTo(Schema.OPTIONAL_INT32_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Integer) { + return (Integer) value; + } + return (int) asLong(value, schema, null); } /** @@ -164,7 +162,12 @@ public static Integer convertToInteger(Schema schema, Object value) throws DataE * @throws DataException if the value could not be converted to a long */ public static Long convertToLong(Schema schema, Object value) throws DataException { - return (Long) convertTo(Schema.OPTIONAL_INT64_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Long) { + return (Long) value; + } + return asLong(value, schema, null); } /** @@ -177,7 +180,12 @@ public static Long convertToLong(Schema schema, Object value) throws DataExcepti * @throws DataException if the value could not be converted to a float */ public static Float convertToFloat(Schema schema, Object value) throws DataException { - return (Float) convertTo(Schema.OPTIONAL_FLOAT32_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Float) { + return (Float) value; + } + return (float) asDouble(value, schema, null); } /** @@ -190,7 +198,12 @@ public static Float convertToFloat(Schema schema, Object value) throws DataExcep * @throws DataException if the value could not be converted to a double */ public static Double convertToDouble(Schema schema, Object value) throws DataException { - return (Double) convertTo(Schema.OPTIONAL_FLOAT64_SCHEMA, schema, value); + if (value == null) { + return null; + } else if (value instanceof Double) { + return (Double) value; + } + return asDouble(value, schema, null); } /** @@ -202,7 +215,12 @@ public static Double convertToDouble(Schema schema, Object value) throws DataExc * @return the representation as a string, or null if the supplied value was null */ public static String convertToString(Schema schema, Object value) { - return (String) convertTo(Schema.OPTIONAL_STRING_SCHEMA, schema, value); + if (value == null) { + return null; + } + StringBuilder sb = new StringBuilder(); + append(sb, value, false); + return sb.toString(); } /** @@ -219,7 +237,7 @@ public static String convertToString(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a list value */ public static List convertToList(Schema schema, Object value) { - return (List) convertTo(ARRAY_SELECTOR_SCHEMA, schema, value); + return convertToArray(ARRAY_SELECTOR_SCHEMA, value); } /** @@ -235,7 +253,7 @@ public static List convertToList(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a map value */ public static Map convertToMap(Schema schema, Object value) { - return (Map) convertTo(MAP_SELECTOR_SCHEMA, schema, value); + return convertToMapInternal(MAP_SELECTOR_SCHEMA, value); } /** @@ -250,7 +268,7 @@ public static List convertToList(Schema schema, Object value) { * @throws DataException if the value is not a struct */ public static Struct convertToStruct(Schema schema, Object value) { - return (Struct) convertTo(STRUCT_SELECTOR_SCHEMA, schema, value); + return convertToStructInternal(STRUCT_SELECTOR_SCHEMA, value); } /** @@ -263,7 +281,10 @@ public static Struct convertToStruct(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a time value */ public static java.util.Date convertToTime(Schema schema, Object value) { - return (java.util.Date) convertTo(Time.SCHEMA, schema, value); + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } + return convertToTime(Time.SCHEMA, schema, value); } /** @@ -276,7 +297,10 @@ public static java.util.Date convertToTime(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a date value */ public static java.util.Date convertToDate(Schema schema, Object value) { - return (java.util.Date) convertTo(Date.SCHEMA, schema, value); + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } + return convertToDate(Date.SCHEMA, schema, value); } /** @@ -289,7 +313,10 @@ public static java.util.Date convertToDate(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a timestamp value */ public static java.util.Date convertToTimestamp(Schema schema, Object value) { - return (java.util.Date) convertTo(Timestamp.SCHEMA, schema, value); + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } + return convertToTimestamp(Timestamp.SCHEMA, schema, value); } /** @@ -302,7 +329,10 @@ public static java.util.Date convertToTimestamp(Schema schema, Object value) { * @throws DataException if the value cannot be converted to a decimal value */ public static BigDecimal convertToDecimal(Schema schema, Object value, int scale) { - return (BigDecimal) convertTo(Decimal.schema(scale), schema, value); + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } + return convertToDecimal(Decimal.schema(scale), value); } /** @@ -314,65 +344,59 @@ public static BigDecimal convertToDecimal(Schema schema, Object value, int scale public static Schema inferSchema(Object value) { if (value instanceof String) { return Schema.STRING_SCHEMA; - } - if (value instanceof Boolean) { + } else if (value instanceof Boolean) { return Schema.BOOLEAN_SCHEMA; - } - if (value instanceof Byte) { + } else if (value instanceof Byte) { return Schema.INT8_SCHEMA; - } - if (value instanceof Short) { + } else if (value instanceof Short) { return Schema.INT16_SCHEMA; - } - if (value instanceof Integer) { + } else if (value instanceof Integer) { return Schema.INT32_SCHEMA; - } - if (value instanceof Long) { + } else if (value instanceof Long) { return Schema.INT64_SCHEMA; - } - if (value instanceof Float) { + } else if (value instanceof Float) { return Schema.FLOAT32_SCHEMA; - } - if (value instanceof Double) { + } else if (value instanceof Double) { return Schema.FLOAT64_SCHEMA; - } - if (value instanceof byte[] || value instanceof ByteBuffer) { + } else if (value instanceof byte[] || value instanceof ByteBuffer) { return Schema.BYTES_SCHEMA; + } else if (value instanceof List) { + return inferListSchema((List) value); + } else if (value instanceof Map) { + return inferMapSchema((Map) value); + } else if (value instanceof Struct) { + return ((Struct) value).schema(); } - if (value instanceof List) { - List list = (List) value; - if (list.isEmpty()) { + return null; + } + + private static Schema inferListSchema(List list) { + if (list.isEmpty()) { + return null; + } + SchemaDetector detector = new SchemaDetector(); + for (Object element : list) { + if (!detector.canDetect(element)) { return null; } - SchemaDetector detector = new SchemaDetector(); - for (Object element : list) { - if (!detector.canDetect(element)) { - return null; - } - } - return SchemaBuilder.array(detector.schema()).build(); } - if (value instanceof Map) { - Map map = (Map) value; - if (map.isEmpty()) { + return SchemaBuilder.array(detector.schema()).build(); + } + + private static Schema inferMapSchema(Map map) { + if (map.isEmpty()) { + return null; + } + SchemaDetector keyDetector = new SchemaDetector(); + SchemaDetector valueDetector = new SchemaDetector(); + for (Map.Entry entry : map.entrySet()) { + if (!keyDetector.canDetect(entry.getKey()) || !valueDetector.canDetect(entry.getValue())) { return null; } - SchemaDetector keyDetector = new SchemaDetector(); - SchemaDetector valueDetector = new SchemaDetector(); - for (Map.Entry entry : map.entrySet()) { - if (!keyDetector.canDetect(entry.getKey()) || !valueDetector.canDetect(entry.getValue())) { - return null; - } - } - return SchemaBuilder.map(keyDetector.schema(), valueDetector.schema()).build(); } - if (value instanceof Struct) { - return ((Struct) value).schema(); - } - return null; + return SchemaBuilder.map(keyDetector.schema(), valueDetector.schema()).build(); } - /** * Parse the specified string representation of a value into its schema and value. * @@ -387,8 +411,8 @@ public static SchemaAndValue parseString(String value) { if (value.isEmpty()) { return new SchemaAndValue(Schema.STRING_SCHEMA, value); } - Parser parser = new Parser(value); - return parse(parser, false); + ValueParser parser = new ValueParser(new Parser(value)); + return parser.parse(false); } /** @@ -396,7 +420,7 @@ public static SchemaAndValue parseString(String value) { * * @param toSchema the schema for the desired type; may not be null * @param fromSchema the schema for the supplied value; may be null if not known - * @return the converted value; never null + * @return the converted value; null if the passed-in schema was optional, and the input value was null. * @throws DataException if the value could not be converted to the desired type */ protected static Object convertTo(Schema toSchema, Schema fromSchema, Object value) throws DataException { @@ -408,184 +432,213 @@ protected static Object convertTo(Schema toSchema, Schema fromSchema, Object val } switch (toSchema.type()) { case BYTES: - if (Decimal.LOGICAL_NAME.equals(toSchema.name())) { - if (value instanceof ByteBuffer) { - value = Utils.toArray((ByteBuffer) value); - } - if (value instanceof byte[]) { - return Decimal.toLogical(toSchema, (byte[]) value); - } - if (value instanceof BigDecimal) { - return value; - } - if (value instanceof Number) { - // Not already a decimal, so treat it as a double ... - double converted = ((Number) value).doubleValue(); - return BigDecimal.valueOf(converted); - } - if (value instanceof String) { - return new BigDecimal(value.toString()); - } - } - if (value instanceof ByteBuffer) { - return Utils.toArray((ByteBuffer) value); - } - if (value instanceof byte[]) { - return value; - } - if (value instanceof BigDecimal) { - return Decimal.fromLogical(toSchema, (BigDecimal) value); - } - break; + return convertMaybeLogicalBytes(toSchema, value); case STRING: - StringBuilder sb = new StringBuilder(); - append(sb, value, false); - return sb.toString(); + return convertToString(fromSchema, value); case BOOLEAN: - if (value instanceof Boolean) { - return value; - } - if (value instanceof String) { - SchemaAndValue parsed = parseString(value.toString()); - if (parsed.value() instanceof Boolean) { - return parsed.value(); - } - } - return asLong(value, fromSchema, null) == 0L ? Boolean.FALSE : Boolean.TRUE; + return convertToBoolean(fromSchema, value); case INT8: - if (value instanceof Byte) { - return value; - } - return (byte) asLong(value, fromSchema, null); + return convertToByte(fromSchema, value); case INT16: - if (value instanceof Short) { - return value; - } - return (short) asLong(value, fromSchema, null); + return convertToShort(fromSchema, value); case INT32: - if (Date.LOGICAL_NAME.equals(toSchema.name())) { - if (value instanceof String) { - SchemaAndValue parsed = parseString(value.toString()); - value = parsed.value(); - } - if (value instanceof java.util.Date) { - if (fromSchema != null) { - String fromSchemaName = fromSchema.name(); - if (Date.LOGICAL_NAME.equals(fromSchemaName)) { - return value; - } - if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { - // Just get the number of days from this timestamp - long millis = ((java.util.Date) value).getTime(); - int days = (int) (millis / MILLIS_PER_DAY); // truncates - return Date.toLogical(toSchema, days); - } - } else { - // There is no fromSchema, so no conversion is needed - return value; - } - } - long numeric = asLong(value, fromSchema, null); - return Date.toLogical(toSchema, (int) numeric); - } - if (Time.LOGICAL_NAME.equals(toSchema.name())) { - if (value instanceof String) { - SchemaAndValue parsed = parseString(value.toString()); - value = parsed.value(); - } - if (value instanceof java.util.Date) { - if (fromSchema != null) { - String fromSchemaName = fromSchema.name(); - if (Time.LOGICAL_NAME.equals(fromSchemaName)) { - return value; - } - if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { - // Just get the time portion of this timestamp - Calendar calendar = Calendar.getInstance(UTC); - calendar.setTime((java.util.Date) value); - calendar.set(Calendar.YEAR, 1970); - calendar.set(Calendar.MONTH, 0); // Months are zero-based - calendar.set(Calendar.DAY_OF_MONTH, 1); - return Time.toLogical(toSchema, (int) calendar.getTimeInMillis()); - } - } else { - // There is no fromSchema, so no conversion is needed - return value; - } - } - long numeric = asLong(value, fromSchema, null); - return Time.toLogical(toSchema, (int) numeric); - } - if (value instanceof Integer) { - return value; - } - return (int) asLong(value, fromSchema, null); + return convertMaybeLogicalInteger(toSchema, fromSchema, value); case INT64: - if (Timestamp.LOGICAL_NAME.equals(toSchema.name())) { - if (value instanceof String) { - SchemaAndValue parsed = parseString(value.toString()); - value = parsed.value(); - } - if (value instanceof java.util.Date) { - java.util.Date date = (java.util.Date) value; - if (fromSchema != null) { - String fromSchemaName = fromSchema.name(); - if (Date.LOGICAL_NAME.equals(fromSchemaName)) { - int days = Date.fromLogical(fromSchema, date); - long millis = days * MILLIS_PER_DAY; - return Timestamp.toLogical(toSchema, millis); - } - if (Time.LOGICAL_NAME.equals(fromSchemaName)) { - long millis = Time.fromLogical(fromSchema, date); - return Timestamp.toLogical(toSchema, millis); - } - if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { - return value; - } - } else { - // There is no fromSchema, so no conversion is needed - return value; - } - } - long numeric = asLong(value, fromSchema, null); - return Timestamp.toLogical(toSchema, numeric); - } - if (value instanceof Long) { - return value; - } - return asLong(value, fromSchema, null); + return convertMaybeLogicalLong(toSchema, fromSchema, value); case FLOAT32: - if (value instanceof Float) { - return value; - } - return (float) asDouble(value, fromSchema, null); + return convertToFloat(fromSchema, value); case FLOAT64: - if (value instanceof Double) { - return value; - } - return asDouble(value, fromSchema, null); + return convertToDouble(fromSchema, value); case ARRAY: - if (value instanceof String) { - SchemaAndValue schemaAndValue = parseString(value.toString()); - value = schemaAndValue.value(); + return convertToArray(toSchema, value); + case MAP: + return convertToMapInternal(toSchema, value); + case STRUCT: + return convertToStructInternal(toSchema, value); + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static Serializable convertMaybeLogicalBytes(Schema toSchema, Object value) { + if (Decimal.LOGICAL_NAME.equals(toSchema.name())) { + return convertToDecimal(toSchema, value); + } + return convertToBytes(toSchema, value); + } + + private static BigDecimal convertToDecimal(Schema toSchema, Object value) { + if (value instanceof ByteBuffer) { + value = Utils.toArray((ByteBuffer) value); + } + if (value instanceof byte[]) { + return Decimal.toLogical(toSchema, (byte[]) value); + } + if (value instanceof BigDecimal) { + return (BigDecimal) value; + } + if (value instanceof Number) { + // Not already a decimal, so treat it as a double ... + double converted = ((Number) value).doubleValue(); + return BigDecimal.valueOf(converted); + } + if (value instanceof String) { + return new BigDecimal(value.toString()); + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static byte[] convertToBytes(Schema toSchema, Object value) { + if (value instanceof ByteBuffer) { + return Utils.toArray((ByteBuffer) value); + } + if (value instanceof byte[]) { + return (byte[]) value; + } + if (value instanceof BigDecimal) { + return Decimal.fromLogical(toSchema, (BigDecimal) value); + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static Serializable convertMaybeLogicalInteger(Schema toSchema, Schema fromSchema, Object value) { + if (Date.LOGICAL_NAME.equals(toSchema.name())) { + return convertToDate(toSchema, fromSchema, value); + } + if (Time.LOGICAL_NAME.equals(toSchema.name())) { + return convertToTime(toSchema, fromSchema, value); + } + return convertToInteger(fromSchema, value); + } + + private static java.util.Date convertToDate(Schema toSchema, Schema fromSchema, Object value) { + if (value == null) { + return null; + } else if (value instanceof String) { + SchemaAndValue parsed = parseString(value.toString()); + value = parsed.value(); + } + if (value instanceof java.util.Date) { + if (fromSchema != null) { + String fromSchemaName = fromSchema.name(); + if (Date.LOGICAL_NAME.equals(fromSchemaName)) { + return (java.util.Date) value; } - if (value instanceof List) { - return value; + if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { + // Just get the number of days from this timestamp + long millis = ((java.util.Date) value).getTime(); + int days = (int) (millis / MILLIS_PER_DAY); // truncates + return Date.toLogical(toSchema, days); } - break; - case MAP: - if (value instanceof String) { - SchemaAndValue schemaAndValue = parseString(value.toString()); - value = schemaAndValue.value(); + } else { + // There is no fromSchema, so no conversion is needed + return (java.util.Date) value; + } + } + long numeric = asLong(value, fromSchema, null); + return Date.toLogical(toSchema, (int) numeric); + } + + private static java.util.Date convertToTime(Schema toSchema, Schema fromSchema, Object value) { + if (value == null) { + return null; + } else if (value instanceof String) { + SchemaAndValue parsed = parseString(value.toString()); + value = parsed.value(); + } + if (value instanceof java.util.Date) { + if (fromSchema != null) { + String fromSchemaName = fromSchema.name(); + if (Time.LOGICAL_NAME.equals(fromSchemaName)) { + return (java.util.Date) value; } - if (value instanceof Map) { - return value; + if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { + // Just get the time portion of this timestamp + Calendar calendar = Calendar.getInstance(UTC); + calendar.setTime((java.util.Date) value); + calendar.set(Calendar.YEAR, 1970); + calendar.set(Calendar.MONTH, 0); // Months are zero-based + calendar.set(Calendar.DAY_OF_MONTH, 1); + return Time.toLogical(toSchema, (int) calendar.getTimeInMillis()); } - break; - case STRUCT: - if (value instanceof Struct) { - return value; + } else { + // There is no fromSchema, so no conversion is needed + return (java.util.Date) value; + } + } + long numeric = asLong(value, fromSchema, null); + return Time.toLogical(toSchema, (int) numeric); + } + + private static Serializable convertMaybeLogicalLong(Schema toSchema, Schema fromSchema, Object value) { + if (Timestamp.LOGICAL_NAME.equals(toSchema.name())) { + return convertToTimestamp(toSchema, fromSchema, value); + } + return convertToLong(fromSchema, value); + } + + private static java.util.Date convertToTimestamp(Schema toSchema, Schema fromSchema, Object value) { + if (value == null) { + return null; + } else if (value instanceof String) { + SchemaAndValue parsed = parseString(value.toString()); + value = parsed.value(); + } + if (value instanceof java.util.Date) { + java.util.Date date = (java.util.Date) value; + if (fromSchema != null) { + String fromSchemaName = fromSchema.name(); + if (Date.LOGICAL_NAME.equals(fromSchemaName)) { + int days = Date.fromLogical(fromSchema, date); + long millis = days * MILLIS_PER_DAY; + return Timestamp.toLogical(toSchema, millis); } + if (Time.LOGICAL_NAME.equals(fromSchemaName)) { + long millis = Time.fromLogical(fromSchema, date); + return Timestamp.toLogical(toSchema, millis); + } + if (Timestamp.LOGICAL_NAME.equals(fromSchemaName)) { + return date; + } + } else { + // There is no fromSchema, so no conversion is needed + return date; + } + } + long numeric = asLong(value, fromSchema, null); + return Timestamp.toLogical(toSchema, numeric); + } + + private static List convertToArray(Schema toSchema, Object value) { + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } else if (value instanceof String) { + SchemaAndValue schemaAndValue = parseString(value.toString()); + value = schemaAndValue.value(); + } + if (value instanceof List) { + return (List) value; + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static Map convertToMapInternal(Schema toSchema, Object value) { + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } else if (value instanceof String) { + SchemaAndValue schemaAndValue = parseString(value.toString()); + value = schemaAndValue.value(); + } + if (value instanceof Map) { + return (Map) value; + } + throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); + } + + private static Struct convertToStructInternal(Schema toSchema, Object value) { + if (value == null) { + throw new DataException("Unable to convert a null value to a schema that requires a value"); + } else if (value instanceof Struct) { + return (Struct) value; } throw new DataException("Unable to convert " + value + " (" + value.getClass() + ") to " + toSchema); } @@ -744,174 +797,212 @@ public static DateFormat dateFormatFor(java.util.Date value) { return new SimpleDateFormat(ISO_8601_TIMESTAMP_FORMAT_PATTERN); } - protected static boolean canParseSingleTokenLiteral(Parser parser, boolean embedded, String tokenLiteral) { - int startPosition = parser.mark(); - // If the next token is what we expect, then either... - if (parser.canConsume(tokenLiteral)) { - // ...we're reading an embedded value, in which case the next token will be handled appropriately - // by the caller if it's something like an end delimiter for a map or array, or a comma to - // separate multiple embedded values... - // ...or it's being parsed as part of a top-level string, in which case, any other tokens should - // cause use to stop parsing this single-token literal as such and instead just treat it like - // a string. For example, the top-level string "true}" will be tokenized as the tokens "true" and - // "}", but should ultimately be parsed as just the string "true}" instead of the boolean true. - if (embedded || !parser.hasNext()) { - return true; - } + private static class ValueParser { + + private static final Logger log = LoggerFactory.getLogger(ValueParser.class); + private static final SchemaAndValue TRUE_SCHEMA_AND_VALUE = new SchemaAndValue(Schema.BOOLEAN_SCHEMA, Boolean.TRUE); + private static final SchemaAndValue FALSE_SCHEMA_AND_VALUE = new SchemaAndValue(Schema.BOOLEAN_SCHEMA, Boolean.FALSE); + private static final String TRUE_LITERAL = Boolean.TRUE.toString(); + private static final String FALSE_LITERAL = Boolean.FALSE.toString(); + private static final BigInteger LONG_MIN = BigInteger.valueOf(Long.MIN_VALUE); + private static final BigInteger LONG_MAX = BigInteger.valueOf(Long.MAX_VALUE); + private static final String QUOTE_DELIMITER = "\""; + private static final String COMMA_DELIMITER = ","; + private static final String ENTRY_DELIMITER = ":"; + private static final String ARRAY_BEGIN_DELIMITER = "["; + private static final String ARRAY_END_DELIMITER = "]"; + private static final String MAP_BEGIN_DELIMITER = "{"; + private static final String MAP_END_DELIMITER = "}"; + private static final int ISO_8601_DATE_LENGTH = ISO_8601_DATE_FORMAT_PATTERN.length(); + private static final int ISO_8601_TIME_LENGTH = ISO_8601_TIME_FORMAT_PATTERN.length() - 2; // subtract single quotes + private static final int ISO_8601_TIMESTAMP_LENGTH = ISO_8601_TIMESTAMP_FORMAT_PATTERN.length() - 4; // subtract single quotes + + private final Parser parser; + + private ValueParser(Parser parser) { + this.parser = parser; } - parser.rewindTo(startPosition); - return false; - } - protected static SchemaAndValue parse(Parser parser, boolean embedded) throws NoSuchElementException { - if (!parser.hasNext()) { - return null; - } - if (embedded) { - if (parser.canConsume(QUOTE_DELIMITER)) { - StringBuilder sb = new StringBuilder(); - while (parser.hasNext()) { - if (parser.canConsume(QUOTE_DELIMITER)) { - break; - } - sb.append(parser.next()); - } - String content = sb.toString(); - // We can parse string literals as temporal logical types, but all others - // are treated as strings - SchemaAndValue parsed = parseString(content); - if (parsed != null && TEMPORAL_LOGICAL_TYPE_NAMES.contains(parsed.schema().name())) { - return parsed; + private boolean canParseSingleTokenLiteral(boolean embedded, String tokenLiteral) { + int startPosition = parser.mark(); + // If the next token is what we expect, then either... + if (parser.canConsume(tokenLiteral)) { + // ...we're reading an embedded value, in which case the next token will be handled appropriately + // by the caller if it's something like an end delimiter for a map or array, or a comma to + // separate multiple embedded values... + // ...or it's being parsed as part of a top-level string, in which case, any other tokens should + // cause use to stop parsing this single-token literal as such and instead just treat it like + // a string. For example, the top-level string "true}" will be tokenized as the tokens "true" and + // "}", but should ultimately be parsed as just the string "true}" instead of the boolean true. + if (embedded || !parser.hasNext()) { + return true; } - return new SchemaAndValue(Schema.STRING_SCHEMA, content); } + parser.rewindTo(startPosition); + return false; } - if (canParseSingleTokenLiteral(parser, embedded, NULL_VALUE)) { - return null; - } - if (canParseSingleTokenLiteral(parser, embedded, TRUE_LITERAL)) { - return TRUE_SCHEMA_AND_VALUE; - } - if (canParseSingleTokenLiteral(parser, embedded, FALSE_LITERAL)) { - return FALSE_SCHEMA_AND_VALUE; + public SchemaAndValue parse(boolean embedded) throws NoSuchElementException { + if (!parser.hasNext()) { + return null; + } else if (embedded && parser.canConsume(QUOTE_DELIMITER)) { + return parseQuotedString(); + } else if (canParseSingleTokenLiteral(embedded, NULL_VALUE)) { + return null; + } else if (canParseSingleTokenLiteral(embedded, TRUE_LITERAL)) { + return TRUE_SCHEMA_AND_VALUE; + } else if (canParseSingleTokenLiteral(embedded, FALSE_LITERAL)) { + return FALSE_SCHEMA_AND_VALUE; + } + + int startPosition = parser.mark(); + + try { + if (parser.canConsume(ARRAY_BEGIN_DELIMITER)) { + return parseArray(); + } else if (parser.canConsume(MAP_BEGIN_DELIMITER)) { + return parseMap(); + } + } catch (DataException e) { + log.trace("Unable to parse the value as a map or an array; reverting to string", e); + parser.rewindTo(startPosition); + } + + String token = parser.next(); + if (Utils.isBlank(token)) { + return new SchemaAndValue(Schema.STRING_SCHEMA, token); + } else { + return parseNextToken(embedded, token.trim()); + } } - int startPosition = parser.mark(); + private SchemaAndValue parseNextToken(boolean embedded, String token) { + char firstChar = token.charAt(0); + boolean firstCharIsDigit = Character.isDigit(firstChar); - try { - if (parser.canConsume(ARRAY_BEGIN_DELIMITER)) { - List result = new ArrayList<>(); - boolean compatible = true; - Schema elementSchema = null; - while (parser.hasNext()) { - if (parser.canConsume(ARRAY_END_DELIMITER)) { - Schema listSchema; - if (elementSchema != null && compatible) { - listSchema = SchemaBuilder.array(elementSchema).schema(); - result = alignListEntriesWithSchema(listSchema, result); - } else { - // Every value is null - listSchema = SchemaBuilder.arrayOfNull().build(); - } - return new SchemaAndValue(listSchema, result); - } + // Temporal types are more restrictive, so try them first + if (firstCharIsDigit) { + SchemaAndValue temporal = parseMultipleTokensAsTemporal(token); + if (temporal != null) { + return temporal; + } + } + if (firstCharIsDigit || firstChar == '+' || firstChar == '-') { + try { + return parseAsNumber(token); + } catch (NumberFormatException e) { + // can't parse as a number + } + } + if (embedded) { + throw new DataException("Failed to parse embedded value"); + } + // At this point, the only thing this non-embedded value can be is a string. + return new SchemaAndValue(Schema.STRING_SCHEMA, parser.original()); + } - if (parser.canConsume(COMMA_DELIMITER)) { - throw new DataException("Unable to parse an empty array element: " + parser.original()); - } - SchemaAndValue element = parse(parser, true); - elementSchema = commonSchemaFor(elementSchema, element); - if (elementSchema == null && element != null && element.schema() != null) { - compatible = false; - } - result.add(element != null ? element.value() : null); + private SchemaAndValue parseQuotedString() { + StringBuilder sb = new StringBuilder(); + while (parser.hasNext()) { + if (parser.canConsume(QUOTE_DELIMITER)) { + break; + } + sb.append(parser.next()); + } + String content = sb.toString(); + // We can parse string literals as temporal logical types, but all others + // are treated as strings + SchemaAndValue parsed = parseAsTemporal(content); + if (parsed != null) { + return parsed; + } + return new SchemaAndValue(Schema.STRING_SCHEMA, content); + } - int currentPosition = parser.mark(); - if (parser.canConsume(ARRAY_END_DELIMITER)) { - parser.rewindTo(currentPosition); - } else if (!parser.canConsume(COMMA_DELIMITER)) { - throw new DataException("Array elements missing '" + COMMA_DELIMITER + "' delimiter"); + private SchemaAndValue parseArray() { + List result = new ArrayList<>(); + SchemaMerger elementSchema = new SchemaMerger(); + while (parser.hasNext()) { + if (parser.canConsume(ARRAY_END_DELIMITER)) { + Schema listSchema; + if (elementSchema.hasCommonSchema()) { + listSchema = SchemaBuilder.array(elementSchema.schema()).schema(); + result = alignListEntriesWithSchema(listSchema, result); + } else { + // Every value is null + listSchema = SchemaBuilder.arrayOfNull().build(); } + return new SchemaAndValue(listSchema, result); } - // Missing either a comma or an end delimiter - if (COMMA_DELIMITER.equals(parser.previous())) { - throw new DataException("Array is missing element after ',': " + parser.original()); + if (parser.canConsume(COMMA_DELIMITER)) { + throw new DataException("Unable to parse an empty array element: " + parser.original()); + } + SchemaAndValue element = parse(true); + elementSchema.merge(element); + result.add(element != null ? element.value() : null); + + int currentPosition = parser.mark(); + if (parser.canConsume(ARRAY_END_DELIMITER)) { + parser.rewindTo(currentPosition); + } else if (!parser.canConsume(COMMA_DELIMITER)) { + throw new DataException("Array elements missing '" + COMMA_DELIMITER + "' delimiter"); } - throw new DataException("Array is missing terminating ']': " + parser.original()); } - if (parser.canConsume(MAP_BEGIN_DELIMITER)) { - Map result = new LinkedHashMap<>(); - boolean keyCompatible = true; - Schema keySchema = null; - boolean valueCompatible = true; - Schema valueSchema = null; - while (parser.hasNext()) { - if (parser.canConsume(MAP_END_DELIMITER)) { - Schema mapSchema; - if (keySchema != null && valueSchema != null && keyCompatible && valueCompatible) { - mapSchema = SchemaBuilder.map(keySchema, valueSchema).build(); - result = alignMapKeysAndValuesWithSchema(mapSchema, result); - } else if (keySchema != null && keyCompatible) { - mapSchema = SchemaBuilder.mapWithNullValues(keySchema); - result = alignMapKeysWithSchema(mapSchema, result); - } else { - mapSchema = SchemaBuilder.mapOfNull().build(); - } - return new SchemaAndValue(mapSchema, result); - } + // Missing either a comma or an end delimiter + if (COMMA_DELIMITER.equals(parser.previous())) { + throw new DataException("Array is missing element after ',': " + parser.original()); + } + throw new DataException("Array is missing terminating ']': " + parser.original()); + } - if (parser.canConsume(COMMA_DELIMITER)) { - throw new DataException("Unable to parse a map entry with no key or value: " + parser.original()); - } - SchemaAndValue key = parse(parser, true); - if (key == null || key.value() == null) { - throw new DataException("Map entry may not have a null key: " + parser.original()); + private SchemaAndValue parseMap() { + Map result = new LinkedHashMap<>(); + SchemaMerger keySchema = new SchemaMerger(); + SchemaMerger valueSchema = new SchemaMerger(); + while (parser.hasNext()) { + if (parser.canConsume(MAP_END_DELIMITER)) { + Schema mapSchema; + if (keySchema.hasCommonSchema() && valueSchema.hasCommonSchema()) { + mapSchema = SchemaBuilder.map(keySchema.schema(), valueSchema.schema()).build(); + result = alignMapKeysAndValuesWithSchema(mapSchema, result); + } else if (keySchema.hasCommonSchema()) { + mapSchema = SchemaBuilder.mapWithNullValues(keySchema.schema()); + result = alignMapKeysWithSchema(mapSchema, result); + } else { + mapSchema = SchemaBuilder.mapOfNull().build(); } + return new SchemaAndValue(mapSchema, result); + } - if (!parser.canConsume(ENTRY_DELIMITER)) { - throw new DataException("Map entry is missing '" + ENTRY_DELIMITER - + "' at " + parser.position() - + " in " + parser.original()); - } - SchemaAndValue value = parse(parser, true); - Object entryValue = value != null ? value.value() : null; - result.put(key.value(), entryValue); - - parser.canConsume(COMMA_DELIMITER); - keySchema = commonSchemaFor(keySchema, key); - if (keySchema == null && key.schema() != null) { - keyCompatible = false; - } - valueSchema = commonSchemaFor(valueSchema, value); - if (valueSchema == null && value != null && value.schema() != null) { - valueCompatible = false; - } + if (parser.canConsume(COMMA_DELIMITER)) { + throw new DataException("Unable to parse a map entry with no key or value: " + parser.original()); } - // Missing either a comma or an end delimiter - if (COMMA_DELIMITER.equals(parser.previous())) { - throw new DataException("Map is missing element after ',': " + parser.original()); + SchemaAndValue key = parse(true); + if (key == null || key.value() == null) { + throw new DataException("Map entry may not have a null key: " + parser.original()); + } else if (!parser.canConsume(ENTRY_DELIMITER)) { + throw new DataException("Map entry is missing '" + ENTRY_DELIMITER + + "' at " + parser.position() + + " in " + parser.original()); } - throw new DataException("Map is missing terminating '}': " + parser.original()); - } - } catch (DataException e) { - LOG.trace("Unable to parse the value as a map or an array; reverting to string", e); - parser.rewindTo(startPosition); - } + SchemaAndValue value = parse(true); + Object entryValue = value != null ? value.value() : null; + result.put(key.value(), entryValue); - String token = parser.next(); - if (Utils.isBlank(token)) { - return new SchemaAndValue(Schema.STRING_SCHEMA, token); + parser.canConsume(COMMA_DELIMITER); + keySchema.merge(key); + valueSchema.merge(value); + } + // Missing either a comma or an end delimiter + if (COMMA_DELIMITER.equals(parser.previous())) { + throw new DataException("Map is missing element after ',': " + parser.original()); + } + throw new DataException("Map is missing terminating '}': " + parser.original()); } - token = token.trim(); - char firstChar = token.charAt(0); - boolean firstCharIsDigit = Character.isDigit(firstChar); - - // Temporal types are more restrictive, so try them first - if (firstCharIsDigit) { + private SchemaAndValue parseMultipleTokensAsTemporal(String token) { // The time and timestamp literals may be split into 5 tokens since an unescaped colon // is a delimiter. Check these first since the first of these tokens is a simple numeric int position = parser.mark(); @@ -925,143 +1016,146 @@ protected static SchemaAndValue parse(Parser parser, boolean embedded) throws No } // No match was found using the 5 tokens, so rewind and see if the current token has a date, time, or timestamp parser.rewindTo(position); - SchemaAndValue temporal = parseAsTemporal(token); - if (temporal != null) { - return temporal; - } + return parseAsTemporal(token); } - if (firstCharIsDigit || firstChar == '+' || firstChar == '-') { - try { - // Try to parse as a number ... - BigDecimal decimal = new BigDecimal(token); - try { - return new SchemaAndValue(Schema.INT8_SCHEMA, decimal.byteValueExact()); - } catch (ArithmeticException e) { - // continue - } - try { - return new SchemaAndValue(Schema.INT16_SCHEMA, decimal.shortValueExact()); - } catch (ArithmeticException e) { - // continue - } - try { - return new SchemaAndValue(Schema.INT32_SCHEMA, decimal.intValueExact()); - } catch (ArithmeticException e) { - // continue - } - try { - return new SchemaAndValue(Schema.INT64_SCHEMA, decimal.longValueExact()); - } catch (ArithmeticException e) { - // continue - } - float fValue = decimal.floatValue(); - if (fValue != Float.NEGATIVE_INFINITY && fValue != Float.POSITIVE_INFINITY + + private static SchemaAndValue parseAsNumber(String token) { + // Try to parse as a number ... + BigDecimal decimal = new BigDecimal(token); + SchemaAndValue exactDecimal = parseAsExactDecimal(decimal); + float fValue = decimal.floatValue(); + double dValue = decimal.doubleValue(); + if (exactDecimal != null) { + return exactDecimal; + } else if (fValue != Float.NEGATIVE_INFINITY && fValue != Float.POSITIVE_INFINITY && decimal.scale() != 0) { - return new SchemaAndValue(Schema.FLOAT32_SCHEMA, fValue); - } - double dValue = decimal.doubleValue(); - if (dValue != Double.NEGATIVE_INFINITY && dValue != Double.POSITIVE_INFINITY + return new SchemaAndValue(Schema.FLOAT32_SCHEMA, fValue); + } else if (dValue != Double.NEGATIVE_INFINITY && dValue != Double.POSITIVE_INFINITY && decimal.scale() != 0) { - return new SchemaAndValue(Schema.FLOAT64_SCHEMA, dValue); - } + return new SchemaAndValue(Schema.FLOAT64_SCHEMA, dValue); + } else { Schema schema = Decimal.schema(decimal.scale()); return new SchemaAndValue(schema, decimal); - } catch (NumberFormatException e) { - // can't parse as a number } } - if (embedded) { - throw new DataException("Failed to parse embedded value"); - } - // At this point, the only thing this non-embedded value can be is a string. - return new SchemaAndValue(Schema.STRING_SCHEMA, parser.original()); - } - private static SchemaAndValue parseAsTemporal(String token) { - if (token == null) { + private static SchemaAndValue parseAsExactDecimal(BigDecimal decimal) { + BigDecimal ceil = decimal.setScale(0, RoundingMode.CEILING); + BigDecimal floor = decimal.setScale(0, RoundingMode.FLOOR); + if (ceil.equals(floor)) { + BigInteger num = ceil.toBigIntegerExact(); + if (ceil.precision() >= 19 && (num.compareTo(LONG_MIN) < 0 || num.compareTo(LONG_MAX) > 0)) { + return null; + } + long integral = num.longValue(); + byte int8 = (byte) integral; + short int16 = (short) integral; + int int32 = (int) integral; + if (int8 == integral) { + return new SchemaAndValue(Schema.INT8_SCHEMA, int8); + } else if (int16 == integral) { + return new SchemaAndValue(Schema.INT16_SCHEMA, int16); + } else if (int32 == integral) { + return new SchemaAndValue(Schema.INT32_SCHEMA, int32); + } else { + return new SchemaAndValue(Schema.INT64_SCHEMA, integral); + } + } return null; } - // If the colons were escaped, we'll see the escape chars and need to remove them - token = token.replace("\\:", ":"); - int tokenLength = token.length(); - if (tokenLength == ISO_8601_TIME_LENGTH) { - try { - return new SchemaAndValue(Time.SCHEMA, new SimpleDateFormat(ISO_8601_TIME_FORMAT_PATTERN).parse(token)); - } catch (ParseException e) { - // not a valid date + + private static SchemaAndValue parseAsTemporal(String token) { + if (token == null) { + return null; } - } else if (tokenLength == ISO_8601_TIMESTAMP_LENGTH) { - try { - return new SchemaAndValue(Timestamp.SCHEMA, new SimpleDateFormat(ISO_8601_TIMESTAMP_FORMAT_PATTERN).parse(token)); - } catch (ParseException e) { - // not a valid date + // If the colons were escaped, we'll see the escape chars and need to remove them + token = token.replace("\\:", ":"); + int tokenLength = token.length(); + if (tokenLength == ISO_8601_TIME_LENGTH) { + return parseAsTemporalType(token, Time.SCHEMA, ISO_8601_TIME_FORMAT_PATTERN); + } else if (tokenLength == ISO_8601_TIMESTAMP_LENGTH) { + return parseAsTemporalType(token, Timestamp.SCHEMA, ISO_8601_TIMESTAMP_FORMAT_PATTERN); + } else if (tokenLength == ISO_8601_DATE_LENGTH) { + return parseAsTemporalType(token, Date.SCHEMA, ISO_8601_DATE_FORMAT_PATTERN); + } else { + return null; } - } else if (tokenLength == ISO_8601_DATE_LENGTH) { - try { - return new SchemaAndValue(Date.SCHEMA, new SimpleDateFormat(ISO_8601_DATE_FORMAT_PATTERN).parse(token)); - } catch (ParseException e) { - // not a valid date + } + + private static SchemaAndValue parseAsTemporalType(String token, Schema schema, String pattern) { + ParsePosition pos = new ParsePosition(0); + java.util.Date result = new SimpleDateFormat(pattern).parse(token, pos); + if (pos.getIndex() != 0) { + return new SchemaAndValue(schema, result); } + return null; } - return null; } - protected static Schema commonSchemaFor(Schema previous, SchemaAndValue latest) { - if (latest == null) { - return previous; + /** + * Utility for merging various optional primitive numeric schemas into a common schema. + * If a non-numeric type appears (including logical numeric types), no common schema will be inferred. + * This class is not thread-safe and should only be accessed by one thread. + */ + private static class SchemaMerger { + /** + * Schema which applies to all of the values passed to {@link #merge(SchemaAndValue)} + * Null if no non-null schemas have been seen, or if the values seen do not have a common schema + */ + private Schema common = null; + /** + * Flag to determine the meaning of the null sentinel in {@link #common} + * If true, null means "any optional type", as no non-null values have appeared. + * If false, null means "no common type", as one or more non-null values had mutually exclusive schemas. + */ + private boolean compatible = true; + + protected void merge(SchemaAndValue latest) { + if (latest != null && latest.schema() != null && compatible) { + if (common == null) { + // This null means any type is valid, so choose the new schema. + common = latest.schema(); + } else { + // There is a previous type restriction, so merge the new schema into the old one. + common = mergeSchemas(common, latest.schema()); + // If there isn't a common schema any longer, then give up on finding further compatible schemas. + compatible = common != null; + } + } } - if (previous == null) { - return latest.schema(); + + protected boolean hasCommonSchema() { + return common != null; } - Schema newSchema = latest.schema(); + + protected Schema schema() { + return common; + } + } + + /** + * Merge two schemas to a common schema which can represent values from both input schemas. + * @param previous One Schema, non-null + * @param newSchema Another schema, non-null + * @return A schema that is a superset of both input schemas, or null if no common schema is found. + */ + private static Schema mergeSchemas(Schema previous, Schema newSchema) { Type previousType = previous.type(); Type newType = newSchema.type(); if (previousType != newType) { switch (previous.type()) { case INT8: - if (newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64 || newType == Type.FLOAT32 || newType == - Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForInt8(newSchema, newType); case INT16: - if (newType == Type.INT8) { - return previous; - } - if (newType == Type.INT32 || newType == Type.INT64 || newType == Type.FLOAT32 || newType == Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForInt16(previous, newSchema, newType); case INT32: - if (newType == Type.INT8 || newType == Type.INT16) { - return previous; - } - if (newType == Type.INT64 || newType == Type.FLOAT32 || newType == Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForInt32(previous, newSchema, newType); case INT64: - if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32) { - return previous; - } - if (newType == Type.FLOAT32 || newType == Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForInt64(previous, newSchema, newType); case FLOAT32: - if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64) { - return previous; - } - if (newType == Type.FLOAT64) { - return newSchema; - } - break; + return commonSchemaForFloat32(previous, newSchema, newType); case FLOAT64: - if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64 || newType == - Type.FLOAT32) { - return previous; - } - break; + return commonSchemaForFloat64(previous, newType); } return null; } @@ -1075,6 +1169,59 @@ protected static Schema commonSchemaFor(Schema previous, SchemaAndValue latest) return previous; } + private static Schema commonSchemaForInt8(Schema newSchema, Type newType) { + if (newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64 + || newType == Type.FLOAT32 || newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForInt16(Schema previous, Schema newSchema, Type newType) { + if (newType == Type.INT8) { + return previous; + } else if (newType == Type.INT32 || newType == Type.INT64 + || newType == Type.FLOAT32 || newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForInt32(Schema previous, Schema newSchema, Type newType) { + if (newType == Type.INT8 || newType == Type.INT16) { + return previous; + } else if (newType == Type.INT64 || newType == Type.FLOAT32 || newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForInt64(Schema previous, Schema newSchema, Type newType) { + if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32) { + return previous; + } else if (newType == Type.FLOAT32 || newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForFloat32(Schema previous, Schema newSchema, Type newType) { + if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64) { + return previous; + } else if (newType == Type.FLOAT64) { + return newSchema; + } + return null; + } + + private static Schema commonSchemaForFloat64(Schema previous, Type newType) { + if (newType == Type.INT8 || newType == Type.INT16 || newType == Type.INT32 || newType == Type.INT64 + || newType == Type.FLOAT32) { + return previous; + } + return null; + } + protected static List alignListEntriesWithSchema(Schema schema, List input) { Schema valueSchema = schema.valueSchema(); List result = new ArrayList<>(); diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index 3aad588fc220b..df3c2ade5dd83 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -81,6 +81,20 @@ public class ValuesTest { INT_LIST.add(-987654321); } + @Test + public void shouldParseNullString() { + SchemaAndValue schemaAndValue = Values.parseString(null); + assertNull(schemaAndValue.schema()); + assertNull(schemaAndValue.value()); + } + + @Test + public void shouldParseEmptyString() { + SchemaAndValue schemaAndValue = Values.parseString(""); + assertEquals(Schema.STRING_SCHEMA, schemaAndValue.schema()); + assertEquals("", schemaAndValue.value()); + } + @Test @Timeout(5) public void shouldNotEncounterInfiniteLoop() { @@ -246,6 +260,20 @@ public void shouldEscapeStringsWithEmbeddedQuotesAndBackslashes() { @Test public void shouldConvertNullValue() { + assertRoundTrip(Schema.INT8_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_INT8_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.INT16_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_INT16_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.INT32_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_INT32_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.INT64_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_INT64_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.FLOAT32_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_FLOAT32_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.FLOAT64_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_FLOAT64_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.BOOLEAN_SCHEMA, Schema.STRING_SCHEMA, null); + assertRoundTrip(Schema.OPTIONAL_BOOLEAN_SCHEMA, Schema.STRING_SCHEMA, null); assertRoundTrip(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA, null); assertRoundTrip(Schema.OPTIONAL_STRING_SCHEMA, Schema.STRING_SCHEMA, null); } @@ -253,14 +281,22 @@ public void shouldConvertNullValue() { @Test public void shouldConvertBooleanValues() { assertRoundTrip(Schema.BOOLEAN_SCHEMA, Schema.BOOLEAN_SCHEMA, Boolean.FALSE); + assertShortCircuit(Schema.BOOLEAN_SCHEMA, Boolean.FALSE); SchemaAndValue resultFalse = roundTrip(Schema.BOOLEAN_SCHEMA, "false"); assertEquals(Schema.BOOLEAN_SCHEMA, resultFalse.schema()); assertEquals(Boolean.FALSE, resultFalse.value()); + resultFalse = roundTrip(Schema.BOOLEAN_SCHEMA, "0"); + assertEquals(Schema.BOOLEAN_SCHEMA, resultFalse.schema()); + assertEquals(Boolean.FALSE, resultFalse.value()); assertRoundTrip(Schema.BOOLEAN_SCHEMA, Schema.BOOLEAN_SCHEMA, Boolean.TRUE); + assertShortCircuit(Schema.BOOLEAN_SCHEMA, Boolean.TRUE); SchemaAndValue resultTrue = roundTrip(Schema.BOOLEAN_SCHEMA, "true"); assertEquals(Schema.BOOLEAN_SCHEMA, resultTrue.schema()); assertEquals(Boolean.TRUE, resultTrue.value()); + resultTrue = roundTrip(Schema.BOOLEAN_SCHEMA, "1"); + assertEquals(Schema.BOOLEAN_SCHEMA, resultTrue.schema()); + assertEquals(Boolean.TRUE, resultTrue.value()); } @Test @@ -268,6 +304,38 @@ public void shouldFailToParseInvalidBooleanValueString() { assertThrows(DataException.class, () -> Values.convertToBoolean(Schema.STRING_SCHEMA, "\"green\"")); } + @Test + public void shouldConvertInt8() { + assertRoundTrip(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA, (byte) 0); + assertRoundTrip(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA, (byte) 1); + } + + @Test + public void shouldConvertInt64() { + assertRoundTrip(Schema.INT64_SCHEMA, Schema.INT64_SCHEMA, (long) 1); + assertShortCircuit(Schema.INT64_SCHEMA, (long) 1); + } + + @Test + public void shouldConvertFloat32() { + assertRoundTrip(Schema.FLOAT32_SCHEMA, Schema.FLOAT32_SCHEMA, (float) 1); + assertShortCircuit(Schema.FLOAT32_SCHEMA, (float) 1); + } + + @Test + public void shouldConvertFloat64() { + assertRoundTrip(Schema.FLOAT64_SCHEMA, Schema.FLOAT64_SCHEMA, (double) 1); + assertShortCircuit(Schema.FLOAT64_SCHEMA, (double) 1); + } + + @Test + public void shouldConvertEmptyStruct() { + Struct struct = new Struct(SchemaBuilder.struct().build()); + assertThrows(DataException.class, () -> Values.convertToStruct(struct.schema(), null)); + assertThrows(DataException.class, () -> Values.convertToStruct(struct.schema(), "")); + Values.convertToStruct(struct.schema(), struct); + } + @Test public void shouldConvertSimpleString() { assertRoundTrip(Schema.STRING_SCHEMA, "simple"); @@ -361,7 +429,27 @@ public void shouldConvertStringOfListWithOnlyNumericElementTypesIntoListOfLarges assertEquals(3, list.size()); assertEquals(1, ((Number) list.get(0)).intValue()); assertEquals(2, ((Number) list.get(1)).intValue()); - assertEquals(thirdValue, ((Number) list.get(2)).intValue()); + assertEquals(thirdValue, list.get(2)); + } + + @Test + public void shouldConvertIntegralTypesToFloat() { + float thirdValue = Float.MAX_VALUE; + List list = Values.convertToList(Schema.STRING_SCHEMA, "[1, 2, " + thirdValue + "]"); + assertEquals(3, list.size()); + assertEquals(1, ((Number) list.get(0)).intValue()); + assertEquals(2, ((Number) list.get(1)).intValue()); + assertEquals(thirdValue, list.get(2)); + } + + @Test + public void shouldConvertIntegralTypesToDouble() { + double thirdValue = Double.MAX_VALUE; + List list = Values.convertToList(Schema.STRING_SCHEMA, "[1, 2, " + thirdValue + "]"); + assertEquals(3, list.size()); + assertEquals(1, ((Number) list.get(0)).intValue()); + assertEquals(2, ((Number) list.get(1)).intValue()); + assertEquals(thirdValue, list.get(2)); } /** @@ -434,6 +522,34 @@ private void assertParseStringMapWithNoSchema(Map expected, Stri assertEquals(expected, list); } + @Test + public void shouldParseNestedArray() { + SchemaAndValue schemaAndValue = Values.parseString("[[]]"); + assertEquals(Type.ARRAY, schemaAndValue.schema().type()); + assertEquals(Type.ARRAY, schemaAndValue.schema().valueSchema().type()); + } + + @Test + public void shouldParseArrayContainingMap() { + SchemaAndValue schemaAndValue = Values.parseString("[{}]"); + assertEquals(Type.ARRAY, schemaAndValue.schema().type()); + assertEquals(Type.MAP, schemaAndValue.schema().valueSchema().type()); + } + + @Test + public void shouldParseNestedMap() { + SchemaAndValue schemaAndValue = Values.parseString("{\"a\":{}}"); + assertEquals(Type.MAP, schemaAndValue.schema().type()); + assertEquals(Type.MAP, schemaAndValue.schema().valueSchema().type()); + } + + @Test + public void shouldParseMapContainingArray() { + SchemaAndValue schemaAndValue = Values.parseString("{\"a\":[]}"); + assertEquals(Type.MAP, schemaAndValue.schema().type()); + assertEquals(Type.ARRAY, schemaAndValue.schema().valueSchema().type()); + } + /** * We can't infer or successfully parse into a different type, so this returns the same string. */ @@ -445,6 +561,22 @@ public void shouldParseStringListWithExtraDelimitersAndReturnString() { assertEquals(str, result.value()); } + @Test + public void shouldParseStringListWithNullLastAsString() { + String str = "[1, null]"; + SchemaAndValue result = Values.parseString(str); + assertEquals(Type.STRING, result.schema().type()); + assertEquals(str, result.value()); + } + + @Test + public void shouldParseStringListWithNullFirstAsString() { + String str = "[null, 1]"; + SchemaAndValue result = Values.parseString(str); + assertEquals(Type.STRING, result.schema().type()); + assertEquals(str, result.value()); + } + @Test public void shouldParseTimestampStringAsTimestamp() throws Exception { String str = "2019-08-23T14:34:54.346Z"; @@ -585,6 +717,13 @@ public void shouldParseTimeStringAsTimeInMap() throws Exception { assertEquals(Collections.singletonMap(keyStr, expected), result.value()); } + @Test + public void shouldFailToConvertNullTime() { + assertThrows(DataException.class, () -> Values.convertToTime(null, null)); + assertThrows(DataException.class, () -> Values.convertToDate(null, null)); + assertThrows(DataException.class, () -> Values.convertToTimestamp(null, null)); + } + /** * This is technically invalid JSON, and we don't want to simply ignore the blank elements. */ @@ -802,6 +941,51 @@ public void shouldConvertDecimalValues() { assertEquals(value, Values.convertToDecimal(null, buffer, 1)); } + @Test + public void shouldFailToConvertNullToDecimal() { + assertThrows(DataException.class, () -> Values.convertToDecimal(null, null, 1)); + } + + @Test + public void shouldInferByteSchema() { + byte[] bytes = new byte[1]; + Schema byteSchema = Values.inferSchema(bytes); + assertEquals(Schema.BYTES_SCHEMA, byteSchema); + Schema byteBufferSchema = Values.inferSchema(ByteBuffer.wrap(bytes)); + assertEquals(Schema.BYTES_SCHEMA, byteBufferSchema); + } + + @Test + public void shouldInferStructSchema() { + Struct struct = new Struct(SchemaBuilder.struct().build()); + Schema structSchema = Values.inferSchema(struct); + assertEquals(struct.schema(), structSchema); + } + + @Test + public void shouldInferNoSchemaForEmptyList() { + Schema listSchema = Values.inferSchema(Collections.emptyList()); + assertNull(listSchema); + } + + @Test + public void shouldInferNoSchemaForListContainingObject() { + Schema listSchema = Values.inferSchema(Collections.singletonList(new Object())); + assertNull(listSchema); + } + + @Test + public void shouldInferNoSchemaForEmptyMap() { + Schema listSchema = Values.inferSchema(Collections.emptyMap()); + assertNull(listSchema); + } + + @Test + public void shouldInferNoSchemaForMapContainingObject() { + Schema listSchema = Values.inferSchema(Collections.singletonMap(new Object(), new Object())); + assertNull(listSchema); + } + /** * Test parsing distinct number-like types (strings containing numbers, and logical Decimals) in the same list * The parser does not convert Numbers to Decimals, or Strings containing numbers to Numbers automatically. @@ -819,6 +1003,17 @@ public void shouldNotConvertArrayValuesToDecimal() { assertEquals(expected, schemaAndValue.value()); } + @Test + public void shouldParseArrayOfOnlyDecimals() { + List decimals = Arrays.asList(BigDecimal.valueOf(Long.MAX_VALUE).add(BigDecimal.ONE), + BigDecimal.valueOf(Long.MIN_VALUE).subtract(BigDecimal.ONE)); + SchemaAndValue schemaAndValue = Values.parseString(decimals.toString()); + Schema schema = schemaAndValue.schema(); + assertEquals(Type.ARRAY, schema.type()); + assertEquals(Decimal.schema(0), schema.valueSchema()); + assertEquals(decimals, schemaAndValue.value()); + } + @Test public void canConsume() { } @@ -949,6 +1144,16 @@ public void shouldParseDoubleAsFloat64() { assertEquals(value, (Double) schemaAndValue.value(), 0); } + @Test + public void shouldParseFractionalPartsAsIntegerWhenNoFractionalPart() { + assertEquals(new SchemaAndValue(Schema.INT8_SCHEMA, (byte) 1), Values.parseString("1.0")); + assertEquals(new SchemaAndValue(Schema.FLOAT32_SCHEMA, 1.1f), Values.parseString("1.1")); + assertEquals(new SchemaAndValue(Schema.INT16_SCHEMA, (short) 300), Values.parseString("300.0")); + assertEquals(new SchemaAndValue(Schema.FLOAT32_SCHEMA, 300.01f), Values.parseString("300.01")); + assertEquals(new SchemaAndValue(Schema.INT32_SCHEMA, 66000), Values.parseString("66000.0")); + assertEquals(new SchemaAndValue(Schema.FLOAT32_SCHEMA, 66000.0008f), Values.parseString("66000.0008")); + } + protected void assertParsed(String input) { assertParsed(input, input); } @@ -1011,47 +1216,48 @@ protected SchemaAndValue roundTrip(Schema desiredSchema, SchemaAndValue input) { desiredSchema = Values.inferSchema(input); assertNotNull(desiredSchema); } + return convertTo(desiredSchema, serialized); + } + + protected SchemaAndValue convertTo(Schema desiredSchema, Object value) { Object newValue = null; - Schema newSchema = null; switch (desiredSchema.type()) { case STRING: - newValue = Values.convertToString(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToString(Schema.STRING_SCHEMA, value); break; case INT8: - newValue = Values.convertToByte(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToByte(Schema.STRING_SCHEMA, value); break; case INT16: - newValue = Values.convertToShort(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToShort(Schema.STRING_SCHEMA, value); break; case INT32: - newValue = Values.convertToInteger(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToInteger(Schema.STRING_SCHEMA, value); break; case INT64: - newValue = Values.convertToLong(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToLong(Schema.STRING_SCHEMA, value); break; case FLOAT32: - newValue = Values.convertToFloat(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToFloat(Schema.STRING_SCHEMA, value); break; case FLOAT64: - newValue = Values.convertToDouble(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToDouble(Schema.STRING_SCHEMA, value); break; case BOOLEAN: - newValue = Values.convertToBoolean(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToBoolean(Schema.STRING_SCHEMA, value); break; case ARRAY: - newValue = Values.convertToList(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToList(Schema.STRING_SCHEMA, value); break; case MAP: - newValue = Values.convertToMap(Schema.STRING_SCHEMA, serialized); + newValue = Values.convertToMap(Schema.STRING_SCHEMA, value); break; case STRUCT: - newValue = Values.convertToStruct(Schema.STRING_SCHEMA, serialized); - break; case BYTES: fail("unexpected schema type"); break; } - newSchema = Values.inferSchema(newValue); + Schema newSchema = Values.inferSchema(newValue); return new SchemaAndValue(newSchema, newValue); } @@ -1075,4 +1281,16 @@ protected void assertRoundTrip(Schema schema, Schema currentSchema, Object value assertEquals(result, result2); } } + + protected void assertShortCircuit(Schema schema, Object value) { + SchemaAndValue result = convertTo(schema, value); + + if (value == null) { + assertNull(result.schema()); + assertNull(result.value()); + } else { + assertEquals(value, result.value()); + assertEquals(schema, result.schema()); + } + } } diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java index 8f186400dd29d..353d2eedb9592 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/Checkpoint.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.HashMap; import java.nio.ByteBuffer; +import java.util.Objects; /** Checkpoint records emitted from MirrorCheckpointConnector. Encodes remote consumer group state. */ public class Checkpoint { @@ -180,5 +181,18 @@ byte[] recordKey() { byte[] recordValue() { return serializeValue(VERSION).array(); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Checkpoint that = (Checkpoint) o; + return upstreamOffset == that.upstreamOffset && downstreamOffset == that.downstreamOffset && Objects.equals(consumerGroupId, that.consumerGroupId) && Objects.equals(topicPartition, that.topicPartition) && Objects.equals(metadata, that.metadata); + } + + @Override + public int hashCode() { + return Objects.hash(consumerGroupId, topicPartition, upstreamOffset, downstreamOffset, metadata); + } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java new file mode 100644 index 0000000000000..cbe76efecb38c --- /dev/null +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/CheckpointStore.java @@ -0,0 +1,203 @@ +/* + * 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.connect.mirror; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.AuthorizationException; +import org.apache.kafka.common.protocol.types.SchemaException; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.util.Callback; +import org.apache.kafka.connect.util.KafkaBasedLog; +import org.apache.kafka.connect.util.TopicAdmin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.connect.mirror.MirrorCheckpointConfig.CHECKPOINTS_TARGET_CONSUMER_ROLE; + +/** + * Reads once the Kafka log for checkpoints and populates a map of + * checkpoints per consumer group. + * + * The Kafka log is closed after the initial load and only the in memory map is + * used after start. + */ +public class CheckpointStore implements AutoCloseable { + + private static final Logger log = LoggerFactory.getLogger(CheckpointStore.class); + + private final MirrorCheckpointTaskConfig config; + private final Set consumerGroups; + + private TopicAdmin cpAdmin = null; + private KafkaBasedLog backingStore = null; + // accessible for testing + Map> checkpointsPerConsumerGroup; + + private volatile boolean loadSuccess = false; + private volatile boolean isInitialized = false; + + public CheckpointStore(MirrorCheckpointTaskConfig config, Set consumerGroups) { + this.config = config; + this.consumerGroups = new HashSet<>(consumerGroups); + } + + // constructor for testing only + CheckpointStore(Map> checkpointsPerConsumerGroup) { + this.config = null; //ignored by tests + this.consumerGroups = null; //ignored by tests + this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; + isInitialized = true; + loadSuccess = true; + } + + // potentially long running + public boolean start() { + checkpointsPerConsumerGroup = readCheckpoints(); + isInitialized = true; + if (log.isTraceEnabled()) { + log.trace("CheckpointStore started, load success={}, map={}", loadSuccess, checkpointsPerConsumerGroup); + } else { + log.debug("CheckpointStore started, load success={}, map.size={}", loadSuccess, checkpointsPerConsumerGroup.size()); + } + return loadSuccess; + } + + public boolean isInitialized() { + return isInitialized; + } + + public void update(String group, Map newCheckpoints) { + Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); + oldCheckpoints.putAll(newCheckpoints); + } + + public Map get(String group) { + Map result = checkpointsPerConsumerGroup.get(group); + return result == null ? null : Collections.unmodifiableMap(result); + } + + public Map> computeConvertedUpstreamOffset() { + Map> result = new HashMap<>(); + + for (Map.Entry> entry : checkpointsPerConsumerGroup.entrySet()) { + String consumerId = entry.getKey(); + Map convertedUpstreamOffset = new HashMap<>(); + for (Checkpoint checkpoint : entry.getValue().values()) { + convertedUpstreamOffset.put(checkpoint.topicPartition(), checkpoint.offsetAndMetadata()); + } + result.put(consumerId, convertedUpstreamOffset); + } + return result; + } + + @Override + public void close() { + releaseResources(); + } + + private void releaseResources() { + Utils.closeQuietly(backingStore != null ? backingStore::stop : null, "backing store for previous Checkpoints"); + Utils.closeQuietly(cpAdmin, "admin client for previous Checkpoints"); + cpAdmin = null; + backingStore = null; + } + + // read the checkpoints topic to initialize the checkpointsPerConsumerGroup state + // the callback may only handle errors thrown by consumer.poll in KafkaBasedLog + // e.g. unauthorized to read from topic (non-retriable) + // if any are encountered, treat the loading of Checkpoints as failed. + private Map> readCheckpoints() { + Map> checkpoints = new HashMap<>(); + Callback> consumedCallback = (error, cpRecord) -> { + if (error != null) { + // if there is no authorization to READ from the topic, we must throw an error + // to stop the KafkaBasedLog forever looping attempting to read to end + checkpoints.clear(); + if (error instanceof RuntimeException) { + throw (RuntimeException) error; + } else { + throw new RuntimeException(error); + } + } else { + try { + Checkpoint cp = Checkpoint.deserializeRecord(cpRecord); + if (consumerGroups.contains(cp.consumerGroupId())) { + Map cps = checkpoints.computeIfAbsent(cp.consumerGroupId(), ignored1 -> new HashMap<>()); + cps.put(cp.topicPartition(), cp); + } + } catch (SchemaException ex) { + log.warn("Ignored invalid checkpoint record at offset {}", cpRecord.offset(), ex); + } + } + }; + + try { + long startTime = System.currentTimeMillis(); + readCheckpointsImpl(config, consumedCallback); + log.debug("starting+stopping KafkaBasedLog took {}ms", System.currentTimeMillis() - startTime); + loadSuccess = true; + } catch (Exception error) { + loadSuccess = false; + if (error instanceof AuthorizationException) { + log.warn("Not authorized to access checkpoints topic {} - " + + "this may degrade offset translation as only checkpoints " + + "for offsets which were mirrored after the task started will be emitted", + config.checkpointsTopic(), error); + } else { + log.info("Exception encountered loading checkpoints topic {} - " + + "this may degrade offset translation as only checkpoints " + + "for offsets which were mirrored after the task started will be emitted", + config.checkpointsTopic(), error); + } + } + return checkpoints; + } + + // accessible for testing + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + try { + cpAdmin = new TopicAdmin( + config.targetAdminConfig("checkpoint-target-admin"), + config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin"))); + + backingStore = KafkaBasedLog.withExistingClients( + config.checkpointsTopic(), + MirrorUtils.newConsumer(config.targetConsumerConfig(CHECKPOINTS_TARGET_CONSUMER_ROLE)), + null, + cpAdmin, + consumedCallback, + Time.SYSTEM, + ignored -> { + }, + topicPartition -> topicPartition.partition() == 0); + + backingStore.start(true); + backingStore.stop(); + } finally { + releaseResources(); + } + } +} diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java index e37cee4a79b8b..8be52a9c9be98 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java @@ -75,6 +75,7 @@ public class MirrorCheckpointConfig extends MirrorConnectorConfig { public static final Class GROUP_FILTER_CLASS_DEFAULT = DefaultGroupFilter.class; public static final String OFFSET_SYNCS_SOURCE_CONSUMER_ROLE = "offset-syncs-source-consumer"; public static final String OFFSET_SYNCS_TARGET_CONSUMER_ROLE = "offset-syncs-target-consumer"; + public static final String CHECKPOINTS_TARGET_CONSUMER_ROLE = "checkpoints-target-consumer"; public static final String OFFSET_SYNCS_SOURCE_ADMIN_ROLE = "offset-syncs-source-admin"; public static final String OFFSET_SYNCS_TARGET_ADMIN_ROLE = "offset-syncs-target-admin"; diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java index 96c287add5f14..7f446efea5df3 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTask.java @@ -69,21 +69,25 @@ public class MirrorCheckpointTask extends SourceTask { private MirrorCheckpointMetrics metrics; private Scheduler scheduler; private Map> idleConsumerGroupsOffset; - private Map> checkpointsPerConsumerGroup; + private CheckpointStore checkpointStore; + public MirrorCheckpointTask() {} // for testing MirrorCheckpointTask(String sourceClusterAlias, String targetClusterAlias, - ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, + ReplicationPolicy replicationPolicy, OffsetSyncStore offsetSyncStore, Set consumerGroups, Map> idleConsumerGroupsOffset, - Map> checkpointsPerConsumerGroup) { + CheckpointStore checkpointStore) { this.sourceClusterAlias = sourceClusterAlias; this.targetClusterAlias = targetClusterAlias; this.replicationPolicy = replicationPolicy; this.offsetSyncStore = offsetSyncStore; + this.consumerGroups = consumerGroups; this.idleConsumerGroupsOffset = idleConsumerGroupsOffset; - this.checkpointsPerConsumerGroup = checkpointsPerConsumerGroup; + this.checkpointStore = checkpointStore; this.topicFilter = topic -> true; + this.interval = Duration.ofNanos(1); + this.pollTimeout = Duration.ofNanos(1); } @Override @@ -103,15 +107,18 @@ public void start(Map props) { targetAdminClient = config.forwardingAdmin(config.targetAdminConfig("checkpoint-target-admin")); metrics = config.metrics(); idleConsumerGroupsOffset = new HashMap<>(); - checkpointsPerConsumerGroup = new HashMap<>(); + checkpointStore = new CheckpointStore(config, consumerGroups); scheduler = new Scheduler(getClass(), config.entityLabel(), config.adminTimeout()); scheduler.execute(() -> { - offsetSyncStore.start(); + // loading the stores are potentially long running operations, so they run asynchronously + // to avoid blocking task::start (until a task has completed starting it cannot be stopped) + boolean checkpointsReadOk = checkpointStore.start(); + offsetSyncStore.start(!checkpointsReadOk); scheduler.scheduleRepeating(this::refreshIdleConsumerGroupOffset, config.syncGroupOffsetsInterval(), "refreshing idle consumers group offsets at target cluster"); scheduler.scheduleRepeatingDelayed(this::syncGroupOffset, config.syncGroupOffsetsInterval(), "sync idle consumer group offset from source to target"); - }, "starting offset sync store"); + }, "starting checkpoint and offset sync stores"); log.info("{} checkpointing {} consumer groups {}->{}: {}.", Thread.currentThread().getName(), consumerGroups.size(), sourceClusterAlias, config.targetClusterAlias(), consumerGroups); } @@ -126,6 +133,7 @@ public void stop() { long start = System.currentTimeMillis(); stopping = true; Utils.closeQuietly(topicFilter, "topic filter"); + Utils.closeQuietly(checkpointStore, "checkpoints store"); Utils.closeQuietly(offsetSyncStore, "offset sync store"); Utils.closeQuietly(sourceAdminClient, "source admin client"); Utils.closeQuietly(targetAdminClient, "target admin client"); @@ -146,8 +154,8 @@ public List poll() throws InterruptedException { while (!stopping && System.currentTimeMillis() < deadline) { Thread.sleep(pollTimeout.toMillis()); } - if (stopping) { - // we are stopping, return early. + if (stopping || !checkpointStore.isInitialized()) { + // we are stopping, or not fully initialized, return early. return null; } List records = new ArrayList<>(); @@ -166,14 +174,13 @@ public List poll() throws InterruptedException { } } - - private List sourceRecordsForGroup(String group) throws InterruptedException { + // visible for testing + List sourceRecordsForGroup(String group) throws InterruptedException { try { long timestamp = System.currentTimeMillis(); Map upstreamGroupOffsets = listConsumerGroupOffsets(group); Map newCheckpoints = checkpointsForGroup(upstreamGroupOffsets, group); - Map oldCheckpoints = checkpointsPerConsumerGroup.computeIfAbsent(group, ignored -> new HashMap<>()); - oldCheckpoints.putAll(newCheckpoints); + checkpointStore.update(group, newCheckpoints); return newCheckpoints.values().stream() .map(x -> checkpointRecord(x, timestamp)) .collect(Collectors.toList()); @@ -195,7 +202,7 @@ Map checkpointsForGroup(Map checkpoints = checkpointsPerConsumerGroup.get(checkpoint.consumerGroupId()); + Map checkpoints = checkpointStore.get(checkpoint.consumerGroupId()); if (checkpoints == null) { log.trace("Emitting {} (first for this group)", checkpoint); return true; @@ -314,7 +321,7 @@ Map> syncGroupOffset() throws Exe Map> offsetToSyncAll = new HashMap<>(); // first, sync offsets for the idle consumers at target - for (Entry> group : getConvertedUpstreamOffset().entrySet()) { + for (Entry> group : checkpointStore.computeConvertedUpstreamOffset().entrySet()) { String consumerGroupId = group.getKey(); // for each idle consumer at target, read the checkpoints (converted upstream offset) // from the pre-populated map @@ -391,18 +398,4 @@ void syncGroupOffset(String consumerGroupId, Map> getConvertedUpstreamOffset() { - Map> result = new HashMap<>(); - - for (Entry> entry : checkpointsPerConsumerGroup.entrySet()) { - String consumerId = entry.getKey(); - Map convertedUpstreamOffset = new HashMap<>(); - for (Checkpoint checkpoint : entry.getValue().values()) { - convertedUpstreamOffset.put(checkpoint.topicPartition(), checkpoint.offsetAndMetadata()); - } - result.put(consumerId, convertedUpstreamOffset); - } - return result; - } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java index eca5cc68f997f..16038044ddd2b 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncStore.java @@ -54,7 +54,7 @@ * started after the position of the consumer group, or if relevant offset syncs for the topic were potentially used as * for translation in an earlier generation of the sync store. */ -class OffsetSyncStore implements AutoCloseable { +public class OffsetSyncStore implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(OffsetSyncStore.class); // Store one offset sync for each bit of the topic offset. @@ -63,8 +63,10 @@ class OffsetSyncStore implements AutoCloseable { private final KafkaBasedLog backingStore; private final Map offsetSyncs = new ConcurrentHashMap<>(); private final TopicAdmin admin; + protected volatile boolean initializationMustReadToEnd = true; protected volatile boolean readToEnd = false; + // package access to avoid Java 21 "this-escape" warning OffsetSyncStore(MirrorCheckpointConfig config) { Consumer consumer = null; TopicAdmin admin = null; @@ -97,6 +99,7 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig ); } + // for testing OffsetSyncStore() { this.admin = null; this.backingStore = null; @@ -105,12 +108,19 @@ private KafkaBasedLog createBackingStore(MirrorCheckpointConfig /** * Start the OffsetSyncStore, blocking until all previous Offset Syncs have been read from backing storage. */ - public void start() { - backingStore.start(); + public void start(boolean initializationMustReadToEnd) { + this.initializationMustReadToEnd = initializationMustReadToEnd; + log.debug("OffsetSyncStore starting - must read to OffsetSync end = {}", initializationMustReadToEnd); + backingStoreStart(); readToEnd = true; } - OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { + // overridable for testing + void backingStoreStart() { + backingStore.start(false); + } + + public OptionalLong translateDownstream(String group, TopicPartition sourceTopicPartition, long upstreamOffset) { if (!readToEnd) { // If we have not read to the end of the syncs topic at least once, decline to translate any offsets. // This prevents emitting stale offsets while initially reading the offset syncs topic. @@ -214,7 +224,9 @@ private void updateSyncArray(OffsetSync[] syncs, OffsetSync[] original, OffsetSy // While reading to the end of the topic, ensure that our earliest sync is later than // any earlier sync that could have been used for translation, to preserve monotonicity // If the upstream offset rewinds, all previous offsets are invalid, so overwrite them all. - if (!readToEnd || syncs[0].upstreamOffset() > upstreamOffset) { + boolean onlyLoadLastOffset = !readToEnd && initializationMustReadToEnd; + boolean upstreamRewind = upstreamOffset < syncs[0].upstreamOffset(); + if (onlyLoadLastOffset || upstreamRewind) { clearSyncArray(syncs, offsetSync); return; } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java new file mode 100644 index 0000000000000..b7b3904899f1a --- /dev/null +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/CheckpointStoreTest.java @@ -0,0 +1,97 @@ +/* + * 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.connect.mirror; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.connect.util.Callback; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CheckpointStoreTest { + + @Test + public void testReadCheckpointsTopic() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + try (CheckpointStore store = new CheckpointStore(config, consumerGroups) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 0, 0)); + // this record must be ignored as not part of consumerGroups for task + consumedCallback.onCompletion(null, newCheckpointRecord("group2", "t1", 0, 0, 0)); + // this record must be ignored as malformed + consumedCallback.onCompletion(null, + new ConsumerRecord<>("checkpoint.topic", 0, 0L, new byte[0], new byte[0])); + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "t1", 0, 1, 1)); + } + }) { + assertFalse(store.isInitialized()); + + assertTrue(store.start(), "expected start to return success"); + assertTrue(store.isInitialized()); + + Map> expected = new HashMap<>(); + expected.put("group1", Collections.singletonMap(new TopicPartition("t1", 0), + new Checkpoint("group1", new TopicPartition("t1", 0), 1, 1, ""))); + assertEquals(expected, store.checkpointsPerConsumerGroup); + } + } + + @Test + public void testReadCheckpointsTopicError() { + Set consumerGroups = new HashSet<>(); + consumerGroups.add("group1"); + + MirrorCheckpointTaskConfig config = mock(MirrorCheckpointTaskConfig.class); + when(config.checkpointsTopic()).thenReturn("checkpoint.topic"); + + try (CheckpointStore store = new CheckpointStore(config, consumerGroups) { + @Override + void readCheckpointsImpl(MirrorCheckpointTaskConfig config, Callback> consumedCallback) { + consumedCallback.onCompletion(null, newCheckpointRecord("group1", "topic", 1, 0, 0)); + consumedCallback.onCompletion(new TopicAuthorizationException("test"), null); + } + }) { + + assertFalse(store.start(), "expected start to return failure"); + assertTrue(store.isInitialized()); + assertTrue(store.checkpointsPerConsumerGroup.isEmpty()); + } + } + + ConsumerRecord newCheckpointRecord(String gid, String topic, int partition, long upo, long dwo) { + Checkpoint cp = new Checkpoint(gid, new TopicPartition(topic, partition), upo, dwo, ""); + return new ConsumerRecord<>("checkpoint.topic", 0, 0L, cp.recordKey(), cp.recordValue()); + } +} diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java index c7aec3e547881..0afc4f74f2f97 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.mirror; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Collections; import java.util.Optional; @@ -32,14 +33,19 @@ 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.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + public class MirrorCheckpointTaskTest { @Test public void testDownstreamTopicRenaming() { MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), null, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); assertEquals(new TopicPartition("source1.topic3", 4), mirrorCheckpointTask.renameTopicPartition(new TopicPartition("topic3", 4)), "Renaming source1.topic3 failed"); @@ -58,9 +64,10 @@ public void testCheckpoint() { long t2UpstreamOffset = 7L; long t2DownstreamOffset = 8L; OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), + Collections.emptyMap(), new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 2), t1UpstreamOffset, t1DownstreamOffset); offsetSyncStore.sync(new TopicPartition("target2.topic5", 6), t2UpstreamOffset, t2DownstreamOffset); Optional optionalCheckpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 2), @@ -160,7 +167,8 @@ public void testSyncOffset() throws ExecutionException, InterruptedException { checkpointsPerConsumerGroup.put(consumer2, checkpointMapC2); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), null, idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, + new CheckpointStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -190,7 +198,8 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu checkpointsPerConsumerGroup.put(consumer, checkpointMap); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source", "target", - new DefaultReplicationPolicy(), null, idleConsumerGroupsOffset, checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), null, Collections.emptySet(), idleConsumerGroupsOffset, + new CheckpointStore(checkpointsPerConsumerGroup)); Map> output = mirrorCheckpointTask.syncGroupOffset(); @@ -200,9 +209,10 @@ public void testSyncOffsetForTargetGroupWithNullOffsetAndMetadata() throws Execu @Test public void testNoCheckpointForTopicWithoutOffsetSyncs() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 3L, 4L); Optional checkpoint1 = mirrorCheckpointTask.checkpoint("group9", new TopicPartition("topic1", 1), @@ -216,9 +226,10 @@ public void testNoCheckpointForTopicWithoutOffsetSyncs() { @Test public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), Collections.emptyMap()); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); offsetSyncStore.sync(new TopicPartition("topic1", 0), 1L, 3L); Optional checkpoint = mirrorCheckpointTask.checkpoint("g1", new TopicPartition("topic1", 0), null); assertFalse(checkpoint.isPresent()); @@ -227,10 +238,11 @@ public void testNoCheckpointForTopicWithNullOffsetAndMetadata() { @Test public void testCheckpointRecordsMonotonicIfStoreRewinds() { OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore(); - offsetSyncStore.start(); + offsetSyncStore.start(true); Map> checkpointsPerConsumerGroup = new HashMap<>(); MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", - new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptyMap(), checkpointsPerConsumerGroup); + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(checkpointsPerConsumerGroup)); TopicPartition tp = new TopicPartition("topic1", 0); TopicPartition targetTP = new TopicPartition("source1.topic1", 0); @@ -252,7 +264,7 @@ public void testCheckpointRecordsMonotonicIfStoreRewinds() { offsetSyncStore.sync(tp, upstream++, downstream++); offsetSyncStore.sync(tp, upstream++, downstream++); offsetSyncStore.sync(tp, upstream++, downstream++); - offsetSyncStore.sync(tp, upstream++, downstream++); + offsetSyncStore.sync(tp, upstream, downstream); // The OffsetSyncStore will change its translation of the same offset assertNotEquals(OptionalLong.of(expectedDownstreamOffset), offsetSyncStore.translateDownstream("g1", tp, consumerGroupOffset)); // But the task will filter this out and not emit a checkpoint @@ -271,4 +283,93 @@ private Map assertCheckpointForTopic( assertEquals(truth, checkpoints.containsKey(remoteTp), "should" + (truth ? "" : " not") + " emit offset sync"); return checkpoints; } + + @Test + public void testCheckpointsTaskRestartUsesExistingCheckpoints() { + TopicPartition t1p0 = new TopicPartition("t1", 0); + TopicPartition sourceT1p0 = new TopicPartition("source1.t1", 0); + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + // OffsetSyncStore contains entries for: 100->100, 200->200, 300->300 + for (int i = 100; i <= 300; i += 100) { + sync(t1p0, i, i); + } + } + }; + offsetSyncStore.start(false); + + MirrorCheckpointTask mirrorCheckpointTask = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(Collections.emptyMap())); + + // Generate a checkpoint for upstream offset 250, and assert it maps to downstream 201 + // (as nearest mapping in OffsetSyncStore is 200->200) + Map upstreamGroupOffsets = new HashMap<>(); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); + Map checkpoints = mirrorCheckpointTask.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, checkpoints.size()); + assertEquals(new Checkpoint("group1", sourceT1p0, 250, 201, ""), checkpoints.get(sourceT1p0)); + + // Simulate task restart, during which more offsets are added to the sync topic, and thus the + // corresponding OffsetSyncStore no longer has a mapping for 100->100 + // Now OffsetSyncStore contains entries for: 175->175, 375->375, 475->475 + OffsetSyncStoreTest.FakeOffsetSyncStore offsetSyncStore2 = new OffsetSyncStoreTest.FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int i = 175; i <= 475; i += 100) { + sync(t1p0, i, i); + } + } + }; + offsetSyncStore2.start(false); + + // Simulate loading existing checkpoints into checkpointsPerConsumerGroup (250->201) + Map> checkpointsPerConsumerGroup = new HashMap<>(); + checkpointsPerConsumerGroup.put("group1", checkpoints); + MirrorCheckpointTask mirrorCheckpointTask2 = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), offsetSyncStore2, Collections.emptySet(), Collections.emptyMap(), + new CheckpointStore(checkpointsPerConsumerGroup)); + + // Upstream offsets 250 and 370 now have the closest downstream value of 176, but this is + // earlier than the downstream value of the last checkpoint (201) - so they are not emitted. + assertEquals(OptionalLong.of(176), offsetSyncStore2.translateDownstream(null, t1p0, 250)); + assertEquals(OptionalLong.of(176), offsetSyncStore2.translateDownstream(null, t1p0, 370)); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(250)); + assertTrue(mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1").isEmpty()); + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(370)); + assertTrue(mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1").isEmpty()); + + // Upstream offset 400 has a closes downstream value of 376, and is emitted because it has + // a later downstream offset than the last checkpoint's downstream (201) + upstreamGroupOffsets.put(t1p0, new OffsetAndMetadata(400)); + Map checkpoints2 = mirrorCheckpointTask2.checkpointsForGroup(upstreamGroupOffsets, "group1"); + assertEquals(1, checkpoints2.size()); + assertEquals(new Checkpoint("group1", sourceT1p0, 400, 376, ""), checkpoints2.get(sourceT1p0)); + } + + @Test + public void testCheckpointStoreInitialized() throws InterruptedException { + CheckpointStore checkpointStore = mock(CheckpointStore.class); + + MirrorCheckpointTask task = new MirrorCheckpointTask("source1", "target2", + new DefaultReplicationPolicy(), + new OffsetSyncStoreTest.FakeOffsetSyncStore(), + Collections.singleton("group"), + Collections.emptyMap(), + checkpointStore) { + + @Override + List sourceRecordsForGroup(String group) { + SourceRecord sr = new SourceRecord(Collections.emptyMap(), Collections.emptyMap(), "", 0, null, null); + return Collections.singletonList(sr); + } + }; + + assertNull(task.poll()); + + when(checkpointStore.isInitialized()).thenReturn(true); + List polled = task.poll(); + assertEquals(1, polled.size()); + } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java index bc76a1994db92..3f2ddbc62e932 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/OffsetSyncStoreTest.java @@ -34,18 +34,22 @@ public class OffsetSyncStoreTest { static TopicPartition tp = new TopicPartition("topic1", 2); static class FakeOffsetSyncStore extends OffsetSyncStore { + private boolean startCalled = false; - FakeOffsetSyncStore() { - super(); + @Override + public void start(boolean initializationMustReadToEnd) { + startCalled = true; + super.start(initializationMustReadToEnd); } @Override - public void start() { - // do not call super to avoid NPE without a KafkaBasedLog. - readToEnd = true; + void backingStoreStart() { + // do not start KafkaBasedLog } + // simulate OffsetSync load as from KafkaBasedLog void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOffset) { + assertTrue(startCalled); // sync in tests should only be called after store.start OffsetSync offsetSync = new OffsetSync(topicPartition, upstreamOffset, downstreamOffset); byte[] key = offsetSync.recordKey(); byte[] value = offsetSync.recordValue(); @@ -57,7 +61,7 @@ void sync(TopicPartition topicPartition, long upstreamOffset, long downstreamOff @Test public void testOffsetTranslation() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); // Emit synced downstream offset without dead-reckoning store.sync(tp, 100, 200); @@ -82,20 +86,24 @@ public void testOffsetTranslation() { @Test public void testNoTranslationIfStoreNotStarted() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + // read a sync during startup + sync(tp, 100, 200); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 0)); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 100)); + assertEquals(OptionalLong.empty(), translateDownstream(null, tp, 200)); + } + }) { // no offsets exist and store is not started assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); - // read a sync during startup - store.sync(tp, 100, 200); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 100)); - assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 200)); - // After the store is started all offsets are visible - store.start(); + store.start(true); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 0)); assertEquals(OptionalLong.of(200), store.translateDownstream(null, tp, 100)); assertEquals(OptionalLong.of(201), store.translateDownstream(null, tp, 200)); @@ -105,26 +113,29 @@ public void testNoTranslationIfStoreNotStarted() { @Test public void testNoTranslationIfNoOffsetSync() { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); assertEquals(OptionalLong.empty(), store.translateDownstream(null, tp, 0)); } } @Test public void testPastOffsetTranslation() { - try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - int maxOffsetLag = 10; - int offset = 0; - for (; offset <= 1000; offset += maxOffsetLag) { - store.sync(tp, offset, offset); - assertSparseSyncInvariant(store, tp); + int maxOffsetLag = 10; + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { + sync(tp, offset, offset); + assertSparseSyncInvariant(this, tp); + } } - store.start(); + }) { + store.start(true); // After starting but before seeing new offsets, only the latest startup offset can be translated assertSparseSync(store, 1000, -1); - for (; offset <= 10000; offset += maxOffsetLag) { + for (int offset = 1000 + maxOffsetLag; offset <= 10000; offset += maxOffsetLag) { store.sync(tp, offset, offset); assertSparseSyncInvariant(store, tp); } @@ -155,6 +166,55 @@ public void testPastOffsetTranslation() { } } + // this test has been written knowing the exact offsets syncs stored + @Test + public void testPastOffsetTranslationWithoutInitializationReadToEnd() { + final int maxOffsetLag = 10; + + try (FakeOffsetSyncStore store = new FakeOffsetSyncStore() { + @Override + void backingStoreStart() { + for (int offset = 0; offset <= 1000; offset += maxOffsetLag) { + sync(tp, offset, offset); + assertSparseSyncInvariant(this, tp); + } + } + }) { + + store.start(false); + + // After starting but before seeing new offsets + assertSparseSync(store, 480, 0); + assertSparseSync(store, 720, 480); + assertSparseSync(store, 1000, 990); + + for (int offset = 1000; offset <= 10000; offset += maxOffsetLag) { + store.sync(tp, offset, offset); + assertSparseSyncInvariant(store, tp); + } + + // After seeing new offsets, 1000 was kicked out of the store, so + // offsets before 3840 can only be translated to 1, only previously stored offset is 0 + assertSparseSync(store, 3840, 0); + assertSparseSync(store, 7680, 3840); + assertSparseSync(store, 8640, 7680); + assertSparseSync(store, 9120, 8640); + assertSparseSync(store, 9600, 9120); + assertSparseSync(store, 9840, 9600); + assertSparseSync(store, 9900, 9840); + assertSparseSync(store, 9960, 9900); + assertSparseSync(store, 9990, 9960); + assertSparseSync(store, 10000, 9990); + + // Rewinding upstream offsets should clear all historical syncs + store.sync(tp, 1500, 11000); + assertSparseSyncInvariant(store, tp); + assertEquals(OptionalLong.of(-1), store.translateDownstream(null, tp, 1499)); + assertEquals(OptionalLong.of(11000), store.translateDownstream(null, tp, 1500)); + assertEquals(OptionalLong.of(11001), store.translateDownstream(null, tp, 2000)); + } + } + @Test public void testConsistentlySpacedSyncs() { // Under normal operation, the incoming syncs will be regularly spaced and the store should keep a set of syncs @@ -215,7 +275,7 @@ public void testDroppedSyncsSpacing() { */ private void assertSyncSpacingHasBoundedExpirations(long firstOffset, LongStream steps, int maximumExpirations) { try (FakeOffsetSyncStore store = new FakeOffsetSyncStore()) { - store.start(); + store.start(true); store.sync(tp, firstOffset, firstOffset); PrimitiveIterator.OfLong iterator = steps.iterator(); long offset = firstOffset; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index 36203399766c3..c1f19e3349101 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -107,6 +107,8 @@ public class KafkaBasedLog { private boolean stopRequested; private final Queue> readLogEndOffsetCallbacks; private final java.util.function.Consumer initializer; + // initialized as false for backward compatibility + private volatile boolean reportErrorsToCallback = false; /** * Create a new KafkaBasedLog object. This does not start reading the log and writing is not permitted until @@ -243,7 +245,12 @@ public void stop() { } public void start() { - log.info("Starting KafkaBasedLog with topic " + topic); + start(false); + } + + public void start(boolean reportErrorsToCallback) { + this.reportErrorsToCallback = reportErrorsToCallback; + log.info("Starting KafkaBasedLog with topic {} reportErrorsToCallback={}", topic, reportErrorsToCallback); // Create the topic admin client and initialize the topic ... admin = topicAdminSupplier.get(); // may be null @@ -468,6 +475,9 @@ private void poll(long timeoutMs) { throw e; } catch (KafkaException e) { log.error("Error polling: " + e); + if (reportErrorsToCallback) { + consumedCallback.onCompletion(e, null); + } } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index e5e58f78761b9..f959d225377e9 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -57,6 +57,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.metadata.BrokerState; import org.apache.kafka.network.SocketServerConfigs; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.apache.kafka.storage.internals.log.CleanerConfig; import org.slf4j.Logger; @@ -163,7 +164,7 @@ public void start() { private void doStart() { brokerConfig.put(ZkConfigs.ZK_CONNECT_CONFIG, zKConnectString()); - putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true); + putIfAbsent(brokerConfig, ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, true); putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 0); putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) brokers.length); putIfAbsent(brokerConfig, AUTO_CREATE_TOPICS_ENABLE_CONFIG, false); @@ -178,7 +179,7 @@ private void doStart() { listenerName = new ListenerName(listenerConfig.toString()); for (int i = 0; i < brokers.length; i++) { - brokerConfig.put(KafkaConfig.BrokerIdProp(), i); + brokerConfig.put(ServerConfigs.BROKER_ID_CONFIG, i); currentBrokerLogDirs[i] = currentBrokerLogDirs[i] == null ? createLogDir() : currentBrokerLogDirs[i]; brokerConfig.put(LOG_DIR_CONFIG, currentBrokerLogDirs[i]); if (!hasListenerConfig) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 6555b7c0cda1a..62e5f56adc56e 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -21,7 +21,6 @@ import kafka.cluster.Partition; import kafka.log.UnifiedLog; import kafka.server.BrokerTopicStats; -import kafka.server.KafkaConfig; import kafka.server.StopPartition; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; @@ -44,6 +43,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.log.remote.metadata.storage.ClassLoaderAwareRemoteLogMetadataManager; import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager; import org.apache.kafka.server.log.remote.storage.LogSegmentData; @@ -252,7 +252,7 @@ public RemoteStorageManager run() { private void configureRSM() { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); - rsmProps.put(KafkaConfig.BrokerIdProp(), brokerId); + rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); remoteLogStorageManager.configure(rsmProps); } @@ -286,7 +286,7 @@ private void configureRLMM() { // update the remoteLogMetadataProps here to override endpoint config if any rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps()); - rlmmProps.put(KafkaConfig.BrokerIdProp(), brokerId); + rlmmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); @@ -605,6 +605,7 @@ boolean isLeader() { // the task's run() method. private volatile Optional copiedOffsetOption = Optional.empty(); private volatile boolean isLogStartOffsetUpdatedOnBecomingLeader = false; + private volatile Optional logDirectory = Optional.empty(); public void convertToLeader(int leaderEpochVal) { if (leaderEpochVal < 0) { @@ -815,6 +816,13 @@ public void run() { } UnifiedLog log = unifiedLogOptional.get(); + // In the first run after completing altering logDir within broker, we should make sure the state is reset. (KAFKA-16711) + if (!log.parentDir().equals(logDirectory.orElse(null))) { + copiedOffsetOption = Optional.empty(); + isLogStartOffsetUpdatedOnBecomingLeader = false; + logDirectory = Optional.of(log.parentDir()); + } + if (isLeader()) { // Copy log segments to remote storage copyLogSegmentsToRemote(log); diff --git a/core/src/main/java/kafka/server/ReplicaAlterLogDirsTierStateMachine.java b/core/src/main/java/kafka/server/ReplicaAlterLogDirsTierStateMachine.java deleted file mode 100644 index 8561fae01990e..0000000000000 --- a/core/src/main/java/kafka/server/ReplicaAlterLogDirsTierStateMachine.java +++ /dev/null @@ -1,41 +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 kafka.server; - -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.message.FetchResponseData.PartitionData; - -import java.util.Optional; - -/** - The replica alter log dirs tier state machine is unsupported but is provided to the ReplicaAlterLogDirsThread. - */ -public class ReplicaAlterLogDirsTierStateMachine implements TierStateMachine { - - public PartitionFetchState start(TopicPartition topicPartition, - PartitionFetchState currentFetchState, - PartitionData fetchPartitionData) throws Exception { - // JBOD is not supported with tiered storage. - throw new UnsupportedOperationException("Building remote log aux state is not supported in ReplicaAlterLogDirsThread."); - } - - public Optional maybeAdvanceState(TopicPartition topicPartition, - PartitionFetchState currentFetchState) { - return Optional.empty(); - } -} diff --git a/core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java b/core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java deleted file mode 100644 index 0462e12c05bca..0000000000000 --- a/core/src/main/java/kafka/server/ReplicaFetcherTierStateMachine.java +++ /dev/null @@ -1,280 +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 kafka.server; - -import java.io.BufferedReader; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.StandardCopyOption; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -import kafka.cluster.Partition; -import kafka.log.UnifiedLog; -import kafka.log.remote.RemoteLogManager; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.message.FetchResponseData.PartitionData; -import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset; -import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition; -import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.common.CheckpointFile; -import org.apache.kafka.server.common.OffsetAndEpoch; -import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; -import org.apache.kafka.server.log.remote.storage.RemoteStorageException; -import org.apache.kafka.server.log.remote.storage.RemoteStorageManager; -import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; -import org.apache.kafka.storage.internals.log.EpochEntry; -import org.apache.kafka.storage.internals.log.LogFileUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import scala.Option; -import scala.collection.JavaConverters; - -import static org.apache.kafka.storage.internals.log.LogStartOffsetIncrementReason.LeaderOffsetIncremented; - -/** - The replica fetcher tier state machine follows a state machine progression. - - Currently, the tier state machine follows a synchronous execution, and we only need to start the machine. - There is no need to advance the state. - - When started, the tier state machine will fetch the local log start offset of the - leader and then build the follower's remote log aux state until the leader's - local log start offset. - */ -public class ReplicaFetcherTierStateMachine implements TierStateMachine { - private static final Logger log = LoggerFactory.getLogger(ReplicaFetcherTierStateMachine.class); - - private final LeaderEndPoint leader; - private final ReplicaManager replicaMgr; - - public ReplicaFetcherTierStateMachine(LeaderEndPoint leader, - ReplicaManager replicaMgr) { - this.leader = leader; - this.replicaMgr = replicaMgr; - } - - - /** - * Start the tier state machine for the provided topic partition. Currently, this start method will build the - * entire remote aux log state synchronously. - * - * @param topicPartition the topic partition - * @param currentFetchState the current PartitionFetchState which will - * be used to derive the return value - * @param fetchPartitionData the data from the fetch response that returned the offset moved to tiered storage error - * - * @return the new PartitionFetchState after the successful start of the - * tier state machine - */ - public PartitionFetchState start(TopicPartition topicPartition, - PartitionFetchState currentFetchState, - PartitionData fetchPartitionData) throws Exception { - - OffsetAndEpoch epochAndLeaderLocalStartOffset = leader.fetchEarliestLocalOffset(topicPartition, currentFetchState.currentLeaderEpoch()); - int epoch = epochAndLeaderLocalStartOffset.leaderEpoch(); - long leaderLocalStartOffset = epochAndLeaderLocalStartOffset.offset(); - - long offsetToFetch = 0; - replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).buildRemoteLogAuxStateRequestRate().mark(); - replicaMgr.brokerTopicStats().allTopicsStats().buildRemoteLogAuxStateRequestRate().mark(); - - try { - offsetToFetch = buildRemoteLogAuxState(topicPartition, currentFetchState.currentLeaderEpoch(), leaderLocalStartOffset, epoch, fetchPartitionData.logStartOffset()); - } catch (RemoteStorageException e) { - replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).failedBuildRemoteLogAuxStateRate().mark(); - replicaMgr.brokerTopicStats().allTopicsStats().failedBuildRemoteLogAuxStateRate().mark(); - throw e; - } - - OffsetAndEpoch fetchLatestOffsetResult = leader.fetchLatestOffset(topicPartition, currentFetchState.currentLeaderEpoch()); - long leaderEndOffset = fetchLatestOffsetResult.offset(); - - long initialLag = leaderEndOffset - offsetToFetch; - - return PartitionFetchState.apply(currentFetchState.topicId(), offsetToFetch, Option.apply(initialLag), currentFetchState.currentLeaderEpoch(), - Fetching$.MODULE$, replicaMgr.localLogOrException(topicPartition).latestEpoch()); - } - - /** - * This is currently a no-op but will be used for implementing async tiering logic in KAFKA-13560. - * - * @param topicPartition the topic partition - * @param currentFetchState the current PartitionFetchState which will - * be used to derive the return value - * - * @return the original PartitionFetchState - */ - public Optional maybeAdvanceState(TopicPartition topicPartition, - PartitionFetchState currentFetchState) { - // No-op for now - return Optional.of(currentFetchState); - } - - private EpochEndOffset fetchEarlierEpochEndOffset(Integer epoch, - TopicPartition partition, - Integer currentLeaderEpoch) { - int previousEpoch = epoch - 1; - - // Find the end-offset for the epoch earlier to the given epoch from the leader - Map partitionsWithEpochs = new HashMap<>(); - partitionsWithEpochs.put(partition, new OffsetForLeaderPartition().setPartition(partition.partition()).setCurrentLeaderEpoch(currentLeaderEpoch).setLeaderEpoch(previousEpoch)); - Option maybeEpochEndOffset = leader.fetchEpochEndOffsets(JavaConverters.mapAsScalaMap(partitionsWithEpochs)).get(partition); - if (maybeEpochEndOffset.isEmpty()) { - throw new KafkaException("No response received for partition: " + partition); - } - - EpochEndOffset epochEndOffset = maybeEpochEndOffset.get(); - if (epochEndOffset.errorCode() != Errors.NONE.code()) { - throw Errors.forCode(epochEndOffset.errorCode()).exception(); - } - - return epochEndOffset; - } - - private List readLeaderEpochCheckpoint(RemoteLogManager rlm, - RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws IOException, RemoteStorageException { - InputStream inputStream = rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.LEADER_EPOCH); - try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) { - CheckpointFile.CheckpointReadBuffer readBuffer = new CheckpointFile.CheckpointReadBuffer<>("", bufferedReader, 0, LeaderEpochCheckpointFile.FORMATTER); - return readBuffer.read(); - } - } - - private void buildProducerSnapshotFile(File snapshotFile, - RemoteLogSegmentMetadata remoteLogSegmentMetadata, - RemoteLogManager rlm) throws IOException, RemoteStorageException { - File tmpSnapshotFile = new File(snapshotFile.getAbsolutePath() + ".tmp"); - // Copy it to snapshot file in atomic manner. - Files.copy(rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT), - tmpSnapshotFile.toPath(), StandardCopyOption.REPLACE_EXISTING); - Utils.atomicMoveWithFallback(tmpSnapshotFile.toPath(), snapshotFile.toPath(), false); - } - - /** - * It tries to build the required state for this partition from leader and remote storage so that it can start - * fetching records from the leader. The return value is the next offset to fetch from the leader, which is the - * next offset following the end offset of the remote log portion. - */ - private Long buildRemoteLogAuxState(TopicPartition topicPartition, - Integer currentLeaderEpoch, - Long leaderLocalLogStartOffset, - Integer epochForLeaderLocalLogStartOffset, - Long leaderLogStartOffset) throws IOException, RemoteStorageException { - - UnifiedLog unifiedLog = replicaMgr.localLogOrException(topicPartition); - - long nextOffset; - - if (unifiedLog.remoteStorageSystemEnable() && unifiedLog.config().remoteStorageEnable()) { - if (replicaMgr.remoteLogManager().isEmpty()) throw new IllegalStateException("RemoteLogManager is not yet instantiated"); - - RemoteLogManager rlm = replicaMgr.remoteLogManager().get(); - - // Find the respective leader epoch for (leaderLocalLogStartOffset - 1). We need to build the leader epoch cache - // until that offset - long previousOffsetToLeaderLocalLogStartOffset = leaderLocalLogStartOffset - 1; - int targetEpoch; - // If the existing epoch is 0, no need to fetch from earlier epoch as the desired offset(leaderLogStartOffset - 1) - // will have the same epoch. - if (epochForLeaderLocalLogStartOffset == 0) { - targetEpoch = epochForLeaderLocalLogStartOffset; - } else { - // Fetch the earlier epoch/end-offset(exclusive) from the leader. - EpochEndOffset earlierEpochEndOffset = fetchEarlierEpochEndOffset(epochForLeaderLocalLogStartOffset, topicPartition, currentLeaderEpoch); - // Check if the target offset lies within the range of earlier epoch. Here, epoch's end-offset is exclusive. - if (earlierEpochEndOffset.endOffset() > previousOffsetToLeaderLocalLogStartOffset) { - // Always use the leader epoch from returned earlierEpochEndOffset. - // This gives the respective leader epoch, that will handle any gaps in epochs. - // For ex, leader epoch cache contains: - // leader-epoch start-offset - // 0 20 - // 1 85 - // <2> - gap no messages were appended in this leader epoch. - // 3 90 - // 4 98 - // There is a gap in leader epoch. For leaderLocalLogStartOffset as 90, leader-epoch is 3. - // fetchEarlierEpochEndOffset(2) will return leader-epoch as 1, end-offset as 90. - // So, for offset 89, we should return leader epoch as 1 like below. - targetEpoch = earlierEpochEndOffset.leaderEpoch(); - } else { - targetEpoch = epochForLeaderLocalLogStartOffset; - } - } - - Optional maybeRlsm = rlm.fetchRemoteLogSegmentMetadata(topicPartition, targetEpoch, previousOffsetToLeaderLocalLogStartOffset); - - if (maybeRlsm.isPresent()) { - RemoteLogSegmentMetadata remoteLogSegmentMetadata = maybeRlsm.get(); - // Build leader epoch cache, producer snapshots until remoteLogSegmentMetadata.endOffset() and start - // segments from (remoteLogSegmentMetadata.endOffset() + 1) - // Assign nextOffset with the offset from which next fetch should happen. - nextOffset = remoteLogSegmentMetadata.endOffset() + 1; - - // Truncate the existing local log before restoring the leader epoch cache and producer snapshots. - Partition partition = replicaMgr.getPartitionOrException(topicPartition); - partition.truncateFullyAndStartAt(nextOffset, false, Option.apply(leaderLogStartOffset)); - - // Increment start offsets - unifiedLog.maybeIncrementLogStartOffset(leaderLogStartOffset, LeaderOffsetIncremented); - unifiedLog.maybeIncrementLocalLogStartOffset(nextOffset, LeaderOffsetIncremented); - - // Build leader epoch cache. - List epochs = readLeaderEpochCheckpoint(rlm, remoteLogSegmentMetadata); - if (unifiedLog.leaderEpochCache().isDefined()) { - unifiedLog.leaderEpochCache().get().assign(epochs); - } - - log.debug("Updated the epoch cache from remote tier till offset: {} with size: {} for {}", leaderLocalLogStartOffset, epochs.size(), partition); - - // Restore producer snapshot - File snapshotFile = LogFileUtils.producerSnapshotFile(unifiedLog.dir(), nextOffset); - buildProducerSnapshotFile(snapshotFile, remoteLogSegmentMetadata, rlm); - - // Reload producer snapshots. - unifiedLog.producerStateManager().truncateFullyAndReloadSnapshots(); - unifiedLog.loadProducerState(nextOffset); - log.debug("Built the leader epoch cache and producer snapshots from remote tier for {}, " + - "with active producers size: {}, leaderLogStartOffset: {}, and logEndOffset: {}", - partition, unifiedLog.producerStateManager().activeProducers().size(), leaderLogStartOffset, nextOffset); - } else { - throw new RemoteStorageException("Couldn't build the state from remote store for partition: " + topicPartition + - ", currentLeaderEpoch: " + currentLeaderEpoch + - ", leaderLocalLogStartOffset: " + leaderLocalLogStartOffset + - ", leaderLogStartOffset: " + leaderLogStartOffset + - ", epoch: " + targetEpoch + - "as the previous remote log segment metadata was not found"); - } - } else { - // If the tiered storage is not enabled throw an exception back so that it will retry until the tiered storage - // is set as expected. - throw new RemoteStorageException("Couldn't build the state from remote store for partition " + topicPartition + ", as remote log storage is not yet enabled"); - } - - return nextOffset; - } -} diff --git a/core/src/main/java/kafka/server/TierStateMachine.java b/core/src/main/java/kafka/server/TierStateMachine.java index 58a44cc647232..085e6c025dd4c 100644 --- a/core/src/main/java/kafka/server/TierStateMachine.java +++ b/core/src/main/java/kafka/server/TierStateMachine.java @@ -17,15 +17,65 @@ package kafka.server; -import java.util.Optional; +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import kafka.cluster.Partition; +import kafka.log.UnifiedLog; +import kafka.log.remote.RemoteLogManager; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.FetchResponseData.PartitionData; +import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData; +import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.common.CheckpointFile; +import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteStorageException; +import org.apache.kafka.server.log.remote.storage.RemoteStorageManager; +import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile; +import org.apache.kafka.storage.internals.log.EpochEntry; +import org.apache.kafka.storage.internals.log.LogFileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.collection.JavaConverters; + +import static org.apache.kafka.storage.internals.log.LogStartOffsetIncrementReason.LeaderOffsetIncremented; /** - * This interface defines the APIs needed to handle any state transitions related to tiering + * This class defines the APIs and implementation needed to handle any state transitions related to tiering + * + * When started, the tier state machine will fetch the local log start offset of the + * leader and then build the follower's remote log aux state until the leader's + * local log start offset. */ -public interface TierStateMachine { +public class TierStateMachine { + private static final Logger log = LoggerFactory.getLogger(TierStateMachine.class); + + private final LeaderEndPoint leader; + private final ReplicaManager replicaMgr; + private final boolean useFutureLog; + public TierStateMachine(LeaderEndPoint leader, + ReplicaManager replicaMgr, + boolean useFutureLog) { + this.leader = leader; + this.replicaMgr = replicaMgr; + this.useFutureLog = useFutureLog; + } /** * Start the tier state machine for the provided topic partition. @@ -40,19 +90,176 @@ public interface TierStateMachine { */ PartitionFetchState start(TopicPartition topicPartition, PartitionFetchState currentFetchState, - PartitionData fetchPartitionData) throws Exception; + PartitionData fetchPartitionData) throws Exception { + OffsetAndEpoch epochAndLeaderLocalStartOffset = leader.fetchEarliestLocalOffset(topicPartition, currentFetchState.currentLeaderEpoch()); + int epoch = epochAndLeaderLocalStartOffset.leaderEpoch(); + long leaderLocalStartOffset = epochAndLeaderLocalStartOffset.offset(); + + long offsetToFetch; + replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).buildRemoteLogAuxStateRequestRate().mark(); + replicaMgr.brokerTopicStats().allTopicsStats().buildRemoteLogAuxStateRequestRate().mark(); + + UnifiedLog unifiedLog; + if (useFutureLog) { + unifiedLog = replicaMgr.futureLogOrException(topicPartition); + } else { + unifiedLog = replicaMgr.localLogOrException(topicPartition); + } + + try { + offsetToFetch = buildRemoteLogAuxState(topicPartition, currentFetchState.currentLeaderEpoch(), leaderLocalStartOffset, epoch, fetchPartitionData.logStartOffset(), unifiedLog); + } catch (RemoteStorageException e) { + replicaMgr.brokerTopicStats().topicStats(topicPartition.topic()).failedBuildRemoteLogAuxStateRate().mark(); + replicaMgr.brokerTopicStats().allTopicsStats().failedBuildRemoteLogAuxStateRate().mark(); + throw e; + } + + OffsetAndEpoch fetchLatestOffsetResult = leader.fetchLatestOffset(topicPartition, currentFetchState.currentLeaderEpoch()); + long leaderEndOffset = fetchLatestOffsetResult.offset(); + + long initialLag = leaderEndOffset - offsetToFetch; + + return PartitionFetchState.apply(currentFetchState.topicId(), offsetToFetch, Option.apply(initialLag), currentFetchState.currentLeaderEpoch(), + Fetching$.MODULE$, unifiedLog.latestEpoch()); + + } + + private OffsetForLeaderEpochResponseData.EpochEndOffset fetchEarlierEpochEndOffset(Integer epoch, + TopicPartition partition, + Integer currentLeaderEpoch) { + int previousEpoch = epoch - 1; + + // Find the end-offset for the epoch earlier to the given epoch from the leader + Map partitionsWithEpochs = new HashMap<>(); + partitionsWithEpochs.put(partition, new OffsetForLeaderEpochRequestData.OffsetForLeaderPartition().setPartition(partition.partition()).setCurrentLeaderEpoch(currentLeaderEpoch).setLeaderEpoch(previousEpoch)); + Option maybeEpochEndOffset = leader.fetchEpochEndOffsets(JavaConverters.mapAsScalaMap(partitionsWithEpochs)).get(partition); + if (maybeEpochEndOffset.isEmpty()) { + throw new KafkaException("No response received for partition: " + partition); + } + + OffsetForLeaderEpochResponseData.EpochEndOffset epochEndOffset = maybeEpochEndOffset.get(); + if (epochEndOffset.errorCode() != Errors.NONE.code()) { + throw Errors.forCode(epochEndOffset.errorCode()).exception(); + } + + return epochEndOffset; + } + + private List readLeaderEpochCheckpoint(RemoteLogManager rlm, + RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws IOException, RemoteStorageException { + InputStream inputStream = rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.LEADER_EPOCH); + try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8))) { + CheckpointFile.CheckpointReadBuffer readBuffer = new CheckpointFile.CheckpointReadBuffer<>("", bufferedReader, 0, LeaderEpochCheckpointFile.FORMATTER); + return readBuffer.read(); + } + } + + private void buildProducerSnapshotFile(UnifiedLog unifiedLog, + long nextOffset, + RemoteLogSegmentMetadata remoteLogSegmentMetadata, + RemoteLogManager rlm) throws IOException, RemoteStorageException { + // Restore producer snapshot + File snapshotFile = LogFileUtils.producerSnapshotFile(unifiedLog.dir(), nextOffset); + Path tmpSnapshotFile = Paths.get(snapshotFile.getAbsolutePath() + ".tmp"); + // Copy it to snapshot file in atomic manner. + Files.copy(rlm.storageManager().fetchIndex(remoteLogSegmentMetadata, RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT), + tmpSnapshotFile, StandardCopyOption.REPLACE_EXISTING); + Utils.atomicMoveWithFallback(tmpSnapshotFile, snapshotFile.toPath(), false); + + // Reload producer snapshots. + unifiedLog.producerStateManager().truncateFullyAndReloadSnapshots(); + unifiedLog.loadProducerState(nextOffset); + } /** - * Optionally advance the state of the tier state machine, based on the - * current PartitionFetchState. The decision to advance the tier - * state machine is implementation specific. - * - * @param topicPartition the topic partition - * @param currentFetchState the current PartitionFetchState which will - * be used to derive the return value - * - * @return the new PartitionFetchState if the tier state machine was advanced, otherwise, return the currentFetchState + * It tries to build the required state for this partition from leader and remote storage so that it can start + * fetching records from the leader. The return value is the next offset to fetch from the leader, which is the + * next offset following the end offset of the remote log portion. */ - Optional maybeAdvanceState(TopicPartition topicPartition, - PartitionFetchState currentFetchState); + private Long buildRemoteLogAuxState(TopicPartition topicPartition, + Integer currentLeaderEpoch, + Long leaderLocalLogStartOffset, + Integer epochForLeaderLocalLogStartOffset, + Long leaderLogStartOffset, + UnifiedLog unifiedLog) throws IOException, RemoteStorageException { + + if (!unifiedLog.remoteStorageSystemEnable() || !unifiedLog.config().remoteStorageEnable()) { + // If the tiered storage is not enabled throw an exception back so that it will retry until the tiered storage + // is set as expected. + throw new RemoteStorageException("Couldn't build the state from remote store for partition " + topicPartition + ", as remote log storage is not yet enabled"); + } + + if (replicaMgr.remoteLogManager().isEmpty()) + throw new IllegalStateException("RemoteLogManager is not yet instantiated"); + + RemoteLogManager rlm = replicaMgr.remoteLogManager().get(); + + // Find the respective leader epoch for (leaderLocalLogStartOffset - 1). We need to build the leader epoch cache + // until that offset + long previousOffsetToLeaderLocalLogStartOffset = leaderLocalLogStartOffset - 1; + int targetEpoch; + // If the existing epoch is 0, no need to fetch from earlier epoch as the desired offset(leaderLogStartOffset - 1) + // will have the same epoch. + if (epochForLeaderLocalLogStartOffset == 0) { + targetEpoch = epochForLeaderLocalLogStartOffset; + } else { + // Fetch the earlier epoch/end-offset(exclusive) from the leader. + OffsetForLeaderEpochResponseData.EpochEndOffset earlierEpochEndOffset = fetchEarlierEpochEndOffset(epochForLeaderLocalLogStartOffset, topicPartition, currentLeaderEpoch); + // Check if the target offset lies within the range of earlier epoch. Here, epoch's end-offset is exclusive. + if (earlierEpochEndOffset.endOffset() > previousOffsetToLeaderLocalLogStartOffset) { + // Always use the leader epoch from returned earlierEpochEndOffset. + // This gives the respective leader epoch, that will handle any gaps in epochs. + // For ex, leader epoch cache contains: + // leader-epoch start-offset + // 0 20 + // 1 85 + // <2> - gap no messages were appended in this leader epoch. + // 3 90 + // 4 98 + // There is a gap in leader epoch. For leaderLocalLogStartOffset as 90, leader-epoch is 3. + // fetchEarlierEpochEndOffset(2) will return leader-epoch as 1, end-offset as 90. + // So, for offset 89, we should return leader epoch as 1 like below. + targetEpoch = earlierEpochEndOffset.leaderEpoch(); + } else { + targetEpoch = epochForLeaderLocalLogStartOffset; + } + } + + RemoteLogSegmentMetadata remoteLogSegmentMetadata = rlm.fetchRemoteLogSegmentMetadata(topicPartition, targetEpoch, previousOffsetToLeaderLocalLogStartOffset) + .orElseThrow(() -> new RemoteStorageException("Couldn't build the state from remote store for partition: " + topicPartition + + ", currentLeaderEpoch: " + currentLeaderEpoch + + ", leaderLocalLogStartOffset: " + leaderLocalLogStartOffset + + ", leaderLogStartOffset: " + leaderLogStartOffset + + ", epoch: " + targetEpoch + + "as the previous remote log segment metadata was not found")); + + + // Build leader epoch cache, producer snapshots until remoteLogSegmentMetadata.endOffset() and start + // segments from (remoteLogSegmentMetadata.endOffset() + 1) + // Assign nextOffset with the offset from which next fetch should happen. + long nextOffset = remoteLogSegmentMetadata.endOffset() + 1; + + // Truncate the existing local log before restoring the leader epoch cache and producer snapshots. + Partition partition = replicaMgr.getPartitionOrException(topicPartition); + partition.truncateFullyAndStartAt(nextOffset, useFutureLog, Option.apply(leaderLogStartOffset)); + // Increment start offsets + unifiedLog.maybeIncrementLogStartOffset(leaderLogStartOffset, LeaderOffsetIncremented); + unifiedLog.maybeIncrementLocalLogStartOffset(nextOffset, LeaderOffsetIncremented); + + // Build leader epoch cache. + List epochs = readLeaderEpochCheckpoint(rlm, remoteLogSegmentMetadata); + if (unifiedLog.leaderEpochCache().isDefined()) { + unifiedLog.leaderEpochCache().get().assign(epochs); + } + + log.info("Updated the epoch cache from remote tier till offset: {} with size: {} for {}", leaderLocalLogStartOffset, epochs.size(), partition); + + buildProducerSnapshotFile(unifiedLog, nextOffset, remoteLogSegmentMetadata, rlm); + + log.debug("Built the leader epoch cache and producer snapshots from remote tier for {}, " + + "with active producers size: {}, leaderLogStartOffset: {}, and logEndOffset: {}", + partition, unifiedLog.producerStateManager().activeProducers().size(), leaderLogStartOffset, nextOffset); + + return nextOffset; + } } diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 980c6ccb258fe..e6783b7d190c1 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -427,7 +427,7 @@ class Partition(val topicPartition: TopicPartition, * @param highWatermarkCheckpoints Checkpoint to load initial high watermark from * @return true iff the future replica is created */ - def maybeCreateFutureReplica(logDir: String, highWatermarkCheckpoints: OffsetCheckpoints): Boolean = { + def maybeCreateFutureReplica(logDir: String, highWatermarkCheckpoints: OffsetCheckpoints, topicId: Option[Uuid] = topicId): Boolean = { // The writeLock is needed to make sure that while the caller checks the log directory of the // current replica and the existence of the future replica, no other thread can update the log directory of the // current replica or remove the future replica. diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index a2d6853ddc5de..e89876e443ce3 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -494,9 +494,9 @@ class GroupMetadataManager(brokerId: Int, if (isTxnOffsetCommit) { addProducerGroup(producerId, group.groupId) - group.prepareTxnOffsetCommit(producerId, offsetMetadata) + group.prepareTxnOffsetCommit(producerId, filteredOffsetMetadata) } else { - group.prepareOffsetCommit(offsetMetadata) + group.prepareOffsetCommit(filteredOffsetMetadata) } appendForGroup(group, records, requestLocal, putCacheCallback, verificationGuards) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index c4782dd95cea3..94445bed2c96b 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.Defaults +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation} @@ -812,7 +812,7 @@ private[transaction] case class TransactionConfig(transactionalIdExpirationMs: I transactionLogMinInsyncReplicas: Int = TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_DEFAULT, abortTimedOutTransactionsIntervalMs: Int = TransactionStateManagerConfigs.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_DEFAULT, removeExpiredTransactionalIdsIntervalMs: Int = TransactionStateManagerConfigs.TRANSACTIONS_REMOVE_EXPIRED_TRANSACTIONAL_ID_CLEANUP_INTERVAL_MS_DEFAULT, - requestTimeoutMs: Int = Defaults.REQUEST_TIMEOUT_MS) + requestTimeoutMs: Int = ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT) case class TransactionalIdAndProducerIdEpoch(transactionalId: String, producerId: Long, producerEpoch: Short) { override def toString: String = { diff --git a/core/src/main/scala/kafka/log/LocalLog.scala b/core/src/main/scala/kafka/log/LocalLog.scala index b2121f5312d7b..09fa464af043c 100644 --- a/core/src/main/scala/kafka/log/LocalLog.scala +++ b/core/src/main/scala/kafka/log/LocalLog.scala @@ -383,12 +383,19 @@ class LocalLog(@volatile private var _dir: File, val segment = segmentOpt.get val baseOffset = segment.baseOffset - val maxPosition = - // Use the max offset position if it is on this segment; otherwise, the segment size is the limit. - if (maxOffsetMetadata.segmentBaseOffset == segment.baseOffset) maxOffsetMetadata.relativePositionInSegment - else segment.size - - fetchDataInfo = segment.read(startOffset, maxLength, maxPosition, minOneMessage) + // 1. If `maxOffsetMetadata#segmentBaseOffset < segment#baseOffset`, then return maxPosition as empty. + // 2. Use the max-offset position if it is on this segment; otherwise, the segment size is the limit. + // 3. When maxOffsetMetadata is message-offset-only, then we don't know the relativePositionInSegment so + // return maxPosition as empty to avoid reading beyond the max-offset + val maxPositionOpt: Optional[java.lang.Long] = + if (segment.baseOffset < maxOffsetMetadata.segmentBaseOffset) + Optional.of(segment.size) + else if (segment.baseOffset == maxOffsetMetadata.segmentBaseOffset && !maxOffsetMetadata.messageOffsetOnly()) + Optional.of(maxOffsetMetadata.relativePositionInSegment) + else + Optional.empty() + + fetchDataInfo = segment.read(startOffset, maxLength, maxPositionOpt, minOneMessage) if (fetchDataInfo != null) { if (includeAbortedTxns) fetchDataInfo = addAbortedTransactions(startOffset, segment, fetchDataInfo) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 45aee545f82ce..0b166c62535e9 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit import kafka.common._ import kafka.log.LogCleaner.{CleanerRecopyPercentMetricName, DeadThreadCountMetricName, MaxBufferUtilizationPercentMetricName, MaxCleanTimeMetricName, MaxCompactionDelayMetricsName} import kafka.server.{BrokerReconfigurable, KafkaConfig} -import kafka.utils._ +import kafka.utils.{Logging, Pool} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.errors.{CorruptRecordException, KafkaStorageException} @@ -32,9 +32,11 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{BufferSupplier, Time} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.ShutdownableThread import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, LastRecord, LogDirFailureChannel, LogSegment, LogSegmentOffsetOverflowException, OffsetMap, SkimpyOffsetMap, TransactionIndex} +import org.apache.kafka.storage.internals.utils.Throttler import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -109,12 +111,7 @@ class LogCleaner(initialConfig: CleanerConfig, private[log] val cleanerManager = new LogCleanerManager(logDirs, logs, logDirFailureChannel) /* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */ - private[log] val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, - checkIntervalMs = 300, - throttleDown = true, - "cleaner-io", - "bytes", - time = time) + private[log] val throttler = new Throttler(config.maxIoBytesPerSecond, 300, "cleaner-io", "bytes", time) private[log] val cleaners = mutable.ArrayBuffer[CleanerThread]() @@ -503,7 +500,7 @@ object LogCleaner { CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, - KafkaConfig.MessageMaxBytesProp, + ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP ) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 5508660a41aad..b68a9f35a0c5d 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -361,7 +361,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, val offsetMetadata = highWatermarkMetadata if (offsetMetadata.messageOffsetOnly) { lock.synchronized { - val fullOffset = convertToOffsetMetadataOrThrow(highWatermark) + val fullOffset = maybeConvertToOffsetMetadata(highWatermark) updateHighWatermarkMetadata(fullOffset) fullOffset } @@ -405,7 +405,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, case Some(offsetMetadata) if offsetMetadata.messageOffset < highWatermarkMetadata.messageOffset => if (offsetMetadata.messageOffsetOnly) { lock synchronized { - val fullOffset = convertToOffsetMetadataOrThrow(offsetMetadata.messageOffset) + val fullOffset = maybeConvertToOffsetMetadata(offsetMetadata.messageOffset) if (firstUnstableOffsetMetadata.contains(offsetMetadata)) firstUnstableOffsetMetadata = Some(fullOffset) fullOffset @@ -964,7 +964,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, val updatedFirstUnstableOffset = producerStateManager.firstUnstableOffset.asScala match { case Some(logOffsetMetadata) if logOffsetMetadata.messageOffsetOnly || logOffsetMetadata.messageOffset < logStartOffset => val offset = math.max(logOffsetMetadata.messageOffset, logStartOffset) - Some(convertToOffsetMetadataOrThrow(offset)) + Some(maybeConvertToOffsetMetadata(offset)) case other => other } @@ -1425,11 +1425,18 @@ class UnifiedLog(@volatile var logStartOffset: Long, /** * Given a message offset, find its corresponding offset metadata in the log. - * If the message offset is out of range, throw an OffsetOutOfRangeException + * 1. If the message offset is less than the log-start-offset (or) local-log-start-offset, then it returns the + * message-only metadata. + * 2. If the message offset is beyond the log-end-offset, then it returns the message-only metadata. + * 3. For all other cases, it returns the offset metadata from the log. */ - private def convertToOffsetMetadataOrThrow(offset: Long): LogOffsetMetadata = { - checkLogStartOffset(offset) - localLog.convertToOffsetMetadataOrThrow(offset) + private[log] def maybeConvertToOffsetMetadata(offset: Long): LogOffsetMetadata = { + try { + localLog.convertToOffsetMetadataOrThrow(offset) + } catch { + case _: OffsetOutOfRangeException => + new LogOffsetMetadata(offset) + } } /** diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala index e2064602c81a3..83973ede23dbf 100755 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala @@ -20,10 +20,12 @@ package kafka.metrics -import kafka.utils.{CoreUtils, VerifiableProperties} +import kafka.utils.VerifiableProperties import org.apache.kafka.server.metrics.MetricConfigs +import org.apache.kafka.server.util.Csv import scala.collection.Seq +import scala.jdk.CollectionConverters._ class KafkaMetricsConfig(props: VerifiableProperties) { @@ -31,8 +33,8 @@ class KafkaMetricsConfig(props: VerifiableProperties) { * Comma-separated list of reporter types. These classes should be on the * classpath and will be instantiated at run-time. */ - val reporters: Seq[String] = CoreUtils.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, - MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT)) + val reporters: Seq[String] = Csv.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, + MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT)).asScala /** * The metrics polling interval (in seconds). diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 6dd4bc5051328..42d1e6585e6cf 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -47,7 +47,7 @@ import org.apache.kafka.common.utils.{KafkaThread, LogContext, Time, Utils} import org.apache.kafka.common.{Endpoint, KafkaException, MetricName, Reconfigurable} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.CredentialProvider -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.FutureUtils import org.slf4j.event.Level @@ -425,7 +425,7 @@ object SocketServer { object DataPlaneAcceptor { val ThreadPrefix: String = "data-plane" val MetricPrefix: String = "" - val ListenerReconfigurableConfigs: Set[String] = Set(KafkaConfig.NumNetworkThreadsProp) + val ListenerReconfigurableConfigs: Set[String] = Set(ServerConfigs.NUM_NETWORK_THREADS_CONFIG) } class DataPlaneAcceptor(socketServer: SocketServer, @@ -506,7 +506,7 @@ class DataPlaneAcceptor(socketServer: SocketServer, * the configs have passed validation using [[validateReconfiguration( Map )]]. */ override def reconfigure(configs: util.Map[String, _]): Unit = { - val newNumNetworkThreads = configs.get(KafkaConfig.NumNetworkThreadsProp).asInstanceOf[Int] + val newNumNetworkThreads = configs.get(ServerConfigs.NUM_NETWORK_THREADS_CONFIG).asInstanceOf[Int] if (newNumNetworkThreads != processors.length) { info(s"Resizing network thread pool size for ${endPoint.listenerName} listener from ${processors.length} to $newNumNetworkThreads") @@ -522,7 +522,7 @@ class DataPlaneAcceptor(socketServer: SocketServer, * Configure this class with the given key-value pairs */ override def configure(configs: util.Map[String, _]): Unit = { - addProcessors(configs.get(KafkaConfig.NumNetworkThreadsProp).asInstanceOf[Int]) + addProcessors(configs.get(ServerConfigs.NUM_NETWORK_THREADS_CONFIG).asInstanceOf[Int]) } } diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 15f87b9c247e9..5e107aa148772 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -223,7 +223,7 @@ final class KafkaMetadataLog private ( override def highWatermark: LogOffsetMetadata = { val hwm = log.fetchOffsetSnapshot.highWatermark - val segmentPosition: Optional[OffsetMetadata] = if (hwm.messageOffsetOnly) { + val segmentPosition: Optional[OffsetMetadata] = if (!hwm.messageOffsetOnly) { Optional.of(SegmentPosition(hwm.segmentBaseOffset, hwm.relativePositionInSegment)) } else { Optional.empty() diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index f44197abd38c7..6bf8bd893ba70 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -30,9 +30,7 @@ import kafka.utils.CoreUtils import kafka.utils.FileLock import kafka.utils.Logging import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient} -import org.apache.kafka.common.KafkaException -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.Uuid +import org.apache.kafka.common.{KafkaException, Node, TopicPartition, Uuid} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} @@ -41,7 +39,7 @@ import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time, Utils} -import org.apache.kafka.raft.{FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, RaftClient, QuorumConfig, ReplicatedLog} +import org.apache.kafka.raft.{FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, QuorumConfig, RaftClient, ReplicatedLog} import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.util.KafkaScheduler @@ -49,6 +47,7 @@ import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.server.util.timer.SystemTimer import scala.jdk.CollectionConverters._ +import scala.jdk.OptionConverters._ object KafkaRaftManager { private def createLogDirectory(logDir: File, logDirName: String): File = { @@ -133,6 +132,8 @@ trait RaftManager[T] { def client: RaftClient[T] def replicatedLog: ReplicatedLog + + def voterNode(id: Int, listener: String): Option[Node] } class KafkaRaftManager[T]( @@ -313,4 +314,8 @@ class KafkaRaftManager[T]( override def leaderAndEpoch: LeaderAndEpoch = { client.leaderAndEpoch } + + override def voterNode(id: Int, listener: String): Option[Node] = { + client.voterNode(id, listener).toScala + } } diff --git a/core/src/main/scala/kafka/serializer/Decoder.scala b/core/src/main/scala/kafka/serializer/Decoder.scala index ce166689cfa6e..a0220a83913d5 100644 --- a/core/src/main/scala/kafka/serializer/Decoder.scala +++ b/core/src/main/scala/kafka/serializer/Decoder.scala @@ -27,13 +27,15 @@ import kafka.utils.VerifiableProperties * An implementation is required to provide a constructor that * takes a VerifiableProperties instance. */ -trait Decoder[T] { +@deprecated(since = "3.8.0") +trait Decoder[T] { def fromBytes(bytes: Array[Byte]): T } /** * The default implementation does nothing, just returns the same byte array it takes in. */ +@deprecated(since = "3.8.0") class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[Byte]] { def fromBytes(bytes: Array[Byte]): Array[Byte] = bytes } @@ -42,6 +44,7 @@ class DefaultDecoder(props: VerifiableProperties = null) extends Decoder[Array[B * The string decoder translates bytes into strings. It uses UTF8 by default but takes * an optional property serializer.encoding to control this. */ +@deprecated(since = "3.8.0") class StringDecoder(props: VerifiableProperties = null) extends Decoder[String] { val encoding: String = if (props == null) @@ -57,6 +60,7 @@ class StringDecoder(props: VerifiableProperties = null) extends Decoder[String] /** * The long decoder translates bytes into longs. */ +@deprecated(since = "3.8.0") class LongDecoder(props: VerifiableProperties = null) extends Decoder[Long] { def fromBytes(bytes: Array[Byte]): Long = { ByteBuffer.wrap(bytes).getLong @@ -66,6 +70,7 @@ class LongDecoder(props: VerifiableProperties = null) extends Decoder[Long] { /** * The integer decoder translates bytes into integers. */ +@deprecated(since = "3.8.0") class IntegerDecoder(props: VerifiableProperties = null) extends Decoder[Integer] { def fromBytes(bytes: Array[Byte]): Integer = { ByteBuffer.wrap(bytes).getInt() diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index 1a10c6f659574..5f3fdc81887ef 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -33,6 +33,7 @@ import org.apache.kafka.queue.{EventQueue, KafkaEventQueue} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import java.util.{Comparator, OptionalLong} +import scala.collection.mutable import scala.jdk.CollectionConverters._ /** @@ -58,7 +59,8 @@ class BrokerLifecycleManager( val time: Time, val threadNamePrefix: String, val isZkBroker: Boolean, - val logDirs: Set[Uuid] + val logDirs: Set[Uuid], + val shutdownHook: () => Unit = () => {} ) extends Logging { private def logPrefix(): String = { @@ -149,10 +151,11 @@ class BrokerLifecycleManager( private var readyToUnfence = false /** - * List of accumulated offline directories. + * Map of accumulated offline directories. The value is true if the directory couldn't be communicated + * to the Controller. * This variable can only be read or written from the event queue thread. */ - private var offlineDirs = Set[Uuid]() + private var offlineDirs = mutable.Map[Uuid, Boolean]() /** * True if we sent a event queue to the active controller requesting controlled @@ -253,8 +256,12 @@ class BrokerLifecycleManager( * Propagate directory failures to the controller. * @param directory The ID for the directory that failed. */ - def propagateDirectoryFailure(directory: Uuid): Unit = { + def propagateDirectoryFailure(directory: Uuid, timeout: Long): Unit = { eventQueue.append(new OfflineDirEvent(directory)) + // If we can't communicate the offline directory to the controller, we should shut down. + eventQueue.scheduleDeferred("offlineDirFailure", + new DeadlineFunction(time.nanoseconds() + MILLISECONDS.toNanos(timeout)), + new OfflineDirBrokerFailureEvent(directory)) } def handleKraftJBODMetadataVersionUpdate(): Unit = { @@ -327,9 +334,9 @@ class BrokerLifecycleManager( private class OfflineDirEvent(val dir: Uuid) extends EventQueue.Event { override def run(): Unit = { if (offlineDirs.isEmpty) { - offlineDirs = Set(dir) + offlineDirs = mutable.Map(dir -> false) } else { - offlineDirs = offlineDirs + dir + offlineDirs += (dir -> false) } if (registered) { scheduleNextCommunicationImmediately() @@ -337,6 +344,15 @@ class BrokerLifecycleManager( } } + private class OfflineDirBrokerFailureEvent(offlineDir: Uuid) extends EventQueue.Event { + override def run(): Unit = { + if (!offlineDirs.getOrElse(offlineDir, false)) { + error(s"Shutting down because couldn't communicate offline log dir $offlineDir with controllers") + shutdownHook() + } + } + } + private class StartupEvent(highestMetadataOffsetProvider: () => Long, channelManager: NodeToControllerChannelManager, clusterId: String, @@ -456,11 +472,11 @@ class BrokerLifecycleManager( setCurrentMetadataOffset(metadataOffset). setWantFence(!readyToUnfence). setWantShutDown(_state == BrokerState.PENDING_CONTROLLED_SHUTDOWN). - setOfflineLogDirs(offlineDirs.toSeq.asJava) + setOfflineLogDirs(offlineDirs.keys.toSeq.asJava) if (isTraceEnabled) { trace(s"Sending broker heartbeat $data") } - val handler = new BrokerHeartbeatResponseHandler() + val handler = new BrokerHeartbeatResponseHandler(offlineDirs.keys) _channelManager.sendRequest(new BrokerHeartbeatRequest.Builder(data), handler) communicationInFlight = true } @@ -468,18 +484,19 @@ class BrokerLifecycleManager( // the response handler is not invoked from the event handler thread, // so it is not safe to update state here, instead, schedule an event // to continue handling the response on the event handler thread - private class BrokerHeartbeatResponseHandler extends ControllerRequestCompletionHandler { + private class BrokerHeartbeatResponseHandler(currentOfflineDirs: Iterable[Uuid]) extends ControllerRequestCompletionHandler { override def onComplete(response: ClientResponse): Unit = { - eventQueue.prepend(new BrokerHeartbeatResponseEvent(response, false)) + eventQueue.prepend(new BrokerHeartbeatResponseEvent(response, false, currentOfflineDirs)) } override def onTimeout(): Unit = { info("Unable to send a heartbeat because the RPC got timed out before it could be sent.") - eventQueue.prepend(new BrokerHeartbeatResponseEvent(null, true)) + eventQueue.prepend(new BrokerHeartbeatResponseEvent(null, true, currentOfflineDirs)) } } - private class BrokerHeartbeatResponseEvent(response: ClientResponse, timedOut: Boolean) extends EventQueue.Event { + private class BrokerHeartbeatResponseEvent(response: ClientResponse, timedOut: Boolean, + currentOfflineDirs: Iterable[Uuid]) extends EventQueue.Event { override def run(): Unit = { communicationInFlight = false if (timedOut) { @@ -507,6 +524,7 @@ class BrokerLifecycleManager( if (errorCode == Errors.NONE) { val responseData = message.data() failedAttempts = 0 + currentOfflineDirs.foreach(cur => offlineDirs.put(cur, true)) _state match { case BrokerState.STARTING => if (responseData.isCaughtUp) { diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 94da0fe58375f..5c2c06cabdf9a 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -34,12 +34,11 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.utils.{LogContext, Time} -import org.apache.kafka.common.{ClusterResource, KafkaException, TopicPartition, Uuid} +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, GroupCoordinatorConfig, GroupCoordinatorService, CoordinatorRecordSerde} import org.apache.kafka.image.publisher.MetadataPublisher import org.apache.kafka.metadata.{BrokerState, ListenerInfo, VersionRange} -import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.{AssignmentsManager, ClientMetricsManager, NodeToControllerChannelManager} import org.apache.kafka.server.authorizer.Authorizer @@ -57,7 +56,7 @@ import java.util import java.util.Optional import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.{Condition, ReentrantLock} -import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit, TimeoutException} +import java.util.concurrent.{CompletableFuture, ExecutionException, TimeoutException, TimeUnit} import scala.collection.Map import scala.compat.java8.OptionConverters.RichOptionForJava8 import scala.jdk.CollectionConverters._ @@ -210,19 +209,19 @@ class BrokerServer( time, s"broker-${config.nodeId}-", isZkBroker = false, - logDirs = logManager.directoryIdsSet) + logDirs = logManager.directoryIdsSet, + () => new Thread(() => shutdown(), "kafka-shutdown-thread").start()) // Enable delegation token cache for all SCRAM mechanisms to simplify dynamic update. // This keeps the cache up-to-date if new SCRAM mechanisms are enabled dynamically. tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames) credentialProvider = new CredentialProvider(ScramMechanism.mechanismNames, tokenCache) - val voterConnections = FutureUtils.waitWithLogging(logger.underlying, logIdent, + FutureUtils.waitWithLogging(logger.underlying, logIdent, "controller quorum voters future", sharedServer.controllerQuorumVotersFuture, startupDeadline, time) - val controllerNodes = QuorumConfig.voterConnectionsToNodes(voterConnections).asScala - val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes) + val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config) clientToControllerChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider, @@ -304,7 +303,7 @@ class BrokerServer( assignmentsManager.onAssignment(partition, directoryId, reason, callback) override def handleFailure(directoryId: Uuid): Unit = - lifecycleManager.propagateDirectoryFailure(directoryId) + lifecycleManager.propagateDirectoryFailure(directoryId, config.logDirFailureTimeoutMs) } this._replicaManager = new ReplicaManager( @@ -615,10 +614,6 @@ class BrokerServer( protected def createRemoteLogManager(): Option[RemoteLogManager] = { if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) { - if (config.logDirs.size > 1) { - throw new KafkaException("Tiered storage is not supported with multiple log dirs.") - } - Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, (tp: TopicPartition) => logManager.getLog(tp).asJava, (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index d0ea5387a26be..fdc241fa81b2f 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -436,7 +436,7 @@ class ControllerServer( /** * Start the KIP-919 controller registration manager. */ - val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes.asScala) + val controllerNodeProvider = RaftControllerNodeProvider(raftManager, config) registrationChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider, time, diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index f8b60b6071d05..cc89631ab405f 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -91,19 +91,19 @@ class DelayedFetch( // Go directly to the check for Case G if the message offsets are the same. If the log segment // has just rolled, then the high watermark offset will remain the same but be on the old segment, // which would incorrectly be seen as an instance of Case F. - if (endOffset.messageOffset != fetchOffset.messageOffset) { - if (endOffset.onOlderSegment(fetchOffset)) { - // Case F, this can happen when the new fetch operation is on a truncated leader - debug(s"Satisfying fetch $this since it is fetching later segments of partition $topicIdPartition.") - return forceComplete() - } else if (fetchOffset.onOlderSegment(endOffset)) { + if (fetchOffset.messageOffset > endOffset.messageOffset) { + // Case F, this can happen when the new fetch operation is on a truncated leader + debug(s"Satisfying fetch $this since it is fetching later segments of partition $topicIdPartition.") + return forceComplete() + } else if (fetchOffset.messageOffset < endOffset.messageOffset) { + if (fetchOffset.onOlderSegment(endOffset)) { // Case F, this can happen when the fetch operation is falling behind the current segment // or the partition has just rolled a new segment debug(s"Satisfying fetch $this immediately since it is fetching older segments.") // We will not force complete the fetch request if a replica should be throttled. if (!params.isFromFollower || !replicaManager.shouldLeaderThrottle(quota, partition, params.replicaId)) return forceComplete() - } else if (fetchOffset.messageOffset < endOffset.messageOffset) { + } else if (fetchOffset.onSameSegment(endOffset)) { // we take the partition fetch size as upper bound when accumulating the bytes (skip if a throttled partition) val bytesAvailable = math.min(endOffset.positionDiff(fetchOffset), fetchStatus.fetchInfo.maxBytes) if (!params.isFromFollower || !replicaManager.shouldLeaderThrottle(quota, partition, params.replicaId)) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 4c320e7c6f5f8..e2879d4378e9e 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -39,7 +39,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, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals} +import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, 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 @@ -98,7 +98,7 @@ object DynamicBrokerConfig { DynamicProducerStateManagerConfig ++ DynamicRemoteLogConfig.ReconfigurableConfigs - private val ClusterLevelListenerConfigs = Set(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, KafkaConfig.NumNetworkThreadsProp) + private val ClusterLevelListenerConfigs = Set(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, ServerConfigs.NUM_NETWORK_THREADS_CONFIG) private val PerBrokerConfigs = (DynamicSecurityConfigs ++ DynamicListenerConfig.ReconfigurableConfigs).diff( ClusterLevelListenerConfigs) private val ListenerMechanismConfigs = Set(KafkaSecurityConfigs.SASL_JAAS_CONFIG, @@ -769,10 +769,10 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok object DynamicThreadPool { val ReconfigurableConfigs = Set( - KafkaConfig.NumIoThreadsProp, + ServerConfigs.NUM_IO_THREADS_CONFIG, ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, - KafkaConfig.BackgroundThreadsProp) + ServerConfigs.BACKGROUND_THREADS_CONFIG) def validateReconfiguration(currentConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { newConfig.values.forEach { (k, v) => @@ -794,10 +794,10 @@ object DynamicThreadPool { def getValue(config: KafkaConfig, name: String): Int = { name match { - case KafkaConfig.NumIoThreadsProp => config.numIoThreads + case ServerConfigs.NUM_IO_THREADS_CONFIG => config.numIoThreads case ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG => config.numReplicaFetchers case ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG => config.numRecoveryThreadsPerDataDir - case KafkaConfig.BackgroundThreadsProp => config.backgroundThreads + case ServerConfigs.BACKGROUND_THREADS_CONFIG => config.backgroundThreads case n => throw new IllegalStateException(s"Unexpected config $n") } } @@ -806,7 +806,7 @@ object DynamicThreadPool { class ControllerDynamicThreadPool(controller: ControllerServer) extends BrokerReconfigurable { override def reconfigurableConfigs: Set[String] = { - Set(KafkaConfig.NumIoThreadsProp) + Set(ServerConfigs.NUM_IO_THREADS_CONFIG) } override def validateReconfiguration(newConfig: KafkaConfig): Unit = { @@ -899,7 +899,7 @@ class DynamicMetricsReporters(brokerId: Int, config: KafkaConfig, metrics: Metri class DynamicMetricReporterState(brokerId: Int, config: KafkaConfig, metrics: Metrics, clusterId: String) { private[server] val dynamicConfig = config.dynamicConfig - private val propsOverride = Map[String, AnyRef](KafkaConfig.BrokerIdProp -> brokerId.toString) + private val propsOverride = Map[String, AnyRef](ServerConfigs.BROKER_ID_CONFIG -> brokerId.toString) private[server] val currentReporters = mutable.Map[String, MetricsReporter]() createReporters(config, clusterId, metricsReporterClasses(dynamicConfig.currentKafkaConfig.values()).asJava, Collections.emptyMap[String, Object]) @@ -1009,7 +1009,7 @@ object DynamicListenerConfig { SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, // Network threads - KafkaConfig.NumNetworkThreadsProp + ServerConfigs.NUM_NETWORK_THREADS_CONFIG ) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9a228e014365a..6dda655bccdff 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -21,10 +21,8 @@ import java.{lang, util} import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} import kafka.cluster.EndPoint -import kafka.utils.CoreUtils.parseCsvList import kafka.utils.{CoreUtils, Logging} import kafka.utils.Implicits._ -import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.Reconfigurable import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression} import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig} @@ -38,7 +36,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy import org.apache.kafka.coordinator.group.Group.GroupType import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.coordinator.group.assignor.PartitionAssignor +import org.apache.kafka.coordinator.group.assignor.ConsumerGroupPartitionAssignor import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig @@ -48,7 +46,7 @@ import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator} import org.apache.kafka.server.common.MetadataVersion._ -import org.apache.kafka.server.config.{Defaults, KRaftConfigs, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, KafkaSecurityConfigs, ServerConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.server.record.BrokerCompressionType @@ -93,114 +91,6 @@ object KafkaConfig { zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_SSL_KEY_STORE_LOCATION_CONFIG).isDefined } - /** ********* General Configuration ***********/ - val BrokerIdGenerationEnableProp = "broker.id.generation.enable" - val MaxReservedBrokerIdProp = "reserved.broker.max.id" - val BrokerIdProp = "broker.id" - val MessageMaxBytesProp = "message.max.bytes" - val NumNetworkThreadsProp = "num.network.threads" - val NumIoThreadsProp = "num.io.threads" - val BackgroundThreadsProp = "background.threads" - val NumReplicaAlterLogDirsThreadsProp = "num.replica.alter.log.dirs.threads" - val QueuedMaxRequestsProp = "queued.max.requests" - val QueuedMaxBytesProp = "queued.max.request.bytes" - val RequestTimeoutMsProp = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG - val ConnectionSetupTimeoutMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG - val ConnectionSetupTimeoutMaxMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG - val DeleteTopicEnableProp = "delete.topic.enable" - val CompressionTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG) - val CompressionGzipLevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG) - val CompressionLz4LevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG) - val CompressionZstdLevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG) - - /************* Authorizer Configuration ***********/ - val AuthorizerClassNameProp = "authorizer.class.name" - val EarlyStartListenersProp = "early.start.listeners" - - /***************** rack configuration *************/ - val RackProp = "broker.rack" - - /** ********* Controlled shutdown configuration ***********/ - val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" - val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" - val ControlledShutdownEnableProp = "controlled.shutdown.enable" - - /** ********* Fetch Configuration **************/ - val MaxIncrementalFetchSessionCacheSlots = "max.incremental.fetch.session.cache.slots" - val FetchMaxBytes = "fetch.max.bytes" - - /** ********* Request Limit Configuration **************/ - val MaxRequestPartitionSizeLimit = "max.request.partition.size.limit" - - /** ********* Delegation Token Configuration ****************/ - val DelegationTokenSecretKeyAliasProp = "delegation.token.master.key" - val DelegationTokenSecretKeyProp = "delegation.token.secret.key" - val DelegationTokenMaxLifeTimeProp = "delegation.token.max.lifetime.ms" - val DelegationTokenExpiryTimeMsProp = "delegation.token.expiry.time.ms" - val DelegationTokenExpiryCheckIntervalMsProp = "delegation.token.expiry.check.interval.ms" - - /** Internal Configurations **/ - val UnstableApiVersionsEnableProp = "unstable.api.versions.enable" - val UnstableMetadataVersionsEnableProp = "unstable.metadata.versions.enable" - - /* Documentation */ - /** ********* General Configuration ***********/ - val BrokerIdGenerationEnableDoc = s"Enable automatic broker id generation on the server. When enabled the value configured for $MaxReservedBrokerIdProp should be reviewed." - val MaxReservedBrokerIdDoc = "Max number that can be used for a broker.id" - val BrokerIdDoc = "The broker id for this server. If unset, a unique broker id will be generated." + - "To avoid conflicts between ZooKeeper generated broker id's and user configured broker id's, generated broker ids " + - "start from " + MaxReservedBrokerIdProp + " + 1." - val MessageMaxBytesDoc = TopicConfig.MAX_MESSAGE_BYTES_DOC + - s"This can be set per topic with the topic level ${TopicConfig.MAX_MESSAGE_BYTES_CONFIG} config." - val NumNetworkThreadsDoc = s"The number of threads that the server uses for receiving requests from the network and sending responses to the network. Noted: each listener (except for controller listener) creates its own thread pool." - val NumIoThreadsDoc = "The number of threads that the server uses for processing requests, which may include disk I/O" - val NumReplicaAlterLogDirsThreadsDoc = "The number of threads that can move replicas between log directories, which may include disk I/O" - val BackgroundThreadsDoc = "The number of threads to use for various background processing tasks" - val QueuedMaxRequestsDoc = "The number of queued requests allowed for data-plane, before blocking the network threads" - val QueuedMaxRequestBytesDoc = "The number of queued bytes allowed before no more requests are read" - val RequestTimeoutMsDoc = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC - val ConnectionSetupTimeoutMsDoc = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC - val ConnectionSetupTimeoutMaxMsDoc = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC - val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" - val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + - "('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + - "'producer' which means retain the original compression codec set by the producer." - val CompressionGzipLevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'gzip'." - val CompressionLz4LevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'lz4'." - val CompressionZstdLevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'zstd'." - - /************* Authorizer Configuration ***********/ - val AuthorizerClassNameDoc = s"The fully qualified name of a class that implements ${classOf[Authorizer].getName}" + - " interface, which is used by the broker for authorization." - val EarlyStartListenersDoc = "A comma-separated list of listener names which may be started before the authorizer has finished " + - "initialization. This is useful when the authorizer is dependent on the cluster itself for bootstrapping, as is the case for " + - "the StandardAuthorizer (which stores ACLs in the metadata log.) By default, all listeners included in controller.listener.names " + - "will also be early start listeners. A listener should not appear in this list if it accepts external traffic." - - /************* Rack Configuration **************/ - val RackDoc = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: RACK1, us-east-1d" - - /** ********* Controlled shutdown configuration ***********/ - val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" - val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." - val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server." - - /** ********* Fetch Configuration **************/ - val MaxIncrementalFetchSessionCacheSlotsDoc = "The maximum number of total incremental fetch sessions that we will maintain. FetchSessionCache is sharded into 8 shards and the limit is equally divided among all shards. Sessions are allocated to each shard in round-robin. Only entries within a shard are considered eligible for eviction." - val FetchMaxBytesDoc = "The maximum number of bytes we will return for a fetch request. Must be at least 1024." - - /** ********* Request Limit Configuration **************/ - val MaxRequestPartitionSizeLimitDoc = "The maximum number of partitions can be served in one request." - - /** ********* Delegation Token Configuration ****************/ - val DelegationTokenSecretKeyAliasDoc = s"DEPRECATED: An alias for $DelegationTokenSecretKeyProp, which should be used instead of this config." - val DelegationTokenSecretKeyDoc = "Secret key to generate and verify delegation tokens. The same key must be configured across all the brokers. " + - " If using Kafka with KRaft, the key must also be set across all controllers. " + - " If the key is not set or set to empty string, brokers will disable the delegation token support." - val DelegationTokenMaxLifeTimeDoc = "The token has a maximum lifetime beyond which it cannot be renewed anymore. Default value 7 days." - val DelegationTokenExpiryTimeMsDoc = "The token validity time in milliseconds before the token needs to be renewed. Default value 1 day." - val DelegationTokenExpiryCheckIntervalDoc = "Scan interval to remove expired delegation tokens." - @nowarn("cat=deprecation") val configDef = { import ConfigDef.Importance._ @@ -232,19 +122,19 @@ object KafkaConfig { .define(ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG, BOOLEAN, ZkConfigs.ZK_SSL_OCSP_ENABLE, LOW, ZkConfigs.ZK_SSL_OCSP_ENABLE_DOC) /** ********* General Configuration ***********/ - .define(BrokerIdGenerationEnableProp, BOOLEAN, Defaults.BROKER_ID_GENERATION_ENABLE, MEDIUM, BrokerIdGenerationEnableDoc) - .define(MaxReservedBrokerIdProp, INT, Defaults.MAX_RESERVED_BROKER_ID, atLeast(0), MEDIUM, MaxReservedBrokerIdDoc) - .define(BrokerIdProp, INT, Defaults.BROKER_ID, HIGH, BrokerIdDoc) - .define(MessageMaxBytesProp, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, MessageMaxBytesDoc) - .define(NumNetworkThreadsProp, INT, Defaults.NUM_NETWORK_THREADS, atLeast(1), HIGH, NumNetworkThreadsDoc) - .define(NumIoThreadsProp, INT, Defaults.NUM_IO_THREADS, atLeast(1), HIGH, NumIoThreadsDoc) - .define(NumReplicaAlterLogDirsThreadsProp, INT, null, HIGH, NumReplicaAlterLogDirsThreadsDoc) - .define(BackgroundThreadsProp, INT, Defaults.BACKGROUND_THREADS, atLeast(1), HIGH, BackgroundThreadsDoc) - .define(QueuedMaxRequestsProp, INT, Defaults.QUEUED_MAX_REQUESTS, atLeast(1), HIGH, QueuedMaxRequestsDoc) - .define(QueuedMaxBytesProp, LONG, Defaults.QUEUED_MAX_REQUEST_BYTES, MEDIUM, QueuedMaxRequestBytesDoc) - .define(RequestTimeoutMsProp, INT, Defaults.REQUEST_TIMEOUT_MS, HIGH, RequestTimeoutMsDoc) - .define(ConnectionSetupTimeoutMsProp, LONG, Defaults.CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, ConnectionSetupTimeoutMsDoc) - .define(ConnectionSetupTimeoutMaxMsProp, LONG, Defaults.CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, ConnectionSetupTimeoutMaxMsDoc) + .define(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, BOOLEAN, ServerConfigs.BROKER_ID_GENERATION_ENABLE_DEFAULT, MEDIUM, ServerConfigs.BROKER_ID_GENERATION_ENABLE_DOC) + .define(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, INT, ServerConfigs.RESERVED_BROKER_MAX_ID_DEFAULT, atLeast(0), MEDIUM, ServerConfigs.RESERVED_BROKER_MAX_ID_DOC) + .define(ServerConfigs.BROKER_ID_CONFIG, INT, ServerConfigs.BROKER_ID_DEFAULT, HIGH, ServerConfigs.BROKER_ID_DOC) + .define(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, ServerConfigs.MESSAGE_MAX_BYTES_DOC) + .define(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, INT, ServerConfigs.NUM_NETWORK_THREADS_DEFAULT, atLeast(1), HIGH, ServerConfigs.NUM_NETWORK_THREADS_DOC) + .define(ServerConfigs.NUM_IO_THREADS_CONFIG, INT, ServerConfigs.NUM_IO_THREADS_DEFAULT, atLeast(1), HIGH, ServerConfigs.NUM_IO_THREADS_DOC) + .define(ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG, INT, null, HIGH, ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_DOC) + .define(ServerConfigs.BACKGROUND_THREADS_CONFIG, INT, ServerConfigs.BACKGROUND_THREADS_DEFAULT, atLeast(1), HIGH, ServerConfigs.BACKGROUND_THREADS_DOC) + .define(ServerConfigs.QUEUED_MAX_REQUESTS_CONFIG, INT, ServerConfigs.QUEUED_MAX_REQUESTS_DEFAULT, atLeast(1), HIGH, ServerConfigs.QUEUED_MAX_REQUESTS_DOC) + .define(ServerConfigs.QUEUED_MAX_BYTES_CONFIG, LONG, ServerConfigs.QUEUED_MAX_REQUEST_BYTES_DEFAULT, MEDIUM, ServerConfigs.QUEUED_MAX_REQUEST_BYTES_DOC) + .define(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, INT, ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT, HIGH, ServerConfigs.REQUEST_TIMEOUT_MS_DOC) + .define(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, LONG, ServerConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC) + .define(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, LONG, ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC) /* * KRaft mode configs. @@ -272,8 +162,8 @@ object KafkaConfig { MEDIUM, KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_DOC) /************* Authorizer Configuration ***********/ - .define(AuthorizerClassNameProp, STRING, Defaults.AUTHORIZER_CLASS_NAME, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) - .define(EarlyStartListenersProp, STRING, null, HIGH, EarlyStartListenersDoc) + .define(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, STRING, ServerConfigs.AUTHORIZER_CLASS_NAME_DEFAULT, new ConfigDef.NonNullValidator(), LOW, ServerConfigs.AUTHORIZER_CLASS_NAME_DOC) + .define(ServerConfigs.EARLY_START_LISTENERS_CONFIG, STRING, null, HIGH, ServerConfigs.EARLY_START_LISTENERS_DOC) /** ********* Socket Server Configuration ***********/ .define(SocketServerConfigs.LISTENERS_CONFIG, STRING, SocketServerConfigs.LISTENERS_DEFAULT, HIGH, SocketServerConfigs.LISTENERS_DOC) @@ -292,7 +182,7 @@ object KafkaConfig { .define(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG, INT, SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_DEFAULT, atLeast(0), LOW, SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_DOC) /************ Rack Configuration ******************/ - .define(RackProp, STRING, null, MEDIUM, RackDoc) + .define(ServerConfigs.BROKER_RACK_CONFIG, STRING, null, MEDIUM, ServerConfigs.BROKER_RACK_DOC) /** ********* Log Configuration ***********/ .define(ServerLogConfigs.NUM_PARTITIONS_CONFIG, INT, ServerLogConfigs.NUM_PARTITIONS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.NUM_PARTITIONS_DOC) @@ -345,6 +235,7 @@ object KafkaConfig { .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DEFAULT, LOW, ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_DOC) .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) /** ********* Replication configuration ***********/ .define(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, INT, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DEFAULT, MEDIUM, ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_DOC) @@ -373,9 +264,9 @@ object KafkaConfig { /** ********* Controlled shutdown configuration ***********/ - .define(ControlledShutdownMaxRetriesProp, INT, Defaults.CONTROLLED_SHUTDOWN_MAX_RETRIES, MEDIUM, ControlledShutdownMaxRetriesDoc) - .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS, MEDIUM, ControlledShutdownRetryBackoffMsDoc) - .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.CONTROLLED_SHUTDOWN_ENABLE, MEDIUM, ControlledShutdownEnableDoc) + .define(ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG, INT, ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_DEFAULT, MEDIUM, ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_DOC) + .define(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG, LONG, ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DEFAULT, MEDIUM, ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DOC) + .define(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, BOOLEAN, ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_DEFAULT, MEDIUM, ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_DOC) /** ********* Group coordinator configuration ***********/ .define(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, INT, GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_DEFAULT, MEDIUM, GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_DOC) @@ -412,11 +303,11 @@ object KafkaConfig { .define(GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_CONFIG, LONG, GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), HIGH, GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_DOC) .define(GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG, INT, GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_DOC) .define(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG, SHORT, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DOC) - .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc) - .define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc) - .define(CompressionGzipLevelProp, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, CompressionGzipLevelDoc) - .define(CompressionLz4LevelProp, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, CompressionLz4LevelDoc) - .define(CompressionZstdLevelProp, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, CompressionZstdLevelDoc) + .define(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, BOOLEAN, ServerConfigs.DELETE_TOPIC_ENABLE_DEFAULT, HIGH, ServerConfigs.DELETE_TOPIC_ENABLE_DOC) + .define(ServerConfigs.COMPRESSION_TYPE_CONFIG, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, ServerConfigs.COMPRESSION_TYPE_DOC) + .define(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, ServerConfigs.COMPRESSION_GZIP_LEVEL_DOC) + .define(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, ServerConfigs.COMPRESSION_LZ4_LEVEL_DOC) + .define(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, ServerConfigs.COMPRESSION_ZSTD_LEVEL_DOC) /** ********* Transaction management configuration ***********/ .define(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG, INT, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DEFAULT, atLeast(1), HIGH, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DOC) @@ -436,11 +327,11 @@ object KafkaConfig { .defineInternal(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_CONFIG, INT, TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), LOW, TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DOC) /** ********* Fetch Configuration **************/ - .define(MaxIncrementalFetchSessionCacheSlots, INT, Defaults.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS, atLeast(0), MEDIUM, MaxIncrementalFetchSessionCacheSlotsDoc) - .define(FetchMaxBytes, INT, Defaults.FETCH_MAX_BYTES, atLeast(1024), MEDIUM, FetchMaxBytesDoc) + .define(ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG, INT, ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DEFAULT, atLeast(0), MEDIUM, ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DOC) + .define(ServerConfigs.FETCH_MAX_BYTES_CONFIG, INT, ServerConfigs.FETCH_MAX_BYTES_DEFAULT, atLeast(1024), MEDIUM, ServerConfigs.FETCH_MAX_BYTES_DOC) /** ********* Request Limit Configuration ***********/ - .define(MaxRequestPartitionSizeLimit, INT, Defaults.MAX_REQUEST_PARTITION_SIZE_LIMIT, atLeast(1), MEDIUM, MaxRequestPartitionSizeLimitDoc) + .define(ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG, INT, ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_DEFAULT, atLeast(1), MEDIUM, ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_DOC) /** ********* Kafka Metrics Configuration ***********/ .define(MetricConfigs.METRIC_NUM_SAMPLES_CONFIG, INT, MetricConfigs.METRIC_NUM_SAMPLES_DEFAULT, atLeast(1), LOW, MetricConfigs.METRIC_NUM_SAMPLES_DOC) @@ -532,11 +423,11 @@ object KafkaConfig { .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_CONFIG, STRING, null, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC) /** ********* Delegation Token Configuration ****************/ - .define(DelegationTokenSecretKeyAliasProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyAliasDoc) - .define(DelegationTokenSecretKeyProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyDoc) - .define(DelegationTokenMaxLifeTimeProp, LONG, Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS, atLeast(1), MEDIUM, DelegationTokenMaxLifeTimeDoc) - .define(DelegationTokenExpiryTimeMsProp, LONG, Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS, atLeast(1), MEDIUM, DelegationTokenExpiryTimeMsDoc) - .define(DelegationTokenExpiryCheckIntervalMsProp, LONG, Defaults.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS, atLeast(1), LOW, DelegationTokenExpiryCheckIntervalDoc) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG, PASSWORD, null, MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, PASSWORD, null, MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_MS_DEFAULT, atLeast(1), MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DEFAULT, atLeast(1), MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), LOW, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_DOC) /** ********* Password encryption configuration for dynamic configs *********/ .define(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, PASSWORD, null, MEDIUM, PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_DOC) @@ -557,9 +448,9 @@ object KafkaConfig { /** Internal Configurations **/ // This indicates whether unreleased APIs should be advertised by this node. - .defineInternal(UnstableApiVersionsEnableProp, BOOLEAN, false, HIGH) + .defineInternal(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, BOOLEAN, false, HIGH) // This indicates whether unreleased MetadataVersions should be enabled on this node. - .defineInternal(UnstableMetadataVersionsEnableProp, BOOLEAN, false, HIGH) + .defineInternal(ServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, BOOLEAN, false, HIGH) } /** ********* Remote Log Management Configuration *********/ @@ -636,10 +527,10 @@ object KafkaConfig { */ def populateSynonyms(input: util.Map[_, _]): util.Map[Any, Any] = { val output = new util.HashMap[Any, Any](input) - val brokerId = output.get(KafkaConfig.BrokerIdProp) + val brokerId = output.get(ServerConfigs.BROKER_ID_CONFIG) val nodeId = output.get(KRaftConfigs.NODE_ID_CONFIG) if (brokerId == null && nodeId != null) { - output.put(KafkaConfig.BrokerIdProp, nodeId) + output.put(ServerConfigs.BROKER_ID_CONFIG, nodeId) } else if (brokerId != null && nodeId == null) { output.put(KRaftConfigs.NODE_ID_CONFIG, brokerId) } @@ -773,9 +664,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val ZkSslCrlEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG) val ZkSslOcspEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG) /** ********* General Configuration ***********/ - val brokerIdGenerationEnable: Boolean = getBoolean(KafkaConfig.BrokerIdGenerationEnableProp) - val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) - var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) + val brokerIdGenerationEnable: Boolean = getBoolean(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG) + val maxReservedBrokerId: Int = getInt(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG) + var brokerId: Int = getInt(ServerConfigs.BROKER_ID_CONFIG) val nodeId: Int = getInt(KRaftConfigs.NODE_ID_CONFIG) val initialRegistrationTimeoutMs: Int = getInt(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG) val brokerHeartbeatIntervalMs: Int = getInt(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG) @@ -825,18 +716,18 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def metadataLogSegmentMinBytes = getInt(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG) val serverMaxStartupTimeMs = getLong(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG) - def numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp) - def backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp) - val queuedMaxRequests = getInt(KafkaConfig.QueuedMaxRequestsProp) - val queuedMaxBytes = getLong(KafkaConfig.QueuedMaxBytesProp) - def numIoThreads = getInt(KafkaConfig.NumIoThreadsProp) - def messageMaxBytes = getInt(KafkaConfig.MessageMaxBytesProp) - val requestTimeoutMs = getInt(KafkaConfig.RequestTimeoutMsProp) - val connectionSetupTimeoutMs = getLong(KafkaConfig.ConnectionSetupTimeoutMsProp) - val connectionSetupTimeoutMaxMs = getLong(KafkaConfig.ConnectionSetupTimeoutMaxMsProp) + def numNetworkThreads = getInt(ServerConfigs.NUM_NETWORK_THREADS_CONFIG) + def backgroundThreads = getInt(ServerConfigs.BACKGROUND_THREADS_CONFIG) + val queuedMaxRequests = getInt(ServerConfigs.QUEUED_MAX_REQUESTS_CONFIG) + val queuedMaxBytes = getLong(ServerConfigs.QUEUED_MAX_BYTES_CONFIG) + def numIoThreads = getInt(ServerConfigs.NUM_IO_THREADS_CONFIG) + def messageMaxBytes = getInt(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG) + val requestTimeoutMs = getInt(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG) + val connectionSetupTimeoutMs = getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG) + val connectionSetupTimeoutMaxMs = getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG) def getNumReplicaAlterLogDirsThreads: Int = { - val numThreads: Integer = Option(getInt(KafkaConfig.NumReplicaAlterLogDirsThreadsProp)).getOrElse(logDirs.size) + val numThreads: Integer = Option(getInt(ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG)).getOrElse(logDirs.size) numThreads } @@ -850,7 +741,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami /************* Authorizer Configuration ***********/ def createNewAuthorizer(): Option[Authorizer] = { - val className = getString(KafkaConfig.AuthorizerClassNameProp) + val className = getString(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG) if (className == null || className.isEmpty) None else { @@ -861,13 +752,13 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val earlyStartListeners: Set[ListenerName] = { val listenersSet = listeners.map(_.listenerName).toSet val controllerListenersSet = controllerListeners.map(_.listenerName).toSet - Option(getString(KafkaConfig.EarlyStartListenersProp)) match { + Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match { case None => controllerListenersSet case Some(str) => str.split(",").map(_.trim()).filterNot(_.isEmpty).map { str => val listenerName = new ListenerName(str) if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName)) - throw new ConfigException(s"${KafkaConfig.EarlyStartListenersProp} contains " + + throw new ConfigException(s"${ServerConfigs.EARLY_START_LISTENERS_CONFIG} contains " + s"listener ${listenerName.value()}, but this is not contained in " + s"${SocketServerConfigs.LISTENERS_CONFIG} or ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}") listenerName @@ -889,13 +780,13 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val failedAuthenticationDelayMs = getInt(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG) /***************** rack configuration **************/ - val rack = Option(getString(KafkaConfig.RackProp)) + val rack = Option(getString(ServerConfigs.BROKER_RACK_CONFIG)) val replicaSelectorClassName = Option(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG)) /** ********* Log Configuration ***********/ val autoCreateTopicsEnable = getBoolean(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG) val numPartitions = getInt(ServerLogConfigs.NUM_PARTITIONS_CONFIG) - val logDirs = CoreUtils.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))) + val logDirs: Seq[String] = Csv.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))).asScala def logSegmentBytes = getInt(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG) def logFlushIntervalMessages = getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG) val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP) @@ -975,6 +866,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG) + def logDirFailureTimeoutMs: Long = getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG) + /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG) val defaultReplicationFactor: Int = getInt(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG) @@ -1020,9 +913,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami } /** ********* Controlled shutdown configuration ***********/ - val controlledShutdownMaxRetries = getInt(KafkaConfig.ControlledShutdownMaxRetriesProp) - val controlledShutdownRetryBackoffMs = getLong(KafkaConfig.ControlledShutdownRetryBackoffMsProp) - val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp) + val controlledShutdownMaxRetries = getInt(ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG) + val controlledShutdownRetryBackoffMs = getLong(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG) + val controlledShutdownEnable = getBoolean(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG) /** ********* Feature configuration ***********/ def isFeatureVersioningSupported = interBrokerProtocolVersion.isFeatureVersioningSupported @@ -1060,7 +953,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val consumerGroupMinHeartbeatIntervalMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG) val consumerGroupMaxHeartbeatIntervalMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG) val consumerGroupMaxSize = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG) - val consumerGroupAssignors = getConfiguredInstances(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, classOf[PartitionAssignor]) + val consumerGroupAssignors = getConfiguredInstances(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, classOf[ConsumerGroupPartitionAssignor]) val consumerGroupMigrationPolicy = ConsumerGroupMigrationPolicy.parse(getString(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG)) /** ********* Offset management configuration ***********/ @@ -1118,12 +1011,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion.isSaslInterBrokerHandshakeRequestEnabled /** ********* DelegationToken Configuration **************/ - val delegationTokenSecretKey = Option(getPassword(KafkaConfig.DelegationTokenSecretKeyProp)) - .getOrElse(getPassword(KafkaConfig.DelegationTokenSecretKeyAliasProp)) + val delegationTokenSecretKey = Option(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG)) + .getOrElse(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG)) val tokenAuthEnabled = delegationTokenSecretKey != null && delegationTokenSecretKey.value.nonEmpty - val delegationTokenMaxLifeMs = getLong(KafkaConfig.DelegationTokenMaxLifeTimeProp) - val delegationTokenExpiryTimeMs = getLong(KafkaConfig.DelegationTokenExpiryTimeMsProp) - val delegationTokenExpiryCheckIntervalMs = getLong(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp) + val delegationTokenMaxLifeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG) + val delegationTokenExpiryTimeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG) + val delegationTokenExpiryCheckIntervalMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG) /** ********* Password encryption configuration for dynamic configs *********/ def passwordEncoderSecret = Option(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG)) @@ -1144,17 +1037,18 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val controllerQuotaWindowSizeSeconds = getInt(QuotaConfigs.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS_CONFIG) /** ********* Fetch Configuration **************/ - val maxIncrementalFetchSessionCacheSlots = getInt(KafkaConfig.MaxIncrementalFetchSessionCacheSlots) - val fetchMaxBytes = getInt(KafkaConfig.FetchMaxBytes) + val maxIncrementalFetchSessionCacheSlots = getInt(ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG) + val fetchMaxBytes = getInt(ServerConfigs.FETCH_MAX_BYTES_CONFIG) /** ********* Request Limit Configuration ***********/ - val maxRequestPartitionSizeLimit = getInt(KafkaConfig.MaxRequestPartitionSizeLimit) + val maxRequestPartitionSizeLimit = getInt(ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG) + + val deleteTopicEnable = getBoolean(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG) + def compressionType = getString(ServerConfigs.COMPRESSION_TYPE_CONFIG) - val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) - def compressionType = getString(KafkaConfig.CompressionTypeProp) - def gzipCompressionLevel = getInt(KafkaConfig.CompressionGzipLevelProp) - def lz4CompressionLevel = getInt(KafkaConfig.CompressionLz4LevelProp) - def zstdCompressionLevel = getInt(KafkaConfig.CompressionZstdLevelProp) + def gzipCompressionLevel = getInt(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG) + def lz4CompressionLevel = getInt(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG) + def zstdCompressionLevel = getInt(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG) /** ********* Raft Quorum Configuration *********/ val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG) @@ -1166,8 +1060,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val quorumRetryBackoffMs = getInt(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG) /** Internal Configurations **/ - val unstableApiVersionsEnabled = getBoolean(KafkaConfig.UnstableApiVersionsEnableProp) - val unstableMetadataVersionsEnabled = getBoolean(KafkaConfig.UnstableMetadataVersionsEnableProp) + val unstableApiVersionsEnabled = getBoolean(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG) + val unstableMetadataVersionsEnabled = getBoolean(ServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG) def addReconfigurable(reconfigurable: Reconfigurable): Unit = { dynamicConfig.addReconfigurable(reconfigurable) @@ -1292,7 +1186,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami // check controller listener names (they won't appear in listeners when process.roles=broker) // as well as listeners for occurrences of SSL or SASL_* if (controllerListenerNames.exists(isSslOrSasl) || - parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).exists(listenerValue => isSslOrSasl(EndPoint.parseListenerName(listenerValue)))) { + Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).asScala.exists(listenerValue => isSslOrSasl(EndPoint.parseListenerName(listenerValue)))) { mapValue // don't add default mappings since we found something that is SSL or SASL_* } else { // add the PLAINTEXT mappings for all controller listener names that are not explicitly PLAINTEXT @@ -1319,7 +1213,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami @nowarn("cat=deprecation") private def validateValues(): Unit = { if (nodeId != brokerId) { - throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${KafkaConfig.BrokerIdProp}`.") + throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${ServerConfigs.BROKER_ID_CONFIG}`.") } if (requiresZookeeper) { if (zkConnect == null) { @@ -1346,9 +1240,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be greater than or equal to 0") require(logRetentionTimeMillis >= 1 || logRetentionTimeMillis == -1, "log.retention.ms must be unlimited (-1) or, greater than or equal to 1") require(logDirs.nonEmpty, "At least one log directory must be defined via log.dirs or log.dir.") - if (isRemoteLogStorageSystemEnabled && logDirs.size > 1) { - throw new ConfigException(s"Multiple log directories `${logDirs.mkString(",")}` are not supported when remote log storage is enabled") - } require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + " to prevent unnecessary socket timeouts") @@ -1519,7 +1410,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami require(!interBrokerUsesSasl || saslEnabledMechanisms(interBrokerListenerName).contains(saslMechanismInterBrokerProtocol), s"${KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG} must be included in ${KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG} when SASL is used for inter-broker communication") require(queuedMaxBytes <= 0 || queuedMaxBytes >= socketRequestMaxBytes, - s"${KafkaConfig.QueuedMaxBytesProp} must be larger or equal to ${SocketServerConfigs.SOCKET_RECEIVE_BUFFER_BYTES_CONFIG}") + s"${ServerConfigs.QUEUED_MAX_BYTES_CONFIG} must be larger or equal to ${SocketServerConfigs.SOCKET_REQUEST_MAX_BYTES_CONFIG}") if (maxConnectionsPerIp == 0) require(maxConnectionsPerIpOverrides.nonEmpty, s"${SocketServerConfigs.MAX_CONNECTIONS_PER_IP_CONFIG} can be set to zero only if" + diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 8dc35ff628e41..ce155e92ffd76 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -44,7 +44,7 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.security.{JaasContext, JaasUtils} import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils} -import org.apache.kafka.common.{Endpoint, KafkaException, Node, TopicPartition} +import org.apache.kafka.common.{Endpoint, Node, TopicPartition} import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.image.loader.metrics.MetadataLoaderMetrics import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag @@ -165,8 +165,10 @@ class KafkaServer( var kafkaScheduler: KafkaScheduler = _ - var kraftControllerNodes: Seq[Node] = _ @volatile var metadataCache: ZkMetadataCache = _ + + @volatile var quorumControllerNodeProvider: RaftControllerNodeProvider = _ + var quotaManagers: QuotaFactory.QuotaManagers = _ val zkClientConfig: ZKClientConfig = KafkaServer.zkClientConfigFromKafkaConfig(config) @@ -324,20 +326,13 @@ class KafkaServer( remoteLogManagerOpt = createRemoteLogManager() - if (config.migrationEnabled) { - kraftControllerNodes = QuorumConfig.voterConnectionsToNodes( - QuorumConfig.parseVoterConnections(config.quorumVoters) - ).asScala - } else { - kraftControllerNodes = Seq.empty - } metadataCache = MetadataCache.zkMetadataCache( config.brokerId, config.interBrokerProtocolVersion, brokerFeatures, - kraftControllerNodes, config.migrationEnabled) - val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config) + val controllerNodeProvider = new MetadataCacheControllerNodeProvider(metadataCache, config, + () => Option(quorumControllerNodeProvider).map(_.getControllerInfo())) /* initialize feature change listener */ _featureChangeListener = new FinalizedFeatureChangeListener(metadataCache, _zkClient) @@ -446,8 +441,7 @@ class KafkaServer( CompletableFuture.completedFuture(quorumVoters), fatalFaultHandler = new LoggingFaultHandler("raftManager", () => shutdown()) ) - val controllerNodes = QuorumConfig.voterConnectionsToNodes(quorumVoters).asScala - val quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config, controllerNodes) + quorumControllerNodeProvider = RaftControllerNodeProvider(raftManager, config) val brokerToQuorumChannelManager = new NodeToControllerChannelManagerImpl( controllerNodeProvider = quorumControllerNodeProvider, time = time, @@ -696,10 +690,6 @@ class KafkaServer( protected def createRemoteLogManager(): Option[RemoteLogManager] = { if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) { - if (config.logDirs.size > 1) { - throw new KafkaException("Tiered storage is not supported with multiple log dirs.") - } - Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, (tp: TopicPartition) => logManager.getLog(tp).asJava, (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { @@ -1076,6 +1066,8 @@ class KafkaServer( } _brokerState = BrokerState.NOT_RUNNING + quorumControllerNodeProvider = null + startupComplete.set(false) isShuttingDown.set(false) CoreUtils.swallow(AppInfoParser.unregisterAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics), this) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 015e46a76523d..b8eda3fe4dc34 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -116,10 +116,9 @@ object MetadataCache { def zkMetadataCache(brokerId: Int, metadataVersion: MetadataVersion, brokerFeatures: BrokerFeatures = BrokerFeatures.createEmpty(), - kraftControllerNodes: collection.Seq[Node] = collection.Seq.empty[Node], zkMigrationEnabled: Boolean = false) : ZkMetadataCache = { - new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, kraftControllerNodes, zkMigrationEnabled) + new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, zkMigrationEnabled) } def kRaftMetadataCache(brokerId: Int): KRaftMetadataCache = { diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index 2190b3b3d7263..0017a5876af13 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -55,22 +55,15 @@ trait ControllerNodeProvider { class MetadataCacheControllerNodeProvider( val metadataCache: ZkMetadataCache, - val config: KafkaConfig + val config: KafkaConfig, + val quorumControllerNodeProvider: () => Option[ControllerInformation] ) extends ControllerNodeProvider { private val zkControllerListenerName = config.controlPlaneListenerName.getOrElse(config.interBrokerListenerName) private val zkControllerSecurityProtocol = config.controlPlaneSecurityProtocol.getOrElse(config.interBrokerSecurityProtocol) private val zkControllerSaslMechanism = config.saslMechanismInterBrokerProtocol - private val kraftControllerListenerName = if (config.controllerListenerNames.nonEmpty) - new ListenerName(config.controllerListenerNames.head) else null - private val kraftControllerSecurityProtocol = Option(kraftControllerListenerName) - .map( listener => config.effectiveListenerSecurityProtocolMap.getOrElse( - listener, SecurityProtocol.forName(kraftControllerListenerName.value()))) - .orNull - private val kraftControllerSaslMechanism = config.saslMechanismControllerProtocol - - private val emptyZkControllerInfo = ControllerInformation( + val emptyZkControllerInfo = ControllerInformation( None, zkControllerListenerName, zkControllerSecurityProtocol, @@ -85,12 +78,8 @@ class MetadataCacheControllerNodeProvider( zkControllerSecurityProtocol, zkControllerSaslMechanism, isZkController = true) - case KRaftCachedControllerId(id) => ControllerInformation( - metadataCache.getAliveBrokerNode(id, kraftControllerListenerName), - kraftControllerListenerName, - kraftControllerSecurityProtocol, - kraftControllerSaslMechanism, - isZkController = false) + case KRaftCachedControllerId(_) => + quorumControllerNodeProvider.apply().getOrElse(emptyZkControllerInfo) }.getOrElse(emptyZkControllerInfo) } } @@ -99,14 +88,12 @@ object RaftControllerNodeProvider { def apply( raftManager: RaftManager[ApiMessageAndVersion], config: KafkaConfig, - controllerQuorumVoterNodes: Seq[Node] ): RaftControllerNodeProvider = { val controllerListenerName = new ListenerName(config.controllerListenerNames.head) val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse(controllerListenerName, SecurityProtocol.forName(controllerListenerName.value())) val controllerSaslMechanism = config.saslMechanismControllerProtocol new RaftControllerNodeProvider( raftManager, - controllerQuorumVoterNodes, controllerListenerName, controllerSecurityProtocol, controllerSaslMechanism @@ -120,15 +107,15 @@ object RaftControllerNodeProvider { */ class RaftControllerNodeProvider( val raftManager: RaftManager[ApiMessageAndVersion], - controllerQuorumVoterNodes: Seq[Node], val listenerName: ListenerName, val securityProtocol: SecurityProtocol, val saslMechanism: String ) extends ControllerNodeProvider with Logging { - private val idToNode = controllerQuorumVoterNodes.map(node => node.id() -> node).toMap + + private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName.value()) override def getControllerInfo(): ControllerInformation = - ControllerInformation(raftManager.leaderAndEpoch.leaderId.asScala.map(idToNode), + ControllerInformation(raftManager.leaderAndEpoch.leaderId.asScala.flatMap(idToNode), listenerName, securityProtocol, saslMechanism, isZkController = false) } diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala index 2ea09390b3dcb..95c7a5ac3d4b1 100644 --- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsThread.scala @@ -40,7 +40,7 @@ class ReplicaAlterLogDirsThread(name: String, clientId = name, leader = leader, failedPartitions, - fetchTierStateMachine = new ReplicaAlterLogDirsTierStateMachine(), + fetchTierStateMachine = new TierStateMachine(leader, replicaMgr, true), fetchBackOffMs = fetchBackOffMs, isInterruptible = false, brokerTopicStats) { diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index c45a5d629a69b..bb073682bdfb6 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -37,7 +37,7 @@ class ReplicaFetcherThread(name: String, clientId = name, leader = leader, failedPartitions, - fetchTierStateMachine = new ReplicaFetcherTierStateMachine(leader, replicaMgr), + fetchTierStateMachine = new TierStateMachine(leader, replicaMgr, false), fetchBackOffMs = brokerConfig.replicaFetchBackoffMs, isInterruptible = false, replicaMgr.brokerTopicStats) { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 35499430d6679..aa56269a2f40d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -703,6 +703,10 @@ class ReplicaManager(val config: KafkaConfig, getPartitionOrException(topicPartition).futureLog.isDefined } + def futureLogOrException(topicPartition: TopicPartition): UnifiedLog = { + getPartitionOrException(topicPartition).futureLocalLogOrException + } + def localLog(topicPartition: TopicPartition): Option[UnifiedLog] = { onlinePartition(topicPartition).flatMap(_.log) } @@ -1744,8 +1748,8 @@ class ReplicaManager(val config: KafkaConfig, val leaderLogStartOffset = log.logStartOffset val leaderLogEndOffset = log.logEndOffset - if (params.isFromFollower) { - // If it is from a follower then send the offset metadata only as the data is already available in remote + if (params.isFromFollower || params.isFromFuture) { + // If it is from a follower or from a future replica, then send the offset metadata only as the data is already available in remote // storage and throw an error saying that this offset is moved to tiered storage. createLogReadResult(highWatermark, leaderLogStartOffset, leaderLogEndOffset, new OffsetMovedToTieredStorageException("Given offset" + offset + " is moved to tiered storage")) @@ -2114,16 +2118,12 @@ class ReplicaManager(val config: KafkaConfig, partition.log.foreach { _ => val leader = BrokerEndPoint(config.brokerId, "localhost", -1) - // Add future replica log to partition's map - partition.createLogIfNotExists( - isNew = false, - isFutureReplica = true, - offsetCheckpoints, - topicIds(partition.topic)) - - // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move - // replica from source dir to destination dir - logManager.abortAndPauseCleaning(topicPartition) + // Add future replica log to partition's map if it's not existed + if (partition.maybeCreateFutureReplica(futureLog.parentDir, offsetCheckpoints, topicIds(partition.topic))) { + // pause cleaning for partitions that are being moved and start ReplicaAlterDirThread to move + // replica from source dir to destination dir + logManager.abortAndPauseCleaning(topicPartition) + } futureReplicasAndInitialOffset.put(topicPartition, InitialFetchState(topicIds(topicPartition.topic), leader, partition.getLeaderEpoch, futureLog.highWatermark)) @@ -2131,8 +2131,11 @@ class ReplicaManager(val config: KafkaConfig, } } - if (futureReplicasAndInitialOffset.nonEmpty) + if (futureReplicasAndInitialOffset.nonEmpty) { + // Even though it's possible that there is another thread adding fetcher for this future log partition, + // but it's fine because `BrokerIdAndFetcherId` will be identical and the operation will be no-op. replicaAlterLogDirsManager.addFetcherForPartitions(futureReplicasAndInitialOffset) + } } /* @@ -2440,7 +2443,9 @@ class ReplicaManager(val config: KafkaConfig, def handleLogDirFailure(dir: String, notifyController: Boolean = true): Unit = { if (!logManager.isLogDirOnline(dir)) return - warn(s"Stopping serving replicas in dir $dir") + // retrieve the UUID here because logManager.handleLogDirFailure handler removes it + val uuid = logManager.directoryId(dir) + warn(s"Stopping serving replicas in dir $dir with uuid $uuid because the log directory has failed.") replicaStateChangeLock synchronized { val newOfflinePartitions = onlinePartitionsIterator.filter { partition => partition.log.exists { _.parentDir == dir } @@ -2465,8 +2470,6 @@ class ReplicaManager(val config: KafkaConfig, warn(s"Broker $localBrokerId stopped fetcher for partitions ${newOfflinePartitions.mkString(",")} and stopped moving logs " + s"for partitions ${partitionsWithOfflineFutureReplica.mkString(",")} because they are in the failed log directory $dir.") } - // retrieve the UUID here because logManager.handleLogDirFailure handler removes it - val uuid = logManager.directoryId(dir) logManager.handleLogDirFailure(dir) if (dir == new File(config.metadataLogDir).getAbsolutePath && (config.processRoles.nonEmpty || config.migrationEnabled)) { fatal(s"Shutdown broker because the metadata log dir $dir has failed") diff --git a/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala index d13f9e0b7f141..9616f059a0eb9 100755 --- a/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala +++ b/core/src/main/scala/kafka/server/metadata/ZkMetadataCache.scala @@ -158,7 +158,6 @@ class ZkMetadataCache( brokerId: Int, metadataVersion: MetadataVersion, brokerFeatures: BrokerFeatures, - kraftControllerNodes: Seq[Node] = Seq.empty, zkMigrationEnabled: Boolean = false) extends MetadataCache with ZkFinalizedFeatureCache with Logging { @@ -182,8 +181,6 @@ class ZkMetadataCache( private val featureLock = new ReentrantLock() private val featureCond = featureLock.newCondition() - private val kraftControllerNodeMap = kraftControllerNodes.map(node => node.id() -> node).toMap - // This method is the main hotspot when it comes to the performance of metadata requests, // we should be careful about adding additional logic here. Relatedly, `brokers` is // `List[Integer]` instead of `List[Int]` to avoid a collection copy. @@ -350,11 +347,7 @@ class ZkMetadataCache( override def getAliveBrokerNode(brokerId: Int, listenerName: ListenerName): Option[Node] = { val snapshot = metadataSnapshot - brokerId match { - case id if snapshot.controllerId.filter(_.isInstanceOf[KRaftCachedControllerId]).exists(_.id == id) => - kraftControllerNodeMap.get(id) - case _ => snapshot.aliveBrokers.get(brokerId).flatMap(_.getNode(listenerName)) - } + snapshot.aliveBrokers.get(brokerId).flatMap(_.getNode(listenerName)) } override def getAliveBrokerNodes(listenerName: ListenerName): Iterable[Node] = { diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index a3f97d5fb9e5a..a4777f98541d3 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -23,9 +23,8 @@ import java.io._ import com.fasterxml.jackson.databind.node.{IntNode, JsonNodeFactory, ObjectNode, TextNode} import kafka.coordinator.transaction.TransactionLog import kafka.log._ -import kafka.serializer.Decoder -import kafka.utils._ import kafka.utils.Implicits._ +import kafka.utils.{CoreUtils, VerifiableProperties} import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.message.ConsumerProtocolAssignment import org.apache.kafka.common.message.ConsumerProtocolAssignmentJsonConverter @@ -47,6 +46,7 @@ import org.apache.kafka.metadata.bootstrap.BootstrapDirectory import org.apache.kafka.snapshot.Snapshots import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils} import org.apache.kafka.storage.internals.log.{CorruptSnapshotException, LogFileUtils, OffsetIndex, ProducerStateManager, TimeIndex, TransactionIndex} +import org.apache.kafka.tools.api.{Decoder, DefaultDecoder, IntegerDecoder, LongDecoder, StringDecoder} import java.nio.ByteBuffer import scala.jdk.CollectionConverters._ @@ -604,14 +604,14 @@ object DumpLogSegments { .ofType(classOf[java.lang.Integer]) .defaultsTo(Integer.MAX_VALUE) private val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration. Automatically set if print-data-log is enabled.") - private val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.") + private val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement org.apache.kafka.tools.api.Decoder trait. Custom jar should be available in kafka/libs directory.") .withOptionalArg() .ofType(classOf[java.lang.String]) - .defaultsTo("kafka.serializer.StringDecoder") - private val keyDecoderOpt = parser.accepts("key-decoder-class", "if set, used to deserialize the keys. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.") + .defaultsTo(classOf[StringDecoder].getName) + private val keyDecoderOpt = parser.accepts("key-decoder-class", "if set, used to deserialize the keys. This class should implement org.apache.kafka.tools.api.Decoder trait. Custom jar should be available in kafka/libs directory.") .withOptionalArg() .ofType(classOf[java.lang.String]) - .defaultsTo("kafka.serializer.StringDecoder") + .defaultsTo(classOf[StringDecoder].getName) private val offsetsOpt = parser.accepts("offsets-decoder", "if set, log data will be parsed as offset data from the " + "__consumer_offsets topic.") private val transactionLogOpt = parser.accepts("transaction-log-decoder", "if set, log data will be parsed as " + @@ -628,8 +628,8 @@ object DumpLogSegments { } else if (options.has(clusterMetadataOpt)) { new ClusterMetadataLogMessageParser } else { - val valueDecoder: Decoder[_] = CoreUtils.createObject[Decoder[_]](options.valueOf(valueDecoderOpt), new VerifiableProperties) - val keyDecoder: Decoder[_] = CoreUtils.createObject[Decoder[_]](options.valueOf(keyDecoderOpt), new VerifiableProperties) + val valueDecoder = newDecoder(options.valueOf(valueDecoderOpt)) + val keyDecoder = newDecoder(options.valueOf(keyDecoderOpt)) new DecoderMessageParser(keyDecoder, valueDecoder) } @@ -651,4 +651,42 @@ object DumpLogSegments { def checkArgs(): Unit = CommandLineUtils.checkRequiredArgs(parser, options, filesOpt) } + + /* + * The kafka.serializer.Decoder is deprecated in 3.8.0. This method is used to transfer the deprecated + * decoder to the new org.apache.kafka.tools.api.Decoder. Old decoders have an input VerifiableProperties. + * Remove it in new interface since it's always empty. + */ + private[tools] def newDecoder(className: String): Decoder[_] = { + try { + CoreUtils.createObject[org.apache.kafka.tools.api.Decoder[_]](convertDeprecatedDecoderClass(className)) + } catch { + case _: Exception => + // Old decoders always have an default VerifiableProperties input, because DumpLogSegments didn't provide + // any way to pass custom configs. + val decoder = CoreUtils.createObject[kafka.serializer.Decoder[_]](className, new VerifiableProperties()) + (bytes: Array[Byte]) => decoder.fromBytes(bytes) + } + } + + /* + * Covert deprecated decoder implementation to new decoder class. + */ + private[tools] def convertDeprecatedDecoderClass(className: String): String = { + if (className == "kafka.serializer.StringDecoder") { + println("kafka.serializer.StringDecoder is deprecated. Please use org.apache.kafka.tools.api.StringDecoder instead") + classOf[StringDecoder].getName + } else if (className == "kafka.serializer.LongDecoder") { + println("kafka.serializer.LongDecoder is deprecated. Please use org.apache.kafka.tools.api.LongDecoder instead") + classOf[LongDecoder].getName + } else if (className == "kafka.serializer.IntegerDecoder") { + println("kafka.serializer.IntegerDecoder is deprecated. Please use org.apache.kafka.tools.api.IntegerDecoder instead") + classOf[IntegerDecoder].getName + } else if (className == "kafka.serializer.DefaultDecoder") { + println("kafka.serializer.DefaultDecoder is deprecated. Please use org.apache.kafka.tools.api.DefaultDecoder instead") + classOf[DefaultDecoder].getName + } else { + className + } + } } diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala index 148be404bf734..86c82f1a1957b 100644 --- a/core/src/main/scala/kafka/tools/StorageTool.scala +++ b/core/src/main/scala/kafka/tools/StorageTool.scala @@ -253,7 +253,7 @@ object StorageTool extends Logging { .setServerKey(formatter.serverKey(saltedPassword)) .setIterations(iterations) } catch { - case e: Throwable => + case e: Throwable => throw new TerseFailure(s"Error attempting to create UserScramCredentialRecord: ${e.getMessage}") } myrecord @@ -464,8 +464,8 @@ object StorageTool extends Logging { copier.setWriteErrorHandler((logDir, e) => { throw new TerseFailure(s"Error while writing meta.properties file $logDir: ${e.getMessage}") }) - copier.writeLogDirChanges() }) + copier.writeLogDirChanges() } 0 } diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 0b445ed1a3cd5..8da7a4e7cc147 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -32,6 +32,7 @@ import org.apache.commons.validator.routines.InetAddressValidator import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.util.Csv import org.slf4j.event.Level import java.util @@ -109,17 +110,6 @@ object CoreUtils { } } - /** - * Parse a comma separated string into a sequence of strings. - * Whitespace surrounding the comma will be removed. - */ - def parseCsvList(csvList: String): Seq[String] = { - if (csvList == null || csvList.isEmpty) - Seq.empty[String] - else - csvList.split("\\s*,\\s*").filter(v => !v.equals("")) - } - /** * Create an instance of the class with the given class name */ @@ -219,8 +209,8 @@ object CoreUtils { } val endPoints = try { - val listenerList = parseCsvList(listeners) - listenerList.map(EndPoint.createEndPoint(_, Some(securityProtocolMap))) + val listenerList = Csv.parseCsvList(listeners) + listenerList.asScala.map(EndPoint.createEndPoint(_, Some(securityProtocolMap))) } catch { case e: Exception => throw new IllegalArgumentException(s"Error creating broker listeners from '$listeners': ${e.getMessage}", e) diff --git a/core/src/main/scala/kafka/utils/Throttler.scala b/core/src/main/scala/kafka/utils/Throttler.scala deleted file mode 100644 index 286343cd4449f..0000000000000 --- a/core/src/main/scala/kafka/utils/Throttler.scala +++ /dev/null @@ -1,111 +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 kafka.utils - -import org.apache.kafka.common.utils.Time -import org.apache.kafka.server.metrics.KafkaMetricsGroup - -import java.util.concurrent.TimeUnit -import java.util.Random - -import scala.math._ - -/** - * A class to measure and throttle the rate of some process. The throttler takes a desired rate-per-second - * (the units of the process don't matter, it could be bytes or a count of some other thing), and will sleep for - * an appropriate amount of time when maybeThrottle() is called to attain the desired rate. - * - * @param desiredRatePerSec: The rate we want to hit in units/sec - * @param checkIntervalMs: The interval at which to check our rate - * @param throttleDown: Does throttling increase or decrease our rate? - * @param time: The time implementation to use - */ -@threadsafe -class Throttler(@volatile var desiredRatePerSec: Double, - checkIntervalMs: Long = 100L, - throttleDown: Boolean = true, - metricName: String = "throttler", - units: String = "entries", - time: Time = Time.SYSTEM) extends Logging { - - private val metricsGroup = new KafkaMetricsGroup(this.getClass) - - private val lock = new Object - private val meter = metricsGroup.newMeter(metricName, units, TimeUnit.SECONDS) - private val checkIntervalNs = TimeUnit.MILLISECONDS.toNanos(checkIntervalMs) - private var periodStartNs: Long = time.nanoseconds - private var observedSoFar: Double = 0.0 - - def maybeThrottle(observed: Double): Unit = { - val msPerSec = TimeUnit.SECONDS.toMillis(1) - val nsPerSec = TimeUnit.SECONDS.toNanos(1) - val currentDesiredRatePerSec = desiredRatePerSec - - meter.mark(observed.toLong) - lock synchronized { - observedSoFar += observed - val now = time.nanoseconds - val elapsedNs = now - periodStartNs - // if we have completed an interval AND we have observed something, maybe - // we should take a little nap - if (elapsedNs > checkIntervalNs && observedSoFar > 0) { - val rateInSecs = (observedSoFar * nsPerSec) / elapsedNs - val needAdjustment = !(throttleDown ^ (rateInSecs > currentDesiredRatePerSec)) - if (needAdjustment) { - // solve for the amount of time to sleep to make us hit the desired rate - val desiredRateMs = currentDesiredRatePerSec / msPerSec.toDouble - val elapsedMs = TimeUnit.NANOSECONDS.toMillis(elapsedNs) - val sleepTime = round(observedSoFar / desiredRateMs - elapsedMs) - if (sleepTime > 0) { - trace("Natural rate is %f per second but desired rate is %f, sleeping for %d ms to compensate.".format(rateInSecs, currentDesiredRatePerSec, sleepTime)) - time.sleep(sleepTime) - } - } - periodStartNs = time.nanoseconds() - observedSoFar = 0 - } - } - } - - def updateDesiredRatePerSec(updatedDesiredRatePerSec: Double): Unit = { - desiredRatePerSec = updatedDesiredRatePerSec - } -} - -object Throttler { - - def main(args: Array[String]): Unit = { - val rand = new Random() - val throttler = new Throttler(100000, 100, true, time = Time.SYSTEM) - val interval = 30000 - var start = System.currentTimeMillis - var total = 0 - while (true) { - val value = rand.nextInt(1000) - Thread.sleep(1) - throttler.maybeThrottle(value) - total += value - val now = System.currentTimeMillis - if (now - start >= interval) { - println(total / (interval/1000.0)) - start = now - total = 0 - } - } - } -} diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 308212ff58ddb..5dd6fbe34cc46 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -41,6 +41,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager; import org.apache.kafka.server.log.remote.storage.LogSegmentData; import org.apache.kafka.server.log.remote.storage.NoOpRemoteLogMetadataManager; @@ -202,7 +203,7 @@ public List read() { }; private final AtomicLong currentLogStartOffset = new AtomicLong(0L); - private final UnifiedLog mockLog = mock(UnifiedLog.class); + private UnifiedLog mockLog = mock(UnifiedLog.class); @BeforeEach void setUp() throws Exception { @@ -356,7 +357,7 @@ void testRemoteLogMetadataManagerWithEndpointConfig() { assertEquals(host + ":" + port, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers")); assertEquals(securityProtocol, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); assertEquals(clusterId, capture.getValue().get("cluster.id")); - assertEquals(brokerId, capture.getValue().get(KafkaConfig.BrokerIdProp())); + assertEquals(brokerId, capture.getValue().get(ServerConfigs.BROKER_ID_CONFIG)); } @Test @@ -395,7 +396,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { // should be overridden as SSL assertEquals("SSL", capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); assertEquals(clusterId, capture.getValue().get("cluster.id")); - assertEquals(brokerId, capture.getValue().get(KafkaConfig.BrokerIdProp())); + assertEquals(brokerId, capture.getValue().get(ServerConfigs.BROKER_ID_CONFIG)); } } @@ -660,6 +661,7 @@ void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc long nextSegmentStartOffset = 150L; int segmentCount = 3; when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + when(mockLog.parentDir()).thenReturn("dir1"); // leader epoch preparation checkpoint.write(totalEpochEntries); @@ -728,7 +730,7 @@ void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc Partition mockLeaderPartition = mockPartition(leaderTopicIdPartition); Partition mockFollowerPartition = mockPartition(followerTopicIdPartition); List list = listRemoteLogSegmentMetadata(leaderTopicIdPartition, segmentCount, 100, 1024, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); - // return 3 metadata and then return 0 to simulate all segments are deleted + // return the metadataList 3 times, then return empty list to simulate all segments are deleted when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)).thenReturn(list.iterator()).thenReturn(Collections.emptyIterator()); when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 0)).thenReturn(list.iterator()).thenReturn(list.iterator()); when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 1)).thenReturn(list.iterator()); @@ -771,12 +773,126 @@ void testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc safeLongYammerMetricValue("RemoteLogSizeBytes"))); } + @Test + void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() throws Exception { + long oldSegmentStartOffset = 0L; + long nextSegmentStartOffset = 150L; + int segmentCount = 3; + when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + when(mockLog.parentDir()).thenReturn("dir1"); + + // leader epoch preparation + checkpoint.write(totalEpochEntries); + LeaderEpochFileCache cache = new LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class), anyInt())) + .thenReturn(Optional.of(0L)) + .thenReturn(Optional.of(nextSegmentStartOffset - 1)); + + 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); + + 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(250L); + when(mockLog.logEndOffset()).thenReturn(500L); + Map logProps = new HashMap<>(); + logProps.put("retention.bytes", 100L); + logProps.put("retention.ms", -1L); + LogConfig logConfig = new LogConfig(logProps); + when(mockLog.config()).thenReturn(logConfig); + + 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(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))).thenReturn(dummyFuture); + + CountDownLatch copyLogSegmentLatch = new CountDownLatch(1); + doAnswer(ans -> { + // waiting for verification + copyLogSegmentLatch.await(5000, TimeUnit.MILLISECONDS); + return Optional.empty(); + }).when(remoteStorageManager).copyLogSegmentData(any(RemoteLogSegmentMetadata.class), any(LogSegmentData.class)); + + Partition mockLeaderPartition = mockPartition(leaderTopicIdPartition); + List metadataList = listRemoteLogSegmentMetadata(leaderTopicIdPartition, segmentCount, 100, 1024, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition)).thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 0)).thenReturn(metadataList.iterator()).thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 1)).thenReturn(metadataList.iterator()); + when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 2)).thenReturn(metadataList.iterator()); + + // leadership change to log in dir1 + remoteLogManager.onLeadershipChange(Collections.singleton(mockLeaderPartition), Collections.emptySet(), topicIds); + + TestUtils.waitForCondition(() -> { + ArgumentCaptor argument = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(1)).updateHighestOffsetInRemoteStorage(argument.capture()); + return 0L == argument.getValue(); + }, "Timed out waiting for updateHighestOffsetInRemoteStorage(0) get invoked for dir1 log"); + + UnifiedLog oldMockLog = mockLog; + Mockito.clearInvocations(oldMockLog); + // simulate altering log dir completes, and the new partition leader changes to the same broker in different log dir (dir2) + mockLog = mock(UnifiedLog.class); + when(mockLog.parentDir()).thenReturn("dir2"); + when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache)); + when(mockLog.config()).thenReturn(logConfig); + when(mockLog.logEndOffset()).thenReturn(500L); + + remoteLogManager.onLeadershipChange(Collections.singleton(mockLeaderPartition), Collections.emptySet(), topicIds); + + // after copyLogSegment completes for log (in dir1), updateHighestOffsetInRemoteStorage will be triggered with new offset + // even though the leader replica has changed to log in dir2 + copyLogSegmentLatch.countDown(); + TestUtils.waitForCondition(() -> { + ArgumentCaptor argument = ArgumentCaptor.forClass(Long.class); + verify(oldMockLog, times(1)).updateHighestOffsetInRemoteStorage(argument.capture()); + return nextSegmentStartOffset - 1 == argument.getValue(); + }, "Timed out waiting for updateHighestOffsetInRemoteStorage(149) get invoked for dir1 log"); + + // On the next run of RLMTask, the log in dir2 will be picked and start by updateHighestOffsetInRemoteStorage to the expected offset + TestUtils.waitForCondition(() -> { + ArgumentCaptor argument = ArgumentCaptor.forClass(Long.class); + verify(mockLog, times(1)).updateHighestOffsetInRemoteStorage(argument.capture()); + return nextSegmentStartOffset - 1 == argument.getValue(); + }, "Timed out waiting for updateHighestOffsetInRemoteStorage(149) get invoked for dir2 log"); + + } + @Test void testRemoteLogManagerRemoteMetrics() throws Exception { long oldestSegmentStartOffset = 0L; long olderSegmentStartOffset = 75L; long nextSegmentStartOffset = 150L; when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition()); + when(mockLog.parentDir()).thenReturn("dir1"); // leader epoch preparation checkpoint.write(totalEpochEntries); diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index df8990b0c92f2..9857d4c92cd39 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -48,7 +48,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -67,15 +66,11 @@ public class RaftClusterInvocationContext implements TestTemplateInvocationConte private final String baseDisplayName; private final ClusterConfig clusterConfig; - private final AtomicReference clusterReference; - private final AtomicReference zkReference; private final boolean isCombined; public RaftClusterInvocationContext(String baseDisplayName, ClusterConfig clusterConfig, boolean isCombined) { this.baseDisplayName = baseDisplayName; this.clusterConfig = clusterConfig; - this.clusterReference = new AtomicReference<>(); - this.zkReference = new AtomicReference<>(); this.isCombined = isCombined; } @@ -86,67 +81,43 @@ public String getDisplayName(int invocationIndex) { @Override public List getAdditionalExtensions() { - RaftClusterInstance clusterInstance = new RaftClusterInstance(clusterReference, zkReference, clusterConfig, isCombined); + RaftClusterInstance clusterInstance = new RaftClusterInstance(clusterConfig, isCombined); return Arrays.asList( - (BeforeTestExecutionCallback) context -> { - TestKitNodes nodes = new TestKitNodes.Builder(). - setBootstrapMetadataVersion(clusterConfig.metadataVersion()). - setCombined(isCombined). - setNumBrokerNodes(clusterConfig.numBrokers()). - setPerServerProperties(clusterConfig.perServerOverrideProperties()). - setNumDisksPerBroker(clusterConfig.numDisksPerBroker()). - setNumControllerNodes(clusterConfig.numControllers()).build(); - KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder(nodes); - - if (Boolean.parseBoolean(clusterConfig.serverProperties().getOrDefault("zookeeper.metadata.migration.enable", "false"))) { - zkReference.set(new EmbeddedZookeeper()); - builder.setConfigProp("zookeeper.connect", String.format("localhost:%d", zkReference.get().port())); - } - // Copy properties into the TestKit builder - clusterConfig.serverProperties().forEach(builder::setConfigProp); - // KAFKA-12512 need to pass security protocol and listener name here - KafkaClusterTestKit cluster = builder.build(); - clusterReference.set(cluster); - cluster.format(); - if (clusterConfig.isAutoStart()) { - cluster.startup(); - kafka.utils.TestUtils.waitUntilTrue( - () -> cluster.brokers().get(0).brokerState() == BrokerState.RUNNING, - () -> "Broker never made it to RUNNING state.", - org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, - 100L); - } - }, - (AfterTestExecutionCallback) context -> clusterInstance.stop(), - new ClusterInstanceParameterResolver(clusterInstance) + (BeforeTestExecutionCallback) context -> { + clusterInstance.format(); + if (clusterConfig.isAutoStart()) { + clusterInstance.start(); + } + }, + (AfterTestExecutionCallback) context -> clusterInstance.stop(), + new ClusterInstanceParameterResolver(clusterInstance) ); } public static class RaftClusterInstance implements ClusterInstance { - private final AtomicReference clusterReference; - private final AtomicReference zkReference; private final ClusterConfig clusterConfig; final AtomicBoolean started = new AtomicBoolean(false); final AtomicBoolean stopped = new AtomicBoolean(false); + final AtomicBoolean formated = new AtomicBoolean(false); private final ConcurrentLinkedQueue admins = new ConcurrentLinkedQueue<>(); + private EmbeddedZookeeper embeddedZookeeper; + private KafkaClusterTestKit clusterTestKit; private final boolean isCombined; - RaftClusterInstance(AtomicReference clusterReference, AtomicReference zkReference, ClusterConfig clusterConfig, boolean isCombined) { - this.clusterReference = clusterReference; - this.zkReference = zkReference; + RaftClusterInstance(ClusterConfig clusterConfig, boolean isCombined) { this.clusterConfig = clusterConfig; this.isCombined = isCombined; } @Override public String bootstrapServers() { - return clusterReference.get().bootstrapServers(); + return clusterTestKit.bootstrapServers(); } @Override public String bootstrapControllers() { - return clusterReference.get().bootstrapControllers(); + return clusterTestKit.bootstrapControllers(); } @Override @@ -193,25 +164,30 @@ public Set controllerIds() { @Override public KafkaClusterTestKit getUnderlying() { - return clusterReference.get(); + return clusterTestKit; } @Override public Admin createAdminClient(Properties configOverrides) { - Admin admin = Admin.create(clusterReference.get(). - newClientPropertiesBuilder(configOverrides).build()); + Admin admin = Admin.create(clusterTestKit.newClientPropertiesBuilder(configOverrides).build()); admins.add(admin); return admin; } @Override public void start() { - if (started.compareAndSet(false, true)) { - try { - clusterReference.get().startup(); - } catch (Exception e) { - throw new RuntimeException("Failed to start Raft server", e); + try { + format(); + if (started.compareAndSet(false, true)) { + clusterTestKit.startup(); + kafka.utils.TestUtils.waitUntilTrue( + () -> this.clusterTestKit.brokers().get(0).brokerState() == BrokerState.RUNNING, + () -> "Broker never made it to RUNNING state.", + org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, + 100L); } + } catch (Exception e) { + throw new RuntimeException("Failed to start Raft server", e); } } @@ -220,9 +196,9 @@ public void stop() { if (stopped.compareAndSet(false, true)) { admins.forEach(admin -> Utils.closeQuietly(admin, "admin")); admins.clear(); - Utils.closeQuietly(clusterReference.get(), "cluster"); - if (zkReference.get() != null) { - Utils.closeQuietly(zkReference.get(), "zk"); + Utils.closeQuietly(clusterTestKit, "cluster"); + if (embeddedZookeeper != null) { + Utils.closeQuietly(embeddedZookeeper, "zk"); } } } @@ -240,27 +216,51 @@ public void startBroker(int brokerId) { @Override public void waitForReadyBrokers() throws InterruptedException { try { - clusterReference.get().waitForReadyBrokers(); + clusterTestKit.waitForReadyBrokers(); } catch (ExecutionException e) { throw new AssertionError("Failed while waiting for brokers to become ready", e); } } - private BrokerServer findBrokerOrThrow(int brokerId) { - return Optional.ofNullable(clusterReference.get().brokers().get(brokerId)) - .orElseThrow(() -> new IllegalArgumentException("Unknown brokerId " + brokerId)); - } @Override public Map brokers() { - return clusterReference.get().brokers().entrySet() + return clusterTestKit.brokers().entrySet() .stream() .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } @Override public Map controllers() { - return Collections.unmodifiableMap(clusterReference.get().controllers()); + return Collections.unmodifiableMap(clusterTestKit.controllers()); + } + + public void format() throws Exception { + if (formated.compareAndSet(false, true)) { + TestKitNodes nodes = new TestKitNodes.Builder() + .setBootstrapMetadataVersion(clusterConfig.metadataVersion()) + .setCombined(isCombined) + .setNumBrokerNodes(clusterConfig.numBrokers()) + .setNumDisksPerBroker(clusterConfig.numDisksPerBroker()) + .setPerServerProperties(clusterConfig.perServerOverrideProperties()) + .setNumControllerNodes(clusterConfig.numControllers()).build(); + KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder(nodes); + if (Boolean.parseBoolean(clusterConfig.serverProperties() + .getOrDefault("zookeeper.metadata.migration.enable", "false"))) { + this.embeddedZookeeper = new EmbeddedZookeeper(); + builder.setConfigProp("zookeeper.connect", String.format("localhost:%d", embeddedZookeeper.port())); + } + // Copy properties into the TestKit builder + clusterConfig.serverProperties().forEach(builder::setConfigProp); + // KAFKA-12512 need to pass security protocol and listener name here + this.clusterTestKit = builder.build(); + this.clusterTestKit.format(); + } + } + + private BrokerServer findBrokerOrThrow(int brokerId) { + return Optional.ofNullable(clusterTestKit.brokers().get(brokerId)) + .orElseThrow(() -> new IllegalArgumentException("Unknown brokerId " + brokerId)); } } diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 4d34ce040142b..803809945fa9f 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -23,7 +23,6 @@ import kafka.server.FaultHandlerFactory; import kafka.server.SharedServer; import kafka.server.KafkaConfig; -import kafka.server.KafkaConfig$; import kafka.server.KafkaRaftServer; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClientConfig; @@ -41,6 +40,7 @@ import org.apache.kafka.raft.QuorumConfig; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.config.KRaftConfigs; +import org.apache.kafka.server.config.ServerConfigs; import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.storage.internals.log.CleanerConfig; @@ -213,8 +213,8 @@ private KafkaConfig createNodeConfig(TestKitNode node) { if (controllerNode != null) { props.putAll(controllerNode.propertyOverrides()); } - props.putIfAbsent(KafkaConfig$.MODULE$.UnstableMetadataVersionsEnableProp(), "true"); - props.putIfAbsent(KafkaConfig$.MODULE$.UnstableApiVersionsEnableProp(), "true"); + props.putIfAbsent(ServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, "true"); + props.putIfAbsent(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true"); return new KafkaConfig(props, false, Option.empty()); } diff --git a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala index 79729bae5147a..90f719dff8c8c 100644 --- a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala +++ b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.message.ApiMessageType import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.ApiVersionsResponse import org.apache.kafka.network.SocketServerConfigs +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotNull, assertTrue} import org.junit.jupiter.api.Timeout import org.junit.jupiter.params.ParameterizedTest @@ -43,7 +44,7 @@ class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { TestUtils.createBrokerConfigs(1, null).map(props => { // Enable unstable api versions to be compatible with the new APIs under development, // maybe we can remove this after the new APIs is complete. - props.setProperty(KafkaConfig.UnstableApiVersionsEnableProp, "true") + props.setProperty(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") props }).map(KafkaConfig.fromProps) } else { @@ -54,7 +55,7 @@ class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT") props.setProperty("listeners", "PLAINTEXT://localhost:0,CONTROLLER://localhost:0") props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "PLAINTEXT://localhost:0,CONTROLLER://localhost:0") - props.setProperty(KafkaConfig.UnstableApiVersionsEnableProp, "true") + props.setProperty(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") props }).map(KafkaConfig.fromProps) } diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala index e5c2d08e0d1ca..e45689b67f02c 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala @@ -13,7 +13,7 @@ package kafka.api import kafka.security.authorizer.AclAuthorizer -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerConfig @@ -31,6 +31,7 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.metadata.authorizer.StandardAuthorizer +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.{BeforeEach, TestInfo} import java.util.Properties @@ -93,7 +94,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group) override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) + properties.put(ServerConfigs.BROKER_ID_CONFIG, brokerId.toString) addNodeProperties(properties) } @@ -105,10 +106,10 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { private def addNodeProperties(properties: Properties): Unit = { if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) } properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") @@ -116,7 +117,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, "1") properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, "1") - properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true") + properties.put(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[PrincipalBuilder].getName) } diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index b08b25a8adfdd..d242ea105e665 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.TopicPartition import kafka.utils.TestUtils -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} @@ -33,6 +33,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.errors.WakeupException import org.apache.kafka.coordinator.group.GroupCoordinatorConfig +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.util.ShutdownableThread import scala.collection.mutable @@ -66,7 +67,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest { override protected def brokerPropertyOverrides(properties: Properties): Unit = { - properties.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + properties.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // speed up shutdown properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "3") // don't want to lose offset properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") // set small enough session timeout diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index 30b0b55f349ca..3fadf51d4f050 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException} import org.apache.kafka.common.utils.Utils import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, ServerLogConfigs} import org.apache.kafka.server.policy.AlterConfigPolicy import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} @@ -127,10 +127,10 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with // Set a mutable broker config val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, brokers.head.config.brokerId.toString) - val brokerConfigs = Seq(new ConfigEntry(KafkaConfig.MessageMaxBytesProp, "50000")).asJava + val brokerConfigs = Seq(new ConfigEntry(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "50000")).asJava val alterResult1 = client.alterConfigs(Map(brokerResource -> new Config(brokerConfigs)).asJava) alterResult1.all.get - assertEquals(Set(KafkaConfig.MessageMaxBytesProp), validationsForResource(brokerResource).head.configs().keySet().asScala) + assertEquals(Set(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG), validationsForResource(brokerResource).head.configs().keySet().asScala) validations.clear() val topicConfigEntries1 = Seq( diff --git a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala index bcfa7f2aa6c24..6f5a16c1f2d57 100644 --- a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala @@ -19,7 +19,6 @@ package kafka.api import java.util import java.util.Properties import java.util.concurrent.ExecutionException -import kafka.server.KafkaConfig import kafka.utils.Logging import kafka.utils.TestUtils._ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription} @@ -30,8 +29,7 @@ import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} @@ -71,7 +69,7 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg @Test def testCreateDeleteTopics(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topics = Seq("mytopic", "mytopic2", "mytopic3") val newTopics = Seq( new NewTopic("mytopic", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava), @@ -163,7 +161,7 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg @Test def testAuthorizedOperations(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // without includeAuthorizedOperations flag var result = client.describeCluster @@ -204,10 +202,10 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg }) } configs.foreach { config => - config.setProperty(KafkaConfig.DeleteTopicEnableProp, "true") + config.setProperty(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") config.setProperty(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") config.setProperty(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, "false") - config.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + config.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // We set this in order to test that we don't expose sensitive data via describe configs. This will already be // set for subclasses with security enabled and we don't want to overwrite it. if (!config.containsKey(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)) @@ -226,12 +224,17 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg val config = new util.HashMap[String, Object] config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) config.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "20000") - val securityProps: util.Map[Object, Object] = - adminClientSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties) - securityProps.forEach { (key, value) => config.put(key.asInstanceOf[String], value) } + config.put(AdminClientConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, "40000") config } + def createAdminClient: Admin = { + val props = new Properties() + props.putAll(createConfig) + val client = createAdminClient(configOverrides = props) + client + } + def waitForTopics(client: Admin, expectedPresent: Seq[String], expectedMissing: Seq[String]): Unit = { waitUntilTrue(() => { val topics = client.listTopics.names.get() diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala index 53429f4183e1e..e27fa687f11dd 100644 --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala @@ -20,7 +20,7 @@ import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} import com.yammer.metrics.core.{Histogram, Meter} import kafka.api.QuotaTestClients._ -import kafka.server.{ClientQuotaManager, KafkaBroker, KafkaConfig, QuotaType} +import kafka.server.{ClientQuotaManager, KafkaBroker, QuotaType} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} @@ -33,7 +33,7 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration import org.apache.kafka.common.quota.ClientQuotaEntity import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} @@ -51,7 +51,7 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { protected def consumerClientId = "QuotasTestConsumer-1" protected def createQuotaTestClients(topic: String, leaderNode: KafkaBroker): QuotaTestClients - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "2") this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") this.serverConfig.setProperty(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala index d01476f999df1..a82aca2f76f9f 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala @@ -21,14 +21,11 @@ import java.lang.{Boolean => JBoolean} import java.time.Duration import java.util import java.util.Collections - -import kafka.server.KafkaConfig import kafka.utils.{EmptyTestInfo, TestUtils} import org.apache.kafka.clients.admin.NewTopic import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} -import org.apache.kafka.server.config.ServerLogConfigs - +import org.apache.kafka.server.config.{ServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} @@ -57,7 +54,7 @@ object ConsumerTopicCreationTest { private val consumerClientId = "ConsumerTestConsumer" // configure server properties - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // speed up shutdown this.serverConfig.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, brokerAutoTopicCreationEnable.toString) // configure client properties diff --git a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala index 9818b6dbd554d..81e3d497e4ab4 100644 --- a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala +++ b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.{Cluster, Reconfigurable} import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth._ -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, QuotaConfigs} import org.apache.kafka.server.quota._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -65,7 +65,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { this.serverConfig.setProperty(QuotaConfigs.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, classOf[GroupedUserQuotaCallback].getName) this.serverConfig.setProperty(s"${listenerName.configPrefix}${KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG}", classOf[GroupedUserPrincipalBuilder].getName) - this.serverConfig.setProperty(KafkaConfig.DeleteTopicEnableProp, "true") + this.serverConfig.setProperty(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") super.setUp(testInfo) producerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, @@ -367,7 +367,7 @@ class GroupedUserQuotaCallback extends ClientQuotaCallback with Reconfigurable w val partitionRatio = new ConcurrentHashMap[String, Double]() override def configure(configs: util.Map[String, _]): Unit = { - brokerId = configs.get(KafkaConfig.BrokerIdProp).toString.toInt + brokerId = configs.get(ServerConfigs.BROKER_ID_CONFIG).toString.toInt callbackInstances.incrementAndGet } diff --git a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala index a36f54abdc682..95cfc201cd4bd 100644 --- a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala @@ -17,8 +17,6 @@ package kafka.api import java.util.Properties - -import kafka.server.KafkaConfig import kafka.utils._ import kafka.tools.StorageTool import kafka.zk.ConfigEntityChangeNotificationZNode @@ -35,6 +33,7 @@ import org.junit.jupiter.api.{BeforeEach, TestInfo} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.config.DelegationTokenManagerConfigs class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest { @@ -54,8 +53,8 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest protected val privilegedAdminClientConfig = new Properties() - this.serverConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") def createDelegationTokenOptions(): CreateDelegationTokenOptions = new CreateDelegationTokenOptions() diff --git a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala index 0be5e306e2948..c150b70415dfe 100644 --- a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala +++ b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala @@ -14,9 +14,7 @@ package kafka.api import java.util import java.util.Properties - import kafka.security.authorizer.AclAuthorizer -import kafka.server.KafkaConfig import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import org.apache.kafka.clients.admin._ import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, CLUSTER_ACTION, DELETE, DESCRIBE} @@ -27,7 +25,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.Utils import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.authorizer.Authorizer -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNull} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -78,7 +76,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS override val brokerCount = 1 this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) var client: Admin = _ diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 60dee252420fb..245277d981146 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -23,7 +23,6 @@ import java.util.{Collections, Properties} import java.util.concurrent.ExecutionException import kafka.security.authorizer.AclAuthorizer import org.apache.kafka.metadata.authorizer.StandardAuthorizer -import kafka.server._ import kafka.utils._ import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords} @@ -39,8 +38,7 @@ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.security.auth._ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST -import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} @@ -153,13 +151,13 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas if (TestInfoUtils.isKRaft(testInfo)) { this.serverConfig.setProperty(StandardAuthorizer.SUPER_USERS_CONFIG, kafkaPrincipal.toString) this.controllerConfig.setProperty(StandardAuthorizer.SUPER_USERS_CONFIG, kafkaPrincipal.toString + ";" + "User:ANONYMOUS") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) - this.controllerConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) + this.controllerConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { // The next two configuration parameters enable ZooKeeper secure ACLs // and sets the Kafka authorizer, both necessary to enable security. this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, authorizerClass.getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, authorizerClass.getName) // Set the specific principal that can update ACLs. this.serverConfig.setProperty(AclAuthorizer.SuperUsersProp, kafkaPrincipal.toString) diff --git a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala index 6ffd0f7fb19bb..02c8399fa0898 100644 --- a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala @@ -16,7 +16,7 @@ import java.util.Properties import java.util.concurrent.ExecutionException import kafka.api.GroupAuthorizerIntegrationTest._ import kafka.security.authorizer.AclAuthorizer -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerRecord @@ -32,6 +32,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -76,16 +77,16 @@ class GroupAuthorizerIntegrationTest extends BaseRequestTest { } override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) + properties.put(ServerConfigs.BROKER_ID_CONFIG, brokerId.toString) addNodeProperties(properties) } private def addNodeProperties(properties: Properties): Unit = { if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) } properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 2133019a4a871..9c8664c6f922a 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -46,7 +46,7 @@ import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEX import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, QuotaConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS import org.junit.jupiter.api.Assertions._ @@ -95,7 +95,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testClose(quorum: String): Unit = { - val client = Admin.create(createConfig) + val client = createAdminClient client.close() client.close() // double close has no effect } @@ -103,7 +103,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testListNodes(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val brokerStrs = bootstrapServers().split(",").toList.sorted var nodeStrs: List[String] = null do { @@ -131,7 +131,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testCreateExistingTopicsThrowTopicExistsException(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "mytopic" val topics = Seq(topic) val newTopics = Seq(new NewTopic(topic, 1, 1.toShort)) @@ -148,7 +148,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDeleteTopicsWithIds(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topics = Seq("mytopic", "mytopic2", "mytopic3") val newTopics = Seq( new NewTopic("mytopic", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava), @@ -170,7 +170,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeNonExistingTopic(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val existingTopic = "existing-topic" client.createTopics(Seq(existingTopic).map(new NewTopic(_, 1, 1.toShort)).asJava).all.get() @@ -188,7 +188,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeTopicsWithIds(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val existingTopic = "existing-topic" client.createTopics(Seq(existingTopic).map(new NewTopic(_, 1, 1.toShort)).asJava).all.get() @@ -207,7 +207,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeCluster(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val result = client.describeCluster val nodes = result.nodes.get() val clusterId = result.clusterId().get() @@ -234,7 +234,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeLogDirs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "topic" val leaderByPartition = createTopic(topic, numPartitions = 10) val partitionsByBroker = leaderByPartition.groupBy { case (_, leaderId) => leaderId }.map { case (k, v) => @@ -266,7 +266,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeReplicaLogDirs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "topic" val leaderByPartition = createTopic(topic, numPartitions = 10) val replicas = leaderByPartition.map { case (partition, brokerId) => @@ -285,7 +285,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testAlterReplicaLogDirs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "topic" val tp = new TopicPartition(topic, 0) val randomNums = brokers.map(server => server -> Random.nextInt(2)).toMap @@ -375,7 +375,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeAndAlterConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic1 = "describe-alter-configs-topic-1" @@ -426,7 +426,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { }) assertEquals(brokers(1).config.nonInternalValues.size + numInternalConfigsSet, configs.get(brokerResource1).entries.size) - assertEquals(brokers(1).config.brokerId.toString, configs.get(brokerResource1).get(KafkaConfig.BrokerIdProp).value) + assertEquals(brokers(1).config.brokerId.toString, configs.get(brokerResource1).get(ServerConfigs.BROKER_ID_CONFIG).value) val listenerSecurityProtocolMap = configs.get(brokerResource1).get(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG) assertEquals(brokers(1).config.getString(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG), listenerSecurityProtocolMap.value) assertEquals(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, listenerSecurityProtocolMap.name) @@ -439,16 +439,16 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertFalse(truststorePassword.isDefault) assertTrue(truststorePassword.isSensitive) assertFalse(truststorePassword.isReadOnly) - val compressionType = configs.get(brokerResource1).get(KafkaConfig.CompressionTypeProp) + val compressionType = configs.get(brokerResource1).get(ServerConfigs.COMPRESSION_TYPE_CONFIG) assertEquals(brokers(1).config.compressionType, compressionType.value) - assertEquals(KafkaConfig.CompressionTypeProp, compressionType.name) + assertEquals(ServerConfigs.COMPRESSION_TYPE_CONFIG, compressionType.name) assertTrue(compressionType.isDefault) assertFalse(compressionType.isSensitive) assertFalse(compressionType.isReadOnly) assertEquals(brokers(2).config.nonInternalValues.size + numInternalConfigsSet, configs.get(brokerResource2).entries.size) - assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(KafkaConfig.BrokerIdProp).value) + assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(ServerConfigs.BROKER_ID_CONFIG).value) assertEquals(brokers(2).config.logCleanerThreads.toString, configs.get(brokerResource2).get(CleanerConfig.LOG_CLEANER_THREADS_PROP).value) @@ -458,7 +458,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testCreatePartitions(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic1 = "create-partitions-topic-1" @@ -731,7 +731,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testSeekAfterDeleteRecords(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) @@ -761,7 +761,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testLogStartOffsetCheckpoint(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) @@ -778,7 +778,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { restartDeadBrokers() client.close() - client = Admin.create(createConfig) + client = createAdminClient TestUtils.waitUntilTrue(() => { // Need to retry if leader is not available for the partition @@ -801,7 +801,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testLogStartOffsetAfterDeleteRecords(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) @@ -840,7 +840,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // we will produce to topic and delete records while one follower is down killBroker(followerIndex) - client = Admin.create(createConfig) + client = createAdminClient val producer = createProducer() sendRecords(producer, 100, topicPartition) @@ -869,7 +869,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testAlterLogDirsAfterDeleteRecords(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient createTopic(topic, replicationFactor = brokerCount) val expectedLEO = 100 val producer = createProducer() @@ -904,7 +904,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testOffsetsForTimesAfterDeleteRecords(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) @@ -928,7 +928,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) - client = Admin.create(createConfig) + client = createAdminClient val producer = createProducer() sendRecords(producer, 10, topicPartition) @@ -952,7 +952,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val consumer = createConsumer() subscribeAndWaitForAssignment(topic, consumer) - client = Admin.create(createConfig) + client = createAdminClient val producer = createProducer() sendRecords(producer, 10, topicPartition) @@ -970,7 +970,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testDescribeConfigsForTopic(quorum: String): Unit = { createTopic(topic, numPartitions = 2, replicationFactor = brokerCount) - client = Admin.create(createConfig) + client = createAdminClient val existingTopic = new ConfigResource(ConfigResource.Type.TOPIC, topic) client.describeConfigs(Collections.singletonList(existingTopic)).values.get(existingTopic).get() @@ -1006,7 +1006,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testInvalidAlterConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient checkInvalidAlterConfigs(this, client) } @@ -1020,7 +1020,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { def testAclOperations(quorum: String): Unit = { val acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW)) - client = Admin.create(createConfig) + client = createAdminClient assertFutureExceptionTypeEquals(client.describeAcls(AclBindingFilter.ANY).values(), classOf[SecurityDisabledException]) assertFutureExceptionTypeEquals(client.createAcls(Collections.singleton(acl)).all(), classOf[SecurityDisabledException]) @@ -1035,7 +1035,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDelayedClose(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topics = Seq("mytopic", "mytopic2") val newTopics = topics.map(new NewTopic(_, 1, 1.toShort)) val future = client.createTopics(newTopics.asJava, new CreateTopicsOptions().validateOnly(true)).all() @@ -1400,7 +1400,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testElectPreferredLeaders(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val prefer0 = Seq(0, 1, 2) val prefer1 = Seq(1, 2, 0) @@ -1560,7 +1560,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersForOnePartition(quorum: String): Unit = { // Case: unclean leader election with one topic partition - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1588,7 +1588,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersForManyPartitions(quorum: String): Unit = { // Case: unclean leader election with many topic partitions - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1628,7 +1628,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersForAllPartitions(quorum: String): Unit = { // Case: noop unclean leader election and valid unclean leader election for all partitions - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1668,7 +1668,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersForUnknownPartitions(quorum: String): Unit = { // Case: unclean leader election for unknown topic - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1694,7 +1694,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersWhenNoLiveBrokers(quorum: String): Unit = { // Case: unclean leader election with no live brokers - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1723,7 +1723,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersNoop(quorum: String): Unit = { // Case: noop unclean leader election with explicit topic partitions - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1751,7 +1751,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) def testElectUncleanLeadersAndNoop(quorum: String): Unit = { // Case: one noop unclean leader election and one valid unclean leader election - client = Admin.create(createConfig) + client = createAdminClient val broker1 = 1 val broker2 = 2 @@ -1790,7 +1790,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testListReassignmentsDoesNotShowNonReassigningPartitions(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic = "list-reassignments-no-reassignments" @@ -1807,7 +1807,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testListReassignmentsDoesNotShowDeletedPartitions(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "list-reassignments-no-reassignments" val tp = new TopicPartition(topic, 0) @@ -1822,7 +1822,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testValidIncrementalAlterConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic1 = "incremental-alter-configs-topic-1" @@ -1942,7 +1942,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testAppendAlreadyExistsConfigsAndSubtractNotExistsConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic = "incremental-alter-configs-topic" @@ -1983,7 +1983,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testIncrementalAlterConfigsDeleteAndSetBrokerConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val broker0Resource = new ConfigResource(ConfigResource.Type.BROKER, "0") client.incrementalAlterConfigs(Map(broker0Resource -> Seq(new AlterConfigOp(new ConfigEntry(QuotaConfigs.LEADER_REPLICATION_THROTTLED_RATE_CONFIG, "123"), @@ -2017,7 +2017,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testIncrementalAlterConfigsDeleteBrokerConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val broker0Resource = new ConfigResource(ConfigResource.Type.BROKER, "0") client.incrementalAlterConfigs(Map(broker0Resource -> Seq(new AlterConfigOp(new ConfigEntry(QuotaConfigs.LEADER_REPLICATION_THROTTLED_RATE_CONFIG, "123"), @@ -2054,7 +2054,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testInvalidIncrementalAlterConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Create topics val topic1 = "incremental-alter-configs-topic-1" @@ -2150,7 +2150,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testInvalidAlterPartitionReassignments(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val topic = "alter-reassignments-topic-1" val tp1 = new TopicPartition(topic, 0) val tp2 = new TopicPartition(topic, 1) @@ -2190,9 +2190,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testLongTopicNames(quorum: String): Unit = { - val client = Admin.create(createConfig) - val longTopicName = List.fill(249)("x").mkString("") - val invalidTopicName = List.fill(250)("x").mkString("") + val client = createAdminClient + val longTopicName = String.join("", Collections.nCopies(249, "x")); + val invalidTopicName = String.join("", Collections.nCopies(250, "x")); val newTopics2 = Seq(new NewTopic(invalidTopicName, 3, 3.toShort), new NewTopic(longTopicName, 3, 3.toShort)) val results = client.createTopics(newTopics2.asJava).values() @@ -2225,7 +2225,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertEquals(ServerLogConfigs.LOG_RETENTION_BYTES_DEFAULT, logConfig.retentionSize) } - client = Admin.create(createConfig) + client = createAdminClient val invalidConfigs = Map[String, String]( TopicConfig.RETENTION_BYTES_CONFIG -> null, TopicConfig.COMPRESSION_TYPE_CONFIG -> "producer" @@ -2258,7 +2258,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testDescribeConfigsForLog4jLogLevels(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient LoggerFactory.getLogger("kafka.cluster.Replica").trace("Message to create the logger") val loggerConfig = describeBrokerLoggers() val kafkaLogLevel = loggerConfig.get("kafka").value() @@ -2276,7 +2276,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) @Disabled // To be re-enabled once KAFKA-8779 is resolved def testIncrementalAlterConfigsForLog4jLogLevels(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val initialLoggerConfig = describeBrokerLoggers() val initialRootLogLevel = initialLoggerConfig.get(Log4jController.ROOT_LOGGER).value() @@ -2341,7 +2341,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) @Disabled // To be re-enabled once KAFKA-8779 is resolved def testIncrementalAlterConfigsForLog4jLogLevelsCanResetLoggerToCurrentRoot(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // step 1 - configure root logger val initialRootLogLevel = LogLevelConfig.TRACE_LOG_LEVEL val alterRootLoggerEntry = Seq( @@ -2384,7 +2384,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) @Disabled // Zk to be re-enabled once KAFKA-8779 is resolved def testIncrementalAlterConfigsForLog4jLogLevelsCannotResetRootLogger(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val deleteRootLoggerEntry = Seq( new AlterConfigOp(new ConfigEntry(Log4jController.ROOT_LOGGER, ""), AlterConfigOp.OpType.DELETE) ).asJavaCollection @@ -2396,7 +2396,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("zk", "kraft")) @Disabled // To be re-enabled once KAFKA-8779 is resolved def testIncrementalAlterConfigsForLog4jLogLevelsDoesNotWorkWithInvalidConfigs(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val validLoggerName = "kafka.server.KafkaRequestHandler" val expectedValidLoggerLogLevel = describeBrokerLoggers().get(validLoggerName) def assertLogLevelDidNotChange(): Unit = { @@ -2440,7 +2440,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ParameterizedTest @ValueSource(strings = Array("kraft")) // Zk to be re-enabled once KAFKA-8779 is resolved def testAlterConfigsForLog4jLogLevelsDoesNotWork(quorum: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val alterLogLevelsEntries = Seq( new ConfigEntry("kafka.controller.KafkaController", LogLevelConfig.INFO_LOG_LEVEL) @@ -2499,7 +2499,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { } private def testAppendConfig(props: Properties, append: String, expected: String): Unit = { - client = Admin.create(createConfig) + client = createAdminClient createTopic(topic, topicConfig = props) val topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic) val topicAlterConfigs = Seq( @@ -2735,7 +2735,7 @@ object PlaintextAdminIntegrationTest { assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value) // Alter configs with validateOnly = true: first and third are invalid, second is valid topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "gzip")).asJava @@ -2764,6 +2764,6 @@ object PlaintextAdminIntegrationTest { assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value) } } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 71cd632c0d8f0..a05f3a20f1eba 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -44,8 +44,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { val overridingProps = new Properties() overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString) - overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) - overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, replicaFetchMaxPartitionBytes.toString) + overridingProps.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, serverMessageMaxBytes.toString) + overridingProps.put(ReplicationConfigs.REPLICA_FETCH_MAX_BYTES_CONFIG, replicaFetchMaxPartitionBytes.toString) overridingProps.put(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_DOC, replicaFetchMaxResponseBytes.toString) // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long diff --git a/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala b/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala index b15e9f9f2d501..1b6a99afb10f4 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} @@ -233,7 +233,7 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 47e8de08031b4..1fc4c3e152a4f 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -12,63 +12,87 @@ */ package kafka.api -import java.util import kafka.security.authorizer.AclAuthorizer -import kafka.server.KafkaConfig -import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import kafka.utils.TestUtils._ +import kafka.utils.{JaasTestUtils, TestUtils} import org.apache.kafka.clients.admin._ import org.apache.kafka.common.Uuid import org.apache.kafka.common.acl._ -import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE} +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, TopicConfig} +import org.apache.kafka.common.config.{ConfigResource, SaslConfigs, TopicConfig} import org.apache.kafka.common.errors.{ClusterAuthorizationException, 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.security.authorizer.AclEntry.{WILDCARD_HOST, WILDCARD_PRINCIPAL_STRING} -import org.apache.kafka.server.authorizer.Authorizer -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs} import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} -import java.util.Collections -import scala.jdk.CollectionConverters._ +import java.util import scala.collection.Seq import scala.compat.java8.OptionConverters._ import scala.concurrent.ExecutionException +import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} +@Timeout(120) class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetup { val clusterResourcePattern = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) - val authorizationAdmin = new AclAuthorizationAdmin(classOf[AclAuthorizer], classOf[AclAuthorizer]) - - this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") + val aclAuthorizerClassName: String = classOf[AclAuthorizer].getName + def kafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, JaasTestUtils.KafkaServerPrincipalUnqualifiedName) + var superUserAdmin: Admin = _ override protected def securityProtocol = SecurityProtocol.SASL_SSL override protected lazy val trustStoreFile = Some(TestUtils.tempFile("truststore", ".jks")) - override def generateConfigs: Seq[KafkaConfig] = { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, authorizationAdmin.authorizerClassName) - super.generateConfigs - } - - override def configureSecurityBeforeServersStart(testInfo: TestInfo): Unit = { - authorizationAdmin.initializeAcls() - } - @BeforeEach override def setUp(testInfo: TestInfo): Unit = { + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, aclAuthorizerClassName) + this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") + this.serverConfig.setProperty(AclAuthorizer.SuperUsersProp, kafkaPrincipal.toString) + setUpSasl() super.setUp(testInfo) + setInitialAcls() } def setUpSasl(): Unit = { startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), Both, JaasTestUtils.KafkaServerContextName)) + + val loginContext = jaasAdminLoginModule("GSSAPI") + superuserClientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, loginContext) + } + + private def setInitialAcls(): Unit = { + superUserAdmin = createSuperuserAdminClient() + val ace = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, ALL, ALLOW) + superUserAdmin.createAcls(List(new AclBinding(new ResourcePattern(TOPIC, "*", LITERAL), ace)).asJava) + superUserAdmin.createAcls(List(new AclBinding(new ResourcePattern(GROUP, "*", LITERAL), ace)).asJava) + + val clusterAcls = List(clusterAcl(ALLOW, CREATE), + clusterAcl(ALLOW, DELETE), + clusterAcl(ALLOW, CLUSTER_ACTION), + clusterAcl(ALLOW, ALTER_CONFIGS), + clusterAcl(ALLOW, ALTER), + clusterAcl(ALLOW, IDEMPOTENT_WRITE)) + + superUserAdmin.createAcls(clusterAcls.map(ace => new AclBinding(clusterResourcePattern, ace)).asJava) + + brokers.foreach { b => + TestUtils.waitAndVerifyAcls(Set(ace), b.dataPlaneRequestProcessor.authorizer.get, new ResourcePattern(TOPIC, "*", LITERAL)) + TestUtils.waitAndVerifyAcls(Set(ace), b.dataPlaneRequestProcessor.authorizer.get, new ResourcePattern(GROUP, "*", LITERAL)) + TestUtils.waitAndVerifyAcls(clusterAcls.toSet, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) + } + } + + private def clusterAcl(permissionType: AclPermissionType, operation: AclOperation): AccessControlEntry = { + new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*").toString, + WILDCARD_HOST, operation, permissionType) } @AfterEach @@ -94,10 +118,10 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclOperations(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW)) - assertEquals(7, getAcls(AclBindingFilter.ANY).size) + assertEquals(8, getAcls(AclBindingFilter.ANY).size) val results = client.createAcls(List(acl2, acl3).asJava) assertEquals(Set(acl2, acl3), results.values.keySet().asScala) results.values.values.forEach(value => value.get) @@ -115,7 +139,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclOperations2(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val results = client.createAcls(List(acl2, acl2, transactionalIdAcl).asJava) assertEquals(Set(acl2, acl2, transactionalIdAcl), results.values.keySet.asScala) results.all.get() @@ -141,7 +165,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclDescribe(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.ANY), AccessControlEntryFilter.ANY) @@ -168,7 +192,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclDelete(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.MATCH), AccessControlEntryFilter.ANY) @@ -218,7 +242,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu //noinspection ScalaDeprecation - test explicitly covers clients using legacy / deprecated constructors @Test def testLegacyAclOpsNeverAffectOrReturnPrefixed(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) // <-- prefixed exists, but should never be returned. val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.MATCH), AccessControlEntryFilter.ANY) @@ -255,7 +279,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAttemptToCreateInvalidAcls(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient val clusterAcl = new AclBinding(new ResourcePattern(ResourceType.CLUSTER, "foobar", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.READ, AclPermissionType.ALLOW)) val emptyResourceNameAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "", PatternType.LITERAL), @@ -266,11 +290,6 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu assertFutureExceptionTypeEquals(results.values.get(emptyResourceNameAcl), classOf[InvalidRequestException]) } - override def configuredClusterPermissions: Set[AclOperation] = { - Set(AclOperation.ALTER, AclOperation.CREATE, AclOperation.CLUSTER_ACTION, AclOperation.ALTER_CONFIGS, - AclOperation.DESCRIBE, AclOperation.DESCRIBE_CONFIGS) - } - private def verifyCauseIsClusterAuth(e: Throwable): Unit = assertEquals(classOf[ClusterAuthorizationException], e.getCause.getClass) private def testAclCreateGetDelete(expectAuth: Boolean): Unit = { @@ -350,36 +369,53 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu @Test def testAclAuthorizationDenied(): Unit = { - client = Admin.create(createConfig) + client = createAdminClient // Test that we cannot create or delete ACLs when ALTER is denied. - authorizationAdmin.addClusterAcl(DENY, ALTER) + addClusterAcl(DENY, ALTER) testAclGet(expectAuth = true) testAclCreateGetDelete(expectAuth = false) // Test that we cannot do anything with ACLs when DESCRIBE and ALTER are denied. - authorizationAdmin.addClusterAcl(DENY, DESCRIBE) + addClusterAcl(DENY, DESCRIBE) testAclGet(expectAuth = false) testAclCreateGetDelete(expectAuth = false) // Test that we can create, delete, and get ACLs with the default ACLs. - authorizationAdmin.removeClusterAcl(DENY, DESCRIBE) - authorizationAdmin.removeClusterAcl(DENY, ALTER) + removeClusterAcl(DENY, DESCRIBE) + removeClusterAcl(DENY, ALTER) testAclGet(expectAuth = true) testAclCreateGetDelete(expectAuth = true) // Test that we can't do anything with ACLs without the ALLOW ALTER ACL in place. - authorizationAdmin.removeClusterAcl(ALLOW, ALTER) - authorizationAdmin.removeClusterAcl(ALLOW, DELETE) + removeClusterAcl(ALLOW, ALTER) + removeClusterAcl(ALLOW, DELETE) testAclGet(expectAuth = false) testAclCreateGetDelete(expectAuth = false) // Test that we can describe, but not alter ACLs, with only the ALLOW DESCRIBE ACL in place. - authorizationAdmin.addClusterAcl(ALLOW, DESCRIBE) + addClusterAcl(ALLOW, DESCRIBE) testAclGet(expectAuth = true) testAclCreateGetDelete(expectAuth = false) } + private def addClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = { + val ace = clusterAcl(permissionType, operation) + superUserAdmin.createAcls(List(new AclBinding(clusterResourcePattern, ace)).asJava) + brokers.foreach { b => + TestUtils.waitAndVerifyAcl(ace, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) + } + } + + private def removeClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = { + val ace = clusterAcl(permissionType, operation) + superUserAdmin.deleteAcls(List(new AclBinding(clusterResourcePattern, ace).toFilter).asJava).values + + brokers.foreach { b => + TestUtils.waitAndVerifyRemovedAcl(ace, b.dataPlaneRequestProcessor.authorizer.get, clusterResourcePattern) + } + } + @Test def testCreateTopicsResponseMetadataAndConfig(): Unit = { val topic1 = "mytopic1" @@ -387,7 +423,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu val denyAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, topic2, PatternType.LITERAL), new AccessControlEntry("User:*", "*", AclOperation.DESCRIBE_CONFIGS, AclPermissionType.DENY)) - client = Admin.create(createConfig) + client = createAdminClient client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get() val topics = Seq(topic1, topic2) @@ -474,54 +510,4 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu private def getAcls(allTopicAcls: AclBindingFilter) = { client.describeAcls(allTopicAcls).values.get().asScala.toSet } - - class AclAuthorizationAdmin(authorizerClass: Class[_ <: AclAuthorizer], authorizerForInitClass: Class[_ <: AclAuthorizer]) { - - def authorizerClassName: String = authorizerClass.getName - - def initializeAcls(): Unit = { - val authorizer = CoreUtils.createObject[Authorizer](authorizerForInitClass.getName) - try { - authorizer.configure(configs.head.originals()) - val ace = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, ALL, ALLOW) - authorizer.createAcls(null, List(new AclBinding(new ResourcePattern(TOPIC, "*", LITERAL), ace)).asJava) - authorizer.createAcls(null, List(new AclBinding(new ResourcePattern(GROUP, "*", LITERAL), ace)).asJava) - - authorizer.createAcls(null, List(clusterAcl(ALLOW, CREATE), - clusterAcl(ALLOW, DELETE), - clusterAcl(ALLOW, CLUSTER_ACTION), - clusterAcl(ALLOW, ALTER_CONFIGS), - clusterAcl(ALLOW, ALTER)) - .map(ace => new AclBinding(clusterResourcePattern, ace)).asJava) - } finally { - authorizer.close() - } - } - - def addClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = { - val ace = clusterAcl(permissionType, operation) - val aclBinding = new AclBinding(clusterResourcePattern, ace) - val authorizer = servers.head.dataPlaneRequestProcessor.authorizer.get - val prevAcls = authorizer.acls(new AclBindingFilter(clusterResourcePattern.toFilter, AccessControlEntryFilter.ANY)) - .asScala.map(_.entry).toSet - authorizer.createAcls(null, Collections.singletonList(aclBinding)) - TestUtils.waitAndVerifyAcls(prevAcls ++ Set(ace), authorizer, clusterResourcePattern) - } - - def removeClusterAcl(permissionType: AclPermissionType, operation: AclOperation): Unit = { - val ace = clusterAcl(permissionType, operation) - val authorizer = servers.head.dataPlaneRequestProcessor.authorizer.get - val clusterFilter = new AclBindingFilter(clusterResourcePattern.toFilter, AccessControlEntryFilter.ANY) - val prevAcls = authorizer.acls(clusterFilter).asScala.map(_.entry).toSet - val deleteFilter = new AclBindingFilter(clusterResourcePattern.toFilter, ace.toFilter) - assertFalse(authorizer.deleteAcls(null, Collections.singletonList(deleteFilter)) - .get(0).toCompletableFuture.get.aclBindingDeleteResults().asScala.head.exception.isPresent) - TestUtils.waitAndVerifyAcls(prevAcls -- Set(ace), authorizer, clusterResourcePattern) - } - - private def clusterAcl(permissionType: AclPermissionType, operation: AclOperation): AccessControlEntry = { - new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*").toString, - WILDCARD_HOST, operation, permissionType) - } - } } diff --git a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala index e1cf49b5af7bf..65906032ca4c1 100644 --- a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala @@ -14,16 +14,21 @@ package kafka.api import java.util import java.util.concurrent._ +import java.util.Properties + import com.yammer.metrics.core.Gauge import kafka.security.authorizer.AclAuthorizer import kafka.utils.TestUtils -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateAclsResult} +import org.apache.kafka.clients.admin.CreateAclsResult import org.apache.kafka.common.acl._ +import org.apache.kafka.common.config.SslConfigs +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceType} -import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} +import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal, SecurityProtocol, SslAuthenticationContext} +import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.common.network.Mode import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotNull, assertTrue} import org.junit.jupiter.api.{AfterEach, Test} @@ -35,6 +40,9 @@ object SslAdminIntegrationTest { @volatile var semaphore: Option[Semaphore] = None @volatile var executor: Option[ExecutorService] = None @volatile var lastUpdateRequestContext: Option[AuthorizableRequestContext] = None + val superuserCn = "super-user" + val serverUser = "server" + class TestableAclAuthorizer extends AclAuthorizer { override def createAcls(requestContext: AuthorizableRequestContext, aclBindings: util.List[AclBinding]): util.List[_ <: CompletionStage[AclCreateResult]] = { @@ -74,16 +82,31 @@ object SslAdminIntegrationTest { futures.asJava } } + + class TestPrincipalBuilder extends DefaultKafkaPrincipalBuilder(null, null) { + private val Pattern = "O=A (.*?),CN=(.*?)".r + + // Use fields from DN as server principal to grant authorisation for servers and super admin client + override def build(context: AuthenticationContext): KafkaPrincipal = { + val peerPrincipal = context.asInstanceOf[SslAuthenticationContext].session.getPeerPrincipal.getName + peerPrincipal match { + case Pattern(name, cn) => + val principal = if ((name == "server") || (cn == superuserCn)) "server" else KafkaPrincipal.ANONYMOUS.getName + new KafkaPrincipal(KafkaPrincipal.USER_TYPE, principal) + case _ => + KafkaPrincipal.ANONYMOUS + } + } + } } class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { - override val authorizationAdmin = new AclAuthorizationAdmin(classOf[SslAdminIntegrationTest.TestableAclAuthorizer], classOf[AclAuthorizer]) - - this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") + override val aclAuthorizerClassName: String = classOf[SslAdminIntegrationTest.TestableAclAuthorizer].getName + this.serverConfig.setProperty(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") + this.serverConfig.setProperty(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[SslAdminIntegrationTest.TestPrincipalBuilder].getName) override protected def securityProtocol = SecurityProtocol.SSL - override protected lazy val trustStoreFile = Some(TestUtils.tempFile("truststore", ".jks")) - private val adminClients = mutable.Buffer.empty[Admin] + override def kafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, SslAdminIntegrationTest.serverUser) override def setUpSasl(): Unit = { SslAdminIntegrationTest.semaphore = None @@ -100,7 +123,6 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { SslAdminIntegrationTest.semaphore = None semaphore.foreach(s => s.release(s.getQueueLength)) - adminClients.foreach(_.close()) super.tearDown() } @@ -206,7 +228,7 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { val testSemaphore = new Semaphore(0) SslAdminIntegrationTest.semaphore = Some(testSemaphore) - client = Admin.create(createConfig) + client = createAdminClient val results = client.createAcls(List(acl2, acl3).asJava).values assertEquals(Set(acl2, acl3), results.keySet().asScala) assertFalse(results.values.asScala.exists(_.isDone)) @@ -228,14 +250,6 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { validateRequestContext(SslAdminIntegrationTest.lastUpdateRequestContext.get, ApiKeys.DELETE_ACLS) } - private def createAdminClient: Admin = { - val config = createConfig - config.put(AdminClientConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, "40000") - val client = Admin.create(config) - adminClients += client - client - } - private def blockedRequestThreads: List[Thread] = { val requestThreads = Thread.getAllStackTraces.keySet.asScala .filter(_.getName.contains("data-plane-kafka-request-handler")) @@ -258,4 +272,12 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { assertTrue(metrics.nonEmpty, s"Unable to find metric $name: allMetrics: ${allMetrics.keySet.map(_.getMBeanName)}") metrics.map(_.asInstanceOf[Gauge[Int]].value).sum } + + // Override the CN to create a principal based on it + override def superuserSecurityProps(certAlias: String): Properties = { + val props = TestUtils.securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, certAlias, SslAdminIntegrationTest.superuserCn, + clientSaslProperties) + props.remove(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG) + props + } } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala index fa13715b87c20..3cb508958a482 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.util.ShutdownableThread -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -46,10 +46,10 @@ class TransactionsBounceTest extends IntegrationTestHarness { val overridingProps = new Properties() overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString) - overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) + overridingProps.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, serverMessageMaxBytes.toString) // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long - overridingProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + overridingProps.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) overridingProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) overridingProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString) diff --git a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala index ee39a764f2ece..a31385224fd08 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -208,7 +208,7 @@ class TransactionsExpirationTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 993e31dc5b0fd..779d5e6f6ade6 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -17,7 +17,6 @@ package kafka.api -import kafka.server.KafkaConfig import kafka.utils.TestUtils import kafka.utils.TestUtils.{consumeRecords, waitUntilTrue} import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerGroupMetadata, ConsumerRecord, OffsetAndMetadata} @@ -26,7 +25,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{InvalidProducerEpochException, ProducerFencedException, TimeoutException} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -67,7 +66,7 @@ class TransactionsTest extends IntegrationTestHarness { props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - props.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + props.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) props.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) props.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala index 4884fc2dc7b56..ae73cde5e848f 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -110,7 +110,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 1.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 1.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala index c42fbd6330e7b..8fa1d2dca6df6 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala @@ -17,13 +17,13 @@ */ package kafka.network -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.Defaults +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -43,8 +43,8 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.put(SocketServerConfigs.LISTENERS_CONFIG, s"$internal://localhost:0, $external://localhost:0") properties.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"$internal:PLAINTEXT, $external:PLAINTEXT") - properties.put(s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", "2") - properties.put(KafkaConfig.NumNetworkThreadsProp, Defaults.NUM_NETWORK_THREADS.toString) + properties.put(s"listener.name.${internal.toLowerCase}.${ServerConfigs.NUM_NETWORK_THREADS_CONFIG}", "2") + properties.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, ServerConfigs.NUM_NETWORK_THREADS_DEFAULT.toString) } @BeforeEach @@ -53,7 +53,7 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { admin = TestUtils.createAdminClient(brokers, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) assertEquals(2, getNumNetworkThreads(internal)) TestUtils.createTopicWithAdmin(admin, "test", brokers, controllerServers) - assertEquals(Defaults.NUM_NETWORK_THREADS, getNumNetworkThreads(external)) + assertEquals(ServerConfigs.NUM_NETWORK_THREADS_DEFAULT, getNumNetworkThreads(external)) } @AfterEach override def tearDown(): Unit = { @@ -71,10 +71,10 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { @ValueSource(strings = Array("zk", "kraft")) def testDynamicNumNetworkThreads(quorum: String): Unit = { // Increase the base network thread count - val newBaseNetworkThreadsCount = Defaults.NUM_NETWORK_THREADS + 1 + val newBaseNetworkThreadsCount = ServerConfigs.NUM_NETWORK_THREADS_DEFAULT + 1 var props = new Properties - props.put(KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString) - reconfigureServers(props, (KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString)) + props.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, newBaseNetworkThreadsCount.toString) + reconfigureServers(props, (ServerConfigs.NUM_NETWORK_THREADS_CONFIG, newBaseNetworkThreadsCount.toString)) // Only the external listener is changed assertEquals(2, getNumNetworkThreads(internal)) @@ -83,8 +83,8 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { // Increase the network thread count for internal val newInternalNetworkThreadsCount = 3 props = new Properties - props.put(s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", newInternalNetworkThreadsCount.toString) - reconfigureServers(props, (s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", newInternalNetworkThreadsCount.toString)) + props.put(s"listener.name.${internal.toLowerCase}.${ServerConfigs.NUM_NETWORK_THREADS_CONFIG}", newInternalNetworkThreadsCount.toString) + reconfigureServers(props, (s"listener.name.${internal.toLowerCase}.${ServerConfigs.NUM_NETWORK_THREADS_CONFIG}", newInternalNetworkThreadsCount.toString)) // The internal listener is changed assertEquals(newInternalNetworkThreadsCount, getNumNetworkThreads(internal)) diff --git a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala index f26f7079d4bfe..ccf4b84b696fe 100644 --- a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala +++ b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala @@ -28,6 +28,8 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, FetchParams, FetchPartitionData, LogOffsetMetadata, LogOffsetSnapshot} import org.junit.jupiter.api.Test import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito.{mock, when} @@ -46,7 +48,7 @@ class DelayedFetchTest { val fetchStatus = FetchPartitionStatus( startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId(), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) var fetchResultOpt: Option[FetchPartitionData] = None @@ -92,7 +94,7 @@ class DelayedFetchTest { val fetchStatus = FetchPartitionStatus( startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId(), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) var fetchResultOpt: Option[FetchPartitionData] = None @@ -116,6 +118,9 @@ class DelayedFetchTest { assertTrue(delayedFetch.tryComplete()) assertTrue(delayedFetch.isCompleted) assertTrue(fetchResultOpt.isDefined) + + val fetchResult = fetchResultOpt.get + assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, fetchResult.error) } @Test @@ -164,18 +169,71 @@ class DelayedFetchTest { assertTrue(delayedFetch.tryComplete()) assertTrue(delayedFetch.isCompleted) assertTrue(fetchResultOpt.isDefined) + + val fetchResult = fetchResultOpt.get + assertEquals(Errors.NONE, fetchResult.error) + } + + @ParameterizedTest(name = "testDelayedFetchWithMessageOnlyHighWatermark endOffset={0}") + @ValueSource(longs = Array(0, 500)) + def testDelayedFetchWithMessageOnlyHighWatermark(endOffset: Long): Unit = { + val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic") + val fetchOffset = 450L + val logStartOffset = 5L + val currentLeaderEpoch = Optional.of[Integer](10) + val replicaId = 1 + + val fetchStatus = FetchPartitionStatus( + startOffsetMetadata = new LogOffsetMetadata(fetchOffset), + fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) + + var fetchResultOpt: Option[FetchPartitionData] = None + def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { + fetchResultOpt = Some(responses.head._2) + } + + val delayedFetch = new DelayedFetch( + params = fetchParams, + fetchPartitionStatus = Seq(topicIdPartition -> fetchStatus), + replicaManager = replicaManager, + quota = replicaQuota, + responseCallback = callback + ) + + val partition: Partition = mock(classOf[Partition]) + when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)).thenReturn(partition) + // Note that the high-watermark does not contain the complete metadata + val endOffsetMetadata = new LogOffsetMetadata(endOffset, -1L, -1) + when(partition.fetchOffsetSnapshot( + currentLeaderEpoch, + fetchOnlyFromLeader = true)) + .thenReturn(new LogOffsetSnapshot(0L, endOffsetMetadata, endOffsetMetadata, endOffsetMetadata)) + when(replicaManager.isAddingReplica(any(), anyInt())).thenReturn(false) + expectReadFromReplica(fetchParams, topicIdPartition, fetchStatus.fetchInfo, Errors.NONE) + + // 1. When `endOffset` is 0, it refers to the truncation case + // 2. When `endOffset` is 500, we won't complete because it doesn't contain offset metadata + val expected = endOffset == 0 + assertEquals(expected, delayedFetch.tryComplete()) + assertEquals(expected, delayedFetch.isCompleted) + assertEquals(expected, fetchResultOpt.isDefined) + if (fetchResultOpt.isDefined) { + assertEquals(Errors.NONE, fetchResultOpt.get.error) + } } private def buildFollowerFetchParams( replicaId: Int, - maxWaitMs: Int + maxWaitMs: Int, + minBytes: Int = 1, ): FetchParams = { new FetchParams( ApiKeys.FETCH.latestVersion, replicaId, 1, maxWaitMs, - 1, + minBytes, maxBytes, FetchIsolation.LOG_END, Optional.empty() diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 379b454bc84f5..5f8dd68022fec 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -62,7 +62,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, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs} import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.ShutdownableThread @@ -542,7 +542,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "20000000") props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, "0.8") props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, "300000") - props.put(KafkaConfig.MessageMaxBytesProp, "40000") + props.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "40000") props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "50000000") props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "6000") @@ -634,7 +634,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, "60000") props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "10000000") props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, TimeUnit.DAYS.toMillis(1).toString) - props.put(KafkaConfig.MessageMaxBytesProp, "100000") + props.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "100000") props.put(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG, "10000") props.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, TimeUnit.DAYS.toMillis(1).toString) props.put(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, "60000") @@ -643,7 +643,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG, "delete") props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false") props.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "2") - props.put(KafkaConfig.CompressionTypeProp, "gzip") + props.put(ServerConfigs.COMPRESSION_TYPE_CONFIG, "gzip") props.put(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG, true.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000") @@ -869,15 +869,15 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } val config = servers.head.config - verifyThreadPoolResize(KafkaConfig.NumIoThreadsProp, config.numIoThreads, + verifyThreadPoolResize(ServerConfigs.NUM_IO_THREADS_CONFIG, config.numIoThreads, requestHandlerPrefix, mayReceiveDuplicates = false) verifyThreadPoolResize(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, config.numReplicaFetchers, fetcherThreadPrefix, mayReceiveDuplicates = false) - verifyThreadPoolResize(KafkaConfig.BackgroundThreadsProp, config.backgroundThreads, + verifyThreadPoolResize(ServerConfigs.BACKGROUND_THREADS_CONFIG, config.backgroundThreads, "kafka-scheduler-", mayReceiveDuplicates = false) verifyThreadPoolResize(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, config.numRecoveryThreadsPerDataDir, "", mayReceiveDuplicates = false) - verifyThreadPoolResize(KafkaConfig.NumNetworkThreadsProp, config.numNetworkThreads, + verifyThreadPoolResize(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, config.numNetworkThreads, networkThreadPrefix, mayReceiveDuplicates = true) verifyThreads("data-plane-kafka-socket-acceptor-", config.listeners.size) @@ -1965,7 +1965,7 @@ class TestMetricsReporter extends MetricsReporter with Reconfigurable with Close } override def configure(configs: util.Map[String, _]): Unit = { - configuredBrokers += configs.get(KafkaConfig.BrokerIdProp).toString.toInt + configuredBrokers += configs.get(ServerConfigs.BROKER_ID_CONFIG).toString.toInt configureCount += 1 pollingInterval = configs.get(PollingIntervalProp).toString.toInt } diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 67606c39e219e..f5db6bd1a9568 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -1541,6 +1541,44 @@ class KRaftClusterTest { cluster.close() } } + + @Test + def testControllerFailover(): Unit = { + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(5).build()).build() + try { + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + TestUtils.waitUntilTrue(() => cluster.brokers().get(0).brokerState == BrokerState.RUNNING, + "Broker never made it to RUNNING state.") + TestUtils.waitUntilTrue(() => cluster.raftManagers().get(0).client.leaderAndEpoch().leaderId.isPresent, + "RaftManager was not initialized.") + + val admin = Admin.create(cluster.clientProperties()) + try { + // Create a test topic + admin.createTopics(Collections.singletonList( + new NewTopic("test-topic", 1, 1.toShort))).all().get() + waitForTopicListing(admin, Seq("test-topic"), Seq()) + + // Shut down active controller + val active = cluster.waitForActiveController() + cluster.raftManagers().get(active.asInstanceOf[QuorumController].nodeId()).shutdown() + + // Create a test topic on the new active controller + admin.createTopics(Collections.singletonList( + new NewTopic("test-topic2", 1, 1.toShort))).all().get() + waitForTopicListing(admin, Seq("test-topic2"), Seq()) + } finally { + admin.close() + } + } finally { + cluster.close() + } + } } class BadAuthorizer extends Authorizer { diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index abc92c8c9267f..eb901f2ef025d 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -38,7 +38,7 @@ import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsem import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs} import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper} @@ -324,7 +324,7 @@ abstract class QuorumTestHarness extends Logging { props.putAll(overridingProps) props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString) props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") - props.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, "true") + props.setProperty(ServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, "true") if (props.getProperty(KRaftConfigs.NODE_ID_CONFIG) == null) { props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1000") } diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 5bf8f32884572..4f2daf3bfa6f3 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -184,6 +184,24 @@ final class KafkaMetadataLogTest { ) } + @Test + def testHighWatermarkOffsetMetadata(): Unit = { + val numberOfRecords = 10 + val epoch = 1 + val log = buildMetadataLog(tempDir, mockTime) + + append(log, numberOfRecords, epoch) + log.updateHighWatermark(new LogOffsetMetadata(numberOfRecords)) + + val highWatermarkMetadata = log.highWatermark + assertEquals(numberOfRecords, highWatermarkMetadata.offset) + assertTrue(highWatermarkMetadata.metadata.isPresent) + + val segmentPosition = highWatermarkMetadata.metadata.get().asInstanceOf[SegmentPosition] + assertEquals(0, segmentPosition.baseOffset) + assertTrue(segmentPosition.relativePosition > 0) + } + @Test def testCreateSnapshotBeforeLogStartOffset(): Unit = { val numberOfRecords = 10 diff --git a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala index 3327dc4073826..dbd84088b14f2 100644 --- a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.{AppInfoParser, SecurityUtils} import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.config.ServerConfigs import org.apache.log4j.Level import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -110,7 +111,7 @@ class AclCommandTest extends QuorumTestHarness with Logging { super.setUp(testInfo) brokerProps = TestUtils.createBrokerConfig(0, zkConnect) - brokerProps.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + brokerProps.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) brokerProps.put(AclAuthorizer.SuperUsersProp, "User:ANONYMOUS") zkArgs = Array("--authorizer-properties", "zookeeper.connect=" + zkConnect) diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index bb2e76e8b2d34..2ddbfa0c23c71 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.errors.{TopicDeletionDisabledException, UnknownTo import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -489,7 +490,7 @@ class DeleteTopicTest extends QuorumTestHarness { if (isKRaftTest()) { // Restart KRaft quorum with the updated config val overridingProps = new Properties() - overridingProps.put(KafkaConfig.DeleteTopicEnableProp, false.toString) + overridingProps.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, false.toString) if (implementation != null) implementation.shutdown() implementation = newKRaftQuorum(overridingProps) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala index a363ef7f5f9bf..d8a932c2f67fe 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_10_2_IV0, IBP_0_9_0, IBP_1_0_IV0, IBP_2_2_IV0, IBP_2_4_IV0, IBP_2_4_IV1, IBP_2_6_IV0, IBP_2_8_IV1, IBP_3_2_IV0, IBP_3_4_IV0} -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -895,7 +895,7 @@ class ControllerChannelManagerTest { private def createConfig(interBrokerVersion: MetadataVersion): KafkaConfig = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, controllerId.toString) + props.put(ServerConfigs.BROKER_ID_CONFIG, controllerId.toString) props.put(ZkConfigs.ZK_CONNECT_CONFIG, "zkConnect") TestUtils.setIbpAndMessageFormatVersions(props, interBrokerVersion) KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala index 16459cbed2f79..f184b625bda87 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala @@ -1664,6 +1664,134 @@ class GroupMetadataManagerTest { assertEquals(0, TestUtils.totalMetricValue(metrics, "offset-commit-count")) } + @Test + def testOffsetMetadataTooLargePartialFailure(): Unit = { + val memberId = "" + val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "foo") + val validTopicIdPartition = new TopicIdPartition(topicIdPartition.topicId, 1, "foo") + val offset = 37 + val requireStable = true; + + groupMetadataManager.addOwnedPartition(groupPartitionId) + val group = new GroupMetadata(groupId, Empty, time) + groupMetadataManager.addGroup(group) + + val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + val offsets = immutable.Map( + topicIdPartition -> OffsetAndMetadata(offset, "s" * (offsetConfig.maxMetadataSize + 1) , time.milliseconds()), + validTopicIdPartition -> OffsetAndMetadata(offset, "", time.milliseconds()) + ) + + expectAppendMessage(Errors.NONE) + + var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None + def callback(errors: immutable.Map[TopicIdPartition, Errors]): Unit = { + commitErrors = Some(errors) + } + + assertEquals(0, TestUtils.totalMetricValue(metrics, "offset-commit-count")) + groupMetadataManager.storeOffsets(group, memberId, offsetTopicPartition, offsets, callback, verificationGuard = None) + assertTrue(group.hasOffsets) + + assertEquals(Some(Map( + topicIdPartition -> Errors.OFFSET_METADATA_TOO_LARGE, + validTopicIdPartition -> Errors.NONE) + ), commitErrors) + + val cachedOffsets = groupMetadataManager.getOffsets( + groupId, + requireStable, + Some(Seq(topicIdPartition.topicPartition, validTopicIdPartition.topicPartition)) + ) + + assertEquals( + Some(OffsetFetchResponse.INVALID_OFFSET), + cachedOffsets.get(topicIdPartition.topicPartition).map(_.offset) + ) + assertEquals( + Some(Errors.NONE), + cachedOffsets.get(topicIdPartition.topicPartition).map(_.error) + ) + assertEquals( + Some(offset), + cachedOffsets.get(validTopicIdPartition.topicPartition).map(_.offset) + ) + + assertEquals(1, TestUtils.totalMetricValue(metrics, "offset-commit-count")) + } + + @Test + def testTransactionalCommitOffsetWithOffsetMetadataTooLargePartialFailure(): Unit = { + val memberId = "" + val foo0 = new TopicIdPartition(Uuid.randomUuid(), 0, "foo") + val foo1 = new TopicIdPartition(Uuid.randomUuid(), 1, "foo") + val producerId = 232L + val producerEpoch = 0.toShort + + groupMetadataManager.addOwnedPartition(groupPartitionId) + + val group = new GroupMetadata(groupId, Empty, time) + groupMetadataManager.addGroup(group) + + val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupMetadataManager.partitionFor(group.groupId)) + val offsets = immutable.Map( + foo0 -> OffsetAndMetadata(37, "", time.milliseconds()), + foo1 -> OffsetAndMetadata(38, "s" * (offsetConfig.maxMetadataSize + 1), time.milliseconds()) + ) + + val capturedResponseCallback: ArgumentCaptor[Map[TopicPartition, PartitionResponse] => Unit] = + ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) + when(replicaManager.getMagic(any())).thenReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE)) + var commitErrors: Option[immutable.Map[TopicIdPartition, Errors]] = None + + def callback(errors: immutable.Map[TopicIdPartition, Errors]): Unit = { + commitErrors = Some(errors) + } + + val verificationGuard = new VerificationGuard() + + groupMetadataManager.storeOffsets( + group, + memberId, + offsetTopicPartition, + offsets, + callback, + producerId, + producerEpoch, + verificationGuard = Some(verificationGuard) + ) + assertTrue(group.hasOffsets) + assertTrue(group.allOffsets.isEmpty) + + verify(replicaManager).appendRecords(anyLong(), + anyShort(), + any(), + any(), + any[Map[TopicPartition, MemoryRecords]], + capturedResponseCallback.capture(), + any[Option[ReentrantLock]], + any(), + any(), + any(), + ArgumentMatchers.eq(Map(offsetTopicPartition -> verificationGuard))) + verify(replicaManager).getMagic(any()) + capturedResponseCallback.getValue.apply(Map(groupTopicPartition -> + new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP, 0L))) + + assertEquals(Some(Map( + foo0 -> Errors.NONE, + foo1 -> Errors.OFFSET_METADATA_TOO_LARGE + )), commitErrors) + + assertTrue(group.hasOffsets) + assertTrue(group.allOffsets.isEmpty) + + group.completePendingTxnOffsetCommit(producerId, isCommit = true) + assertTrue(group.hasOffsets) + assertFalse(group.allOffsets.isEmpty) + assertEquals(offsets.get(foo0), group.offset(foo0.topicPartition)) + } + @Test def testExpireOffset(): Unit = { val memberId = "" diff --git a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala index b46abdc5cfd1c..4733c8ad9e4a8 100644 --- a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala @@ -24,7 +24,7 @@ import kafka.utils.{Logging, TestUtils} import scala.jdk.CollectionConverters._ import org.junit.jupiter.api.{BeforeEach, TestInfo} import com.yammer.metrics.core.Gauge -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -39,7 +39,7 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with private val createDeleteIterations = 3 private val overridingProps = new Properties - overridingProps.put(KafkaConfig.DeleteTopicEnableProp, "true") + overridingProps.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false") // speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation // But the replica.lag.time.max.ms value still need to consider the slow Jenkins testing environment diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala index a0ba0c478c76b..cf1f1a0617782 100644 --- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala @@ -372,6 +372,43 @@ class LocalLogTest { assertTrue(fetchDataInfo.records.records.asScala.isEmpty) } + @Test + def testWhenFetchOffsetHigherThanMaxOffset(): Unit = { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + for (offset <- 0 to 4) { + appendRecords(List(record), initialOffset = offset) + if (offset % 2 != 0) + log.roll() + } + assertEquals(3, log.segments.numberOfSegments) + + // case-0: valid case, `startOffset` < `maxOffsetMetadata.offset` + var fetchDataInfo = readRecords(startOffset = 3L, maxOffsetMetadata = new LogOffsetMetadata(4L, 4L, 0)) + assertEquals(1, fetchDataInfo.records.records.asScala.size) + assertEquals(new LogOffsetMetadata(3, 2L, 69), fetchDataInfo.fetchOffsetMetadata) + + // case-1: `startOffset` == `maxOffsetMetadata.offset` + fetchDataInfo = readRecords(startOffset = 4L, maxOffsetMetadata = new LogOffsetMetadata(4L, 4L, 0)) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + assertEquals(new LogOffsetMetadata(4L, 4L, 0), fetchDataInfo.fetchOffsetMetadata) + + // case-2: `startOffset` > `maxOffsetMetadata.offset` + fetchDataInfo = readRecords(startOffset = 5L, maxOffsetMetadata = new LogOffsetMetadata(4L, 4L, 0)) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + assertEquals(new LogOffsetMetadata(5L, 4L, 69), fetchDataInfo.fetchOffsetMetadata) + + // case-3: `startOffset` < `maxMessageOffset.offset` but `maxMessageOffset.messageOnlyOffset` is true + fetchDataInfo = readRecords(startOffset = 3L, maxOffsetMetadata = new LogOffsetMetadata(4L, -1L, -1)) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + assertEquals(new LogOffsetMetadata(3L, 2L, 69), fetchDataInfo.fetchOffsetMetadata) + + // case-4: `startOffset` < `maxMessageOffset.offset`, `maxMessageOffset.messageOnlyOffset` is false, but + // `maxOffsetMetadata.segmentBaseOffset` < `startOffset.segmentBaseOffset` + fetchDataInfo = readRecords(startOffset = 3L, maxOffsetMetadata = new LogOffsetMetadata(4L, 0L, 40)) + assertTrue(fetchDataInfo.records.records.asScala.isEmpty) + assertEquals(new LogOffsetMetadata(3L, 2L, 69), fetchDataInfo.fetchOffsetMetadata) + } + @Test def testTruncateTo(): Unit = { for (offset <- 0 to 11) { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 876082dd074ac..9f93c241ae39f 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_11_0_IV0, IBP_0_9_0} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.junit.jupiter.api.Assertions._ @@ -259,7 +260,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, cleanerConfig.dedupeBufferSize.toString) props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, cleanerConfig.dedupeBufferLoadFactor.toString) props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, cleanerConfig.ioBufferSize.toString) - props.put(KafkaConfig.MessageMaxBytesProp, cleanerConfig.maxMessageSize.toString) + props.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, cleanerConfig.maxMessageSize.toString) props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, cleanerConfig.backoffMs.toString) props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, cleanerConfig.maxIoBytesPerSecond.toString) KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 7e72a4852cafe..99b1e35e4eed9 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -19,7 +19,7 @@ package kafka.log import kafka.common._ import kafka.server.{BrokerTopicStats, KafkaConfig} -import kafka.utils._ +import kafka.utils.{CoreUtils, Logging, Pool, TestUtils} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig @@ -30,6 +30,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanerConfig, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetMap, ProducerStateManager, ProducerStateManagerConfig} +import org.apache.kafka.storage.internals.utils.Throttler import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.mockito.ArgumentMatchers @@ -59,7 +60,7 @@ class LogCleanerTest extends Logging { logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val time = new MockTime() - val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) + val throttler = new Throttler(Double.MaxValue, Long.MaxValue, "throttler", "entries", time) val tombstoneRetentionMs = 86400000 val largeTimestamp = Long.MaxValue - tombstoneRetentionMs - 1 val producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false) diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index e1c204bf871a7..1a781a93ea667 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -358,7 +358,7 @@ class LogLoaderTest { val wrapper = Mockito.spy(segment) Mockito.doAnswer { in => segmentsWithReads += wrapper - segment.read(in.getArgument(0, classOf[java.lang.Long]), in.getArgument(1, classOf[java.lang.Integer]), in.getArgument(2, classOf[java.lang.Long]), in.getArgument(3, classOf[java.lang.Boolean])) + segment.read(in.getArgument(0, classOf[java.lang.Long]), in.getArgument(1, classOf[java.lang.Integer]), in.getArgument(2, classOf[java.util.Optional[java.lang.Long]]), in.getArgument(3, classOf[java.lang.Boolean])) }.when(wrapper).read(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any()) Mockito.doAnswer { in => recoveredSegments += wrapper diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index abfa481016043..25d4fc8626b8c 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.storage.internals.log._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.CsvSource +import org.junit.jupiter.params.provider.{CsvSource, ValueSource} import java.io.{File, RandomAccessFile} import java.util @@ -144,6 +144,34 @@ class LogSegmentTest { checkEquals(ms2.records.iterator, read.records.records.iterator) } + @ParameterizedTest(name = "testReadWhenNoMaxPosition minOneMessage = {0}") + @ValueSource(booleans = Array(true, false)) + def testReadWhenNoMaxPosition(minOneMessage: Boolean): Unit = { + val maxPosition: Optional[java.lang.Long] = Optional.empty() + val maxSize = 1 + val seg = createSegment(40) + val ms = records(50, "hello", "there") + seg.append(51, RecordBatch.NO_TIMESTAMP, -1L, ms) + // read before first offset + var read = seg.read(48, maxSize, maxPosition, minOneMessage) + assertEquals(new LogOffsetMetadata(48, 40, 0), read.fetchOffsetMetadata) + assertTrue(read.records.records().iterator().asScala.isEmpty) + // read at first offset + read = seg.read(50, maxSize, maxPosition, minOneMessage) + assertEquals(new LogOffsetMetadata(50, 40, 0), read.fetchOffsetMetadata) + assertTrue(read.records.records().iterator().asScala.isEmpty) + // read at last offset + read = seg.read(51, maxSize, maxPosition, minOneMessage) + assertEquals(new LogOffsetMetadata(51, 40, 39), read.fetchOffsetMetadata) + assertTrue(read.records.records().iterator().asScala.isEmpty) + // read at log-end-offset + read = seg.read(52, maxSize, maxPosition, minOneMessage) + assertNull(read) + // read beyond log-end-offset + read = seg.read(53, maxSize, maxPosition, minOneMessage) + assertNull(read) + } + /** * In a loop append two messages then truncate off the second of those messages and check that we can read * the first but not the second message. @@ -331,7 +359,7 @@ class LogSegmentTest { writeNonsenseToFile(indexFile, 5, indexFile.length.toInt) seg.recover(newProducerStateManager(), Optional.empty()) for (i <- 0 until 100) { - val records = seg.read(i, 1, seg.size(), true).records.records + val records = seg.read(i, 1, Optional.of(seg.size()), true).records.records assertEquals(i, records.iterator.next().offset) } } diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 77b63bf89ac03..c31e1739f3115 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -20,7 +20,7 @@ package kafka.log import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.remote.RemoteLogManager import kafka.server.{BrokerTopicStats, KafkaConfig} -import kafka.utils._ +import kafka.utils.TestUtils import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.{InvalidRecordException, TopicPartition, Uuid} @@ -39,6 +39,7 @@ import org.apache.kafka.server.util.{KafkaScheduler, MockTime, 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, EpochEntry, FetchIsolation, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig, RecordValidationException, VerificationGuard} +import org.apache.kafka.storage.internals.utils.Throttler import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -1046,7 +1047,7 @@ class UnifiedLogTest { ioBufferSize = 64 * 1024, maxIoBufferSize = 64 * 1024, dupBufferLoadFactor = 0.75, - throttler = new Throttler(Double.MaxValue, Long.MaxValue, false, time = mockTime), + throttler = new Throttler(Double.MaxValue, Long.MaxValue, "throttler", "entries", mockTime), time = mockTime, checkDone = _ => {}) @@ -3245,7 +3246,7 @@ class UnifiedLogTest { val readInfo = segment.read(offsetMetadata.messageOffset, 2048, - segment.size, + Optional.of(segment.size), false) if (offsetMetadata.relativePositionInSegment < segment.size) @@ -4210,6 +4211,46 @@ class UnifiedLogTest { assertEquals(31, log.localLogStartOffset()) } + @Test + def testConvertToOffsetMetadataDoesNotThrowOffsetOutOfRangeError(): Unit = { + val logConfig = LogTestUtils.createLogConfig(localRetentionBytes = 1, fileDeleteDelayMs = 0, remoteLogStorageEnable = true) + val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) + + var offset = 0L + for(_ <- 0 until 50) { + val records = TestUtils.singletonRecords("test".getBytes()) + val info = log.appendAsLeader(records, leaderEpoch = 0) + offset = info.lastOffset + if (offset != 0 && offset % 10 == 0) + log.roll() + } + assertEquals(5, log.logSegments.size) + log.updateHighWatermark(log.logEndOffset) + // simulate calls to upload 3 segments to remote storage + log.updateHighestOffsetInRemoteStorage(30) + + log.deleteOldSegments() + assertEquals(2, log.logSegments.size()) + assertEquals(0, log.logStartOffset) + assertEquals(31, log.localLogStartOffset()) + + log.updateLogStartOffsetFromRemoteTier(15) + assertEquals(15, log.logStartOffset) + + // case-1: offset is higher than the local-log-start-offset. + // log-start-offset < local-log-start-offset < offset-to-be-converted < log-end-offset + assertEquals(new LogOffsetMetadata(35, 31, 288), log.maybeConvertToOffsetMetadata(35)) + // case-2: offset is less than the local-log-start-offset + // log-start-offset < offset-to-be-converted < local-log-start-offset < log-end-offset + assertEquals(new LogOffsetMetadata(29, -1L, -1), log.maybeConvertToOffsetMetadata(29)) + // case-3: offset is higher than the log-end-offset + // log-start-offset < local-log-start-offset < log-end-offset < offset-to-be-converted + assertEquals(new LogOffsetMetadata(log.logEndOffset + 1, -1L, -1), log.maybeConvertToOffsetMetadata(log.logEndOffset + 1)) + // case-4: offset is less than the log-start-offset + // offset-to-be-converted < log-start-offset < local-log-start-offset < log-end-offset + assertEquals(new LogOffsetMetadata(14, -1L, -1), log.maybeConvertToOffsetMetadata(14)) + } + private def appendTransactionalToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index ac8307bdf749c..3042064e0eb3e 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -38,7 +38,7 @@ import org.apache.kafka.common.utils._ import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.common.{Features, MetadataVersion} -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.apache.log4j.Level @@ -2051,7 +2051,7 @@ class SocketServerTest { val sslProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, interBrokerSecurityProtocol = Some(SecurityProtocol.SSL), trustStoreFile = Some(trustStoreFile)) sslProps.put(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:0") - sslProps.put(KafkaConfig.NumNetworkThreadsProp, "1") + sslProps.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, "1") sslProps } diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index e6153bd6c4c3a..3416ffe65b690 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.ProcessRole import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -64,7 +64,7 @@ class RaftManagerTest { } props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") - props.setProperty(KafkaConfig.BrokerIdProp, nodeId.toString) + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, nodeId.toString) new KafkaConfig(props) } diff --git a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala index f482d221763f3..c75ad5c66d152 100644 --- a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala +++ b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.network.Session import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST import org.apache.kafka.security.authorizer.AuthorizerUtils import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KafkaSecurityConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -59,8 +59,8 @@ class DelegationTokenManagerTest extends QuorumTestHarness { val tokenManagers = mutable.Buffer[DelegationTokenManager]() val secretKey = "secretKey" - val maxLifeTimeMsDefault = Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS - val renewTimeMsDefault = Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS + val maxLifeTimeMsDefault = DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_MS_DEFAULT + val renewTimeMsDefault = DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DEFAULT var tokenCache: DelegationTokenCache = _ var props: Properties = _ @@ -73,7 +73,7 @@ class DelegationTokenManagerTest extends QuorumTestHarness { super.setUp(testInfo) props = TestUtils.createBrokerConfig(0, zkConnect, enableToken = true) props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, ScramMechanism.mechanismNames().asScala.mkString(",")) - props.put(KafkaConfig.DelegationTokenSecretKeyProp, secretKey) + props.put(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, secretKey) tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames()) } diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala index 11b7ceb2df4b8..f9750618ae0ea 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherManagerTest.scala @@ -33,7 +33,6 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test} import org.mockito.Mockito.{mock, verify, when} -import java.util.Optional import scala.collection.{Map, Set, mutable} import scala.jdk.CollectionConverters._ @@ -313,12 +312,10 @@ class AbstractFetcherManagerTest { override def fetchEarliestLocalOffset(topicPartition: TopicPartition, currentLeaderEpoch: Int): OffsetAndEpoch = new OffsetAndEpoch(1L, 0) } - private class MockResizeFetcherTierStateMachine extends TierStateMachine { + private class MockResizeFetcherTierStateMachine extends TierStateMachine(null, null, false) { override def start(topicPartition: TopicPartition, currentFetchState: PartitionFetchState, fetchPartitionData: PartitionData): PartitionFetchState = { throw new UnsupportedOperationException("Materializing tier state is not supported in this test.") } - - override def maybeAdvanceState(tp: TopicPartition, currentFetchState: PartitionFetchState): Optional[PartitionFetchState] = Optional.empty[PartitionFetchState] } private class TestResizeFetcherThread(sourceBroker: BrokerEndPoint, failedPartitions: FailedPartitions, fetchTierStateMachine: TierStateMachine) diff --git a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala index e35561ef63990..ebe4bd05f3ffb 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.message.MetadataRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.assertEquals abstract class AbstractMetadataRequestTest extends BaseRequestTest { @@ -32,7 +32,7 @@ abstract class AbstractMetadataRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(ServerConfigs.BROKER_ID_CONFIG)}") } protected def requestData(topics: List[String], allowAutoTopicCreation: Boolean): MetadataRequestData = { diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala index 4449857c7b20d..dc659dea941f0 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala @@ -22,7 +22,7 @@ import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse} -import org.apache.kafka.server.config.KafkaSecurityConfigs +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -36,8 +36,8 @@ import scala.jdk.CollectionConverters._ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) + properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) } diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala index b84f6761f5968..bf7331a5984c0 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala @@ -34,8 +34,8 @@ import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrinci import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.junit.jupiter.api.{Test, BeforeEach, TestInfo} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs} +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -55,16 +55,16 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { if (TestInfoUtils.isKRaft(testInfo)) { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) if (testInfo.getDisplayName.contains("quorum=kraft-IBP_3_4")) { testMetadataVersion = MetadataVersion.IBP_3_4_IV0 } } else { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) } this.serverConfig.setProperty(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") super.setUp(testInfo) } diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala index 9ee92469e53f9..d2bb8ea715dd5 100644 --- a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -38,6 +38,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerd import org.apache.kafka.common.utils.{SecurityUtils, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test} @@ -65,7 +66,7 @@ class AutoTopicCreationManagerTest { @BeforeEach def setup(): Unit = { val props = TestUtils.createBrokerConfig(1, "localhost") - props.setProperty(KafkaConfig.RequestTimeoutMsProp, requestTimeout.toString) + props.setProperty(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString) props.setProperty(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString) diff --git a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala index f17b27e1ee54a..cb42739551601 100644 --- a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.Record import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.AfterEach import java.util @@ -36,7 +37,7 @@ class BaseFetchRequestTest extends BaseRequestTest { protected var producer: KafkaProducer[String, String] = _ override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.FetchMaxBytes, Int.MaxValue.toString) + properties.put(ServerConfigs.FETCH_MAX_BYTES_CONFIG, Int.MaxValue.toString) } @AfterEach diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index eddeeb4253d5e..4c1494380ea60 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader} import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.config.ServerConfigs import java.io.{DataInputStream, DataOutputStream} import java.net.Socket @@ -43,7 +44,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness { override def modifyConfigs(props: Seq[Properties]): Unit = { props.foreach { p => - p.put(KafkaConfig.ControlledShutdownEnableProp, "false") + p.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") brokerPropertyOverrides(p) } } diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index 7752f0dae2249..34f9d139a03cc 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -232,11 +232,11 @@ class BrokerLifecycleManagerTest { poll(ctx, manager, prepareResponse[BrokerHeartbeatRequest](ctx, new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData()))) .data().offlineLogDirs().asScala.map(_.toString).toSet assertEquals(Set.empty, nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("h3sC4Yk-Q9-fd0ntJTocCA")) + manager.propagateDirectoryFailure(Uuid.fromString("h3sC4Yk-Q9-fd0ntJTocCA"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA"), nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("ej8Q9_d2Ri6FXNiTxKFiow")) + manager.propagateDirectoryFailure(Uuid.fromString("ej8Q9_d2Ri6FXNiTxKFiow"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow"), nextHeartbeatDirs()) - manager.propagateDirectoryFailure(Uuid.fromString("1iF76HVNRPqC7Y4r6647eg")) + manager.propagateDirectoryFailure(Uuid.fromString("1iF76HVNRPqC7Y4r6647eg"), Integer.MAX_VALUE) assertEquals(Set("h3sC4Yk-Q9-fd0ntJTocCA", "ej8Q9_d2Ri6FXNiTxKFiow", "1iF76HVNRPqC7Y4r6647eg"), nextHeartbeatDirs()) manager.close() } diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 84a58278a42f3..83439cc28b8b4 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -54,7 +54,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock} -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs} import org.apache.kafka.server.util.FutureUtils import org.apache.kafka.storage.internals.log.CleanerConfig import org.junit.jupiter.api.Assertions._ @@ -890,7 +890,7 @@ class ControllerApisTest { val controller = new MockController.Builder(). newInitialTopic("foo", fooId).build() val props = new Properties() - props.put(KafkaConfig.DeleteTopicEnableProp, "false") + props.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "false") controllerApis = createControllerApis(None, controller, props) val request = new DeleteTopicsRequestData() request.topics().add(new DeleteTopicState().setName("foo").setTopicId(ZERO_UUID)) diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala index f4b7f6bcd8d5d..db31be92c5400 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.common.requests.DeleteTopicsResponse import org.apache.kafka.common.security.auth.AuthenticationContext import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, QuotaConfigs} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions.assertEquals @@ -95,7 +95,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest { override def brokerCount: Int = 1 override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala index 1ef6e37720204..6081a0fe3d0c8 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.errors.InvalidPrincipalTypeException import org.apache.kafka.common.errors.DelegationTokenNotFoundException import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.SecurityUtils +import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -43,11 +44,11 @@ class DelegationTokenRequestsTest extends IntegrationTestHarness with SaslSetup override def brokerCount = 1 - this.serverConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") // Remove expired tokens every minute. - this.serverConfig.setProperty(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp, "5000") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp, "5000") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, "5000") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, "5000") @BeforeEach override def setUp(testInfo: TestInfo): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala index e377256ccc67b..91ea2e8840e42 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala @@ -18,12 +18,12 @@ package kafka.server import java.util.Collections - import kafka.utils._ import org.apache.kafka.common.message.DeleteTopicsRequestData import org.apache.kafka.common.network.ListenerName 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.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -34,7 +34,7 @@ class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest { override def kraftControllerConfigs() = { val props = super.kraftControllerConfigs() - props.head.setProperty(KafkaConfig.DeleteTopicEnableProp, "false") + props.head.setProperty(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "false") props } diff --git a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala index eb6acd8d73e57..0148d29061e80 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -40,7 +40,7 @@ class DescribeClusterRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(ServerConfigs.BROKER_ID_CONFIG)}") } @BeforeEach diff --git a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala index e8a84048b1341..bca74ecb06ddf 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala @@ -21,7 +21,7 @@ import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse} import org.apache.kafka.metadata.authorizer.StandardAuthorizer -import org.apache.kafka.server.config.KafkaSecurityConfigs +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -33,11 +33,11 @@ import java.util.Properties */ class DescribeUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[DescribeCredentialsTest.TestAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[DescribeCredentialsTest.TestAuthorizer].getName) } properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) } diff --git a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala index a70cb9cfeeb9e..ef83a4ac36408 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala @@ -28,8 +28,8 @@ import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, De import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal} import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.junit.jupiter.api.{Test, BeforeEach, TestInfo} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs} +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -45,13 +45,13 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { if (TestInfoUtils.isKRaft(testInfo)) { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) + this.serverConfig.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) } this.serverConfig.setProperty(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") super.setUp(testInfo) } diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 945d17b69c0a7..85be6826423ec 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -37,7 +37,7 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.PasswordEncoderConfigs import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.config.{Defaults, KRaftConfigs, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs} import org.apache.kafka.server.util.KafkaScheduler @@ -134,11 +134,11 @@ class DynamicBrokerConfigTest { @Test def testUpdateDynamicThreadPool(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - origProps.put(KafkaConfig.NumIoThreadsProp, "4") - origProps.put(KafkaConfig.NumNetworkThreadsProp, "2") + origProps.put(ServerConfigs.NUM_IO_THREADS_CONFIG, "4") + origProps.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, "2") origProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") origProps.put(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, "1") - origProps.put(KafkaConfig.BackgroundThreadsProp, "3") + origProps.put(ServerConfigs.BACKGROUND_THREADS_CONFIG, "3") val config = KafkaConfig(origProps) val serverMock = Mockito.mock(classOf[KafkaBroker]) @@ -165,18 +165,18 @@ class DynamicBrokerConfigTest { val props = new Properties() - props.put(KafkaConfig.NumIoThreadsProp, "8") + props.put(ServerConfigs.NUM_IO_THREADS_CONFIG, "8") config.dynamicConfig.updateDefaultConfig(props) assertEquals(8, config.numIoThreads) Mockito.verify(handlerPoolMock).resizeThreadPool(newSize = 8) - props.put(KafkaConfig.NumNetworkThreadsProp, "4") + props.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, "4") config.dynamicConfig.updateDefaultConfig(props) assertEquals(4, config.numNetworkThreads) val captor: ArgumentCaptor[JMap[String, String]] = ArgumentCaptor.forClass(classOf[JMap[String, String]]) Mockito.verify(acceptorMock).reconfigure(captor.capture()) - assertTrue(captor.getValue.containsKey(KafkaConfig.NumNetworkThreadsProp)) - assertEquals(4, captor.getValue.get(KafkaConfig.NumNetworkThreadsProp)) + assertTrue(captor.getValue.containsKey(ServerConfigs.NUM_NETWORK_THREADS_CONFIG)) + assertEquals(4, captor.getValue.get(ServerConfigs.NUM_NETWORK_THREADS_CONFIG)) props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") config.dynamicConfig.updateDefaultConfig(props) @@ -188,7 +188,7 @@ class DynamicBrokerConfigTest { assertEquals(2, config.numRecoveryThreadsPerDataDir) Mockito.verify(logManagerMock).resizeRecoveryThreadPool(newSize = 2) - props.put(KafkaConfig.BackgroundThreadsProp, "6") + props.put(ServerConfigs.BACKGROUND_THREADS_CONFIG, "6") config.dynamicConfig.updateDefaultConfig(props) assertEquals(6, config.backgroundThreads) Mockito.verify(schedulerMock).resizeThreadPool(6) @@ -263,7 +263,7 @@ class DynamicBrokerConfigTest { def testReconfigurableValidation(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val config = KafkaConfig(origProps) - val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, KafkaConfig.BrokerIdProp, "some.prop") + 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") def createReconfigurable(configs: Set[String]) = new Reconfigurable { @@ -616,15 +616,15 @@ class DynamicBrokerConfigTest { when(zkClient.getEntityConfigs(anyString(), anyString())).thenReturn(new java.util.Properties()) val initialProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) - initialProps.remove(KafkaConfig.BackgroundThreadsProp) + initialProps.remove(ServerConfigs.BACKGROUND_THREADS_CONFIG) val oldConfig = KafkaConfig.fromProps(initialProps) val dynamicBrokerConfig = new DynamicBrokerConfig(oldConfig) dynamicBrokerConfig.initialize(Some(zkClient), None) dynamicBrokerConfig.addBrokerReconfigurable(new TestDynamicThreadPool) val newprops = new Properties() - newprops.put(KafkaConfig.NumIoThreadsProp, "10") - newprops.put(KafkaConfig.BackgroundThreadsProp, "100") + newprops.put(ServerConfigs.NUM_IO_THREADS_CONFIG, "10") + newprops.put(ServerConfigs.BACKGROUND_THREADS_CONFIG, "100") dynamicBrokerConfig.updateBrokerConfig(0, newprops) } @@ -639,7 +639,7 @@ class DynamicBrokerConfigTest { var newProps = new Properties() newProps.put(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, "9999") - newProps.put(KafkaConfig.MessageMaxBytesProp, "2222") + newProps.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "2222") config.dynamicConfig.updateDefaultConfig(newProps) assertEquals(9999, config.maxConnections) @@ -647,7 +647,7 @@ class DynamicBrokerConfigTest { newProps = new Properties() newProps.put(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, "INVALID_INT") - newProps.put(KafkaConfig.MessageMaxBytesProp, "1111") + newProps.put(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "1111") config.dynamicConfig.updateDefaultConfig(newProps) // Invalid value should be skipped and reassigned as default value @@ -846,8 +846,8 @@ class TestDynamicThreadPool() extends BrokerReconfigurable { } override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { - assertEquals(Defaults.NUM_IO_THREADS, oldConfig.numIoThreads) - assertEquals(Defaults.BACKGROUND_THREADS, oldConfig.backgroundThreads) + assertEquals(ServerConfigs.NUM_IO_THREADS_DEFAULT, oldConfig.numIoThreads) + assertEquals(ServerConfigs.BACKGROUND_THREADS_DEFAULT, oldConfig.backgroundThreads) assertEquals(10, newConfig.numIoThreads) assertEquals(100, newConfig.backgroundThreads) diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala index 1c5d0a76f0f8a..6a7aa13fd4f4a 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} +import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -74,7 +75,7 @@ class FetchRequestMaxBytesTest extends BaseRequestTest { override protected def brokerPropertyOverrides(properties: Properties): Unit = { super.brokerPropertyOverrides(properties) - properties.put(KafkaConfig.FetchMaxBytes, "1024") + properties.put(ServerConfigs.FETCH_MAX_BYTES_CONFIG, "1024") } private def createTopics(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index ce7acfe821f8d..742635578adda 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -1180,7 +1180,7 @@ class KafkaApisTest extends Logging { val requestTimeout = 10 val topicConfigOverride = mutable.Map.empty[String, String] - topicConfigOverride.put(KafkaConfig.RequestTimeoutMsProp, requestTimeout.toString) + topicConfigOverride.put(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) val groupId = "group" val topicName = @@ -3155,7 +3155,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.COMMIT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) when(groupCoordinator.completeTransaction( @@ -3164,7 +3164,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.ABORT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) val entriesPerPartition: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = @@ -3173,7 +3173,7 @@ class KafkaApisTest extends Logging { ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( - ArgumentMatchers.eq(Defaults.REQUEST_TIMEOUT_MS.toLong), + ArgumentMatchers.eq(ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT.toLong), ArgumentMatchers.eq(-1), ArgumentMatchers.eq(true), ArgumentMatchers.eq(AppendOrigin.COORDINATOR), @@ -3274,7 +3274,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.COMMIT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(ServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(FutureUtils.failedFuture[Void](error.exception())) kafkaApis = createKafkaApis(overrideProperties = Map( GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 8513854be7b4f..482393e2bf236 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -40,13 +40,14 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.PasswordEncoderConfigs import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1} -import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, KafkaSecurityConfigs, ServerConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.storage.internals.log.CleanerConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.api.function.Executable + import scala.annotation.nowarn import scala.jdk.CollectionConverters._ @@ -158,7 +159,7 @@ class KafkaConfigTest { val port = 9999 val hostName = "fake-host" val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"PLAINTEXT://$hostName:$port") val serverConfig = KafkaConfig.fromProps(props) @@ -189,7 +190,7 @@ class KafkaConfigTest { @Test def testDuplicateListeners(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") // listeners with duplicate port @@ -215,7 +216,7 @@ class KafkaConfigTest { @Test def testIPv4AndIPv6SamePortListeners(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") + props.put(ServerConfigs.BROKER_ID_CONFIG, "1") props.put(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://[::1]:9092,SSL://[::1]:9092") @@ -455,7 +456,7 @@ class KafkaConfigTest { @Test def testControllerListenerNameDoesNotMapToPlaintextByDefaultForNonKRaft(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://localhost:9092") assertBadConfigContainingMessage(props, @@ -468,7 +469,7 @@ class KafkaConfigTest { @Test def testBadListenerProtocol(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "BAD://localhost:9091") @@ -478,7 +479,7 @@ class KafkaConfigTest { @Test def testListenerNamesWithAdvertisedListenerUnset(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CLIENT://localhost:9091,REPLICATION://localhost:9092,INTERNAL://localhost:9093") @@ -502,7 +503,7 @@ class KafkaConfigTest { @Test def testListenerAndAdvertisedListenerNames(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "EXTERNAL://localhost:9091,INTERNAL://localhost:9093") @@ -533,7 +534,7 @@ class KafkaConfigTest { @Test def testListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091,REPLICATION://localhost:9092") @@ -544,7 +545,7 @@ class KafkaConfigTest { @Test def testInterBrokerListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091") @@ -555,7 +556,7 @@ class KafkaConfigTest { @Test def testInterBrokerListenerNameAndSecurityProtocolSet(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091") @@ -567,7 +568,7 @@ class KafkaConfigTest { @Test def testCaseInsensitiveListenerProtocol(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "plaintext://localhost:9091,SsL://localhost:9092") val config = KafkaConfig.fromProps(props) @@ -582,7 +583,7 @@ class KafkaConfigTest { @Test def testListenerDefaults(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") // configuration with no listeners @@ -596,7 +597,7 @@ class KafkaConfigTest { @Test def testVersionConfiguration(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") val conf = KafkaConfig.fromProps(props) assertEquals(MetadataVersion.latestProduction, conf.interBrokerProtocolVersion) @@ -706,31 +707,31 @@ class KafkaConfigTest { @Test def testInvalidCompressionType(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "abc") + props.setProperty(ServerConfigs.COMPRESSION_TYPE_CONFIG, "abc") assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidGzipCompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "gzip") - props.setProperty(KafkaConfig.CompressionGzipLevelProp, (GzipCompression.MAX_LEVEL + 1).toString) + props.setProperty(ServerConfigs.COMPRESSION_TYPE_CONFIG, "gzip") + props.setProperty(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG, (GzipCompression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidLz4CompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "lz4") - props.setProperty(KafkaConfig.CompressionLz4LevelProp, (Lz4Compression.MAX_LEVEL + 1).toString) + props.setProperty(ServerConfigs.COMPRESSION_TYPE_CONFIG, "lz4") + props.setProperty(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG, (Lz4Compression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidZstdCompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "zstd") - props.setProperty(KafkaConfig.CompressionZstdLevelProp, (ZstdCompression.MAX_LEVEL + 1).toString) + props.setProperty(ServerConfigs.COMPRESSION_TYPE_CONFIG, "zstd") + props.setProperty(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG, (ZstdCompression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @@ -820,16 +821,16 @@ class KafkaConfigTest { case ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean") case ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean") - case KafkaConfig.BrokerIdProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.NumReplicaAlterLogDirsThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.QueuedMaxBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.RequestTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ConnectionSetupTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ConnectionSetupTimeoutMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.BROKER_ID_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.NUM_NETWORK_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.NUM_IO_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.BACKGROUND_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.QUEUED_MAX_REQUESTS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.QUEUED_MAX_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") // KRaft mode configs case KRaftConfigs.PROCESS_ROLES_CONFIG => // ignore @@ -845,7 +846,7 @@ class KafkaConfigTest { case KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG => // ignore string case KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.AuthorizerClassNameProp => //ignore string + case ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG => //ignore string case ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG => //ignore string case SocketServerConfigs.SOCKET_SEND_BUFFER_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -909,9 +910,9 @@ class KafkaConfigTest { case ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") - case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") + case ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") case GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case GroupCoordinatorConfig.GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -934,18 +935,18 @@ class KafkaConfigTest { case TransactionLogConfigs.TRANSACTIONS_TOPIC_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") case TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") case QuotaConfigs.QUOTA_WINDOW_SIZE_SECONDS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") + case ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") case MetricConfigs.METRIC_NUM_SAMPLES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") case MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") case MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG => // ignore string case MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG => // ignore string + case ServerConfigs.BROKER_RACK_CONFIG => // ignore string - case KafkaConfig.CompressionGzipLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.CompressionLz4LevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.CompressionZstdLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1) + case ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1) - case KafkaConfig.RackProp => // ignore string //SSL Configs case KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => @@ -1016,11 +1017,11 @@ class KafkaConfigTest { case PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") //delegation token configs - case KafkaConfig.DelegationTokenSecretKeyAliasProp => // ignore - case KafkaConfig.DelegationTokenSecretKeyProp => // ignore - case KafkaConfig.DelegationTokenMaxLifeTimeProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DelegationTokenExpiryTimeMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG => // ignore + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG => // ignore + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") //Kafka Yammer metrics reporter configs case MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG => // ignore @@ -1180,9 +1181,9 @@ class KafkaConfigTest { defaults.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "127.0.0.1:2181") // For ZkConnectionTimeoutMs defaults.setProperty(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, "1234") - defaults.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") - defaults.setProperty(KafkaConfig.MaxReservedBrokerIdProp, "1") - defaults.setProperty(KafkaConfig.BrokerIdProp, "1") + defaults.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") + defaults.setProperty(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, "1") + defaults.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") defaults.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:1122") defaults.setProperty(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG, "127.0.0.1:2, 127.0.0.2:3") defaults.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, "/tmp1,/tmp2") @@ -1215,7 +1216,7 @@ class KafkaConfigTest { assertEquals(24 * 60L * 60L * 1000L, config.delegationTokenExpiryTimeMs) assertEquals(1 * 60L * 1000L * 60, config.delegationTokenExpiryCheckIntervalMs) - defaults.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "1234567890") + defaults.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "1234567890") val config1 = KafkaConfig.fromProps(defaults) assertEquals(true, config1.tokenAuthEnabled) } @@ -1436,7 +1437,7 @@ class KafkaConfigTest { // -1 is the default for both node.id and broker.id val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) } @@ -1473,7 +1474,7 @@ class KafkaConfigTest { props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.setProperty(KRaftConfigs.NODE_ID_CONFIG, negativeTwoNodeId.toString) - props.setProperty(KafkaConfig.BrokerIdProp, negativeTwoNodeId.toString) + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, negativeTwoNodeId.toString) assertFalse(isValidKafkaConfig(props)) } @@ -1486,7 +1487,7 @@ class KafkaConfigTest { val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") KafkaConfig.fromProps(props) } @@ -1496,7 +1497,7 @@ class KafkaConfigTest { val props = TestUtils.createBrokerConfig(-1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") assertFalse(isValidKafkaConfig(props)) } @@ -1564,9 +1565,9 @@ class KafkaConfigTest { @Test def testPopulateSynonymsOnMapWithoutNodeId(): Unit = { val input = new util.HashMap[String, String]() - input.put(KafkaConfig.BrokerIdProp, "4") + input.put(ServerConfigs.BROKER_ID_CONFIG, "4") val expectedOutput = new util.HashMap[String, String]() - expectedOutput.put(KafkaConfig.BrokerIdProp, "4") + expectedOutput.put(ServerConfigs.BROKER_ID_CONFIG, "4") expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4") assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) } @@ -1576,7 +1577,7 @@ class KafkaConfigTest { val input = new util.HashMap[String, String]() input.put(KRaftConfigs.NODE_ID_CONFIG, "4") val expectedOutput = new util.HashMap[String, String]() - expectedOutput.put(KafkaConfig.BrokerIdProp, "4") + expectedOutput.put(ServerConfigs.BROKER_ID_CONFIG, "4") expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4") assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) } @@ -1584,7 +1585,7 @@ class KafkaConfigTest { @Test def testNodeIdMustNotBeDifferentThanBrokerId(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") assertEquals("You must set `node.id` to the same value as `broker.id`.", assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage()) @@ -1605,13 +1606,13 @@ class KafkaConfigTest { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") - props.setProperty(KafkaConfig.BrokerIdProp, "3") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "3") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") val config = KafkaConfig.fromProps(props) assertEquals(3, config.brokerId) assertEquals(3, config.nodeId) val originals = config.originals() - assertEquals("3", originals.get(KafkaConfig.BrokerIdProp)) + assertEquals("3", originals.get(ServerConfigs.BROKER_ID_CONFIG)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) } @@ -1632,7 +1633,7 @@ class KafkaConfigTest { assertEquals(3, config.brokerId) assertEquals(3, config.nodeId) val originals = config.originals() - assertEquals("3", originals.get(KafkaConfig.BrokerIdProp)) + assertEquals("3", originals.get(ServerConfigs.BROKER_ID_CONFIG)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) } @@ -1649,9 +1650,9 @@ class KafkaConfigTest { def testInvalidAuthorizerClassName(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val configs = new util.HashMap[Object, Object](props) - configs.put(KafkaConfig.AuthorizerClassNameProp, null) + configs.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, null) val ce = assertThrows(classOf[ConfigException], () => KafkaConfig.apply(configs)) - assertTrue(ce.getMessage.contains(KafkaConfig.AuthorizerClassNameProp)) + assertTrue(ce.getMessage.contains(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG)) } @Test @@ -1678,7 +1679,7 @@ class KafkaConfigTest { def testEarlyStartListeners(): Unit = { val props = new Properties() props.putAll(kraftProps()) - props.setProperty(KafkaConfig.EarlyStartListenersProp, "INTERNAL,INTERNAL2") + props.setProperty(ServerConfigs.EARLY_START_LISTENERS_CONFIG, "INTERNAL,INTERNAL2") props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "INTERNAL") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "INTERNAL:PLAINTEXT,INTERNAL2:PLAINTEXT,CONTROLLER:PLAINTEXT") @@ -1693,7 +1694,7 @@ class KafkaConfigTest { def testEarlyStartListenersMustBeListeners(): Unit = { val props = new Properties() props.putAll(kraftProps()) - props.setProperty(KafkaConfig.EarlyStartListenersProp, "INTERNAL") + props.setProperty(ServerConfigs.EARLY_START_LISTENERS_CONFIG, "INTERNAL") assertEquals("early.start.listeners contains listener INTERNAL, but this is not " + "contained in listeners or controller.listener.names", assertThrows(classOf[ConfigException], () => new KafkaConfig(props)).getMessage) @@ -1765,7 +1766,7 @@ class KafkaConfigTest { KafkaConfig.fromProps(props) // Check that we allow authorizer to be set - props.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getCanonicalName) + props.setProperty(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getCanonicalName) KafkaConfig.fromProps(props) // Don't allow migration startup with an older IBP @@ -1892,21 +1893,14 @@ class KafkaConfigTest { } } - @Test - def testMultipleLogDirectoriesNotSupportedWithRemoteLogStorage(): Unit = { - val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true)) - props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a,/tmp/b") - - val caught = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) - assertTrue(caught.getMessage.contains("Multiple log directories `/tmp/a,/tmp/b` are not supported when remote log storage is enabled")) - } - @Test def testSingleLogDirectoryWithRemoteLogStorage(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, String.valueOf(true)) props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a") assertDoesNotThrow(() => KafkaConfig.fromProps(props)) + + props.put(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/a,/tmp/b") + assertDoesNotThrow(() => KafkaConfig.fromProps(props)) } } diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala index a27a49f9b2aa9..9cc0204210712 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala @@ -21,6 +21,7 @@ import kafka.metrics.KafkaMetricsReporter import kafka.utils.{CoreUtils, TestUtils, VerifiableProperties} import kafka.server.QuorumTestHarness import org.apache.kafka.common.{ClusterResource, ClusterResourceListener} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.test.MockMetricsReporter import org.junit.jupiter.api.Assertions._ @@ -61,7 +62,7 @@ object KafkaMetricReporterClusterIdTest { // Because this code is run during the test setUp phase, if we throw an exception here, // it just results in the test itself being declared "not found" rather than failing. // So we track an error message which we will check later in the test body. - val brokerId = configs.get(KafkaConfig.BrokerIdProp) + val brokerId = configs.get(ServerConfigs.BROKER_ID_CONFIG) if (brokerId == null) setupError.compareAndSet("", "No value was set for the broker id.") else if (!brokerId.isInstanceOf[String]) @@ -85,8 +86,8 @@ class KafkaMetricReporterClusterIdTest extends QuorumTestHarness { val props = TestUtils.createBrokerConfig(1, zkConnect) props.setProperty(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricReporterClusterIdTest$MockKafkaMetricsReporter") props.setProperty(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricReporterClusterIdTest$MockBrokerMetricsReporter") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") - props.setProperty(KafkaConfig.BrokerIdProp, "-1") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "true") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "-1") config = KafkaConfig.fromProps(props) server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName)) server.startup() diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala index 9886b1e056f4a..09f14fdcaa7a2 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala @@ -20,6 +20,7 @@ import java.util import java.util.concurrent.atomic.AtomicReference import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsReporter} +import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.metrics.MetricConfigs import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Assertions._ @@ -72,8 +73,8 @@ class KafkaMetricsReporterTest extends QuorumTestHarness { super.setUp(testInfo) val props = TestUtils.createBrokerConfig(1, zkConnectOrNull) props.setProperty(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricsReporterTest$MockMetricsReporter") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "true") + props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") config = KafkaConfig.fromProps(props) broker = createBroker(config, threadNamePrefix = Option(this.getClass.getName)) broker.startup() diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala index 53bd2b0e0da4b..a8b7a280defeb 100644 --- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala @@ -28,7 +28,8 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{KafkaStorageException, NotLeaderOrFollowerException} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.params.provider.ValueSource @@ -52,6 +53,8 @@ class LogDirFailureTest extends IntegrationTestHarness { this.serverConfig.setProperty(ReplicationConfigs.REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS_CONFIG, "60000") this.serverConfig.setProperty(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") + this.serverConfig.setProperty(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, "5000") + this.serverConfig.setProperty(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") @BeforeEach override def setUp(testInfo: TestInfo): Unit = { @@ -66,6 +69,31 @@ class LogDirFailureTest extends IntegrationTestHarness { testProduceErrorsFromLogDirFailureOnLeader(Roll) } + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testLogDirNotificationTimeout(quorum: String): Unit = { + // Disable retries to allow exception to bubble up for validation + this.producerConfig.setProperty(ProducerConfig.RETRIES_CONFIG, "0") + this.producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false") + val producer = createProducer() + + val partition = new TopicPartition(topic, 0) + + val leaderServerId = producer.partitionsFor(topic).asScala.find(_.partition() == 0).get.leader().id() + val leaderServer = brokers.find(_.config.brokerId == leaderServerId).get + + // shut down the controller to simulate the case where the broker is not able to send the log dir notification + controllerServer.shutdown() + controllerServer.awaitShutdown() + + TestUtils.causeLogDirFailure(Checkpoint, leaderServer, partition) + + TestUtils.waitUntilTrue(() => leaderServer.brokerState == BrokerState.SHUTTING_DOWN, + s"Expected broker to be in NOT_RUNNING state but was ${leaderServer.brokerState}", 15000) + // wait for actual shutdown (by default max 5 minutes for graceful shutdown) + leaderServer.awaitShutdown() + } + @ParameterizedTest @ValueSource(strings = Array("zk", "kraft")) def testIOExceptionDuringLogRoll(quorum: String): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/MockTierStateMachine.scala b/core/src/test/scala/unit/kafka/server/MockTierStateMachine.scala index 9e225607325d5..86df92d77daf9 100644 --- a/core/src/test/scala/unit/kafka/server/MockTierStateMachine.scala +++ b/core/src/test/scala/unit/kafka/server/MockTierStateMachine.scala @@ -20,9 +20,7 @@ package kafka.server import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message.FetchResponseData -import java.util.Optional - -class MockTierStateMachine(leader: LeaderEndPoint) extends ReplicaFetcherTierStateMachine(leader, null) { +class MockTierStateMachine(leader: LeaderEndPoint) extends TierStateMachine(leader, null, false) { var fetcher: MockFetcherThread = _ @@ -37,11 +35,6 @@ class MockTierStateMachine(leader: LeaderEndPoint) extends ReplicaFetcherTierSta Fetching, Some(currentFetchState.currentLeaderEpoch)) } - override def maybeAdvanceState(topicPartition: TopicPartition, - currentFetchState: PartitionFetchState): Optional[PartitionFetchState] = { - Optional.of(currentFetchState) - } - def setFetcher(mockFetcherThread: MockFetcherThread): Unit = { fetcher = mockFetcherThread } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 5f1c41ba3f1c6..6b3eb31ad9d9f 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -315,6 +315,81 @@ class ReplicaManagerTest { } } + @ParameterizedTest(name = "testMaybeAddLogDirFetchersPausingCleaning with futureLogCreated: {0}") + @ValueSource(booleans = Array(true, false)) + def testMaybeAddLogDirFetchersPausingCleaning(futureLogCreated: Boolean): Unit = { + val dir1 = TestUtils.tempDir() + val dir2 = TestUtils.tempDir() + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put("log.dirs", dir1.getAbsolutePath + "," + dir2.getAbsolutePath) + val config = KafkaConfig.fromProps(props) + val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties())) + val spyLogManager = spy(logManager) + val metadataCache: MetadataCache = mock(classOf[MetadataCache]) + mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0))) + when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion) + val tp0 = new TopicPartition(topic, 0) + val uuid = Uuid.randomUuid() + val rm = new ReplicaManager( + metrics = metrics, + config = config, + time = time, + scheduler = new MockScheduler(time), + logManager = spyLogManager, + quotaManagers = quotaManager, + metadataCache = metadataCache, + logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), + alterPartitionManager = alterPartitionManager) + + try { + val partition = rm.createPartition(tp0) + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, + new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints), Option.apply(uuid)) + + val response = rm.becomeLeaderOrFollower(0, new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0, brokerEpoch, + Seq(new LeaderAndIsrPartitionState() + .setTopicName(topic) + .setPartitionIndex(0) + .setControllerEpoch(0) + .setLeader(0) + .setLeaderEpoch(0) + .setIsr(Seq[Integer](0).asJava) + .setPartitionEpoch(0) + .setReplicas(Seq[Integer](0).asJava) + .setIsNew(false)).asJava, + Collections.singletonMap(topic, uuid), + Set(new Node(0, "host1", 0)).asJava).build(), (_, _) => ()) + // expect the errorCounts only has 1 entry with Errors.NONE + val errorCounts = response.errorCounts() + assertEquals(1, response.errorCounts().size()) + assertNotNull(errorCounts.get(Errors.NONE)) + spyLogManager.maybeUpdatePreferredLogDir(tp0, dir2.getAbsolutePath) + + if (futureLogCreated) { + // create future log before maybeAddLogDirFetchers invoked + partition.createLogIfNotExists(isNew = false, isFutureReplica = true, + new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints), None) + } else { + val mockLog = mock(classOf[UnifiedLog]) + when(spyLogManager.getLog(tp0, isFuture = true)).thenReturn(Option.apply(mockLog)) + when(mockLog.topicId).thenReturn(Option.apply(uuid)) + when(mockLog.parentDir).thenReturn(dir2.getAbsolutePath) + } + + val topicIdMap: Map[String, Option[Uuid]] = Map(topic -> Option.apply(uuid)) + rm.maybeAddLogDirFetchers(Set(partition), new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints), topicIdMap) + if (futureLogCreated) { + // since the futureLog is already created, we don't have to abort and pause the cleaning + verify(spyLogManager, never).abortAndPauseCleaning(any[TopicPartition]) + } else { + verify(spyLogManager, times(1)).abortAndPauseCleaning(any[TopicPartition]) + } + rm.replicaAlterLogDirsManager.fetcherThreadMap.values.foreach(t => t.fetchState(new TopicPartition(topic, 0)).foreach(s => assertEquals(0L, s.fetchOffset))) + } finally { + rm.shutdown(checkpointHW = false) + } + } + @Test def testClearPurgatoryOnBecomingFollower(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) @@ -3286,12 +3361,8 @@ class ReplicaManagerTest { val props = TestUtils.createBrokerConfig(brokerId, TestUtils.MockZkConnect) val path1 = TestUtils.tempRelativeDir("data").getAbsolutePath val path2 = TestUtils.tempRelativeDir("data2").getAbsolutePath - if (enableRemoteStorage) { - props.put("log.dirs", path1) - props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, enableRemoteStorage.toString) - } else { - props.put("log.dirs", path1 + "," + path2) - } + props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, enableRemoteStorage.toString) + props.put("log.dirs", path1 + "," + path2) propsModifier.apply(props) val config = KafkaConfig.fromProps(props) val logProps = new Properties() @@ -4982,9 +5053,8 @@ class ReplicaManagerTest { assertEquals(followerPartitions, actualFollowerPartitions) } - // KAFKA-16031: Enabling remote storage after JBOD is supported in tiered storage @ParameterizedTest - @ValueSource(booleans = Array(false)) + @ValueSource(booleans = Array(true, false)) def testApplyDeltaShouldHandleReplicaAssignedToOnlineDirectory(enableRemoteStorage: Boolean): Unit = { val localId = 1 val topicPartition0 = new TopicPartition("foo", 0) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 7314d9ed612b7..fac910e98cfed 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -45,7 +45,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.network.Session import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerConfigs, QuotaConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -78,17 +78,17 @@ class RequestQuotaTest extends BaseRequestTest { private val tasks = new ListBuffer[Task] override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.put(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") properties.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName) - properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true") + properties.put(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[RequestQuotaTest.KraftTestAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[RequestQuotaTest.KraftTestAuthorizer].getName) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[RequestQuotaTest.ZkTestAuthorizer].getName) + properties.put(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[RequestQuotaTest.ZkTestAuthorizer].getName) } } diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala index a4fa2b315e5be..6da56191b4df6 100755 --- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -19,6 +19,7 @@ package kafka.server import kafka.utils.TestUtils import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, PropertiesUtils} +import org.apache.kafka.server.config.ServerConfigs import org.apache.zookeeper.KeeperException.NodeExistsException import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -87,9 +88,9 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness { @Test def testDisableGeneratedBrokerId(): Unit = { val props3 = TestUtils.createBrokerConfig(3, zkConnect) - props3.put(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props3.put(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") // Set reserve broker ids to cause collision and ensure disabling broker id generation ignores the setting - props3.put(KafkaConfig.MaxReservedBrokerIdProp, "0") + props3.put(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, "0") val config3 = KafkaConfig.fromProps(props3) val server3 = createServer(config3, threadNamePrefix = Option(this.getClass.getName)) servers = Seq(server3) @@ -151,14 +152,14 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness { servers = Seq(serverA) // adjust the broker config and start again - propsB.setProperty(KafkaConfig.BrokerIdProp, "2") + propsB.setProperty(ServerConfigs.BROKER_ID_CONFIG, "2") val serverB2 = new KafkaServer(KafkaConfig.fromProps(propsB), threadNamePrefix = Option(this.getClass.getName)) val startupException = assertThrows(classOf[RuntimeException], () => serverB2.startup()) assertTrue(startupException.getMessage.startsWith("Stored node id 1 doesn't match previous node id 2"), "Unexpected exception message " + startupException.getMessage) serverB2.config.logDirs.foreach(logDir => Utils.delete(new File(logDir))) - propsB.setProperty(KafkaConfig.BrokerIdProp, "3") + propsB.setProperty(ServerConfigs.BROKER_ID_CONFIG, "3") val serverB3 = new KafkaServer(KafkaConfig.fromProps(propsB), threadNamePrefix = Option(this.getClass.getName)) serverB3.startup() diff --git a/core/src/test/scala/unit/kafka/server/ServerTest.scala b/core/src/test/scala/unit/kafka/server/ServerTest.scala index 8c36aa3464a00..62345c446e27d 100644 --- a/core/src/test/scala/unit/kafka/server/ServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerTest.scala @@ -17,11 +17,10 @@ package kafka.server import java.util.Properties - import org.apache.kafka.common.Uuid import org.apache.kafka.common.metrics.MetricsContext import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -55,7 +54,7 @@ class ServerTest { val clusterId = Uuid.randomUuid().toString val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, brokerId.toString) + props.put(ServerConfigs.BROKER_ID_CONFIG, brokerId.toString) props.put(ZkConfigs.ZK_CONNECT_CONFIG, "127.0.0.1:0") val config = KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherTierStateMachineTest.scala b/core/src/test/scala/unit/kafka/server/TierStateMachineTest.scala similarity index 93% rename from core/src/test/scala/unit/kafka/server/ReplicaFetcherTierStateMachineTest.scala rename to core/src/test/scala/unit/kafka/server/TierStateMachineTest.scala index af7d1ce633f13..139aeb053ffea 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherTierStateMachineTest.scala +++ b/core/src/test/scala/unit/kafka/server/TierStateMachineTest.scala @@ -23,20 +23,21 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.record._ import org.apache.kafka.common.{TopicPartition, Uuid} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test import kafka.server.FetcherThreadTestUtils.{initialFetchState, mkBatch} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.collection.Map -class ReplicaFetcherTierStateMachineTest { +class TierStateMachineTest { - val truncateOnFetch = true val topicIds = Map("topic1" -> Uuid.randomUuid(), "topic2" -> Uuid.randomUuid()) val version = ApiKeys.FETCH.latestVersion() private val failedPartitions = new FailedPartitions - @Test - def testFollowerFetchMovedToTieredStore(): Unit = { + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testFollowerFetchMovedToTieredStore(truncateOnFetch: Boolean): Unit = { val partition = new TopicPartition("topic", 0) val replicaLog = Seq( @@ -94,8 +95,9 @@ class ReplicaFetcherTierStateMachineTest { * tiered storage as well. Hence, `X < globalLogStartOffset`. * 4. Follower comes online and tries to fetch X from leader. */ - @Test - def testFollowerFetchOffsetOutOfRangeWithTieredStore(): Unit = { + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testFollowerFetchOffsetOutOfRangeWithTieredStore(truncateOnFetch: Boolean): Unit = { val partition = new TopicPartition("topic", 0) val replicaLog = Seq( @@ -105,7 +107,7 @@ class ReplicaFetcherTierStateMachineTest { val replicaState = PartitionState(replicaLog, leaderEpoch = 7, highWatermark = 0L, rlmEnabled = true) - val mockLeaderEndpoint = new MockLeaderEndPoint + val mockLeaderEndpoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) val mockTierStateMachine = new MockTierStateMachine(mockLeaderEndpoint) val fetcher = new MockFetcherThread(mockLeaderEndpoint, mockTierStateMachine) @@ -153,8 +155,9 @@ class ReplicaFetcherTierStateMachineTest { assertEquals(11L, replicaState.logEndOffset) } - @Test - def testFencedOffsetResetAfterMovedToRemoteTier(): Unit = { + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testFencedOffsetResetAfterMovedToRemoteTier(truncateOnFetch: Boolean): Unit = { val partition = new TopicPartition("topic", 0) var isErrorHandled = false val mockLeaderEndpoint = new MockLeaderEndPoint(truncateOnFetch = truncateOnFetch, version = version) @@ -189,4 +192,5 @@ class ReplicaFetcherTierStateMachineTest { assertTrue(fetcher.fetchState(partition).isEmpty) assertTrue(failedPartitions.contains(partition)) } -} + +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala b/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala index f41487c5d4ad3..ea3f79ddbce3a 100644 --- a/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala +++ b/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse, FetchRequest, FetchResponse, MetadataRequest, MetadataResponse} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.common.MetadataVersion.IBP_2_7_IV0 -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} @@ -41,7 +41,7 @@ class TopicIdWithOldInterBrokerProtocolTest extends BaseRequestTest { properties.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, IBP_2_7_IV0.toString) properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(ServerConfigs.BROKER_ID_CONFIG)}") } @BeforeEach diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index de7736973b2b4..b8764f5fae3d0 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -28,7 +28,7 @@ import kafka.log.{LogTestUtils, UnifiedLog} import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} import kafka.server.{BrokerTopicStats, KafkaRaftServer} import kafka.tools.DumpLogSegments.{OffsetsMessageParser, TimeIndexDumpErrors} -import kafka.utils.TestUtils +import kafka.utils.{TestUtils, VerifiableProperties} import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.{Assignment, Subscription} import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.{TopicPartition, Uuid} @@ -598,6 +598,26 @@ class DumpLogSegmentsTest { ) } + @Test + def testNewDecoder(): Unit = { + // Decoder translate should pass without exception + DumpLogSegments.newDecoder(classOf[DumpLogSegmentsTest.TestDecoder].getName) + DumpLogSegments.newDecoder(classOf[kafka.serializer.DefaultDecoder].getName) + assertThrows(classOf[Exception], () => DumpLogSegments.newDecoder(classOf[DumpLogSegmentsTest.TestDecoderWithoutVerifiableProperties].getName)) + } + + @Test + def testConvertDeprecatedDecoderClass(): Unit = { + assertEquals(classOf[org.apache.kafka.tools.api.DefaultDecoder].getName, DumpLogSegments.convertDeprecatedDecoderClass( + classOf[kafka.serializer.DefaultDecoder].getName)) + assertEquals(classOf[org.apache.kafka.tools.api.IntegerDecoder].getName, DumpLogSegments.convertDeprecatedDecoderClass( + classOf[kafka.serializer.IntegerDecoder].getName)) + assertEquals(classOf[org.apache.kafka.tools.api.LongDecoder].getName, DumpLogSegments.convertDeprecatedDecoderClass( + classOf[kafka.serializer.LongDecoder].getName)) + assertEquals(classOf[org.apache.kafka.tools.api.StringDecoder].getName, DumpLogSegments.convertDeprecatedDecoderClass( + classOf[kafka.serializer.StringDecoder].getName)) + } + private def readBatchMetadata(lines: util.ListIterator[String]): Option[String] = { while (lines.hasNext) { val line = lines.next() @@ -732,3 +752,13 @@ class DumpLogSegmentsTest { } } } + +object DumpLogSegmentsTest { + class TestDecoder(props: VerifiableProperties) extends kafka.serializer.Decoder[Array[Byte]] { + override def fromBytes(bytes: Array[Byte]): Array[Byte] = bytes + } + + class TestDecoderWithoutVerifiableProperties() extends kafka.serializer.Decoder[Array[Byte]] { + override def fromBytes(bytes: Array[Byte]): Array[Byte] = bytes + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 896e7169671ff..9f48348b81250 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.common.metadata.UserScramCredentialRecord import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue} import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.params.ParameterizedTest @@ -206,12 +206,15 @@ Found problem: @Test def testFormatSucceedsIfAllDirectoriesAreAvailable(): Unit = { - val availableDir1 = TestUtils.tempDir() - val availableDir2 = TestUtils.tempDir() + val availableDirs = Seq(TestUtils.tempDir(), TestUtils.tempDir(), TestUtils.tempDir()).map(dir => dir.toString) val stream = new ByteArrayOutputStream() - assertEquals(0, runFormatCommand(stream, Seq(availableDir1.toString, availableDir2.toString))) - assertTrue(stream.toString().contains("Formatting %s".format(availableDir1))) - assertTrue(stream.toString().contains("Formatting %s".format(availableDir2))) + assertEquals(0, runFormatCommand(stream, availableDirs)) + val actual = stream.toString().split("\\r?\\n") + val expect = availableDirs.map("Formatting %s".format(_)) + assertEquals(availableDirs.size, actual.size) + expect.foreach(dir => { + assertEquals(1, actual.count(_.startsWith(dir))) + }) } @Test @@ -464,7 +467,7 @@ Found problem: val propsStream = Files.newOutputStream(propsFile.toPath) try { properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString) - properties.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, enableUnstable.toString) + properties.setProperty(ServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, enableUnstable.toString) properties.store(propsStream, "config.props") } finally { propsStream.close() @@ -489,3 +492,4 @@ Found problem: } } } + diff --git a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala index e96da4cd7d3a5..8acaa9586104b 100755 --- a/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/CoreUtilsTest.scala @@ -89,19 +89,6 @@ class CoreUtilsTest extends Logging { assertEquals(Integer.MAX_VALUE, Utils.abs(Integer.MAX_VALUE)) } - @Test - def testCsvList(): Unit = { - val emptyString:String = "" - val nullString:String = null - val emptyList = CoreUtils.parseCsvList(emptyString) - val emptyListFromNullString = CoreUtils.parseCsvList(nullString) - val emptyStringList = Seq.empty[String] - assertTrue(emptyList!=null) - assertTrue(emptyListFromNullString!=null) - assertTrue(emptyStringList.equals(emptyListFromNullString)) - assertTrue(emptyStringList.equals(emptyList)) - } - @Test def testInLock(): Unit = { val lock = new ReentrantLock() diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 8ba2ea00cdd8e..45fff50c8b821 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -304,18 +304,18 @@ object TestUtils extends Logging { }.mkString(",") val props = new Properties - props.put(KafkaConfig.UnstableMetadataVersionsEnableProp, "true") + props.put(ServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, "true") if (zkConnect == null) { props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString) props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString) - props.put(KafkaConfig.BrokerIdProp, nodeId.toString) + props.put(ServerConfigs.BROKER_ID_CONFIG, nodeId.toString) props.put(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, protocolAndPorts. map(p => "%s:%s".format(p._1, p._1)).mkString(",") + ",CONTROLLER:PLAINTEXT") } else { - if (nodeId >= 0) props.put(KafkaConfig.BrokerIdProp, nodeId.toString) + if (nodeId >= 0) props.put(ServerConfigs.BROKER_ID_CONFIG, nodeId.toString) props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners) } if (logDirCount > 1) { @@ -341,20 +341,20 @@ object TestUtils extends Logging { } props.put(ReplicationConfigs.REPLICA_SOCKET_TIMEOUT_MS_CONFIG, "1500") props.put(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, "1500") - props.put(KafkaConfig.ControlledShutdownEnableProp, enableControlledShutdown.toString) - props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString) + props.put(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, enableControlledShutdown.toString) + props.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, enableDeleteTopic.toString) props.put(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000") - props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100") + props.put(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG, "100") props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152") props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") if (!props.containsKey(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG)) props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "5") if (!props.containsKey(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG)) props.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") - rack.foreach(props.put(KafkaConfig.RackProp, _)) + rack.foreach(props.put(ServerConfigs.BROKER_RACK_CONFIG, _)) // Reduce number of threads per broker - props.put(KafkaConfig.NumNetworkThreadsProp, "2") - props.put(KafkaConfig.BackgroundThreadsProp, "2") + props.put(ServerConfigs.NUM_NETWORK_THREADS_CONFIG, "2") + props.put(ServerConfigs.BACKGROUND_THREADS_CONFIG, "2") if (protocolAndPorts.exists { case (protocol, _) => usesSslTransportLayer(protocol) }) props ++= sslConfigs(Mode.SERVER, false, trustStoreFile, s"server$nodeId") @@ -367,13 +367,13 @@ object TestUtils extends Logging { } if (enableToken) - props.put(KafkaConfig.DelegationTokenSecretKeyProp, "secretkey") + props.put(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "secretkey") props.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numPartitions.toString) props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, defaultReplicationFactor.toString) if (enableFetchFromFollower) { - props.put(KafkaConfig.RackProp, nodeId.toString) + props.put(ServerConfigs.BROKER_RACK_CONFIG, nodeId.toString) props.put(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG, "org.apache.kafka.common.replica.RackAwareReplicaSelector") } props @@ -1417,6 +1417,33 @@ object TestUtils extends Logging { 45000) } + + def waitAndVerifyAcl(expected: AccessControlEntry, + authorizer: JAuthorizer, + resource: ResourcePattern, + accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = { + val newLine = scala.util.Properties.lineSeparator + + val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter) + waitUntilTrue(() => authorizer.acls(filter).asScala.map(_.entry).toSet.contains(expected), + s"expected to contain acl: $expected" + + s"but got:${authorizer.acls(filter).asScala.map(_.entry).mkString(newLine + "\t", newLine + "\t", newLine)}", + 45000) + } + + def waitAndVerifyRemovedAcl(expectedToRemoved: AccessControlEntry, + authorizer: JAuthorizer, + resource: ResourcePattern, + accessControlEntryFilter: AccessControlEntryFilter = AccessControlEntryFilter.ANY): Unit = { + val newLine = scala.util.Properties.lineSeparator + + val filter = new AclBindingFilter(resource.toFilter, accessControlEntryFilter) + waitUntilTrue(() => !authorizer.acls(filter).asScala.map(_.entry).toSet.contains(expectedToRemoved), + s"expected acl to be removed : $expectedToRemoved" + + s"but got:${authorizer.acls(filter).asScala.map(_.entry).mkString(newLine + "\t", newLine + "\t", newLine)}", + 45000) + } + /** * Verifies that this ACL is the secure one. */ diff --git a/core/src/test/scala/unit/kafka/utils/ThrottlerTest.scala b/core/src/test/scala/unit/kafka/utils/ThrottlerTest.scala deleted file mode 100755 index a34e2ea12cf5d..0000000000000 --- a/core/src/test/scala/unit/kafka/utils/ThrottlerTest.scala +++ /dev/null @@ -1,108 +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 kafka.utils - -import org.apache.kafka.server.util.MockTime -import org.junit.jupiter.api.Test -import org.junit.jupiter.api.Assertions.{assertTrue, assertEquals} - - -class ThrottlerTest { - @Test - def testThrottleDesiredRate(): Unit = { - val throttleCheckIntervalMs = 100 - val desiredCountPerSec = 1000.0 - val desiredCountPerInterval = desiredCountPerSec * throttleCheckIntervalMs / 1000.0 - - val mockTime = new MockTime() - val throttler = new Throttler(desiredRatePerSec = desiredCountPerSec, - checkIntervalMs = throttleCheckIntervalMs, - time = mockTime) - - // Observe desiredCountPerInterval at t1 - val t1 = mockTime.milliseconds() - throttler.maybeThrottle(desiredCountPerInterval) - assertEquals(t1, mockTime.milliseconds()) - - // Observe desiredCountPerInterval at t1 + throttleCheckIntervalMs + 1, - mockTime.sleep(throttleCheckIntervalMs + 1) - throttler.maybeThrottle(desiredCountPerInterval) - val t2 = mockTime.milliseconds() - assertTrue(t2 >= t1 + 2 * throttleCheckIntervalMs) - - // Observe desiredCountPerInterval at t2 - throttler.maybeThrottle(desiredCountPerInterval) - assertEquals(t2, mockTime.milliseconds()) - - // Observe desiredCountPerInterval at t2 + throttleCheckIntervalMs + 1 - mockTime.sleep(throttleCheckIntervalMs + 1) - throttler.maybeThrottle(desiredCountPerInterval) - val t3 = mockTime.milliseconds() - assertTrue(t3 >= t2 + 2 * throttleCheckIntervalMs) - - val elapsedTimeMs = t3 - t1 - val actualCountPerSec = 4 * desiredCountPerInterval * 1000 / elapsedTimeMs - assertTrue(actualCountPerSec <= desiredCountPerSec) - } - - @Test - def testUpdateThrottleDesiredRate(): Unit = { - val throttleCheckIntervalMs = 100 - val desiredCountPerSec = 1000.0 - val desiredCountPerInterval = desiredCountPerSec * throttleCheckIntervalMs / 1000.0 - val updatedDesiredCountPerSec = 1500.0 - val updatedDesiredCountPerInterval = updatedDesiredCountPerSec * throttleCheckIntervalMs / 1000.0 - - val mockTime = new MockTime() - val throttler = new Throttler(desiredRatePerSec = desiredCountPerSec, - checkIntervalMs = throttleCheckIntervalMs, - time = mockTime) - - // Observe desiredCountPerInterval at t1 - val t1 = mockTime.milliseconds() - throttler.maybeThrottle(desiredCountPerInterval) - assertEquals(t1, mockTime.milliseconds()) - - // Observe desiredCountPerInterval at t1 + throttleCheckIntervalMs + 1, - mockTime.sleep(throttleCheckIntervalMs + 1) - throttler.maybeThrottle(desiredCountPerInterval) - val t2 = mockTime.milliseconds() - assertTrue(t2 >= t1 + 2 * throttleCheckIntervalMs) - - val elapsedTimeMs = t2 - t1 - val actualCountPerSec = 2 * desiredCountPerInterval * 1000 / elapsedTimeMs - assertTrue(actualCountPerSec <= desiredCountPerSec) - - // Update ThrottleDesiredRate - throttler.updateDesiredRatePerSec(updatedDesiredCountPerSec) - - // Observe updatedDesiredCountPerInterval at t2 - throttler.maybeThrottle(updatedDesiredCountPerInterval) - assertEquals(t2, mockTime.milliseconds()) - - // Observe updatedDesiredCountPerInterval at t2 + throttleCheckIntervalMs + 1 - mockTime.sleep(throttleCheckIntervalMs + 1) - throttler.maybeThrottle(updatedDesiredCountPerInterval) - val t3 = mockTime.milliseconds() - assertTrue(t3 >= t2 + 2 * throttleCheckIntervalMs) - - val updatedElapsedTimeMs = t3 - t2 - val updatedActualCountPerSec = 2 * updatedDesiredCountPerInterval * 1000 / updatedElapsedTimeMs - assertTrue(updatedActualCountPerSec <= updatedDesiredCountPerSec) - } -} diff --git a/docker/docker_official_image_build_test.py b/docker/docker_official_image_build_test.py new file mode 100644 index 0000000000000..6ffe25ee0b89c --- /dev/null +++ b/docker/docker_official_image_build_test.py @@ -0,0 +1,87 @@ +#!/usr/bin/env python + +# 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. + +""" +Python script to build and test a docker image +This script is used to generate a test report + +Usage: + docker_official_image_build_test.py --help + Get detailed description of each option + + Example command:- + docker_official_image_build_test.py --image-tag --image-type --kafka-version + + This command will build an image with as image name, as image_tag (it will be latest by default), + as image type (jvm by default), for the kafka version for which the image is being built, and + run tests on the image. + -b can be passed as additional argument if you just want to build the image. + -t can be passed if you just want to run tests on the image. +""" + +import argparse +from distutils.dir_util import copy_tree +import shutil +from common import execute +from docker_build_test import run_docker_tests +import tempfile +import os + + +def build_docker_official_image(image, tag, kafka_version, image_type): + image = f'{image}:{tag}' + current_dir = os.path.dirname(os.path.realpath(__file__)) + temp_dir_path = tempfile.mkdtemp() + copy_tree(f"{current_dir}/docker_official_images/{kafka_version}/{image_type}", + f"{temp_dir_path}/{image_type}") + copy_tree(f"{current_dir}/docker_official_images/{kafka_version}/jvm/resources", + f"{temp_dir_path}/{image_type}/resources") + command = f"docker build -f $DOCKER_FILE -t {image} $DOCKER_DIR" + command = command.replace("$DOCKER_FILE", f"{temp_dir_path}/{image_type}/Dockerfile") + command = command.replace("$DOCKER_DIR", f"{temp_dir_path}/{image_type}") + try: + execute(command.split()) + except: + raise SystemError("Docker Image Build failed") + finally: + shutil.rmtree(temp_dir_path) + + +if __name__ == '__main__': + parser = argparse.ArgumentParser() + parser.add_argument( + "image", help="Image name that you want to keep for the Docker image") + parser.add_argument("--image-tag", "-tag", default="latest", + dest="tag", help="Image tag that you want to add to the image") + parser.add_argument("--image-type", "-type", choices=[ + "jvm"], default="jvm", dest="image_type", help="Image type you want to build") + parser.add_argument("--kafka-version", "-v", dest="kafka_version", + help="Kafka version for which the source for docker official image is to be built") + parser.add_argument("--build", "-b", action="store_true", dest="build_only", + default=False, help="Only build the image, don't run tests") + parser.add_argument("--test", "-t", action="store_true", dest="test_only", + default=False, help="Only run the tests, don't build the image") + args = parser.parse_args() + kafka_url = f"https://downloads.apache.org/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" + if args.build_only or not (args.build_only or args.test_only): + if args.kafka_version: + build_docker_official_image(args.image, args.tag, args.kafka_version, args.image_type) + else: + raise ValueError( + "--kafka-version is required argument for jvm docker official image image") + if args.test_only or not (args.build_only or args.test_only): + run_docker_tests(args.image, args.tag, kafka_url, args.image_type) diff --git a/docker/docker_official_images/.gitkeep b/docker/docker_official_images/.gitkeep new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/docker/extract_docker_official_image_artifact.py b/docker/extract_docker_official_image_artifact.py new file mode 100644 index 0000000000000..2d362eb50db94 --- /dev/null +++ b/docker/extract_docker_official_image_artifact.py @@ -0,0 +1,77 @@ +#!/usr/bin/env python + +# 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. + +""" +Python script to extract docker official images artifact and give it executable permissions +This script is used to extract docker official images artifact and give it executable permissions + +Usage: + extract_docker_official_image_artifact.py --help + Get detailed description of each option + + Example command:- + extract_docker_official_image_artifact.py --path_to_downloaded_artifact + + This command will build an extract the downloaded artifact, and copy the contents to the + docker_official_images directory. If the extracted artifact contents already exist in the + docker_official_images directory , they will be overwritten, else they will be created. + +""" +import os +import argparse +import zipfile +import shutil +from pathlib import Path + +def set_executable_permissions(directory): + for root, _, files in os.walk(directory): + for file in files: + path = os.path.join(root, file) + os.chmod(path, os.stat(path).st_mode | 0o111) + + +def extract_artifact(artifact_path): + docker_official_images_dir = Path(os.path.dirname(os.path.realpath(__file__)), "docker_official_images") + temp_dir = Path('temp_extracted') + try: + if temp_dir.exists(): + shutil.rmtree(temp_dir) + temp_dir.mkdir() + with zipfile.ZipFile(artifact_path, 'r') as zip_ref: + zip_ref.extractall(temp_dir) + artifact_version_dirs = list(temp_dir.iterdir()) + if len(artifact_version_dirs) != 1: + raise Exception("Unexpected contents in the artifact. Exactly one version directory is expected.") + artifact_version_dir = artifact_version_dirs[0] + target_version_dir = Path(os.path.join(docker_official_images_dir, artifact_version_dir.name)) + target_version_dir.mkdir(parents=True, exist_ok=True) + for image_type_dir in artifact_version_dir.iterdir(): + target_image_type_dir = Path(os.path.join(target_version_dir, image_type_dir.name)) + if target_image_type_dir.exists(): + shutil.rmtree(target_image_type_dir) + shutil.copytree(image_type_dir, target_image_type_dir) + set_executable_permissions(target_image_type_dir) + finally: + if temp_dir.exists(): + shutil.rmtree(temp_dir) + +if __name__ == '__main__': + parser = argparse.ArgumentParser() + parser.add_argument("--path_to_downloaded_artifact", "-artifact_path", required=True, + dest="artifact_path", help="Path to zipped artifacy downloaded from github actions workflow.") + args = parser.parse_args() + extract_artifact(args.artifact_path) diff --git a/docker/generate_kafka_pr_template.py b/docker/generate_kafka_pr_template.py new file mode 100644 index 0000000000000..b3f9577cf89b4 --- /dev/null +++ b/docker/generate_kafka_pr_template.py @@ -0,0 +1,92 @@ +#!/usr/bin/env python + +# 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. + +""" +Python script to prepare the PR template for the docker official image +This script is used to prepare the PR template for the docker official image + +Usage: + Example command:- + generate_kafka_pr_template.py --help + Get detailed description of each option + + generate_kafka_pr_template.py --image-type + + This command will build a PR template for as image type (jvm by default) based docker official image, + on the directories present under docker/docker_official_images. + This PR template will be used to raise a PR in the Docker Official Images Repo. +""" + +import os +import subprocess +import sys +import argparse +from pathlib import Path + + +# Returns the hash of the most recent commit that modified any of the specified files. +def file_commit(*files): + return subprocess.check_output(["git", "log", "-1", "--format=format:%H", "HEAD", "--"] + list(files)).strip().decode('utf-8') + + +# Returns the latest commit hash for all files in a given directory. +def dir_commit(directory): + docker_required_scripts = [str(path) for path in Path(directory).rglob('*') if path.is_file()] + files_to_check = [os.path.join(directory, "Dockerfile")] + docker_required_scripts + return file_commit(*files_to_check) + + +# Split the version string into parts and convert them to integers for version comparision +def get_version_parts(version): + return tuple(int(part) for part in version.name.split('.')) + + +def main(): + parser = argparse.ArgumentParser() + parser.add_argument("--image-type", "-type", choices=[ + "jvm"], default="jvm", dest="image_type", help="Image type you want to build") + args = parser.parse_args() + self = os.path.basename(__file__) + current_dir = os.path.dirname(os.path.abspath(__file__)) + docker_official_images_dir = Path(os.path.join(current_dir, "docker_official_images")) + highest_version = "" + + header = f""" +# This file is generated via https://github.com/apache/kafka/blob/{file_commit(os.path.join(current_dir, self))}/docker/generate_kafka_pr_template.py +Maintainers: The Apache Kafka Project (@ApacheKafka) +GitRepo: https://github.com/apache/kafka.git +""" + print(header) + versions = sorted((d for d in docker_official_images_dir.iterdir() if d.is_dir()), key=get_version_parts, reverse=True) + highest_version = max(versions).name if versions else "" + + for dir in versions: + version = dir.name + tags = version + (", latest" if version == highest_version else "") + commit = dir_commit(dir.joinpath(args.image_type)) + + info = f""" +Tags: {tags} +Architectures: amd64,arm64v8 +GitCommit: {commit} +Directory: ./docker/docker_official_images/{version}/{args.image_type} +""" + print(info.strip(), '\n') + + +if __name__ == "__main__": + main() diff --git a/docker/prepare_docker_official_image_source.py b/docker/prepare_docker_official_image_source.py new file mode 100644 index 0000000000000..a39915c9e51f2 --- /dev/null +++ b/docker/prepare_docker_official_image_source.py @@ -0,0 +1,68 @@ +#!/usr/bin/env python + +# 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. + +""" +Python script to prepare the hardcoded source folder for the docker official image +This script is used to prepare the source folder for the docker official image + +Usage: + prepare_docker_official_image_source.py --help + Get detailed description of each option + + Example command:- + prepare_docker_official_image_source.py --image-type --kafka-version + + This command will build a directory with the name as housing the hardcoded static Dockerfile and scripts for + the docker official image, as image type (jvm by default), for the kafka version for which the + image is being built. +""" + +from datetime import date +import argparse +from distutils.dir_util import copy_tree +import os +import shutil + + +def remove_args_and_hardcode_values(file_path, kafka_url): + with open(file_path, 'r') as file: + filedata = file.read() + filedata = filedata.replace("ARG kafka_url", f"ENV kafka_url {kafka_url}") + filedata = filedata.replace( + "ARG build_date", f"ENV build_date {str(date.today())}") + with open(file_path, 'w') as file: + file.write(filedata) + + +if __name__ == '__main__': + parser = argparse.ArgumentParser() + parser.add_argument("--image-type", "-type", choices=[ + "jvm"], default="jvm", dest="image_type", help="Image type you want to build") + parser.add_argument("--kafka-version", "-v", dest="kafka_version", + help="Kafka version for which the source for docker official image is to be built") + args = parser.parse_args() + kafka_url = f"https://downloads.apache.org/kafka/{args.kafka_version}/kafka_2.13-{args.kafka_version}.tgz" + current_dir = os.path.dirname(os.path.realpath(__file__)) + new_dir = os.path.join( + current_dir, f'docker_official_images', args.kafka_version) + if os.path.exists(new_dir): + shutil.rmtree(new_dir) + os.makedirs(new_dir) + copy_tree(os.path.join(current_dir, args.image_type), os.path.join(new_dir, args.kafka_version, args.image_type)) + copy_tree(os.path.join(current_dir, 'resources'), os.path.join(new_dir, args.kafka_version, args.image_type, 'resources')) + remove_args_and_hardcode_values( + os.path.join(new_dir, args.kafka_version, args.image_type, 'Dockerfile'), kafka_url) diff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html index 5bb106d77a206..0814723523ce6 100644 --- a/docs/streams/quickstart.html +++ b/docs/streams/quickstart.html @@ -200,7 +200,6 @@

    Step 4: St
    > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
         --topic streams-wordcount-output \
         --from-beginning \
    -    --formatter kafka.tools.DefaultMessageFormatter \
         --property print.key=true \
         --property print.value=true \
         --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    @@ -223,7 +222,6 @@ 

    Step 5
    > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
         --topic streams-wordcount-output \
         --from-beginning \
    -    --formatter kafka.tools.DefaultMessageFormatter \
         --property print.key=true \
         --property print.value=true \
         --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    @@ -252,7 +250,6 @@ 

    Step 5
    > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
         --topic streams-wordcount-output \
         --from-beginning \
    -    --formatter kafka.tools.DefaultMessageFormatter \
         --property print.key=true \
         --property print.value=true \
         --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    @@ -283,7 +280,6 @@ 

    Step 5
    > bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 \
         --topic streams-wordcount-output \
         --from-beginning \
    -    --formatter kafka.tools.DefaultMessageFormatter \
         --property print.key=true \
         --property print.value=true \
         --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
    diff --git a/docs/upgrade.html b/docs/upgrade.html
    index 6be2b4db08657..b1e0b0e78a32e 100644
    --- a/docs/upgrade.html
    +++ b/docs/upgrade.html
    @@ -19,6 +19,17 @@