KAFKA-17367: Introduce share coordinator [2/N] (#17011)

Introduces the share coordinator. This coordinator is built on the new coordinator runtime framework. It 
is responsible for persistence of share-group state in a new internal topic named "__share_group_state".
The responsibility for being a share coordinator is distributed across the brokers in a cluster. 

Reviewers: David Arthur <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
This commit is contained in:
Sushant Mahajan 2024-09-10 05:31:24 +05:30 committed by GitHub
parent 92672d1df8
commit 821c10157d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
41 changed files with 4238 additions and 424 deletions

View File

@ -1020,6 +1020,7 @@ project(':core') {
implementation project(':server')
implementation project(':coordinator-common')
implementation project(':share')
implementation project(':share-coordinator')
implementation libs.argparse4j
implementation libs.commonsValidator
@ -1054,6 +1055,7 @@ project(':core') {
testImplementation project(':clients').sourceSets.test.output
testImplementation project(':group-coordinator').sourceSets.test.output
testImplementation project(':share-coordinator').sourceSets.test.output
testImplementation project(':metadata').sourceSets.test.output
testImplementation project(':raft').sourceSets.test.output
testImplementation project(':server-common').sourceSets.test.output
@ -1586,6 +1588,7 @@ project(':coordinator-common') {
implementation project(':storage')
implementation libs.slf4jApi
implementation libs.metrics
implementation libs.hdrHistogram
testImplementation project(':clients').sourceSets.test.output
testImplementation project(':server-common').sourceSets.test.output
@ -1632,7 +1635,10 @@ project(':share-coordinator') {
implementation project(':clients')
implementation project(':coordinator-common')
implementation project(':metadata')
implementation project(':server')
implementation project(':server-common')
implementation libs.slf4jApi
implementation libs.metrics
testImplementation project(':clients').sourceSets.test.output
testImplementation project(':server-common').sourceSets.test.output

View File

@ -49,6 +49,7 @@
<allow pkg="org.apache.kafka.common.internals" />
<allow pkg="org.apache.kafka.common.message" />
<allow pkg="org.apache.kafka.common.metadata" />
<allow pkg="org.apache.kafka.common.metrics" />
<allow pkg="org.apache.kafka.common.network" />
<allow pkg="org.apache.kafka.common.protocol" />
<allow pkg="org.apache.kafka.common.record" />
@ -62,6 +63,7 @@
<allow pkg="org.apache.kafka.storage.internals.log" />
<allow pkg="org.apache.kafka.test" />
<allow pkg="org.apache.kafka.timeline" />
<allow pkg="org.HdrHistogram" />
</subpackage>
</subpackage>
</subpackage>

View File

@ -31,9 +31,12 @@
<!-- anyone can use public classes -->
<allow pkg="org.apache.kafka.common" exact-match="true" />
<allow pkg="org.apache.kafka.common.compress" />
<allow pkg="org.apache.kafka.common.errors" exact-match="true" />
<allow pkg="org.apache.kafka.common.memory" />
<allow pkg="org.apache.kafka.common.metrics" />
<allow pkg="org.apache.kafka.common.protocol" />
<allow pkg="org.apache.kafka.common.record" />
<allow pkg="org.apache.kafka.common.security" />
<allow pkg="org.apache.kafka.common.serialization" />
<allow pkg="org.apache.kafka.common.utils" />
@ -43,15 +46,31 @@
<allow pkg="org.apache.kafka.common.annotation" />
<allow pkg="org.apache.kafka.common.config" />
<allow pkg="org.apache.kafka.common.message" />
<allow pkg="org.apache.kafka.common.network" />
<allow pkg="org.apache.kafka.common.internals" />
<allow pkg="org.apache.kafka.common.requests" />
<allow pkg="org.apache.kafka.coordinator.common" />
<allow pkg="org.apache.kafka.coordinator.share.generated" />
<allow pkg="org.apache.kafka.coordinator.share.metrics" />
<allow pkg="org.apache.kafka.image" />
<allow pkg="org.apache.kafka.metadata" />
<allow pkg="org.apache.kafka.server.common" />
<allow pkg="org.apache.kafka.server.config" />
<allow pkg="org.apache.kafka.server.group.share" />
<allow pkg="org.apache.kafka.server.record" />
<allow pkg="org.apache.kafka.server.util" />
<allow pkg="org.apache.kafka.server.util.timer" />
<allow pkg="org.apache.kafka.timeline" />
<allow pkg="org.junit.jupiter.api" />
<allow pkg="org.mockito" />
<allow pkg="org.slf4j" />
<subpackage name="generated">
<allow pkg="com.fasterxml.jackson" />
</subpackage>
<subpackage name="metrics">
<allow pkg="com.yammer.metrics.core" />
<allow pkg="org.apache.kafka.timeline" />
</subpackage>
</subpackage>
</subpackage>
</import-control>

View File

@ -69,6 +69,16 @@ public class WriteShareGroupStateResponse extends AbstractResponse {
);
}
public static WriteShareGroupStateResponseData toResponseData(Uuid topicId, int partitionId) {
return new WriteShareGroupStateResponseData()
.setResults(Collections.singletonList(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId)
.setPartitions(Collections.singletonList(
new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId)))));
}
public static WriteShareGroupStateResponseData toErrorResponseData(Uuid topicId, int partitionId, Errors error, String errorMessage) {
WriteShareGroupStateResponseData responseData = new WriteShareGroupStateResponseData();
responseData.setResults(Collections.singletonList(new WriteShareGroupStateResponseData.WriteStateResult()
@ -92,4 +102,9 @@ public class WriteShareGroupStateResponse extends AbstractResponse {
.setTopicId(topicId)
.setPartitions(partitionResults);
}
public static WriteShareGroupStateResponseData.PartitionResult toResponsePartitionResult(int partitionId) {
return new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId);
}
}

View File

@ -90,6 +90,10 @@ offsets.topic.replication.factor=1
transaction.state.log.replication.factor=1
transaction.state.log.min.isr=1
# Share state topic settings
share.coordinator.state.topic.replication.factor=1
share.coordinator.state.topic.min.isr=1
############################# Log Flush Policy #############################
# Messages are immediately written to the filesystem but by default we only fsync() to sync

View File

@ -88,6 +88,10 @@ offsets.topic.replication.factor=1
transaction.state.log.replication.factor=1
transaction.state.log.min.isr=1
# Share state topic settings
share.coordinator.state.topic.replication.factor=1
share.coordinator.state.topic.min.isr=1
############################# Log Flush Policy #############################
# Messages are immediately written to the filesystem but by default we only fsync() to sync

View File

@ -93,6 +93,10 @@ offsets.topic.replication.factor=1
transaction.state.log.replication.factor=1
transaction.state.log.min.isr=1
# Share state topic settings
share.coordinator.state.topic.replication.factor=1
share.coordinator.state.topic.min.isr=1
############################# Log Flush Policy #############################
# Messages are immediately written to the filesystem but by default we only fsync() to sync

View File

@ -0,0 +1,83 @@
/*
* 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.coordinator.common.runtime;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.ApiError;
import org.slf4j.Logger;
import java.util.function.BiFunction;
public class CoordinatorOperationExceptionHelper {
/**
* This is the handler commonly used by all the operations that requires to convert errors to
* coordinator errors. The handler also handles and log unexpected errors.
*
* @param operationName The name of the operation.
* @param operationInput The operation's input for logging purposes.
* @param exception The exception to handle.
* @param handler A function which takes an Errors and a String and builds the expected
* output. The String can be null. Note that the function could further
* transform the error depending on the context.
* @return The output built by the handler.
* @param <IN> The type of the operation input. It must be a toString'able object.
* @param <OUT> The type of the value returned by handler.
*/
public static <IN, OUT> OUT handleOperationException(
String operationName,
IN operationInput,
Throwable exception,
BiFunction<Errors, String, OUT> handler,
Logger log
) {
ApiError apiError = ApiError.fromThrowable(exception);
switch (apiError.error()) {
case UNKNOWN_SERVER_ERROR:
log.error("Operation {} with {} hit an unexpected exception: {}.",
operationName, operationInput, exception.getMessage(), exception);
return handler.apply(Errors.UNKNOWN_SERVER_ERROR, null);
case NETWORK_EXCEPTION:
// When committing offsets transactionally, we now verify the transaction with the
// transaction coordinator. Verification can fail with `NETWORK_EXCEPTION`, a
// retriable error which older clients may not expect and retry correctly. We
// translate the error to `COORDINATOR_LOAD_IN_PROGRESS` because it causes clients
// to retry the request without an unnecessary coordinator lookup.
return handler.apply(Errors.COORDINATOR_LOAD_IN_PROGRESS, null);
case UNKNOWN_TOPIC_OR_PARTITION:
case NOT_ENOUGH_REPLICAS:
case REQUEST_TIMED_OUT:
return handler.apply(Errors.COORDINATOR_NOT_AVAILABLE, null);
case NOT_LEADER_OR_FOLLOWER:
case KAFKA_STORAGE_ERROR:
return handler.apply(Errors.NOT_COORDINATOR, null);
case MESSAGE_TOO_LARGE:
case RECORD_LIST_TOO_LARGE:
case INVALID_FETCH_SIZE:
return handler.apply(Errors.UNKNOWN_SERVER_ERROR, null);
default:
return handler.apply(apiError.error(), apiError.message());
}
}
}

View File

@ -0,0 +1,311 @@
/*
* 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.coordinator.common.runtime;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.metrics.stats.Rate;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState;
import java.util.Arrays;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Supplier;
public class CoordinatorRuntimeMetricsImpl implements CoordinatorRuntimeMetrics {
/**
* The metrics group
*/
private final String metricsGroup;
/**
* The partition count metric name.
*/
public static final String NUM_PARTITIONS_METRIC_NAME = "num-partitions";
/**
* The event queue time metric name.
*/
public static final String EVENT_QUEUE_TIME_METRIC_NAME = "event-queue-time-ms";
/**
* The event queue time metric name.
*/
public static final String EVENT_PROCESSING_TIME_METRIC_NAME = "event-processing-time-ms";
/**
* The event purgatory time metric name.
*/
public static final String EVENT_PURGATORY_TIME_METRIC_NAME = "event-purgatory-time-ms";
/**
* The flush time metric name.
*/
public static final String BATCH_FLUSH_TIME_METRIC_NAME = "batch-flush-time-ms";
/**
* Metric to count the number of partitions in Loading state.
*/
private final MetricName numPartitionsLoading;
private final AtomicLong numPartitionsLoadingCounter = new AtomicLong(0);
/**
* Metric to count the number of partitions in Active state.
*/
private final MetricName numPartitionsActive;
private final AtomicLong numPartitionsActiveCounter = new AtomicLong(0);
/**
* Metric to count the number of partitions in Failed state.
*/
private final MetricName numPartitionsFailed;
private final AtomicLong numPartitionsFailedCounter = new AtomicLong(0);
/**
* Metric to count the size of the processor queue.
*/
private final MetricName eventQueueSize;
/**
* The Kafka metrics registry.
*/
private final Metrics metrics;
/**
* The partition load sensor.
*/
private final Sensor partitionLoadSensor;
/**
* The thread idle sensor.
*/
private final Sensor threadIdleSensor;
/**
* The event queue time sensor.
*/
private final Sensor eventQueueTimeSensor;
/**
* The event processing time sensor.
*/
private final Sensor eventProcessingTimeSensor;
/**
* Sensor to measure the time an event stays in the purgatory.
*/
private final Sensor eventPurgatoryTimeSensor;
/**
* Sensor to measure the flush time.
*/
private final Sensor flushTimeSensor;
public CoordinatorRuntimeMetricsImpl(Metrics metrics, String metricsGroup) {
this.metrics = Objects.requireNonNull(metrics);
this.metricsGroup = Objects.requireNonNull(metricsGroup);
this.numPartitionsLoading = kafkaMetricName(
NUM_PARTITIONS_METRIC_NAME,
"The number of partitions in Loading state.",
"state", "loading"
);
this.numPartitionsActive = kafkaMetricName(
NUM_PARTITIONS_METRIC_NAME,
"The number of partitions in Active state.",
"state", "active"
);
this.numPartitionsFailed = kafkaMetricName(
NUM_PARTITIONS_METRIC_NAME,
"The number of partitions in Failed state.",
"state", "failed"
);
this.eventQueueSize = kafkaMetricName("event-queue-size", "The event accumulator queue size.");
metrics.addMetric(numPartitionsLoading, (Gauge<Long>) (config, now) -> numPartitionsLoadingCounter.get());
metrics.addMetric(numPartitionsActive, (Gauge<Long>) (config, now) -> numPartitionsActiveCounter.get());
metrics.addMetric(numPartitionsFailed, (Gauge<Long>) (config, now) -> numPartitionsFailedCounter.get());
this.partitionLoadSensor = metrics.sensor("GroupPartitionLoadTime");
this.partitionLoadSensor.add(
metrics.metricName(
"partition-load-time-max",
this.metricsGroup,
"The max time it took to load the partitions in the last 30 sec."
), new Max());
this.partitionLoadSensor.add(
metrics.metricName(
"partition-load-time-avg",
this.metricsGroup,
"The average time it took to load the partitions in the last 30 sec."
), new Avg());
this.threadIdleSensor = metrics.sensor("ThreadIdleRatio");
this.threadIdleSensor.add(
metrics.metricName(
"thread-idle-ratio-avg",
this.metricsGroup,
"The fraction of time the threads spent waiting for an event. This is an average across " +
"all coordinator event processor threads."),
new Rate(TimeUnit.MILLISECONDS));
KafkaMetricHistogram eventQueueTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
suffix -> kafkaMetricName(
EVENT_QUEUE_TIME_METRIC_NAME + "-" + suffix,
"The " + suffix + " event queue time in milliseconds"
)
);
this.eventQueueTimeSensor = metrics.sensor("EventQueueTime");
this.eventQueueTimeSensor.add(eventQueueTimeHistogram);
KafkaMetricHistogram eventProcessingTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
suffix -> kafkaMetricName(
EVENT_PROCESSING_TIME_METRIC_NAME + "-" + suffix,
"The " + suffix + " event processing time in milliseconds"
)
);
this.eventProcessingTimeSensor = metrics.sensor("EventProcessingTime");
this.eventProcessingTimeSensor.add(eventProcessingTimeHistogram);
KafkaMetricHistogram eventPurgatoryTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
suffix -> kafkaMetricName(
EVENT_PURGATORY_TIME_METRIC_NAME + "-" + suffix,
"The " + suffix + " event purgatory time in milliseconds"
)
);
this.eventPurgatoryTimeSensor = metrics.sensor("EventPurgatoryTime");
this.eventPurgatoryTimeSensor.add(eventPurgatoryTimeHistogram);
KafkaMetricHistogram flushTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
suffix -> kafkaMetricName(
BATCH_FLUSH_TIME_METRIC_NAME + "-" + suffix,
"The " + suffix + " flush time in milliseconds"
)
);
this.flushTimeSensor = metrics.sensor("FlushTime");
this.flushTimeSensor.add(flushTimeHistogram);
}
/**
* Retrieve the kafka metric name.
*
* @param name The name of the metric.
*
* @return The kafka metric name.
*/
private MetricName kafkaMetricName(String name, String description, String... keyValue) {
return metrics.metricName(name, this.metricsGroup, description, keyValue);
}
@Override
public void close() {
Arrays.asList(
numPartitionsLoading,
numPartitionsActive,
numPartitionsFailed,
eventQueueSize
).forEach(metrics::removeMetric);
metrics.removeSensor(partitionLoadSensor.name());
metrics.removeSensor(threadIdleSensor.name());
metrics.removeSensor(eventQueueTimeSensor.name());
metrics.removeSensor(eventProcessingTimeSensor.name());
metrics.removeSensor(eventPurgatoryTimeSensor.name());
metrics.removeSensor(flushTimeSensor.name());
}
/**
* Called when the partition state changes. Decrement the old state and increment the new state.
*
* @param oldState The old state.
* @param newState The new state to transition to.
*/
@Override
public void recordPartitionStateChange(CoordinatorState oldState, CoordinatorState newState) {
switch (oldState) {
case INITIAL:
case CLOSED:
break;
case LOADING:
numPartitionsLoadingCounter.decrementAndGet();
break;
case ACTIVE:
numPartitionsActiveCounter.decrementAndGet();
break;
case FAILED:
numPartitionsFailedCounter.decrementAndGet();
}
switch (newState) {
case INITIAL:
case CLOSED:
break;
case LOADING:
numPartitionsLoadingCounter.incrementAndGet();
break;
case ACTIVE:
numPartitionsActiveCounter.incrementAndGet();
break;
case FAILED:
numPartitionsFailedCounter.incrementAndGet();
}
}
@Override
public void recordPartitionLoadSensor(long startTimeMs, long endTimeMs) {
this.partitionLoadSensor.record(endTimeMs - startTimeMs, endTimeMs, false);
}
@Override
public void recordEventQueueTime(long durationMs) {
eventQueueTimeSensor.record(durationMs);
}
@Override
public void recordEventProcessingTime(long durationMs) {
eventProcessingTimeSensor.record(durationMs);
}
@Override
public void recordEventPurgatoryTime(long purgatoryTimeMs) {
eventPurgatoryTimeSensor.record(purgatoryTimeMs);
}
@Override
public void recordFlushTime(long durationMs) {
flushTimeSensor.record(durationMs);
}
@Override
public void recordThreadIdleTime(long idleTimeMs) {
threadIdleSensor.record(idleTimeMs);
}
@Override
public void registerEventQueueSizeGauge(Supplier<Integer> sizeSupplier) {
metrics.addMetric(eventQueueSize, (Gauge<Long>) (config, now) -> (long) sizeSupplier.get());
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.coordinator.group.metrics;
package org.apache.kafka.coordinator.common.runtime;
import org.HdrHistogram.Histogram;
import org.HdrHistogram.Recorder;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.coordinator.group.metrics;
package org.apache.kafka.coordinator.common.runtime;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.CompoundStat;

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.coordinator.group.metrics;
package org.apache.kafka.coordinator.common.runtime;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.KafkaMetric;
@ -31,17 +31,18 @@ import java.util.Arrays;
import java.util.HashSet;
import java.util.stream.IntStream;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorRuntimeMetrics.BATCH_FLUSH_TIME_METRIC_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorRuntimeMetrics.EVENT_PROCESSING_TIME_METRIC_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorRuntimeMetrics.EVENT_PURGATORY_TIME_METRIC_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorRuntimeMetrics.EVENT_QUEUE_TIME_METRIC_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorRuntimeMetrics.METRICS_GROUP;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorRuntimeMetrics.NUM_PARTITIONS_METRIC_NAME;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl.BATCH_FLUSH_TIME_METRIC_NAME;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl.EVENT_PROCESSING_TIME_METRIC_NAME;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl.EVENT_PURGATORY_TIME_METRIC_NAME;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl.EVENT_QUEUE_TIME_METRIC_NAME;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl.NUM_PARTITIONS_METRIC_NAME;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class GroupCoordinatorRuntimeMetricsTest {
public class CoordinatorRuntimeMetricsImplTest {
private static final String METRICS_GROUP = "test-runtime-metrics";
@Test
public void testMetricNames() {
@ -77,7 +78,7 @@ public class GroupCoordinatorRuntimeMetricsTest {
kafkaMetricName(metrics, "batch-flush-time-ms-p999")
));
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) {
try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP)) {
runtimeMetrics.registerEventQueueSizeGauge(() -> 0);
expectedMetrics.forEach(metricName -> assertTrue(metrics.metrics().containsKey(metricName)));
}
@ -92,7 +93,7 @@ public class GroupCoordinatorRuntimeMetricsTest {
public void testUpdateNumPartitionsMetrics() {
Metrics metrics = new Metrics();
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) {
try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP)) {
IntStream.range(0, 10)
.forEach(__ -> runtimeMetrics.recordPartitionStateChange(CoordinatorState.INITIAL, CoordinatorState.LOADING));
IntStream.range(0, 8)
@ -113,7 +114,7 @@ public class GroupCoordinatorRuntimeMetricsTest {
Time time = new MockTime();
Metrics metrics = new Metrics(time);
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) {
try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP)) {
long startTimeMs = time.milliseconds();
runtimeMetrics.recordPartitionLoadSensor(startTimeMs, startTimeMs + 1000);
runtimeMetrics.recordPartitionLoadSensor(startTimeMs, startTimeMs + 2000);
@ -134,7 +135,7 @@ public class GroupCoordinatorRuntimeMetricsTest {
Time time = new MockTime();
Metrics metrics = new Metrics(time);
GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics);
CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP);
IntStream.range(0, 3).forEach(i -> runtimeMetrics.recordThreadIdleTime((i + 1) * 1000L));
org.apache.kafka.common.MetricName metricName = kafkaMetricName(metrics, "thread-idle-ratio-avg");
@ -147,7 +148,7 @@ public class GroupCoordinatorRuntimeMetricsTest {
Time time = new MockTime();
Metrics metrics = new Metrics(time);
try (GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics)) {
try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP)) {
runtimeMetrics.registerEventQueueSizeGauge(() -> 5);
assertMetricGauge(metrics, kafkaMetricName(metrics, "event-queue-size"), 5);
}
@ -164,7 +165,7 @@ public class GroupCoordinatorRuntimeMetricsTest {
Time time = new MockTime();
Metrics metrics = new Metrics(time);
GroupCoordinatorRuntimeMetrics runtimeMetrics = new GroupCoordinatorRuntimeMetrics(metrics);
CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP);
IntStream.range(1, 1001).forEach(i -> {
switch (metricNamePrefix) {
@ -210,4 +211,4 @@ public class GroupCoordinatorRuntimeMetricsTest {
private static MetricName kafkaMetricName(Metrics metrics, String name, String... keyValue) {
return metrics.metricName(name, METRICS_GROUP, "", keyValue);
}
}
}

View File

@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.coordinator.group.metrics;
package org.apache.kafka.coordinator.common.runtime;
import com.yammer.metrics.core.Histogram;
import com.yammer.metrics.core.MetricName;

View File

@ -35,6 +35,7 @@ import kafka.server.share.SharePartitionManager;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.group.GroupCoordinator;
import org.apache.kafka.coordinator.share.ShareCoordinator;
import org.apache.kafka.server.ClientMetricsManager;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.storage.log.metrics.BrokerTopicStats;
@ -67,6 +68,7 @@ public class KafkaApisBuilder {
private DelegationTokenManager tokenManager = null;
private ApiVersionManager apiVersionManager = null;
private Optional<ClientMetricsManager> clientMetricsManager = Optional.empty();
private Optional<ShareCoordinator> shareCoordinator = Optional.empty();
public KafkaApisBuilder setRequestChannel(RequestChannel requestChannel) {
this.requestChannel = requestChannel;
@ -93,6 +95,11 @@ public class KafkaApisBuilder {
return this;
}
public KafkaApisBuilder setShareCoordinator(Optional<ShareCoordinator> shareCoordinator) {
this.shareCoordinator = shareCoordinator;
return this;
}
public KafkaApisBuilder setAutoTopicCreationManager(AutoTopicCreationManager autoTopicCreationManager) {
this.autoTopicCreationManager = autoTopicCreationManager;
return this;
@ -195,6 +202,7 @@ public class KafkaApisBuilder {
replicaManager,
groupCoordinator,
txnCoordinator,
OptionConverters.toScala(shareCoordinator),
autoTopicCreationManager,
brokerId,
config,

View File

@ -26,13 +26,14 @@ import kafka.utils.Logging
import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.errors.InvalidTopicException
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME}
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME}
import org.apache.kafka.common.message.CreateTopicsRequestData
import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicConfig, CreatableTopicConfigCollection}
import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{ApiError, CreateTopicsRequest, RequestContext, RequestHeader}
import org.apache.kafka.coordinator.group.GroupCoordinator
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager}
import scala.collection.{Map, Seq, Set, mutable}
@ -57,9 +58,10 @@ object AutoTopicCreationManager {
controller: Option[KafkaController],
groupCoordinator: GroupCoordinator,
txnCoordinator: TransactionCoordinator,
shareCoordinator: Option[ShareCoordinator],
): AutoTopicCreationManager = {
new DefaultAutoTopicCreationManager(config, channelManager, adminManager,
controller, groupCoordinator, txnCoordinator)
controller, groupCoordinator, txnCoordinator, shareCoordinator)
}
}
@ -69,7 +71,8 @@ class DefaultAutoTopicCreationManager(
adminManager: Option[ZkAdminManager],
controller: Option[KafkaController],
groupCoordinator: GroupCoordinator,
txnCoordinator: TransactionCoordinator
txnCoordinator: TransactionCoordinator,
shareCoordinator: Option[ShareCoordinator]
) extends AutoTopicCreationManager with Logging {
if (controller.isEmpty && channelManager.isEmpty) {
throw new IllegalArgumentException("Must supply a channel manager if not supplying a controller")
@ -244,6 +247,16 @@ class DefaultAutoTopicCreationManager(
.setReplicationFactor(config.transactionLogConfig.transactionTopicReplicationFactor)
.setConfigs(convertToTopicConfigCollections(
txnCoordinator.transactionTopicConfigs))
case SHARE_GROUP_STATE_TOPIC_NAME =>
val props = shareCoordinator match {
case Some(coordinator) => coordinator.shareGroupStateTopicConfigs()
case None => new Properties()
}
new CreatableTopic()
.setName(topic)
.setNumPartitions(config.shareCoordinatorConfig.shareCoordinatorStateTopicNumPartitions())
.setReplicationFactor(config.shareCoordinatorConfig.shareCoordinatorStateTopicReplicationFactor())
.setConfigs(convertToTopicConfigCollections(props))
case topicName =>
new CreatableTopic()
.setName(topicName)

View File

@ -38,6 +38,8 @@ import org.apache.kafka.common.{ClusterResource, TopicPartition, Uuid}
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord
import org.apache.kafka.coordinator.group.metrics.{GroupCoordinatorMetrics, GroupCoordinatorRuntimeMetrics}
import org.apache.kafka.coordinator.group.{GroupConfigManager, GroupCoordinator, GroupCoordinatorRecordSerde, GroupCoordinatorService}
import org.apache.kafka.coordinator.share.metrics.{ShareCoordinatorMetrics, ShareCoordinatorRuntimeMetrics}
import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorRecordSerde, ShareCoordinatorService}
import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher}
import org.apache.kafka.metadata.{BrokerState, ListenerInfo}
import org.apache.kafka.security.CredentialProvider
@ -121,6 +123,8 @@ class BrokerServer(
var transactionCoordinator: TransactionCoordinator = _
var shareCoordinator: Option[ShareCoordinator] = _
var clientToControllerChannelManager: NodeToControllerChannelManager = _
var forwardingManager: ForwardingManager = _
@ -341,6 +345,8 @@ class BrokerServer(
tokenManager = new DelegationTokenManager(config, tokenCache, time)
tokenManager.startup()
shareCoordinator = createShareCoordinator()
groupCoordinator = createGroupCoordinator()
val producerIdManagerSupplier = () => ProducerIdManager.rpc(
@ -358,7 +364,7 @@ class BrokerServer(
autoTopicCreationManager = new DefaultAutoTopicCreationManager(
config, Some(clientToControllerChannelManager), None, None,
groupCoordinator, transactionCoordinator)
groupCoordinator, transactionCoordinator, shareCoordinator)
dynamicConfigHandlers = Map[String, ConfigHandler](
ConfigType.TOPIC -> new TopicConfigHandler(replicaManager, config, quotaManagers, None),
@ -427,6 +433,7 @@ class BrokerServer(
replicaManager = replicaManager,
groupCoordinator = groupCoordinator,
txnCoordinator = transactionCoordinator,
shareCoordinator = shareCoordinator,
autoTopicCreationManager = autoTopicCreationManager,
brokerId = config.nodeId,
config = config,
@ -473,6 +480,7 @@ class BrokerServer(
replicaManager,
groupCoordinator,
transactionCoordinator,
shareCoordinator,
new DynamicConfigPublisher(
config,
sharedServer.metadataPublishingFaultHandler,
@ -625,6 +633,36 @@ class BrokerServer(
}
}
private def createShareCoordinator(): Option[ShareCoordinator] = {
if (!config.shareGroupConfig.isShareGroupEnabled) {
return None
}
val time = Time.SYSTEM
val timer = new SystemTimerReaper(
"share-coordinator-reaper",
new SystemTimer("share-coordinator")
)
val serde = new ShareCoordinatorRecordSerde
val loader = new CoordinatorLoaderImpl[CoordinatorRecord](
time,
replicaManager,
serde,
config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize()
)
val writer = new CoordinatorPartitionWriter(
replicaManager
)
Some(new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig)
.withTimer(timer)
.withTime(time)
.withLoader(loader)
.withWriter(writer)
.withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics))
.withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics))
.build())
}
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time,
@ -692,6 +730,8 @@ class BrokerServer(
CoreUtils.swallow(transactionCoordinator.shutdown(), this)
if (groupCoordinator != null)
CoreUtils.swallow(groupCoordinator.shutdown(), this)
if (shareCoordinator.isDefined)
CoreUtils.swallow(shareCoordinator.get.shutdown(), this)
if (tokenManager != null)
CoreUtils.swallow(tokenManager.shutdown(), this)

View File

@ -35,7 +35,7 @@ import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal}
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal}
import org.apache.kafka.common.internals.{FatalExitError, Topic}
import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.{AddPartitionsToTxnResult, AddPartitionsToTxnResultCollection}
import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse
@ -70,6 +70,7 @@ import org.apache.kafka.common.security.token.delegation.{DelegationToken, Token
import org.apache.kafka.common.utils.{ProducerIdAndEpoch, Time}
import org.apache.kafka.common.{Node, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.coordinator.group.{Group, GroupCoordinator}
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.server.ClientMetricsManager
import org.apache.kafka.server.authorizer._
import org.apache.kafka.server.common.{GroupVersion, MetadataVersion, RequestLocal}
@ -100,6 +101,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val replicaManager: ReplicaManager,
val groupCoordinator: GroupCoordinator,
val txnCoordinator: TransactionCoordinator,
val shareCoordinator: Option[ShareCoordinator],
val autoTopicCreationManager: AutoTopicCreationManager,
val brokerId: Int,
val config: KafkaConfig,
@ -1695,10 +1697,13 @@ class KafkaApis(val requestChannel: RequestChannel,
(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, Node.noNode)
else if (keyType == CoordinatorType.SHARE.id && request.context.apiVersion < 6)
(Errors.INVALID_REQUEST, Node.noNode)
else if (keyType == CoordinatorType.SHARE.id &&
!authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME))
(Errors.CLUSTER_AUTHORIZATION_FAILED, Node.noNode)
else {
if (keyType == CoordinatorType.SHARE.id) {
authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
if (shareCoordinator.isEmpty) {
return (Errors.INVALID_REQUEST, Node.noNode)
}
}
val (partition, internalTopicName) = CoordinatorType.forId(keyType) match {
case CoordinatorType.GROUP =>
(groupCoordinator.partitionFor(key), GROUP_METADATA_TOPIC_NAME)
@ -1707,8 +1712,8 @@ class KafkaApis(val requestChannel: RequestChannel,
(txnCoordinator.partitionFor(key), TRANSACTION_STATE_TOPIC_NAME)
case CoordinatorType.SHARE =>
// When share coordinator support is implemented in KIP-932, a proper check will go here
return (Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode)
// None check already done above
(shareCoordinator.get.partitionFor(key), SHARE_GROUP_STATE_TOPIC_NAME)
}
val topicMetadata = metadataCache.getTopicMetadata(Set(internalTopicName), request.context.listenerName)
@ -4462,18 +4467,46 @@ class KafkaApis(val requestChannel: RequestChannel,
CompletableFuture.completedFuture[Unit](())
}
def handleReadShareGroupStateRequest(request: RequestChannel.Request): Unit = {
def handleReadShareGroupStateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = {
val readShareGroupStateRequest = request.body[ReadShareGroupStateRequest]
// TODO: Implement the ReadShareGroupStateRequest handling
requestHelper.sendMaybeThrottle(request, readShareGroupStateRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
CompletableFuture.completedFuture[Unit](())
authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
shareCoordinator match {
case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
readShareGroupStateRequest.getErrorResponse(requestThrottleMs,
new ApiException("Share coordinator is not enabled.")))
CompletableFuture.completedFuture[Unit](())
case Some(coordinator) => coordinator.readState(request.context, readShareGroupStateRequest.data)
.handle[Unit] { (response, exception) =>
if (exception != null) {
requestHelper.sendMaybeThrottle(request, readShareGroupStateRequest.getErrorResponse(exception))
} else {
requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateResponse(response))
}
}
}
}
def handleWriteShareGroupStateRequest(request: RequestChannel.Request): Unit = {
val writeShareGroupStateRequest = request.body[WriteShareGroupStateRequest]
// TODO: Implement the WriteShareGroupStateRequest handling
requestHelper.sendMaybeThrottle(request, writeShareGroupStateRequest.getErrorResponse(Errors.UNSUPPORTED_VERSION.exception))
CompletableFuture.completedFuture[Unit](())
def handleWriteShareGroupStateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = {
val writeShareRequest = request.body[WriteShareGroupStateRequest]
authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
shareCoordinator match {
case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
writeShareRequest.getErrorResponse(requestThrottleMs,
new ApiException("Share coordinator is not enabled.")))
CompletableFuture.completedFuture[Unit](())
case Some(coordinator) => coordinator.writeState(request.context, writeShareRequest.data)
.handle[Unit] { (response, exception) =>
if (exception != null) {
requestHelper.sendMaybeThrottle(request, writeShareRequest.getErrorResponse(exception))
} else {
requestHelper.sendMaybeThrottle(request, new WriteShareGroupStateResponse(response))
}
}
}
}
def handleDeleteShareGroupStateRequest(request: RequestChannel.Request): Unit = {

View File

@ -44,7 +44,7 @@ import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion._
import org.apache.kafka.server.config.{AbstractKafkaConfig, DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ShareGroupConfig, ZkConfigs}
import org.apache.kafka.server.config.{AbstractKafkaConfig, DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ShareCoordinatorConfig, ShareGroupConfig, ZkConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.MetricConfigs
import org.apache.kafka.server.util.Csv
@ -238,6 +238,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
private val _shareGroupConfig = new ShareGroupConfig(this)
def shareGroupConfig: ShareGroupConfig = _shareGroupConfig
private val _shareCoordinatorConfig = new ShareCoordinatorConfig(this)
def shareCoordinatorConfig: ShareCoordinatorConfig = _shareCoordinatorConfig
private val _transactionLogConfig = new TransactionLogConfig(this)
private val _transactionStateManagerConfig = new TransactionStateManagerConfig(this)
def transactionLogConfig: TransactionLogConfig = _transactionLogConfig

View File

@ -534,7 +534,8 @@ class KafkaServer(
Some(adminManager),
Some(kafkaController),
groupCoordinator,
transactionCoordinator
transactionCoordinator,
None
)
/* Get the authorizer and initialize it if one is specified.*/
@ -586,6 +587,7 @@ class KafkaServer(
replicaManager = replicaManager,
groupCoordinator = groupCoordinator,
txnCoordinator = transactionCoordinator,
shareCoordinator = None, //share coord only supported in kraft mode
autoTopicCreationManager = autoTopicCreationManager,
brokerId = config.brokerId,
config = config,

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.TimeoutException
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.coordinator.group.GroupCoordinator
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.image.loader.LoaderManifest
import org.apache.kafka.image.publisher.MetadataPublisher
import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta}
@ -67,6 +68,7 @@ class BrokerMetadataPublisher(
replicaManager: ReplicaManager,
groupCoordinator: GroupCoordinator,
txnCoordinator: TransactionCoordinator,
shareCoordinator: Option[ShareCoordinator],
var dynamicConfigPublisher: DynamicConfigPublisher,
dynamicClientQuotaPublisher: DynamicClientQuotaPublisher,
scramPublisher: ScramPublisher,
@ -160,6 +162,19 @@ class BrokerMetadataPublisher(
case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating txn " +
s"coordinator with local changes in $deltaName", t)
}
if (shareCoordinator.isDefined) {
try {
updateCoordinator(newImage,
delta,
Topic.SHARE_GROUP_STATE_TOPIC_NAME,
shareCoordinator.get.onElection,
(partitionIndex, leaderEpochOpt) => shareCoordinator.get.onResignation(partitionIndex, toOptionalInt(leaderEpochOpt))
)
} catch {
case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " +
s"coordinator with local changes in $deltaName", t)
}
}
try {
// Notify the group coordinator about deleted topics.
val deletedTopicPartitions = new mutable.ArrayBuffer[TopicPartition]()
@ -313,6 +328,15 @@ class BrokerMetadataPublisher(
} catch {
case t: Throwable => fatalFaultHandler.handleFault("Error starting TransactionCoordinator", t)
}
if (config.shareGroupConfig.isShareGroupEnabled && shareCoordinator.isDefined) {
try {
// Start the share coordinator.
shareCoordinator.get.startup(() => metadataCache.numPartitions(
Topic.SHARE_GROUP_STATE_TOPIC_NAME).getOrElse(config.shareCoordinatorConfig.shareCoordinatorStateTopicNumPartitions()))
} catch {
case t: Throwable => fatalFaultHandler.handleFault("Error starting Share coordinator", t)
}
}
}
private def finishInitializingReplicaManager(): Unit = {

View File

@ -27,7 +27,7 @@ import kafka.utils.TestUtils
import org.apache.kafka.clients.{ClientResponse, NodeApiVersions, RequestCompletionHandler}
import org.apache.kafka.common.Node
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME}
import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME}
import org.apache.kafka.common.message.{ApiVersionsResponseData, CreateTopicsRequestData}
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic
import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic
@ -37,8 +37,9 @@ import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerde, SecurityProtocol}
import org.apache.kafka.common.utils.{SecurityUtils, Utils}
import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig}
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.server.config.{ServerConfigs, ShareCoordinatorConfig}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
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}
@ -58,6 +59,7 @@ class AutoTopicCreationManagerTest {
private val controller = Mockito.mock(classOf[KafkaController])
private val groupCoordinator = Mockito.mock(classOf[GroupCoordinator])
private val transactionCoordinator = Mockito.mock(classOf[TransactionCoordinator])
private val shareCoordinator = Mockito.mock(classOf[ShareCoordinator])
private var autoTopicCreationManager: AutoTopicCreationManager = _
private val internalTopicPartitions = 2
@ -70,14 +72,16 @@ class AutoTopicCreationManagerTest {
props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString)
props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString)
props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_REPLICATION_FACTOR_CONFIG , internalTopicPartitions.toString)
props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString)
props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString)
props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_NUM_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString)
config = KafkaConfig.fromProps(props)
val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1))
Mockito.reset(metadataCache, controller, brokerToController, groupCoordinator, transactionCoordinator)
Mockito.reset(metadataCache, controller, brokerToController, groupCoordinator, transactionCoordinator, shareCoordinator)
Mockito.when(metadataCache.getAliveBrokerNodes(any(classOf[ListenerName]))).thenReturn(aliveBrokers)
}
@ -94,6 +98,12 @@ class AutoTopicCreationManagerTest {
testCreateTopic(TRANSACTION_STATE_TOPIC_NAME, isInternal = true, internalTopicPartitions, internalTopicReplicationFactor)
}
@Test
def testCreateShareStateTopic(): Unit = {
Mockito.when(shareCoordinator.shareGroupStateTopicConfigs()).thenReturn(new Properties)
testCreateTopic(SHARE_GROUP_STATE_TOPIC_NAME, isInternal = true, internalTopicPartitions, internalTopicReplicationFactor)
}
@Test
def testCreateNonInternalTopic(): Unit = {
testCreateTopic("topic", isInternal = false)
@ -109,7 +119,8 @@ class AutoTopicCreationManagerTest {
Some(adminManager),
Some(controller),
groupCoordinator,
transactionCoordinator)
transactionCoordinator,
Some(shareCoordinator))
val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection
topicsCollection.add(getNewTopic(topicName, numPartitions, replicationFactor))
@ -137,7 +148,8 @@ class AutoTopicCreationManagerTest {
Some(adminManager),
Some(controller),
groupCoordinator,
transactionCoordinator)
transactionCoordinator,
Some(shareCoordinator))
val topicName = "topic"
@ -320,7 +332,8 @@ class AutoTopicCreationManagerTest {
Some(adminManager),
Some(controller),
groupCoordinator,
transactionCoordinator)
transactionCoordinator,
Some(shareCoordinator))
val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection
topicsCollection.add(getNewTopic(topicName))
@ -350,7 +363,8 @@ class AutoTopicCreationManagerTest {
Some(adminManager),
Some(controller),
groupCoordinator,
transactionCoordinator)
transactionCoordinator,
Some(shareCoordinator))
Mockito.when(controller.isActive).thenReturn(false)
val newTopic = if (isInternal) {

View File

@ -76,6 +76,7 @@ import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource
import org.apache.kafka.common.utils.{ImplicitLinkedHashCollection, ProducerIdAndEpoch, SecurityUtils, Utils}
import org.apache.kafka.coordinator.group.GroupConfig.{CONSUMER_HEARTBEAT_INTERVAL_MS_CONFIG, CONSUMER_SESSION_TIMEOUT_MS_CONFIG}
import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig}
import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorConfigTest}
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.network.metrics.{RequestChannelMetrics, RequestMetrics}
@ -117,6 +118,7 @@ class KafkaApisTest extends Logging {
private val requestChannelMetrics: RequestChannelMetrics = mock(classOf[RequestChannelMetrics])
private val replicaManager: ReplicaManager = mock(classOf[ReplicaManager])
private val groupCoordinator: GroupCoordinator = mock(classOf[GroupCoordinator])
private val shareCoordinator: ShareCoordinator = mock(classOf[ShareCoordinator])
private val adminManager: ZkAdminManager = mock(classOf[ZkAdminManager])
private val txnCoordinator: TransactionCoordinator = mock(classOf[TransactionCoordinator])
private val controller: KafkaController = mock(classOf[KafkaController])
@ -225,6 +227,7 @@ class KafkaApisTest extends Logging {
replicaManager = replicaManager,
groupCoordinator = groupCoordinator,
txnCoordinator = txnCoordinator,
shareCoordinator = Some(shareCoordinator),
autoTopicCreationManager = autoTopicCreationManager,
brokerId = brokerId,
config = config,
@ -11537,6 +11540,226 @@ class KafkaApisTest extends Logging {
assertEquals(Errors.NONE.code(), response.data.groups.get(1).errorCode())
}
@Test
def testReadShareGroupStateSuccess(): Unit = {
val topicId = Uuid.randomUuid();
val readRequestData = new ReadShareGroupStateRequestData()
.setGroupId("group1")
.setTopics(List(
new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(topicId)
.setPartitions(List(
new ReadShareGroupStateRequestData.PartitionData()
.setPartition(1)
.setLeaderEpoch(1)
).asJava)
).asJava)
val readStateResultData: util.List[ReadShareGroupStateResponseData.ReadStateResult] = List(
new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId)
.setPartitions(List(
new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(1)
.setErrorCode(Errors.NONE.code())
.setErrorMessage(null)
.setStateEpoch(1)
.setStartOffset(10)
.setStateBatches(List(
new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(11)
.setLastOffset(15)
.setDeliveryState(0)
.setDeliveryCount(1)
).asJava)
).asJava)
).asJava
val config = Map(
ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true",
)
val response = getReadShareGroupResponse(
readRequestData,
config ++ ShareCoordinatorConfigTest.testConfigMap().asScala,
verifyNoErr = true,
null,
readStateResultData
)
assertNotNull(response.data)
assertEquals(1, response.data.results.size)
}
@Test
def testReadShareGroupStateAuthorizationFailed(): Unit = {
val topicId = Uuid.randomUuid();
val readRequestData = new ReadShareGroupStateRequestData()
.setGroupId("group1")
.setTopics(List(
new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(topicId)
.setPartitions(List(
new ReadShareGroupStateRequestData.PartitionData()
.setPartition(1)
.setLeaderEpoch(1)
).asJava)
).asJava)
val readStateResultData: util.List[ReadShareGroupStateResponseData.ReadStateResult] = List(
new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId)
.setPartitions(List(
new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(1)
.setErrorCode(Errors.NONE.code())
.setErrorMessage(null)
.setStateEpoch(1)
.setStartOffset(10)
.setStateBatches(List(
new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(11)
.setLastOffset(15)
.setDeliveryState(0)
.setDeliveryCount(1)
).asJava)
).asJava)
).asJava
val authorizer: Authorizer = mock(classOf[Authorizer])
when(authorizer.authorize(any[RequestContext], any[util.List[Action]]))
.thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava)
val config = Map(
ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true",
)
val response = getReadShareGroupResponse(
readRequestData,
config ++ ShareCoordinatorConfigTest.testConfigMap().asScala,
verifyNoErr = false,
authorizer,
readStateResultData
)
assertNotNull(response.data)
assertEquals(1, response.data.results.size)
response.data.results.forEach(readResult => {
assertEquals(1, readResult.partitions.size)
assertEquals(Errors.CLUSTER_AUTHORIZATION_FAILED.code(), readResult.partitions.get(0).errorCode())
})
}
@Test
def testWriteShareGroupStateSuccess(): Unit = {
val topicId = Uuid.randomUuid();
val writeRequestData = new WriteShareGroupStateRequestData()
.setGroupId("group1")
.setTopics(List(
new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(topicId)
.setPartitions(List(
new WriteShareGroupStateRequestData.PartitionData()
.setPartition(1)
.setLeaderEpoch(1)
.setStateEpoch(2)
.setStartOffset(10)
.setStateBatches(List(
new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(11)
.setLastOffset(15)
.setDeliveryCount(1)
.setDeliveryState(0)
).asJava)
).asJava)
).asJava)
val writeStateResultData: util.List[WriteShareGroupStateResponseData.WriteStateResult] = List(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId)
.setPartitions(List(
new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(1)
.setErrorCode(Errors.NONE.code())
.setErrorMessage(null)
).asJava)
).asJava
val config = Map(
ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true",
)
val response = getWriteShareGroupResponse(
writeRequestData,
config ++ ShareCoordinatorConfigTest.testConfigMap().asScala,
verifyNoErr = true,
null,
writeStateResultData
)
assertNotNull(response.data)
assertEquals(1, response.data.results.size)
}
@Test
def testWriteShareGroupStateAuthorizationFailed(): Unit = {
val topicId = Uuid.randomUuid();
val writeRequestData = new WriteShareGroupStateRequestData()
.setGroupId("group1")
.setTopics(List(
new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(topicId)
.setPartitions(List(
new WriteShareGroupStateRequestData.PartitionData()
.setPartition(1)
.setLeaderEpoch(1)
.setStateEpoch(2)
.setStartOffset(10)
.setStateBatches(List(
new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(11)
.setLastOffset(15)
.setDeliveryCount(1)
.setDeliveryState(0)
).asJava)
).asJava)
).asJava)
val writeStateResultData: util.List[WriteShareGroupStateResponseData.WriteStateResult] = List(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId)
.setPartitions(List(
new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(1)
.setErrorCode(Errors.NONE.code())
.setErrorMessage(null)
).asJava)
).asJava
val config = Map(
ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true",
)
val authorizer: Authorizer = mock(classOf[Authorizer])
when(authorizer.authorize(any[RequestContext], any[util.List[Action]]))
.thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava)
val response = getWriteShareGroupResponse(
writeRequestData,
config ++ ShareCoordinatorConfigTest.testConfigMap().asScala,
verifyNoErr = false,
authorizer,
writeStateResultData
)
assertNotNull(response.data)
assertEquals(1, response.data.results.size)
response.data.results.forEach(writeResult => {
assertEquals(1, writeResult.partitions.size)
assertEquals(Errors.CLUSTER_AUTHORIZATION_FAILED.code(), writeResult.partitions.get(0).errorCode())
})
}
def getShareGroupDescribeResponse(groupIds: util.List[String], configOverrides: Map[String, String] = Map.empty,
verifyNoErr: Boolean = true, authorizer: Authorizer = null,
describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup]): ShareGroupDescribeResponse = {
@ -11567,4 +11790,64 @@ class KafkaApisTest extends Logging {
}
response
}
def getReadShareGroupResponse(requestData: ReadShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty,
verifyNoErr: Boolean = true, authorizer: Authorizer = null,
readStateResult: util.List[ReadShareGroupStateResponseData.ReadStateResult]): ReadShareGroupStateResponse = {
val requestChannelRequest = buildRequest(new ReadShareGroupStateRequest.Builder(requestData, true).build())
val future = new CompletableFuture[ReadShareGroupStateResponseData]()
when(shareCoordinator.readState(
any[RequestContext],
any[ReadShareGroupStateRequestData]
)).thenReturn(future)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0)
kafkaApis = createKafkaApis(
overrideProperties = configOverrides,
authorizer = Option(authorizer),
raftSupport = true
)
kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching())
future.complete(new ReadShareGroupStateResponseData()
.setResults(readStateResult))
val response = verifyNoThrottling[ReadShareGroupStateResponse](requestChannelRequest)
if (verifyNoErr) {
val expectedReadShareGroupStateResponseData = new ReadShareGroupStateResponseData()
.setResults(readStateResult)
assertEquals(expectedReadShareGroupStateResponseData, response.data)
}
response
}
def getWriteShareGroupResponse(requestData: WriteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty,
verifyNoErr: Boolean = true, authorizer: Authorizer = null,
writeStateResult: util.List[WriteShareGroupStateResponseData.WriteStateResult]): WriteShareGroupStateResponse = {
val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData, true).build())
val future = new CompletableFuture[WriteShareGroupStateResponseData]()
when(shareCoordinator.writeState(
any[RequestContext],
any[WriteShareGroupStateRequestData]
)).thenReturn(future)
metadataCache = MetadataCache.kRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0)
kafkaApis = createKafkaApis(
overrideProperties = configOverrides,
authorizer = Option(authorizer),
raftSupport = true
)
kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching())
future.complete(new WriteShareGroupStateResponseData()
.setResults(writeStateResult))
val response = verifyNoThrottling[WriteShareGroupStateResponse](requestChannelRequest)
if (verifyNoErr) {
val expectedWriteShareGroupStateResponseData = new WriteShareGroupStateResponseData()
.setResults(writeStateResult)
assertEquals(expectedWriteShareGroupStateResponseData, response.data)
}
response
}
}

View File

@ -32,6 +32,7 @@ import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.config.ConfigResource.Type.BROKER
import org.apache.kafka.common.utils.Exit
import org.apache.kafka.coordinator.group.GroupCoordinator
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataImageTest, MetadataProvenance}
import org.apache.kafka.image.loader.LogDeltaManifest
import org.apache.kafka.network.SocketServerConfigs
@ -194,6 +195,7 @@ class BrokerMetadataPublisherTest {
replicaManager,
groupCoordinator,
mock(classOf[TransactionCoordinator]),
Some(mock(classOf[ShareCoordinator])),
mock(classOf[DynamicConfigPublisher]),
mock(classOf[DynamicClientQuotaPublisher]),
mock(classOf[ScramPublisher]),

View File

@ -93,10 +93,11 @@ import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.BiFunction;
import java.util.function.IntSupplier;
import java.util.stream.Collectors;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorOperationExceptionHelper.handleOperationException;
/**
* The group coordinator service.
*/
@ -336,7 +337,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
exception,
(error, message) -> new ConsumerGroupHeartbeatResponseData()
.setErrorCode(error.code())
.setErrorMessage(message)
.setErrorMessage(message),
log
));
}
@ -365,7 +367,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
exception,
(error, message) -> new ShareGroupHeartbeatResponseData()
.setErrorCode(error.code())
.setErrorMessage(message)
.setErrorMessage(message),
log
));
}
@ -413,7 +416,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"classic-group-join",
request,
exception,
(error, __) -> new JoinGroupResponseData().setErrorCode(error.code())
(error, __) -> new JoinGroupResponseData().setErrorCode(error.code()),
log
));
}
return null;
@ -456,7 +460,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"classic-group-sync",
request,
exception,
(error, __) -> new SyncGroupResponseData().setErrorCode(error.code())
(error, __) -> new SyncGroupResponseData().setErrorCode(error.code()),
log
));
}
return null;
@ -503,7 +508,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
return new HeartbeatResponseData()
.setErrorCode(error.code());
}
}
},
log
));
}
@ -551,7 +557,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
return new LeaveGroupResponseData()
.setErrorCode(error.code());
}
}
},
log
));
}
@ -595,7 +602,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"list-groups",
request,
exception,
(error, __) -> new ListGroupsResponseData().setErrorCode(error.code())
(error, __) -> new ListGroupsResponseData().setErrorCode(error.code()),
log
));
}
@ -641,7 +649,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"consumer-group-describe",
groupList,
exception,
(error, __) -> ConsumerGroupDescribeRequest.getErrorDescribedGroupList(groupList, error)
(error, __) -> ConsumerGroupDescribeRequest.getErrorDescribedGroupList(groupList, error),
log
));
futures.add(future);
@ -691,7 +700,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"share-group-describe",
groupList,
exception,
(error, __) -> ShareGroupDescribeRequest.getErrorDescribedGroupList(groupList, error)
(error, __) -> ShareGroupDescribeRequest.getErrorDescribedGroupList(groupList, error),
log
));
futures.add(future);
@ -744,7 +754,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"describe-groups",
groupList,
exception,
(error, __) -> DescribeGroupsRequest.getErrorDescribedGroupList(groupList, error)
(error, __) -> DescribeGroupsRequest.getErrorDescribedGroupList(groupList, error),
log
));
futures.add(future);
@ -799,7 +810,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"delete-groups",
groupList,
exception,
(error, __) -> DeleteGroupsRequest.getErrorResultCollection(groupList, error)
(error, __) -> DeleteGroupsRequest.getErrorResultCollection(groupList, error),
log
));
futures.add(future);
@ -954,7 +966,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"commit-offset",
request,
exception,
(error, __) -> OffsetCommitRequest.getErrorResponse(request, error)
(error, __) -> OffsetCommitRequest.getErrorResponse(request, error),
log
));
}
@ -994,7 +1007,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"txn-commit-offset",
request,
exception,
(error, __) -> TxnOffsetCommitRequest.getErrorResponse(request, error)
(error, __) -> TxnOffsetCommitRequest.getErrorResponse(request, error),
log
));
}
@ -1028,7 +1042,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
"delete-offsets",
request,
exception,
(error, __) -> new OffsetDeleteResponseData().setErrorCode(error.code())
(error, __) -> new OffsetDeleteResponseData().setErrorCode(error.code()),
log
));
}
@ -1215,61 +1230,6 @@ public class GroupCoordinatorService implements GroupCoordinator {
return groupId != null && !groupId.isEmpty();
}
/**
* This is the handler commonly used by all the operations that requires to convert errors to
* coordinator errors. The handler also handles and log unexpected errors.
*
* @param operationName The name of the operation.
* @param operationInput The operation's input for logging purposes.
* @param exception The exception to handle.
* @param handler A function which takes an Errors and a String and builds the expected
* output. The String can be null. Note that the function could further
* transform the error depending on the context.
* @return The output built by the handler.
* @param <IN> The type of the operation input. It must be a toString'able object.
* @param <OUT> The type of the value returned by handler.
*/
private <IN, OUT> OUT handleOperationException(
String operationName,
IN operationInput,
Throwable exception,
BiFunction<Errors, String, OUT> handler
) {
ApiError apiError = ApiError.fromThrowable(exception);
switch (apiError.error()) {
case UNKNOWN_SERVER_ERROR:
log.error("Operation {} with {} hit an unexpected exception: {}.",
operationName, operationInput, exception.getMessage(), exception);
return handler.apply(Errors.UNKNOWN_SERVER_ERROR, null);
case NETWORK_EXCEPTION:
// When committing offsets transactionally, we now verify the transaction with the
// transaction coordinator. Verification can fail with `NETWORK_EXCEPTION`, a
// retriable error which older clients may not expect and retry correctly. We
// translate the error to `COORDINATOR_LOAD_IN_PROGRESS` because it causes clients
// to retry the request without an unnecessary coordinator lookup.
return handler.apply(Errors.COORDINATOR_LOAD_IN_PROGRESS, null);
case UNKNOWN_TOPIC_OR_PARTITION:
case NOT_ENOUGH_REPLICAS:
case REQUEST_TIMED_OUT:
return handler.apply(Errors.COORDINATOR_NOT_AVAILABLE, null);
case NOT_LEADER_OR_FOLLOWER:
case KAFKA_STORAGE_ERROR:
return handler.apply(Errors.NOT_COORDINATOR, null);
case MESSAGE_TOO_LARGE:
case RECORD_LIST_TOO_LARGE:
case INVALID_FETCH_SIZE:
return handler.apply(Errors.UNKNOWN_SERVER_ERROR, null);
default:
return handler.apply(apiError.error(), apiError.message());
}
}
/**
* This is the handler used by offset fetch operations to convert errors to coordinator errors.
* The handler also handles and log unexpected errors.
@ -1310,7 +1270,8 @@ public class GroupCoordinatorService implements GroupCoordinator {
exception,
(error, __) -> new OffsetFetchResponseData.OffsetFetchResponseGroup()
.setGroupId(request.groupId())
.setErrorCode(error.code())
.setErrorCode(error.code()),
log
);
}
}

View File

@ -28,6 +28,7 @@ import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.coordinator.group.Group;
import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState;
import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
import org.apache.kafka.server.metrics.KafkaYammerMetrics;
import org.apache.kafka.timeline.SnapshotRegistry;
@ -67,14 +68,18 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
public static final String GROUP_COUNT_METRIC_NAME = "group-count";
public static final String GROUP_COUNT_PROTOCOL_TAG = "protocol";
public static final String SHARE_GROUP_PROTOCOL_TAG = GROUP_COUNT_PROTOCOL_TAG;
public static final String CONSUMER_GROUP_COUNT_METRIC_NAME = "consumer-group-count";
public static final String SHARE_GROUP_COUNT_METRIC_NAME = "group-count";
public static final String CONSUMER_GROUP_COUNT_STATE_TAG = "state";
public static final String SHARE_GROUP_COUNT_STATE_TAG = CONSUMER_GROUP_COUNT_STATE_TAG;
public static final String OFFSET_COMMITS_SENSOR_NAME = "OffsetCommits";
public static final String OFFSET_EXPIRED_SENSOR_NAME = "OffsetExpired";
public static final String OFFSET_DELETIONS_SENSOR_NAME = "OffsetDeletions";
public static final String CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME = "CompletedRebalances";
public static final String CONSUMER_GROUP_REBALANCES_SENSOR_NAME = "ConsumerGroupRebalances";
public static final String SHARE_GROUP_REBALANCES_SENSOR_NAME = "ShareGroupRebalances";
private final MetricName classicGroupCountMetricName;
private final MetricName consumerGroupCountMetricName;
@ -83,6 +88,10 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
private final MetricName consumerGroupCountReconcilingMetricName;
private final MetricName consumerGroupCountStableMetricName;
private final MetricName consumerGroupCountDeadMetricName;
private final MetricName shareGroupCountMetricName;
private final MetricName shareGroupCountEmptyMetricName;
private final MetricName shareGroupCountStableMetricName;
private final MetricName shareGroupCountDeadMetricName;
private final MetricsRegistry registry;
private final Metrics metrics;
@ -150,6 +159,37 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
Collections.singletonMap(CONSUMER_GROUP_COUNT_STATE_TAG, ConsumerGroupState.DEAD.toString())
);
shareGroupCountMetricName = metrics.metricName(
SHARE_GROUP_COUNT_METRIC_NAME,
METRICS_GROUP,
"The total number of share groups.",
Collections.singletonMap(SHARE_GROUP_PROTOCOL_TAG, Group.GroupType.SHARE.toString())
);
shareGroupCountEmptyMetricName = metrics.metricName(
SHARE_GROUP_COUNT_METRIC_NAME,
METRICS_GROUP,
"The number of share groups in empty state.",
SHARE_GROUP_PROTOCOL_TAG, Group.GroupType.SHARE.toString(),
SHARE_GROUP_COUNT_STATE_TAG, ShareGroup.ShareGroupState.EMPTY.toString()
);
shareGroupCountStableMetricName = metrics.metricName(
SHARE_GROUP_COUNT_METRIC_NAME,
METRICS_GROUP,
"The number of share groups in stable state.",
SHARE_GROUP_PROTOCOL_TAG, Group.GroupType.SHARE.toString(),
SHARE_GROUP_COUNT_STATE_TAG, ShareGroup.ShareGroupState.STABLE.toString()
);
shareGroupCountDeadMetricName = metrics.metricName(
SHARE_GROUP_COUNT_METRIC_NAME,
METRICS_GROUP,
"The number of share groups in dead state.",
SHARE_GROUP_PROTOCOL_TAG, Group.GroupType.SHARE.toString(),
SHARE_GROUP_COUNT_STATE_TAG, ShareGroup.ShareGroupState.DEAD.toString()
);
registerGauges();
Sensor offsetCommitsSensor = metrics.sensor(OFFSET_COMMITS_SENSOR_NAME);
@ -197,12 +237,24 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
METRICS_GROUP,
"The total number of consumer group rebalances")));
Sensor shareGroupRebalanceSensor = metrics.sensor(SHARE_GROUP_REBALANCES_SENSOR_NAME);
shareGroupRebalanceSensor.add(new Meter(
metrics.metricName("rebalance-rate",
METRICS_GROUP,
"The rate of share group rebalances",
SHARE_GROUP_PROTOCOL_TAG, Group.GroupType.SHARE.toString()),
metrics.metricName("rebalance-count",
METRICS_GROUP,
"The total number of share group rebalances",
SHARE_GROUP_PROTOCOL_TAG, Group.GroupType.SHARE.toString())));
globalSensors = Collections.unmodifiableMap(Utils.mkMap(
Utils.mkEntry(OFFSET_COMMITS_SENSOR_NAME, offsetCommitsSensor),
Utils.mkEntry(OFFSET_EXPIRED_SENSOR_NAME, offsetExpiredSensor),
Utils.mkEntry(OFFSET_DELETIONS_SENSOR_NAME, offsetDeletionsSensor),
Utils.mkEntry(CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME, classicGroupCompletedRebalancesSensor),
Utils.mkEntry(CONSUMER_GROUP_REBALANCES_SENSOR_NAME, consumerGroupRebalanceSensor)
Utils.mkEntry(CONSUMER_GROUP_REBALANCES_SENSOR_NAME, consumerGroupRebalanceSensor),
Utils.mkEntry(SHARE_GROUP_REBALANCES_SENSOR_NAME, shareGroupRebalanceSensor)
));
}
@ -226,6 +278,14 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
return shards.values().stream().mapToLong(shard -> shard.numConsumerGroups(state)).sum();
}
private long numShareGroups() {
return shards.values().stream().mapToLong(GroupCoordinatorMetricsShard::numShareGroups).sum();
}
private long numShareGroups(ShareGroup.ShareGroupState state) {
return shards.values().stream().mapToLong(shard -> shard.numShareGroups(state)).sum();
}
@Override
public void close() {
Arrays.asList(
@ -245,7 +305,11 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
consumerGroupCountAssigningMetricName,
consumerGroupCountReconcilingMetricName,
consumerGroupCountStableMetricName,
consumerGroupCountDeadMetricName
consumerGroupCountDeadMetricName,
shareGroupCountMetricName,
shareGroupCountEmptyMetricName,
shareGroupCountStableMetricName,
shareGroupCountDeadMetricName
).forEach(metrics::removeMetric);
Arrays.asList(
@ -253,7 +317,8 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
OFFSET_EXPIRED_SENSOR_NAME,
OFFSET_DELETIONS_SENSOR_NAME,
CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME,
CONSUMER_GROUP_REBALANCES_SENSOR_NAME
CONSUMER_GROUP_REBALANCES_SENSOR_NAME,
SHARE_GROUP_REBALANCES_SENSOR_NAME
).forEach(metrics::removeSensor);
}
@ -376,5 +441,25 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
consumerGroupCountDeadMetricName,
(Gauge<Long>) (config, now) -> numConsumerGroups(ConsumerGroupState.DEAD)
);
metrics.addMetric(
shareGroupCountMetricName,
(Gauge<Long>) (config, now) -> numShareGroups()
);
metrics.addMetric(
shareGroupCountEmptyMetricName,
(Gauge<Long>) (config, now) -> numShareGroups(ShareGroup.ShareGroupState.EMPTY)
);
metrics.addMetric(
shareGroupCountStableMetricName,
(Gauge<Long>) (config, now) -> numShareGroups(ShareGroup.ShareGroupState.STABLE)
);
metrics.addMetric(
shareGroupCountDeadMetricName,
(Gauge<Long>) (config, now) -> numShareGroups(ShareGroup.ShareGroupState.DEAD)
);
}
}

View File

@ -22,9 +22,13 @@ import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState;
import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineLong;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicLong;
@ -39,6 +43,8 @@ import java.util.concurrent.atomic.AtomicLong;
*/
public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
private static final Logger log = LoggerFactory.getLogger(GroupCoordinatorMetricsShard.class);
/**
* This class represents a gauge counter for this shard. The TimelineLong object represents a gauge backed by
* the snapshot registry. Once we commit to a certain offset in the snapshot registry, we write the given
@ -66,6 +72,11 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
*/
private final Map<ConsumerGroupState, TimelineGaugeCounter> consumerGroupGauges;
/**
* Share group size gauge counters keyed by the metric name.
*/
private final Map<ShareGroup.ShareGroupState, TimelineGaugeCounter> shareGroupGauges;
/**
* All sensors keyed by the sensor name. A Sensor object is shared across all metrics shards.
*/
@ -116,6 +127,15 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0)))
);
this.shareGroupGauges = Utils.mkMap(
Utils.mkEntry(ShareGroup.ShareGroupState.EMPTY,
new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0))),
Utils.mkEntry(ShareGroup.ShareGroupState.STABLE,
new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0))),
Utils.mkEntry(ShareGroup.ShareGroupState.DEAD,
new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0)))
);
this.globalSensors = Objects.requireNonNull(globalSensors);
this.topicPartition = Objects.requireNonNull(topicPartition);
}
@ -278,6 +298,14 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
long value = numOffsetsTimelineGaugeCounter.timelineLong.get(offset);
numOffsetsTimelineGaugeCounter.atomicLong.set(value);
}
this.shareGroupGauges.forEach((__, gaugeCounter) -> {
long value;
synchronized (gaugeCounter.timelineLong) {
value = gaugeCounter.timelineLong.get(offset);
}
gaugeCounter.atomicLong.set(value);
});
}
/**
@ -379,4 +407,73 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
}
}
}
public void incrementNumShareGroups(ShareGroup.ShareGroupState state) {
TimelineGaugeCounter gaugeCounter = shareGroupGauges.get(state);
if (gaugeCounter != null) {
synchronized (gaugeCounter.timelineLong) {
gaugeCounter.timelineLong.increment();
}
}
}
public void decrementNumShareGroups(ShareGroup.ShareGroupState state) {
TimelineGaugeCounter gaugeCounter = shareGroupGauges.get(state);
if (gaugeCounter != null) {
synchronized (gaugeCounter.timelineLong) {
gaugeCounter.timelineLong.decrement();
}
}
}
public long numShareGroups(ShareGroup.ShareGroupState state) {
TimelineGaugeCounter gaugeCounter = shareGroupGauges.get(state);
if (gaugeCounter != null) {
return gaugeCounter.atomicLong.get();
}
return 0L;
}
public long numShareGroups() {
return shareGroupGauges.values().stream()
.mapToLong(timelineGaugeCounter -> timelineGaugeCounter.atomicLong.get()).sum();
}
// could be called from ShareGroup to indicate state transition
public void onShareGroupStateTransition(
ShareGroup.ShareGroupState oldState,
ShareGroup.ShareGroupState newState
) {
if (newState != null) {
switch (newState) {
case EMPTY:
incrementNumShareGroups(ShareGroup.ShareGroupState.EMPTY);
break;
case STABLE:
incrementNumShareGroups(ShareGroup.ShareGroupState.STABLE);
break;
case DEAD:
incrementNumShareGroups(ShareGroup.ShareGroupState.DEAD);
break;
default:
log.warn("Unknown new share group state: {}", newState);
}
}
if (oldState != null) {
switch (oldState) {
case EMPTY:
decrementNumShareGroups(ShareGroup.ShareGroupState.EMPTY);
break;
case STABLE:
decrementNumShareGroups(ShareGroup.ShareGroupState.STABLE);
break;
case DEAD:
decrementNumShareGroups(ShareGroup.ShareGroupState.DEAD);
break;
default:
log.warn("Unknown previous share group state: {}", oldState);
}
}
}
}

View File

@ -16,295 +16,13 @@
*/
package org.apache.kafka.coordinator.group.metrics;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.metrics.stats.Rate;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl;
import java.util.Arrays;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Supplier;
public class GroupCoordinatorRuntimeMetrics implements CoordinatorRuntimeMetrics {
/**
* The metrics group.
*/
public class GroupCoordinatorRuntimeMetrics extends CoordinatorRuntimeMetricsImpl {
public static final String METRICS_GROUP = "group-coordinator-metrics";
/**
* The partition count metric name.
*/
public static final String NUM_PARTITIONS_METRIC_NAME = "num-partitions";
/**
* The event queue time metric name.
*/
public static final String EVENT_QUEUE_TIME_METRIC_NAME = "event-queue-time-ms";
/**
* The event queue time metric name.
*/
public static final String EVENT_PROCESSING_TIME_METRIC_NAME = "event-processing-time-ms";
/**
* The event purgatory time metric name.
*/
public static final String EVENT_PURGATORY_TIME_METRIC_NAME = "event-purgatory-time-ms";
/**
* The flush time metric name.
*/
public static final String BATCH_FLUSH_TIME_METRIC_NAME = "batch-flush-time-ms";
/**
* Metric to count the number of partitions in Loading state.
*/
private final MetricName numPartitionsLoading;
private final AtomicLong numPartitionsLoadingCounter = new AtomicLong(0);
/**
* Metric to count the number of partitions in Active state.
*/
private final MetricName numPartitionsActive;
private final AtomicLong numPartitionsActiveCounter = new AtomicLong(0);
/**
* Metric to count the number of partitions in Failed state.
*/
private final MetricName numPartitionsFailed;
private final AtomicLong numPartitionsFailedCounter = new AtomicLong(0);
/**
* Metric to count the size of the processor queue.
*/
private final MetricName eventQueueSize;
/**
* The Kafka metrics registry.
*/
private final Metrics metrics;
/**
* The partition load sensor.
*/
private final Sensor partitionLoadSensor;
/**
* The thread idle sensor.
*/
private final Sensor threadIdleSensor;
/**
* The event queue time sensor.
*/
private final Sensor eventQueueTimeSensor;
/**
* The event processing time sensor.
*/
private final Sensor eventProcessingTimeSensor;
/**
* Sensor to measure the time an event stays in the purgatory.
*/
private final Sensor eventPurgatoryTimeSensor;
/**
* Sensor to measure the flush time.
*/
private final Sensor flushTimeSensor;
public GroupCoordinatorRuntimeMetrics(Metrics metrics) {
this.metrics = Objects.requireNonNull(metrics);
this.numPartitionsLoading = kafkaMetricName(
NUM_PARTITIONS_METRIC_NAME,
"The number of partitions in Loading state.",
"state", "loading"
);
this.numPartitionsActive = kafkaMetricName(
NUM_PARTITIONS_METRIC_NAME,
"The number of partitions in Active state.",
"state", "active"
);
this.numPartitionsFailed = kafkaMetricName(
NUM_PARTITIONS_METRIC_NAME,
"The number of partitions in Failed state.",
"state", "failed"
);
this.eventQueueSize = kafkaMetricName("event-queue-size", "The event accumulator queue size.");
metrics.addMetric(numPartitionsLoading, (Gauge<Long>) (config, now) -> numPartitionsLoadingCounter.get());
metrics.addMetric(numPartitionsActive, (Gauge<Long>) (config, now) -> numPartitionsActiveCounter.get());
metrics.addMetric(numPartitionsFailed, (Gauge<Long>) (config, now) -> numPartitionsFailedCounter.get());
this.partitionLoadSensor = metrics.sensor("GroupPartitionLoadTime");
this.partitionLoadSensor.add(
metrics.metricName(
"partition-load-time-max",
METRICS_GROUP,
"The max time it took to load the partitions in the last 30 sec."
), new Max());
this.partitionLoadSensor.add(
metrics.metricName(
"partition-load-time-avg",
METRICS_GROUP,
"The average time it took to load the partitions in the last 30 sec."
), new Avg());
this.threadIdleSensor = metrics.sensor("ThreadIdleRatio");
this.threadIdleSensor.add(metrics.metricName(
"thread-idle-ratio-avg",
METRICS_GROUP,
"The fraction of time the threads spent waiting for an event. This is an average across " +
"all coordinator event processor threads."),
new Rate(TimeUnit.MILLISECONDS));
KafkaMetricHistogram eventQueueTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
suffix -> kafkaMetricName(
EVENT_QUEUE_TIME_METRIC_NAME + "-" + suffix,
"The " + suffix + " event queue time in milliseconds"
)
);
this.eventQueueTimeSensor = metrics.sensor("EventQueueTime");
this.eventQueueTimeSensor.add(eventQueueTimeHistogram);
KafkaMetricHistogram eventProcessingTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
suffix -> kafkaMetricName(
EVENT_PROCESSING_TIME_METRIC_NAME + "-" + suffix,
"The " + suffix + " event processing time in milliseconds"
)
);
this.eventProcessingTimeSensor = metrics.sensor("EventProcessingTime");
this.eventProcessingTimeSensor.add(eventProcessingTimeHistogram);
KafkaMetricHistogram eventPurgatoryTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
suffix -> kafkaMetricName(
EVENT_PURGATORY_TIME_METRIC_NAME + "-" + suffix,
"The " + suffix + " event purgatory time in milliseconds"
)
);
this.eventPurgatoryTimeSensor = metrics.sensor("EventPurgatoryTime");
this.eventPurgatoryTimeSensor.add(eventPurgatoryTimeHistogram);
KafkaMetricHistogram flushTimeHistogram = KafkaMetricHistogram.newLatencyHistogram(
suffix -> kafkaMetricName(
BATCH_FLUSH_TIME_METRIC_NAME + "-" + suffix,
"The " + suffix + " flush time in milliseconds"
)
);
this.flushTimeSensor = metrics.sensor("FlushTime");
this.flushTimeSensor.add(flushTimeHistogram);
}
/**
* Retrieve the kafka metric name.
*
* @param name The name of the metric.
*
* @return The kafka metric name.
*/
private MetricName kafkaMetricName(String name, String description, String... keyValue) {
return metrics.metricName(name, METRICS_GROUP, description, keyValue);
}
@Override
public void close() {
Arrays.asList(
numPartitionsLoading,
numPartitionsActive,
numPartitionsFailed,
eventQueueSize
).forEach(metrics::removeMetric);
metrics.removeSensor(partitionLoadSensor.name());
metrics.removeSensor(threadIdleSensor.name());
metrics.removeSensor(eventQueueTimeSensor.name());
metrics.removeSensor(eventProcessingTimeSensor.name());
metrics.removeSensor(eventPurgatoryTimeSensor.name());
metrics.removeSensor(flushTimeSensor.name());
}
/**
* Called when the partition state changes. Decrement the old state and increment the new state.
*
* @param oldState The old state.
* @param newState The new state to transition to.
*/
@Override
public void recordPartitionStateChange(CoordinatorState oldState, CoordinatorState newState) {
switch (oldState) {
case INITIAL:
case CLOSED:
break;
case LOADING:
numPartitionsLoadingCounter.decrementAndGet();
break;
case ACTIVE:
numPartitionsActiveCounter.decrementAndGet();
break;
case FAILED:
numPartitionsFailedCounter.decrementAndGet();
}
switch (newState) {
case INITIAL:
case CLOSED:
break;
case LOADING:
numPartitionsLoadingCounter.incrementAndGet();
break;
case ACTIVE:
numPartitionsActiveCounter.incrementAndGet();
break;
case FAILED:
numPartitionsFailedCounter.incrementAndGet();
}
}
@Override
public void recordPartitionLoadSensor(long startTimeMs, long endTimeMs) {
this.partitionLoadSensor.record(endTimeMs - startTimeMs, endTimeMs, false);
}
@Override
public void recordEventQueueTime(long durationMs) {
eventQueueTimeSensor.record(durationMs);
}
@Override
public void recordEventProcessingTime(long durationMs) {
eventProcessingTimeSensor.record(durationMs);
}
@Override
public void recordEventPurgatoryTime(long purgatoryTimeMs) {
eventPurgatoryTimeSensor.record(purgatoryTimeMs);
}
@Override
public void recordFlushTime(long durationMs) {
flushTimeSensor.record(durationMs);
}
@Override
public void recordThreadIdleTime(long idleTimeMs) {
threadIdleSensor.record(idleTimeMs);
}
@Override
public void registerEventQueueSizeGauge(Supplier<Integer> sizeSupplier) {
metrics.addMetric(eventQueueSize, (Gauge<Long>) (config, now) -> (long) sizeSupplier.get());
super(metrics, METRICS_GROUP);
}
}

View File

@ -17,13 +17,16 @@
package org.apache.kafka.coordinator.group.metrics;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.ShareGroupState;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Metrics;
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.coordinator.group.Group;
import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState;
import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
import org.apache.kafka.timeline.SnapshotRegistry;
import com.yammer.metrics.core.MetricsRegistry;
@ -40,6 +43,7 @@ import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.METRICS_GROUP;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.OFFSET_COMMITS_SENSOR_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.OFFSET_EXPIRED_SENSOR_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.SHARE_GROUP_REBALANCES_SENSOR_NAME;
import static org.apache.kafka.coordinator.group.metrics.MetricsTestUtils.assertGaugeValue;
import static org.apache.kafka.coordinator.group.metrics.MetricsTestUtils.assertMetricsForTypeEqual;
import static org.apache.kafka.coordinator.group.metrics.MetricsTestUtils.metricName;
@ -92,7 +96,37 @@ public class GroupCoordinatorMetricsTest {
metrics.metricName(
"consumer-group-count",
GroupCoordinatorMetrics.METRICS_GROUP,
Collections.singletonMap("state", ConsumerGroupState.DEAD.toString()))
Collections.singletonMap("state", ConsumerGroupState.DEAD.toString())),
metrics.metricName(
"group-count",
GroupCoordinatorMetrics.METRICS_GROUP,
Collections.singletonMap("protocol", Group.GroupType.SHARE.toString())),
metrics.metricName(
"rebalance-rate",
GroupCoordinatorMetrics.METRICS_GROUP,
Collections.singletonMap("protocol", Group.GroupType.SHARE.toString())),
metrics.metricName(
"rebalance-count",
GroupCoordinatorMetrics.METRICS_GROUP,
Collections.singletonMap("protocol", Group.GroupType.SHARE.toString())),
metrics.metricName(
"group-count",
GroupCoordinatorMetrics.METRICS_GROUP,
"The number of share groups in empty state.",
"protocol", Group.GroupType.SHARE.toString(),
"state", ShareGroupState.EMPTY.toString()),
metrics.metricName(
"group-count",
GroupCoordinatorMetrics.METRICS_GROUP,
"The number of share groups in stable state.",
"protocol", Group.GroupType.SHARE.toString(),
"state", ShareGroupState.STABLE.toString()),
metrics.metricName(
"group-count",
GroupCoordinatorMetrics.METRICS_GROUP,
"The number of share groups in dead state.",
"protocol", Group.GroupType.SHARE.toString(),
"state", ShareGroupState.DEAD.toString())
));
try {
@ -145,6 +179,10 @@ public class GroupCoordinatorMetricsTest {
IntStream.range(0, 2).forEach(__ -> shard1.incrementNumOffsets());
IntStream.range(0, 1).forEach(__ -> shard1.decrementNumOffsets());
IntStream.range(0, 5).forEach(__ -> shard0.incrementNumShareGroups(ShareGroup.ShareGroupState.STABLE));
IntStream.range(0, 5).forEach(__ -> shard1.incrementNumShareGroups(ShareGroup.ShareGroupState.EMPTY));
IntStream.range(0, 3).forEach(__ -> shard1.decrementNumShareGroups(ShareGroup.ShareGroupState.DEAD));
assertEquals(4, shard0.numClassicGroups());
assertEquals(5, shard1.numClassicGroups());
assertGaugeValue(registry, metricName("GroupMetadataManager", "NumGroups"), 9);
@ -169,6 +207,14 @@ public class GroupCoordinatorMetricsTest {
7
);
assertGaugeValue(registry, metricName("GroupMetadataManager", "NumOffsets"), 7);
assertEquals(5, shard0.numShareGroups());
assertEquals(2, shard1.numShareGroups());
assertGaugeValue(
metrics,
metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "share")),
7
);
}
@Test
@ -196,6 +242,20 @@ public class GroupCoordinatorMetricsTest {
shard.record(CONSUMER_GROUP_REBALANCES_SENSOR_NAME, 50);
assertMetricValue(metrics, metrics.metricName("consumer-group-rebalance-rate", GroupCoordinatorMetrics.METRICS_GROUP), 5.0 / 3.0);
assertMetricValue(metrics, metrics.metricName("consumer-group-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), 50);
shard.record(SHARE_GROUP_REBALANCES_SENSOR_NAME, 50);
assertMetricValue(metrics, metrics.metricName(
"rebalance-rate",
GroupCoordinatorMetrics.METRICS_GROUP,
"The rate of share group rebalances",
"protocol", "share"
), 5.0 / 3.0);
assertMetricValue(metrics, metrics.metricName(
"rebalance-count",
GroupCoordinatorMetrics.METRICS_GROUP,
"The total number of share group rebalances",
"protocol", "share"
), 50);
}
private void assertMetricValue(Metrics metrics, MetricName metricName, double val) {

View File

@ -57,6 +57,7 @@ public abstract class AbstractKafkaConfig extends AbstractConfig {
CleanerConfig.CONFIG_DEF,
LogConfig.SERVER_CONFIG_DEF,
ShareGroupConfig.CONFIG_DEF,
ShareCoordinatorConfig.CONFIG_DEF,
TransactionLogConfig.CONFIG_DEF,
TransactionStateManagerConfig.CONFIG_DEF,
QuorumConfig.CONFIG_DEF,

View File

@ -0,0 +1,158 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.server.config;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.utils.Utils;
import java.util.Optional;
import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH;
import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM;
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
import static org.apache.kafka.common.config.ConfigDef.Type.INT;
import static org.apache.kafka.common.config.ConfigDef.Type.SHORT;
public class ShareCoordinatorConfig {
public static final String STATE_TOPIC_NUM_PARTITIONS_CONFIG = "share.coordinator.state.topic.num.partitions";
public static final int STATE_TOPIC_NUM_PARTITIONS_DEFAULT = 50;
public static final String STATE_TOPIC_NUM_PARTITIONS_DOC = "The number of partitions for the share-group state topic (should not change after deployment).";
public static final String STATE_TOPIC_REPLICATION_FACTOR_CONFIG = "share.coordinator.state.topic.replication.factor";
public static final short STATE_TOPIC_REPLICATION_FACTOR_DEFAULT = 3;
public static final String STATE_TOPIC_REPLICATION_FACTOR_DOC = "Replication factor for the share-group state topic. " +
"Topic creation will fail until the cluster size meets this replication factor requirement.";
public static final String STATE_TOPIC_MIN_ISR_CONFIG = "share.coordinator.state.topic.min.isr";
public static final short STATE_TOPIC_MIN_ISR_DEFAULT = 2;
public static final String STATE_TOPIC_MIN_ISR_DOC = "Overridden min.insync.replicas for the share-group state topic.";
public static final String STATE_TOPIC_SEGMENT_BYTES_CONFIG = "share.coordinator.state.topic.segment.bytes";
public static final int STATE_TOPIC_SEGMENT_BYTES_DEFAULT = 100 * 1024 * 1024;
public static final String STATE_TOPIC_SEGMENT_BYTES_DOC = "The log segment size for the share-group state topic.";
public static final String NUM_THREADS_CONFIG = "share.coordinator.threads";
public static final int NUM_THREADS_DEFAULT = 1;
public static final String NUM_THREADS_DOC = "The number of threads used by the share coordinator.";
public static final String SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG = "share.coordinator.snapshot.update.records.per.snapshot";
public static final int SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_DEFAULT = 500;
public static final String SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_DOC = "The number of update records the share coordinator writes between snapshot records.";
public static final String WRITE_TIMEOUT_MS_CONFIG = "share.coordinator.write.timeout.ms";
public static final int WRITE_TIMEOUT_MS_DEFAULT = 5000;
public static final String WRITE_TIMEOUT_MS_DOC = "The duration in milliseconds that the share coordinator will wait for all replicas of the share-group state topic to receive a write.";
public static final String LOAD_BUFFER_SIZE_CONFIG = "share.coordinator.load.buffer.size";
public static final int LOAD_BUFFER_SIZE_DEFAULT = 5 * 1024 * 1024;
public static final String LOAD_BUFFER_SIZE_DOC = "Batch size for reading from the share-group state topic when loading state information into the cache (soft-limit, overridden if records are too large).";
public static final String STATE_TOPIC_COMPRESSION_CODEC_CONFIG = "share.coordinator.state.topic.compression.codec";
public static final CompressionType STATE_TOPIC_COMPRESSION_CODEC_DEFAULT = CompressionType.NONE;
public static final String STATE_TOPIC_COMPRESSION_CODEC_DOC = "Compression codec for the share-group state topic.";
public static final String APPEND_LINGER_MS_CONFIG = "share.coordinator.append.linger.ms";
public static final int APPEND_LINGER_MS_DEFAULT = 10;
public static final String APPEND_LINGER_MS_DOC = "The duration in milliseconds that the share coordinator will wait for writes to accumulate before flushing them to disk.";
public static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(STATE_TOPIC_NUM_PARTITIONS_CONFIG, INT, STATE_TOPIC_NUM_PARTITIONS_DEFAULT, atLeast(1), HIGH, STATE_TOPIC_NUM_PARTITIONS_DOC)
.define(STATE_TOPIC_REPLICATION_FACTOR_CONFIG, SHORT, STATE_TOPIC_REPLICATION_FACTOR_DEFAULT, atLeast(1), HIGH, STATE_TOPIC_REPLICATION_FACTOR_DOC)
.define(STATE_TOPIC_MIN_ISR_CONFIG, SHORT, STATE_TOPIC_MIN_ISR_DEFAULT, atLeast(1), HIGH, STATE_TOPIC_MIN_ISR_DOC)
.define(STATE_TOPIC_SEGMENT_BYTES_CONFIG, INT, STATE_TOPIC_SEGMENT_BYTES_DEFAULT, atLeast(1), HIGH, STATE_TOPIC_SEGMENT_BYTES_DOC)
.define(NUM_THREADS_CONFIG, INT, NUM_THREADS_DEFAULT, atLeast(1), MEDIUM, NUM_THREADS_DOC)
.define(SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG, INT, SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_DEFAULT, atLeast(0), MEDIUM, SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_DOC)
.define(LOAD_BUFFER_SIZE_CONFIG, INT, LOAD_BUFFER_SIZE_DEFAULT, atLeast(1), HIGH, LOAD_BUFFER_SIZE_DOC)
.define(STATE_TOPIC_COMPRESSION_CODEC_CONFIG, INT, (int) STATE_TOPIC_COMPRESSION_CODEC_DEFAULT.id, HIGH, STATE_TOPIC_COMPRESSION_CODEC_DOC)
.define(APPEND_LINGER_MS_CONFIG, INT, APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, APPEND_LINGER_MS_DOC)
.define(WRITE_TIMEOUT_MS_CONFIG, INT, WRITE_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, WRITE_TIMEOUT_MS_DOC);
private final int stateTopicNumPartitions;
private final short stateTopicReplicationFactor;
private final int stateTopicMinIsr;
private final int stateTopicSegmentBytes;
private final int numThreads;
private final int snapshotUpdateRecordsPerSnapshot;
private final int writeTimeoutMs;
private final int loadBufferSize;
private final CompressionType compressionType;
private final int appendLingerMs;
public ShareCoordinatorConfig(AbstractConfig config) {
stateTopicNumPartitions = config.getInt(STATE_TOPIC_NUM_PARTITIONS_CONFIG);
stateTopicReplicationFactor = config.getShort(STATE_TOPIC_REPLICATION_FACTOR_CONFIG);
stateTopicMinIsr = config.getShort(STATE_TOPIC_MIN_ISR_CONFIG);
stateTopicSegmentBytes = config.getInt(STATE_TOPIC_SEGMENT_BYTES_CONFIG);
numThreads = config.getInt(NUM_THREADS_CONFIG);
snapshotUpdateRecordsPerSnapshot = config.getInt(SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG);
writeTimeoutMs = config.getInt(WRITE_TIMEOUT_MS_CONFIG);
loadBufferSize = config.getInt(LOAD_BUFFER_SIZE_CONFIG);
compressionType = Optional.ofNullable(config.getInt(STATE_TOPIC_COMPRESSION_CODEC_CONFIG))
.map(CompressionType::forId)
.orElse(null);
appendLingerMs = config.getInt(APPEND_LINGER_MS_CONFIG);
validate();
}
public int shareCoordinatorStateTopicNumPartitions() {
return stateTopicNumPartitions;
}
public short shareCoordinatorStateTopicReplicationFactor() {
return stateTopicReplicationFactor;
}
public int shareCoordinatorStateTopicMinIsr() {
return stateTopicMinIsr;
}
public int shareCoordinatorStateTopicSegmentBytes() {
return stateTopicSegmentBytes;
}
public int shareCoordinatorNumThreads() {
return numThreads;
}
public int shareCoordinatorSnapshotUpdateRecordsPerSnapshot() {
return snapshotUpdateRecordsPerSnapshot;
}
public int shareCoordinatorWriteTimeoutMs() {
return writeTimeoutMs;
}
public int shareCoordinatorLoadBufferSize() {
return loadBufferSize;
}
public int shareCoordinatorAppendLingerMs() {
return appendLingerMs;
}
public CompressionType shareCoordinatorStateTopicCompressionType() {
return compressionType;
}
private void validate() {
Utils.require(snapshotUpdateRecordsPerSnapshot >= 0 && snapshotUpdateRecordsPerSnapshot <= 500,
String.format("%s must be between [0, 500]", SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG));
}
}

View File

@ -18,6 +18,7 @@
package org.apache.kafka.coordinator.share;
import org.apache.kafka.common.message.WriteShareGroupStateRequestData;
import org.apache.kafka.coordinator.share.generated.ShareUpdateValue;
import org.apache.kafka.server.group.share.PersisterStateBatch;
import java.util.Objects;
@ -64,6 +65,15 @@ public class PersisterOffsetsStateBatch {
);
}
public static PersisterOffsetsStateBatch from(ShareUpdateValue.StateBatch batch) {
return new PersisterOffsetsStateBatch(
batch.firstOffset(),
batch.lastOffset(),
batch.deliveryState(),
batch.deliveryCount()
);
}
@Override
public boolean equals(Object o) {
if (!(o instanceof PersisterOffsetsStateBatch)) {

View File

@ -0,0 +1,537 @@
/*
* 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.coordinator.share;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.internals.Topic;
import org.apache.kafka.common.message.ReadShareGroupStateRequestData;
import org.apache.kafka.common.message.ReadShareGroupStateResponseData;
import org.apache.kafka.common.message.WriteShareGroupStateRequestData;
import org.apache.kafka.common.message.WriteShareGroupStateResponseData;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.ReadShareGroupStateResponse;
import org.apache.kafka.common.requests.RequestContext;
import org.apache.kafka.common.requests.WriteShareGroupStateResponse;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.common.runtime.CoordinatorEventProcessor;
import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorShardBuilderSupplier;
import org.apache.kafka.coordinator.common.runtime.MultiThreadedEventProcessor;
import org.apache.kafka.coordinator.common.runtime.PartitionWriter;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
import org.apache.kafka.server.config.ShareCoordinatorConfig;
import org.apache.kafka.server.group.share.SharePartitionKey;
import org.apache.kafka.server.record.BrokerCompressionType;
import org.apache.kafka.server.util.timer.Timer;
import org.slf4j.Logger;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.OptionalInt;
import java.util.Properties;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.IntSupplier;
import java.util.stream.Collectors;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorOperationExceptionHelper.handleOperationException;
public class ShareCoordinatorService implements ShareCoordinator {
private final ShareCoordinatorConfig config;
private final Logger log;
private final AtomicBoolean isActive = new AtomicBoolean(false); // for controlling start and stop
private final CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime;
private final ShareCoordinatorMetrics shareCoordinatorMetrics;
private volatile int numPartitions = -1; // Number of partitions for __share_group_state. Provided when component is started.
private final Time time;
public static class Builder {
private final int nodeId;
private final ShareCoordinatorConfig config;
private PartitionWriter writer;
private CoordinatorLoader<CoordinatorRecord> loader;
private Time time;
private Timer timer;
private ShareCoordinatorMetrics coordinatorMetrics;
private CoordinatorRuntimeMetrics coordinatorRuntimeMetrics;
public Builder(int nodeId, ShareCoordinatorConfig config) {
this.nodeId = nodeId;
this.config = config;
}
public Builder withWriter(PartitionWriter writer) {
this.writer = writer;
return this;
}
public Builder withLoader(CoordinatorLoader<CoordinatorRecord> loader) {
this.loader = loader;
return this;
}
public Builder withTime(Time time) {
this.time = time;
return this;
}
public Builder withTimer(Timer timer) {
this.timer = timer;
return this;
}
public Builder withCoordinatorMetrics(ShareCoordinatorMetrics coordinatorMetrics) {
this.coordinatorMetrics = coordinatorMetrics;
return this;
}
public Builder withCoordinatorRuntimeMetrics(CoordinatorRuntimeMetrics coordinatorRuntimeMetrics) {
this.coordinatorRuntimeMetrics = coordinatorRuntimeMetrics;
return this;
}
public ShareCoordinatorService build() {
if (config == null) {
throw new IllegalArgumentException("Config must be set.");
}
if (writer == null) {
throw new IllegalArgumentException("Writer must be set.");
}
if (loader == null) {
throw new IllegalArgumentException("Loader must be set.");
}
if (time == null) {
throw new IllegalArgumentException("Time must be set.");
}
if (timer == null) {
throw new IllegalArgumentException("Timer must be set.");
}
if (coordinatorMetrics == null) {
throw new IllegalArgumentException("Share Coordinator metrics must be set.");
}
if (coordinatorRuntimeMetrics == null) {
throw new IllegalArgumentException("Coordinator runtime metrics must be set.");
}
String logPrefix = String.format("ShareCoordinator id=%d", nodeId);
LogContext logContext = new LogContext(String.format("[%s] ", logPrefix));
CoordinatorShardBuilderSupplier<ShareCoordinatorShard, CoordinatorRecord> supplier = () ->
new ShareCoordinatorShard.Builder(config);
CoordinatorEventProcessor processor = new MultiThreadedEventProcessor(
logContext,
"share-coordinator-event-processor-",
config.shareCoordinatorNumThreads(),
time,
coordinatorRuntimeMetrics
);
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime =
new CoordinatorRuntime.Builder<ShareCoordinatorShard, CoordinatorRecord>()
.withTime(time)
.withTimer(timer)
.withLogPrefix(logPrefix)
.withLogContext(logContext)
.withEventProcessor(processor)
.withPartitionWriter(writer)
.withLoader(loader)
.withCoordinatorShardBuilderSupplier(supplier)
.withTime(time)
.withDefaultWriteTimeOut(Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()))
.withCoordinatorRuntimeMetrics(coordinatorRuntimeMetrics)
.withCoordinatorMetrics(coordinatorMetrics)
.withSerializer(new ShareCoordinatorRecordSerde())
.withCompression(Compression.of(config.shareCoordinatorStateTopicCompressionType()).build())
.withAppendLingerMs(config.shareCoordinatorAppendLingerMs())
.build();
return new ShareCoordinatorService(
logContext,
config,
runtime,
coordinatorMetrics,
time
);
}
}
public ShareCoordinatorService(
LogContext logContext,
ShareCoordinatorConfig config,
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime,
ShareCoordinatorMetrics shareCoordinatorMetrics,
Time time) {
this.log = logContext.logger(ShareCoordinatorService.class);
this.config = config;
this.runtime = runtime;
this.shareCoordinatorMetrics = shareCoordinatorMetrics;
this.time = time;
}
@Override
public int partitionFor(String key) {
return Utils.abs(key.hashCode()) % numPartitions;
}
@Override
public Properties shareGroupStateTopicConfigs() {
Properties properties = new Properties();
properties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE); // as defined in KIP-932
properties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name);
properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, config.shareCoordinatorStateTopicSegmentBytes());
return properties;
}
/**
* The share coordinator startup method will get invoked from BrokerMetadataPublisher.
* At the time of writing, the publisher uses metadata cache to fetch the number of partitions
* of the share state topic. In case this information is not available, the user provided
* config will be used to fetch the value.
* This is consistent with the group coordinator startup functionality.
*
* @param shareGroupTopicPartitionCount - supplier returning the number of partitions for __share_group_state topic
*/
@Override
public void startup(
IntSupplier shareGroupTopicPartitionCount
) {
if (!isActive.compareAndSet(false, true)) {
log.warn("Share coordinator is already running.");
return;
}
log.info("Starting up.");
numPartitions = shareGroupTopicPartitionCount.getAsInt();
log.info("Startup complete.");
}
@Override
public void shutdown() {
if (!isActive.compareAndSet(true, false)) {
log.warn("Share coordinator is already shutting down.");
return;
}
log.info("Shutting down.");
Utils.closeQuietly(runtime, "coordinator runtime");
Utils.closeQuietly(shareCoordinatorMetrics, "share coordinator metrics");
log.info("Shutdown complete.");
}
@Override
public CompletableFuture<WriteShareGroupStateResponseData> writeState(RequestContext context, WriteShareGroupStateRequestData request) {
// Send an empty response if topic data is empty
if (isEmpty(request.topics())) {
log.error("Topic Data is empty: {}", request);
return CompletableFuture.completedFuture(
new WriteShareGroupStateResponseData()
);
}
// Send an empty response if partition data is empty for any topic
for (WriteShareGroupStateRequestData.WriteStateData topicData : request.topics()) {
if (isEmpty(topicData.partitions())) {
log.error("Partition Data for topic {} is empty: {}", topicData.topicId(), request);
return CompletableFuture.completedFuture(
new WriteShareGroupStateResponseData()
);
}
}
String groupId = request.groupId();
// Send an empty response if groupId is invalid
if (isGroupIdEmpty(groupId)) {
log.error("Group id must be specified and non-empty: {}", request);
return CompletableFuture.completedFuture(
new WriteShareGroupStateResponseData()
);
}
// Send an empty response if the coordinator is not active
if (!isActive.get()) {
return CompletableFuture.completedFuture(
generateErrorWriteStateResponse(
request,
Errors.COORDINATOR_NOT_AVAILABLE,
"Share coordinator is not available."
)
);
}
// The request received here could have multiple keys of structure group:topic:partition. However,
// the writeState method in ShareCoordinatorShard expects a single key in the request. Hence, we will
// be looping over the keys below and constructing new WriteShareGroupStateRequestData objects to pass
// onto the shard method.
Map<Uuid, Map<Integer, CompletableFuture<WriteShareGroupStateResponseData>>> futureMap = new HashMap<>();
long startTimeMs = time.hiResClockMs();
request.topics().forEach(topicData -> {
Map<Integer, CompletableFuture<WriteShareGroupStateResponseData>> partitionFut =
futureMap.computeIfAbsent(topicData.topicId(), k -> new HashMap<>());
topicData.partitions().forEach(
partitionData -> {
CompletableFuture<WriteShareGroupStateResponseData> future = runtime.scheduleWriteOperation(
"write-share-group-state",
topicPartitionFor(SharePartitionKey.getInstance(groupId, topicData.topicId(), partitionData.partition())),
Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()),
coordinator -> coordinator.writeState(new WriteShareGroupStateRequestData()
.setGroupId(groupId)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(topicData.topicId())
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(partitionData.partition())
.setStartOffset(partitionData.startOffset())
.setLeaderEpoch(partitionData.leaderEpoch())
.setStateEpoch(partitionData.stateEpoch())
.setStateBatches(partitionData.stateBatches())))))))
.exceptionally(exception -> handleOperationException(
"write-share-group-state",
request,
exception,
(error, message) -> WriteShareGroupStateResponse.toErrorResponseData(
topicData.topicId(),
partitionData.partition(),
error,
"Unable to write share group state: " + exception.getMessage()
),
log
));
partitionFut.put(partitionData.partition(), future);
});
});
// Combine all futures into a single CompletableFuture<Void>
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(futureMap.values().stream()
.flatMap(partMap -> partMap.values().stream()).toArray(CompletableFuture[]::new));
// topicId -> {partitionId -> responseFuture}
return combinedFuture.thenApply(v -> {
List<WriteShareGroupStateResponseData.WriteStateResult> writeStateResults = new ArrayList<>(futureMap.size());
futureMap.forEach(
(topicId, topicEntry) -> {
List<WriteShareGroupStateResponseData.PartitionResult> partitionResults = new ArrayList<>(topicEntry.size());
topicEntry.forEach(
// map of partition id -> responses from api
(partitionId, responseFut) -> {
// This is the future returned by runtime.scheduleWriteOperation which returns when the
// operation has completed including error information. When this line executes, the future
// should be complete as we used CompletableFuture::allOf to get a combined future from
// all futures in the map.
WriteShareGroupStateResponseData partitionData = responseFut.getNow(null);
partitionResults.addAll(partitionData.results().get(0).partitions());
}
);
writeStateResults.add(WriteShareGroupStateResponse.toResponseWriteStateResult(topicId, partitionResults));
}
);
// time taken for write
// at this point all futures are completed written above.
shareCoordinatorMetrics.record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME,
time.hiResClockMs() - startTimeMs);
return new WriteShareGroupStateResponseData()
.setResults(writeStateResults);
});
}
@Override
public CompletableFuture<ReadShareGroupStateResponseData> readState(RequestContext context, ReadShareGroupStateRequestData request) {
String groupId = request.groupId();
// A map to store the futures for each topicId and partition.
Map<Uuid, Map<Integer, CompletableFuture<ReadShareGroupStateResponseData>>> futureMap = new HashMap<>();
// Send an empty response if topic data is empty
if (isEmpty(request.topics())) {
log.error("Topic Data is empty: {}", request);
return CompletableFuture.completedFuture(
new ReadShareGroupStateResponseData()
);
}
// Send an empty response if partition data is empty for any topic
for (ReadShareGroupStateRequestData.ReadStateData topicData : request.topics()) {
if (isEmpty(topicData.partitions())) {
log.error("Partition Data for topic {} is empty: {}", topicData.topicId(), request);
return CompletableFuture.completedFuture(
new ReadShareGroupStateResponseData()
);
}
}
// Send an empty response if groupId is invalid
if (isGroupIdEmpty(groupId)) {
log.error("Group id must be specified and non-empty: {}", request);
return CompletableFuture.completedFuture(
new ReadShareGroupStateResponseData()
);
}
// Send an empty response if the coordinator is not active
if (!isActive.get()) {
return CompletableFuture.completedFuture(
generateErrorReadStateResponse(
request,
Errors.COORDINATOR_NOT_AVAILABLE,
"Share coordinator is not available."
)
);
}
// The request received here could have multiple keys of structure group:topic:partition. However,
// the readState method in ShareCoordinatorShard expects a single key in the request. Hence, we will
// be looping over the keys below and constructing new ReadShareGroupStateRequestData objects to pass
// onto the shard method.
request.topics().forEach(topicData -> {
Uuid topicId = topicData.topicId();
topicData.partitions().forEach(partitionData -> {
// Request object containing information of a single topic partition
ReadShareGroupStateRequestData requestForCurrentPartition = new ReadShareGroupStateRequestData()
.setGroupId(groupId)
.setTopics(Collections.singletonList(new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(topicId)
.setPartitions(Collections.singletonList(partitionData))));
SharePartitionKey coordinatorKey = SharePartitionKey.getInstance(request.groupId(), topicId, partitionData.partition());
// Scheduling a runtime read operation to read share partition state from the coordinator in memory state
CompletableFuture<ReadShareGroupStateResponseData> future = runtime.scheduleReadOperation(
"read-share-group-state",
topicPartitionFor(coordinatorKey),
(coordinator, offset) -> coordinator.readState(requestForCurrentPartition, offset)
).exceptionally(exception -> handleOperationException(
"read-share-group-state",
request,
exception,
(error, message) -> ReadShareGroupStateResponse.toErrorResponseData(
topicData.topicId(),
partitionData.partition(),
error,
"Unable to read share group state: " + exception.getMessage()
),
log
));
futureMap.computeIfAbsent(topicId, k -> new HashMap<>())
.put(partitionData.partition(), future);
});
});
// Combine all futures into a single CompletableFuture<Void>
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(futureMap.values().stream()
.flatMap(map -> map.values().stream()).toArray(CompletableFuture[]::new));
// Transform the combined CompletableFuture<Void> into CompletableFuture<ReadShareGroupStateResponseData>
return combinedFuture.thenApply(v -> {
List<ReadShareGroupStateResponseData.ReadStateResult> readStateResult = new ArrayList<>(futureMap.size());
futureMap.forEach(
(topicId, topicEntry) -> {
List<ReadShareGroupStateResponseData.PartitionResult> partitionResults = new ArrayList<>(topicEntry.size());
topicEntry.forEach(
(partitionId, responseFut) -> {
// responseFut would already be completed by now since we have used
// CompletableFuture::allOf to create a combined future from the future map.
partitionResults.add(
responseFut.getNow(null).results().get(0).partitions().get(0)
);
}
);
readStateResult.add(ReadShareGroupStateResponse.toResponseReadStateResult(topicId, partitionResults));
}
);
return new ReadShareGroupStateResponseData()
.setResults(readStateResult);
});
}
private ReadShareGroupStateResponseData generateErrorReadStateResponse(
ReadShareGroupStateRequestData request,
Errors error,
String errorMessage
) {
return new ReadShareGroupStateResponseData().setResults(request.topics().stream()
.map(topicData -> {
ReadShareGroupStateResponseData.ReadStateResult resultData = new ReadShareGroupStateResponseData.ReadStateResult();
resultData.setTopicId(topicData.topicId());
resultData.setPartitions(topicData.partitions().stream()
.map(partitionData -> ReadShareGroupStateResponse.toErrorResponsePartitionResult(
partitionData.partition(), error, errorMessage
)).collect(Collectors.toList()));
return resultData;
}).collect(Collectors.toList()));
}
private WriteShareGroupStateResponseData generateErrorWriteStateResponse(
WriteShareGroupStateRequestData request,
Errors error,
String errorMessage
) {
return new WriteShareGroupStateResponseData()
.setResults(request.topics().stream()
.map(topicData -> {
WriteShareGroupStateResponseData.WriteStateResult resultData = new WriteShareGroupStateResponseData.WriteStateResult();
resultData.setTopicId(topicData.topicId());
resultData.setPartitions(topicData.partitions().stream()
.map(partitionData -> WriteShareGroupStateResponse.toErrorResponsePartitionResult(
partitionData.partition(), error, errorMessage
)).collect(Collectors.toList()));
return resultData;
}).collect(Collectors.toList()));
}
private static boolean isGroupIdEmpty(String groupId) {
return groupId == null || groupId.isEmpty();
}
@Override
public void onElection(int partitionIndex, int partitionLeaderEpoch) {
runtime.scheduleLoadOperation(
new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, partitionIndex),
partitionLeaderEpoch
);
}
@Override
public void onResignation(int partitionIndex, OptionalInt partitionLeaderEpoch) {
runtime.scheduleUnloadOperation(
new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, partitionIndex),
partitionLeaderEpoch
);
}
private TopicPartition topicPartitionFor(SharePartitionKey key) {
return new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, partitionFor(key.toString()));
}
private static <P> boolean isEmpty(List<P> list) {
return list == null || list.isEmpty();
}
}

View File

@ -17,46 +17,232 @@
package org.apache.kafka.coordinator.share;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.ReadShareGroupStateRequestData;
import org.apache.kafka.common.message.ReadShareGroupStateResponseData;
import org.apache.kafka.common.message.WriteShareGroupStateRequestData;
import org.apache.kafka.common.message.WriteShareGroupStateResponseData;
import org.apache.kafka.common.requests.RequestContext;
import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.ReadShareGroupStateResponse;
import org.apache.kafka.common.requests.TransactionResult;
import org.apache.kafka.common.requests.WriteShareGroupStateResponse;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
import org.apache.kafka.coordinator.common.runtime.CoordinatorShard;
import org.apache.kafka.coordinator.common.runtime.CoordinatorShardBuilder;
import org.apache.kafka.coordinator.common.runtime.CoordinatorTimer;
import org.apache.kafka.coordinator.share.generated.ShareSnapshotKey;
import org.apache.kafka.coordinator.share.generated.ShareSnapshotValue;
import org.apache.kafka.coordinator.share.generated.ShareUpdateKey;
import org.apache.kafka.coordinator.share.generated.ShareUpdateValue;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetricsShard;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.config.ShareCoordinatorConfig;
import org.apache.kafka.server.group.share.PartitionFactory;
import org.apache.kafka.server.group.share.SharePartitionKey;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineHashMap;
import org.slf4j.Logger;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedHashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
/**
* The share coordinator shard is a replicated state machine that manages the metadata of all
* share partitions. It holds the hard and the soft state of the share partitions. This class
* has two kinds of methods:
* 1) The request handlers which handle the requests and generate a response and records to
* mutate the hard state. Those records will be written by the runtime and applied to the
* hard state via the replay methods.
* 2) The replay methods which apply records to the hard state. Those are used in the request
* handling as well as during the initial loading of the records from the partitions.
*/
public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord> {
private final Logger log;
private final ShareCoordinatorConfig config;
private final CoordinatorMetrics coordinatorMetrics;
private final CoordinatorMetricsShard metricsShard;
private final TimelineHashMap<SharePartitionKey, ShareGroupOffset> shareStateMap; // coord key -> ShareGroupOffset
// leaderEpochMap can be updated by writeState call
// or if a newer leader makes a readState call.
private final TimelineHashMap<SharePartitionKey, Integer> leaderEpochMap;
private final TimelineHashMap<SharePartitionKey, Integer> snapshotUpdateCount;
private final TimelineHashMap<SharePartitionKey, Integer> stateEpochMap;
private MetadataImage metadataImage;
public static final Exception NULL_TOPIC_ID = new Exception("The topic id cannot be null.");
public static final Exception NEGATIVE_PARTITION_ID = new Exception("The partition id cannot be a negative number.");
public static class Builder implements CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> {
private ShareCoordinatorConfig config;
private LogContext logContext;
private SnapshotRegistry snapshotRegistry;
private CoordinatorMetrics coordinatorMetrics;
private TopicPartition topicPartition;
public Builder(ShareCoordinatorConfig config) {
this.config = config;
}
@Override
public CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> withSnapshotRegistry(SnapshotRegistry snapshotRegistry) {
this.snapshotRegistry = snapshotRegistry;
return this;
}
@Override
public CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> withLogContext(LogContext logContext) {
this.logContext = logContext;
return this;
}
@Override
public CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> withTime(Time time) {
// method is required due to interface
return this;
}
@Override
public CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> withTimer(CoordinatorTimer<Void, CoordinatorRecord> timer) {
// method is required due to interface
return this;
}
@Override
public CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> withCoordinatorMetrics(CoordinatorMetrics coordinatorMetrics) {
this.coordinatorMetrics = coordinatorMetrics;
return this;
}
@Override
public CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> withTopicPartition(TopicPartition topicPartition) {
this.topicPartition = topicPartition;
return this;
}
@Override
@SuppressWarnings("NPathComplexity")
public ShareCoordinatorShard build() {
if (logContext == null) logContext = new LogContext();
if (config == null)
throw new IllegalArgumentException("Config must be set.");
if (snapshotRegistry == null)
throw new IllegalArgumentException("SnapshotRegistry must be set.");
if (coordinatorMetrics == null || !(coordinatorMetrics instanceof ShareCoordinatorMetrics))
throw new IllegalArgumentException("CoordinatorMetrics must be set and be of type ShareCoordinatorMetrics.");
if (topicPartition == null)
throw new IllegalArgumentException("TopicPartition must be set.");
ShareCoordinatorMetricsShard metricsShard = ((ShareCoordinatorMetrics) coordinatorMetrics)
.newMetricsShard(snapshotRegistry, topicPartition);
return new ShareCoordinatorShard(
logContext,
config,
coordinatorMetrics,
metricsShard,
snapshotRegistry
);
}
}
ShareCoordinatorShard(
LogContext logContext,
ShareCoordinatorConfig config,
CoordinatorMetrics coordinatorMetrics,
CoordinatorMetricsShard metricsShard,
SnapshotRegistry snapshotRegistry
) {
this.log = logContext.logger(ShareCoordinatorShard.class);
this.config = config;
this.coordinatorMetrics = coordinatorMetrics;
this.metricsShard = metricsShard;
this.shareStateMap = new TimelineHashMap<>(snapshotRegistry, 0);
this.leaderEpochMap = new TimelineHashMap<>(snapshotRegistry, 0);
this.snapshotUpdateCount = new TimelineHashMap<>(snapshotRegistry, 0);
this.stateEpochMap = new TimelineHashMap<>(snapshotRegistry, 0);
}
@Override
public void onLoaded(MetadataImage newImage) {
CoordinatorShard.super.onLoaded(newImage);
this.metadataImage = newImage;
coordinatorMetrics.activateMetricsShard(metricsShard);
}
@Override
public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
CoordinatorShard.super.onNewMetadataImage(newImage, delta);
this.metadataImage = newImage;
}
@Override
public void onUnloaded() {
CoordinatorShard.super.onUnloaded();
coordinatorMetrics.deactivateMetricsShard(metricsShard);
}
@Override
public void replay(long offset, long producerId, short producerEpoch, CoordinatorRecord record) throws RuntimeException {
ApiMessageAndVersion key = record.key();
ApiMessageAndVersion value = record.value();
switch (key.version()) {
case ShareCoordinator.SHARE_SNAPSHOT_RECORD_KEY_VERSION: // ShareSnapshot
handleShareSnapshot((ShareSnapshotKey) key.message(), (ShareSnapshotValue) messageOrNull(value));
break;
case ShareCoordinator.SHARE_UPDATE_RECORD_KEY_VERSION: // ShareUpdate
handleShareUpdate((ShareUpdateKey) key.message(), (ShareUpdateValue) messageOrNull(value));
break;
default:
// noop
}
}
private void handleShareSnapshot(ShareSnapshotKey key, ShareSnapshotValue value) {
SharePartitionKey mapKey = SharePartitionKey.getInstance(key.groupId(), key.topicId(), key.partition());
maybeUpdateLeaderEpochMap(mapKey, value.leaderEpoch());
maybeUpdateStateEpochMap(mapKey, value.stateEpoch());
ShareGroupOffset offsetRecord = ShareGroupOffset.fromRecord(value);
// this record is the complete snapshot
shareStateMap.put(mapKey, offsetRecord);
// if number of share updates is exceeded, then reset it
if (snapshotUpdateCount.containsKey(mapKey)) {
if (snapshotUpdateCount.get(mapKey) >= config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot()) {
snapshotUpdateCount.put(mapKey, 0);
}
}
}
private void handleShareUpdate(ShareUpdateKey key, ShareUpdateValue value) {
SharePartitionKey mapKey = SharePartitionKey.getInstance(key.groupId(), key.topicId(), key.partition());
maybeUpdateLeaderEpochMap(mapKey, value.leaderEpoch());
// share update does not hold state epoch information.
ShareGroupOffset offsetRecord = ShareGroupOffset.fromRecord(value);
// this is an incremental snapshot
// so, we need to apply it to our current soft state
shareStateMap.compute(mapKey, (k, v) -> v == null ? offsetRecord : merge(v, value));
snapshotUpdateCount.compute(mapKey, (k, v) -> v == null ? 0 : v + 1);
}
private void maybeUpdateLeaderEpochMap(SharePartitionKey mapKey, int leaderEpoch) {
leaderEpochMap.putIfAbsent(mapKey, leaderEpoch);
if (leaderEpochMap.get(mapKey) < leaderEpoch) {
leaderEpochMap.put(mapKey, leaderEpoch);
}
}
private void maybeUpdateStateEpochMap(SharePartitionKey mapKey, int stateEpoch) {
stateEpochMap.putIfAbsent(mapKey, stateEpoch);
if (stateEpochMap.get(mapKey) < stateEpoch) {
stateEpochMap.put(mapKey, stateEpoch);
}
}
@Override
@ -64,11 +250,332 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
CoordinatorShard.super.replayEndTransactionMarker(producerId, producerEpoch, result);
}
public CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> writeState(RequestContext context, WriteShareGroupStateRequestData request) {
throw new RuntimeException("Not implemented");
/**
* This method generates the ShareSnapshotValue record corresponding to the requested topic partition information.
* The generated record is then written to the __share_group_state topic and replayed to the in-memory state
* of the coordinator shard, shareStateMap, by CoordinatorRuntime.
* <p>
* This method as called by the ShareCoordinatorService will be provided with
* the request data which covers only a single key i.e. group1:topic1:partition1. The implementation
* below was done keeping this in mind.
*
* @param request - WriteShareGroupStateRequestData for a single key
* @return CoordinatorResult(records, response)
*/
@SuppressWarnings("NPathComplexity")
public CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> writeState(
WriteShareGroupStateRequestData request
) {
// records to write (with both key and value of snapshot type), response to caller
// only one key will be there in the request by design
metricsShard.record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
Optional<CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord>> error = maybeGetWriteStateError(request);
if (error.isPresent()) {
return error.get();
}
String groupId = request.groupId();
WriteShareGroupStateRequestData.WriteStateData topicData = request.topics().get(0);
WriteShareGroupStateRequestData.PartitionData partitionData = topicData.partitions().get(0);
SharePartitionKey key = SharePartitionKey.getInstance(groupId, topicData.topicId(), partitionData.partition());
List<CoordinatorRecord> recordList;
if (!shareStateMap.containsKey(key)) {
// since this is the first time we are getting a write request, we should be creating a share snapshot record
recordList = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
groupId, topicData.topicId(), partitionData.partition(), ShareGroupOffset.fromRequest(partitionData)
));
} else if (snapshotUpdateCount.getOrDefault(key, 0) >= config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot()) {
int newLeaderEpoch = partitionData.leaderEpoch() == -1 ? shareStateMap.get(key).leaderEpoch() : partitionData.leaderEpoch();
int newStateEpoch = partitionData.stateEpoch() == -1 ? shareStateMap.get(key).stateEpoch() : partitionData.stateEpoch();
long newStartOffset = partitionData.startOffset() == -1 ? shareStateMap.get(key).startOffset() : partitionData.startOffset();
// Since the number of update records for this share part key exceeds snapshotUpdateRecordsPerSnapshot,
// we should be creating a share snapshot record.
List<PersisterOffsetsStateBatch> batchesToAdd = combineStateBatches(
shareStateMap.get(key).stateBatchAsSet(),
partitionData.stateBatches().stream()
.map(PersisterOffsetsStateBatch::from)
.collect(Collectors.toCollection(LinkedHashSet::new)),
newStartOffset);
recordList = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
groupId, topicData.topicId(), partitionData.partition(),
new ShareGroupOffset.Builder()
.setStartOffset(newStartOffset)
.setLeaderEpoch(newLeaderEpoch)
.setStateEpoch(newStateEpoch)
.setStateBatches(batchesToAdd)
.build()));
} else {
// share snapshot is present and number of share snapshot update records < snapshotUpdateRecordsPerSnapshot
recordList = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotUpdateRecord(
groupId, topicData.topicId(), partitionData.partition(), ShareGroupOffset.fromRequest(partitionData, shareStateMap.get(key).snapshotEpoch())
));
}
List<CoordinatorRecord> validRecords = new LinkedList<>();
WriteShareGroupStateResponseData responseData = new WriteShareGroupStateResponseData();
for (CoordinatorRecord record : recordList) { // should be single record
if (!(record.key().message() instanceof ShareSnapshotKey) && !(record.key().message() instanceof ShareUpdateKey)) {
continue;
}
SharePartitionKey mapKey = null;
boolean shouldIncSnapshotEpoch = false;
if (record.key().message() instanceof ShareSnapshotKey) {
ShareSnapshotKey recordKey = (ShareSnapshotKey) record.key().message();
responseData.setResults(Collections.singletonList(WriteShareGroupStateResponse.toResponseWriteStateResult(
recordKey.topicId(), Collections.singletonList(WriteShareGroupStateResponse.toResponsePartitionResult(
recordKey.partition())))));
mapKey = SharePartitionKey.getInstance(recordKey.groupId(), recordKey.topicId(), recordKey.partition());
shouldIncSnapshotEpoch = true;
} else if (record.key().message() instanceof ShareUpdateKey) {
ShareUpdateKey recordKey = (ShareUpdateKey) record.key().message();
responseData.setResults(Collections.singletonList(WriteShareGroupStateResponse.toResponseWriteStateResult(
recordKey.topicId(), Collections.singletonList(WriteShareGroupStateResponse.toResponsePartitionResult(
recordKey.partition())))));
mapKey = SharePartitionKey.getInstance(recordKey.groupId(), recordKey.topicId(), recordKey.partition());
}
if (shareStateMap.containsKey(mapKey) && shouldIncSnapshotEpoch) {
ShareGroupOffset oldValue = shareStateMap.get(mapKey);
((ShareSnapshotValue) record.value().message()).setSnapshotEpoch(oldValue.snapshotEpoch() + 1); // increment the snapshot epoch
}
validRecords.add(record); // this will have updated snapshot epoch and on replay the value will trickle down to the map
}
return new CoordinatorResult<>(validRecords, responseData);
}
/**
* This method finds the ShareSnapshotValue record corresponding to the requested topic partition from the
* in-memory state of coordinator shard, the shareStateMap.
* <p>
* This method as called by the ShareCoordinatorService will be provided with
* the request data which covers only key i.e. group1:topic1:partition1. The implementation
* below was done keeping this in mind.
*
* @param request - WriteShareGroupStateRequestData for a single key
* @param offset - offset to read from the __share_group_state topic partition
* @return CoordinatorResult(records, response)
*/
public ReadShareGroupStateResponseData readState(ReadShareGroupStateRequestData request, Long offset) {
throw new RuntimeException("Not implemented");
// records to read (with the key of snapshot type), response to caller
// only one key will be there in the request by design
Optional<ReadShareGroupStateResponseData> error = maybeGetReadStateError(request, offset);
if (error.isPresent()) {
return error.get();
}
Uuid topicId = request.topics().get(0).topicId();
int partition = request.topics().get(0).partitions().get(0).partition();
int leaderEpoch = request.topics().get(0).partitions().get(0).leaderEpoch();
SharePartitionKey coordinatorKey = SharePartitionKey.getInstance(request.groupId(), topicId, partition);
if (!shareStateMap.containsKey(coordinatorKey)) {
return ReadShareGroupStateResponse.toResponseData(
topicId,
partition,
PartitionFactory.DEFAULT_START_OFFSET,
PartitionFactory.DEFAULT_STATE_EPOCH,
Collections.emptyList()
);
}
ShareGroupOffset offsetValue = shareStateMap.get(coordinatorKey, offset);
if (offsetValue == null) {
// Returning an error response as the snapshot value was not found
return ReadShareGroupStateResponse.toErrorResponseData(
topicId,
partition,
Errors.UNKNOWN_SERVER_ERROR,
"Data not found for topic {}, partition {} for group {}, in the in-memory state of share coordinator"
);
}
List<ReadShareGroupStateResponseData.StateBatch> stateBatches = (offsetValue.stateBatches() != null && !offsetValue.stateBatches().isEmpty()) ?
offsetValue.stateBatches().stream().map(
stateBatch -> new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(stateBatch.firstOffset())
.setLastOffset(stateBatch.lastOffset())
.setDeliveryState(stateBatch.deliveryState())
.setDeliveryCount(stateBatch.deliveryCount())
).collect(java.util.stream.Collectors.toList()) : Collections.emptyList();
// Updating the leader map with the new leader epoch
leaderEpochMap.put(coordinatorKey, leaderEpoch);
// Returning the successfully retrieved snapshot value
return ReadShareGroupStateResponse.toResponseData(topicId, partition, offsetValue.startOffset(), offsetValue.stateEpoch(), stateBatches);
}
private Optional<CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord>> maybeGetWriteStateError(
WriteShareGroupStateRequestData request
) {
String groupId = request.groupId();
WriteShareGroupStateRequestData.WriteStateData topicData = request.topics().get(0);
WriteShareGroupStateRequestData.PartitionData partitionData = topicData.partitions().get(0);
Uuid topicId = topicData.topicId();
int partitionId = partitionData.partition();
if (topicId == null) {
return Optional.of(getWriteErrorResponse(Errors.INVALID_REQUEST, NULL_TOPIC_ID, null, partitionId));
}
if (partitionId < 0) {
return Optional.of(getWriteErrorResponse(Errors.INVALID_REQUEST, NEGATIVE_PARTITION_ID, topicId, partitionId));
}
SharePartitionKey mapKey = SharePartitionKey.getInstance(groupId, topicId, partitionId);
if (leaderEpochMap.containsKey(mapKey) && leaderEpochMap.get(mapKey) > partitionData.leaderEpoch()) {
log.error("Request leader epoch smaller than last recorded.");
return Optional.of(getWriteErrorResponse(Errors.FENCED_LEADER_EPOCH, null, topicId, partitionId));
}
if (stateEpochMap.containsKey(mapKey) && stateEpochMap.get(mapKey) > partitionData.stateEpoch()) {
log.error("Request state epoch smaller than last recorded.");
return Optional.of(getWriteErrorResponse(Errors.FENCED_STATE_EPOCH, null, topicId, partitionId));
}
if (metadataImage == null) {
log.error("Metadata image is null");
return Optional.of(getWriteErrorResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION, null, topicId, partitionId));
}
if (metadataImage.topics().getTopic(topicId) == null ||
metadataImage.topics().getPartition(topicId, partitionId) == null) {
log.error("Topic/TopicPartition not found in metadata image.");
return Optional.of(getWriteErrorResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION, null, topicId, partitionId));
}
return Optional.empty();
}
private Optional<ReadShareGroupStateResponseData> maybeGetReadStateError(ReadShareGroupStateRequestData request, Long offset) {
String groupId = request.groupId();
ReadShareGroupStateRequestData.ReadStateData topicData = request.topics().get(0);
ReadShareGroupStateRequestData.PartitionData partitionData = topicData.partitions().get(0);
Uuid topicId = topicData.topicId();
int partitionId = partitionData.partition();
if (topicId == null) {
log.error("Request topic id is null.");
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(
null, partitionId, Errors.INVALID_REQUEST, NULL_TOPIC_ID.getMessage()));
}
if (partitionId < 0) {
log.error("Request partition id is negative.");
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(
topicId, partitionId, Errors.INVALID_REQUEST, NEGATIVE_PARTITION_ID.getMessage()));
}
SharePartitionKey mapKey = SharePartitionKey.getInstance(groupId, topicId, partitionId);
if (leaderEpochMap.containsKey(mapKey, offset) && leaderEpochMap.get(mapKey, offset) > partitionData.leaderEpoch()) {
log.error("Request leader epoch id is smaller than last recorded.");
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(topicId, partitionId, Errors.FENCED_LEADER_EPOCH, Errors.FENCED_LEADER_EPOCH.message()));
}
if (metadataImage == null) {
log.error("Metadata image is null");
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(topicId, partitionId, Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.UNKNOWN_TOPIC_OR_PARTITION.message()));
}
if (metadataImage.topics().getTopic(topicId) == null ||
metadataImage.topics().getPartition(topicId, partitionId) == null) {
log.error("Topic/TopicPartition not found in metadata image.");
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(topicId, partitionId, Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.UNKNOWN_TOPIC_OR_PARTITION.message()));
}
return Optional.empty();
}
private CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> getWriteErrorResponse(
Errors error,
Exception exception,
Uuid topicId,
int partitionId
) {
String message = exception == null ? error.message() : exception.getMessage();
WriteShareGroupStateResponseData responseData = WriteShareGroupStateResponse.toErrorResponseData(topicId, partitionId, error, message);
return new CoordinatorResult<>(Collections.emptyList(), responseData);
}
// Visible for testing
Integer getLeaderMapValue(SharePartitionKey key) {
return this.leaderEpochMap.get(key);
}
// Visible for testing
Integer getStateEpochMapValue(SharePartitionKey key) {
return this.stateEpochMap.get(key);
}
// Visible for testing
ShareGroupOffset getShareStateMapValue(SharePartitionKey key) {
return this.shareStateMap.get(key);
}
// Visible for testing
CoordinatorMetricsShard getMetricsShard() {
return metricsShard;
}
private static ShareGroupOffset merge(ShareGroupOffset soFar, ShareUpdateValue newData) {
// snapshot epoch should be same as last share snapshot
// state epoch is not present
Set<PersisterOffsetsStateBatch> currentBatches = soFar.stateBatchAsSet();
long newStartOffset = newData.startOffset() == -1 ? soFar.startOffset() : newData.startOffset();
int newLeaderEpoch = newData.leaderEpoch() == -1 ? soFar.leaderEpoch() : newData.leaderEpoch();
return new ShareGroupOffset.Builder()
.setSnapshotEpoch(soFar.snapshotEpoch())
.setStateEpoch(soFar.stateEpoch())
.setStartOffset(newStartOffset)
.setLeaderEpoch(newLeaderEpoch)
.setStateBatches(combineStateBatches(currentBatches, newData.stateBatches().stream()
.map(PersisterOffsetsStateBatch::from)
.collect(Collectors.toCollection(LinkedHashSet::new)), newStartOffset))
.build();
}
/**
* Util method which takes in 2 collections containing {@link PersisterOffsetsStateBatch}
* and the startOffset.
* It removes all batches from the 1st collection which have the same first and last offset
* as the batches in 2nd collection. It then creates a final list of batches which contains the
* former result and all the batches in the 2nd collection. In set notation (A - B) U B (we prefer batches in B
* which have same first and last offset in A).
* Finally, it removes any batches where the lastOffset < startOffset, if the startOffset > -1.
* @param currentBatch - collection containing current soft state of batches
* @param newBatch - collection containing batches in incoming request
* @param startOffset - startOffset to consider when removing old batches.
* @return List containing combined batches
*/
private static List<PersisterOffsetsStateBatch> combineStateBatches(
Collection<PersisterOffsetsStateBatch> currentBatch,
Collection<PersisterOffsetsStateBatch> newBatch,
long startOffset
) {
currentBatch.removeAll(newBatch);
List<PersisterOffsetsStateBatch> batchesToAdd = new LinkedList<>(currentBatch);
batchesToAdd.addAll(newBatch);
// Any batches where the last offset is < the current start offset
// are now expired. We should remove them from the persister.
if (startOffset != -1) {
batchesToAdd.removeIf(batch -> batch.lastOffset() < startOffset);
}
return batchesToAdd;
}
private static ApiMessage messageOrNull(ApiMessageAndVersion apiMessageAndVersion) {
if (apiMessageAndVersion == null) {
return null;
} else {
return apiMessageAndVersion.message();
}
}
}

View File

@ -30,7 +30,7 @@ import java.util.stream.Collectors;
/**
* Container class to represent data encapsulated in {@link ShareSnapshotValue} and {@link ShareUpdateValue}
* This class is effectively immutable (state batches is not modified out of context).
* This class is immutable (state batches is not modified out of context).
*/
public class ShareGroupOffset {
private final int snapshotEpoch;

View File

@ -0,0 +1,156 @@
/*
* 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.coordinator.share.metrics;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.metrics.stats.Meter;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.timeline.SnapshotRegistry;
import com.yammer.metrics.core.MetricsRegistry;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
public class ShareCoordinatorMetrics extends CoordinatorMetrics implements AutoCloseable {
//write (write-rate and write-total) Meter share-coordinator-metric The number of share-group state write calls per second.
//write-latency (write-latency-avg and write-latency-total) Meter share-coordinator-metrics The time taken for a share-group state write call, including the time to write to the share-group state topic.
public static final String METRICS_GROUP = "share-coordinator-metrics";
private final Metrics metrics;
private final Map<TopicPartition, ShareCoordinatorMetricsShard> shards = new ConcurrentHashMap<>();
public static final String SHARE_COORDINATOR_WRITE_SENSOR_NAME = "ShareCoordinatorWrite";
public static final String SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME = "ShareCoordinatorWriteLatency";
/**
* Global sensors. These are shared across all metrics shards.
*/
public final Map<String, Sensor> globalSensors;
public ShareCoordinatorMetrics() {
this(new Metrics());
}
public ShareCoordinatorMetrics(Metrics metrics) {
this.metrics = Objects.requireNonNull(metrics);
Sensor shareCoordinatorWriteSensor = metrics.sensor(SHARE_COORDINATOR_WRITE_SENSOR_NAME);
shareCoordinatorWriteSensor.add(new Meter(
metrics.metricName("write-rate",
METRICS_GROUP,
"The number of share-group state write calls per second."),
metrics.metricName("write-total",
METRICS_GROUP,
"Total number of share-group state write calls.")));
Sensor shareCoordinatorWriteLatencySensor = metrics.sensor(SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME);
shareCoordinatorWriteLatencySensor.add(
metrics.metricName("write-latency-avg",
METRICS_GROUP,
"The average time taken for a share-group state write call, including the time to write to the share-group state topic."),
new Avg());
shareCoordinatorWriteLatencySensor.add(
metrics.metricName("write-latency-max",
METRICS_GROUP,
"The maximum time taken for a share-group state write call, including the time to write to the share-group state topic."),
new Max());
this.globalSensors = Collections.unmodifiableMap(Utils.mkMap(
Utils.mkEntry(SHARE_COORDINATOR_WRITE_SENSOR_NAME, shareCoordinatorWriteSensor),
Utils.mkEntry(SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME, shareCoordinatorWriteLatencySensor)
));
}
@Override
public void close() throws Exception {
Arrays.asList(
SHARE_COORDINATOR_WRITE_SENSOR_NAME,
SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME
).forEach(metrics::removeSensor);
}
@Override
public ShareCoordinatorMetricsShard newMetricsShard(SnapshotRegistry snapshotRegistry, TopicPartition tp) {
return new ShareCoordinatorMetricsShard(snapshotRegistry, globalSensors, tp);
}
@Override
public void activateMetricsShard(CoordinatorMetricsShard shard) {
if (!(shard instanceof ShareCoordinatorMetricsShard)) {
throw new IllegalArgumentException("ShareCoordinatorMetrics can only activate ShareCoordinatorMetricShard");
}
shards.put(shard.topicPartition(), (ShareCoordinatorMetricsShard) shard);
}
@Override
public void deactivateMetricsShard(CoordinatorMetricsShard shard) {
if (!(shard instanceof ShareCoordinatorMetricsShard)) {
throw new IllegalArgumentException("ShareCoordinatorMetrics can only deactivate ShareCoordinatorMetricShard");
}
shards.remove(shard.topicPartition());
}
@Override
public MetricsRegistry registry() {
// we are not using MetricsRegistry in share coordinator
// but this method is part for implemented interface
return null;
}
@Override
public void onUpdateLastCommittedOffset(TopicPartition tp, long offset) {
CoordinatorMetricsShard shard = shards.get(tp);
if (shard != null) {
shard.commitUpTo(offset);
}
}
/**
* This method can be used to record on any sensor
* defined as part of global sensors
*
* @param sensorName - String representing name of sensor
*/
public void record(String sensorName, double value) {
if (globalSensors.containsKey(sensorName)) {
globalSensors.get(sensorName).record(value);
}
}
/**
* This method can be used to record on any sensor
* defined as part of global sensors
*
* @param sensorName - String representing name of sensor
*/
public void record(String sensorName) {
if (globalSensors.containsKey(sensorName)) {
globalSensors.get(sensorName).record();
}
}
}

View File

@ -0,0 +1,64 @@
/*
* 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.coordinator.share.metrics;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.timeline.SnapshotRegistry;
import java.util.Map;
public class ShareCoordinatorMetricsShard implements CoordinatorMetricsShard {
private final SnapshotRegistry snapshotRegistry;
private final Map<String, Sensor> globalSensors;
private final TopicPartition topicPartition;
public ShareCoordinatorMetricsShard(SnapshotRegistry snapshotRegistry,
Map<String, Sensor> globalSensors,
TopicPartition topicPartition) {
this.snapshotRegistry = snapshotRegistry;
this.globalSensors = globalSensors;
this.topicPartition = topicPartition;
}
@Override
public void record(String sensorName) {
if (this.globalSensors.containsKey(sensorName)) {
this.globalSensors.get(sensorName).record();
}
}
@Override
public void record(String sensorName, double val) {
if (this.globalSensors.containsKey(sensorName)) {
this.globalSensors.get(sensorName).record(val);
}
}
@Override
public TopicPartition topicPartition() {
return this.topicPartition;
}
@Override
public void commitUpTo(long offset) {
}
}

View File

@ -0,0 +1,31 @@
/*
* 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.coordinator.share.metrics;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl;
public class ShareCoordinatorRuntimeMetrics extends CoordinatorRuntimeMetricsImpl {
/**
* The metrics group.
*/
public static final String METRICS_GROUP = "share-coordinator-metrics";
public ShareCoordinatorRuntimeMetrics(Metrics metrics) {
super(metrics, METRICS_GROUP);
}
}

View File

@ -0,0 +1,70 @@
/*
* 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.coordinator.share;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.config.ShareCoordinatorConfig;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class ShareCoordinatorConfigTest {
private static final List<ConfigDef> CONFIG_DEF_LIST = Collections.singletonList(
ShareCoordinatorConfig.CONFIG_DEF
);
public static ShareCoordinatorConfig testConfig() {
return createConfig(testConfigMap());
}
private static Map<String, String> testConfigMapRaw() {
Map<String, String> configs = new HashMap<>();
configs.put(ShareCoordinatorConfig.STATE_TOPIC_NUM_PARTITIONS_CONFIG, "1");
configs.put(ShareCoordinatorConfig.STATE_TOPIC_REPLICATION_FACTOR_CONFIG, "1");
configs.put(ShareCoordinatorConfig.STATE_TOPIC_MIN_ISR_CONFIG, "1");
configs.put(ShareCoordinatorConfig.STATE_TOPIC_SEGMENT_BYTES_CONFIG, "1000");
configs.put(ShareCoordinatorConfig.NUM_THREADS_CONFIG, "1");
configs.put(ShareCoordinatorConfig.SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG, "50");
configs.put(ShareCoordinatorConfig.WRITE_TIMEOUT_MS_CONFIG, "5000");
configs.put(ShareCoordinatorConfig.LOAD_BUFFER_SIZE_CONFIG, "555");
configs.put(ShareCoordinatorConfig.APPEND_LINGER_MS_CONFIG, "10");
configs.put(ShareCoordinatorConfig.STATE_TOPIC_COMPRESSION_CODEC_CONFIG, String.valueOf(CompressionType.NONE.id));
return configs;
}
public static Map<String, String> testConfigMap() {
return Collections.unmodifiableMap(testConfigMapRaw());
}
public static Map<String, String> testConfigMap(Map<String, String> overrides) {
Map<String, String> configs = testConfigMapRaw();
configs.putAll(overrides);
return Collections.unmodifiableMap(configs);
}
public static ShareCoordinatorConfig createConfig(Map<String, String> configs) {
return new ShareCoordinatorConfig(
new AbstractConfig(Utils.mergeConfigs(CONFIG_DEF_LIST), configs, false));
}
}

View File

@ -0,0 +1,611 @@
/*
* 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.coordinator.share;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.internals.Topic;
import org.apache.kafka.common.message.ReadShareGroupStateRequestData;
import org.apache.kafka.common.message.ReadShareGroupStateResponseData;
import org.apache.kafka.common.message.WriteShareGroupStateRequestData;
import org.apache.kafka.common.message.WriteShareGroupStateResponseData;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
import org.apache.kafka.server.util.FutureUtils;
import org.junit.jupiter.api.Test;
import org.mockito.ArgumentMatchers;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.apache.kafka.coordinator.common.runtime.TestUtil.requestContext;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
class ShareCoordinatorServiceTest {
@SuppressWarnings("unchecked")
private CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> mockRuntime() {
return (CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord>) mock(CoordinatorRuntime.class);
}
@Test
public void testStartupShutdown() throws Exception {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
new ShareCoordinatorMetrics(),
Time.SYSTEM
);
service.startup(() -> 1);
service.shutdown();
verify(runtime, times(1)).close();
}
@Test
public void testWriteStateSuccess() throws ExecutionException, InterruptedException, TimeoutException {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
Metrics metrics = new Metrics();
ShareCoordinatorMetrics coordinatorMetrics = new ShareCoordinatorMetrics(metrics);
Time time = mock(Time.class);
when(time.hiResClockMs()).thenReturn(0L).thenReturn(100L).thenReturn(150L);
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
coordinatorMetrics,
time
);
service.startup(() -> 1);
String groupId = "group1";
Uuid topicId1 = Uuid.randomUuid();
int partition1 = 0;
Uuid topicId2 = Uuid.randomUuid();
int partition2 = 1;
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(groupId)
.setTopics(Arrays.asList(
new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(
new WriteShareGroupStateRequestData.PartitionData()
.setPartition(partition1)
.setStartOffset(0)
.setStateEpoch(1)
.setLeaderEpoch(1)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0))
)
)),
new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(
new WriteShareGroupStateRequestData.PartitionData()
.setPartition(partition2)
.setStartOffset(0)
.setStateEpoch(1)
.setLeaderEpoch(1)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0))
)
))
)
);
WriteShareGroupStateResponseData response1 = new WriteShareGroupStateResponseData()
.setResults(Collections.singletonList(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)))
));
WriteShareGroupStateResponseData response2 = new WriteShareGroupStateResponseData()
.setResults(Collections.singletonList(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition2)))
));
when(runtime.scheduleWriteOperation(
ArgumentMatchers.eq("write-share-group-state"),
ArgumentMatchers.eq(new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, 0)),
ArgumentMatchers.eq(Duration.ofMillis(5000)),
ArgumentMatchers.any()
))
.thenReturn(CompletableFuture.completedFuture(response1))
.thenReturn(CompletableFuture.completedFuture(response2));
CompletableFuture<WriteShareGroupStateResponseData> future = service.writeState(
requestContext(ApiKeys.WRITE_SHARE_GROUP_STATE),
request
);
HashSet<WriteShareGroupStateResponseData.WriteStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<WriteShareGroupStateResponseData.WriteStateResult> expectedResult = new HashSet<>(Arrays.asList(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition2))),
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)))));
assertEquals(expectedResult, result);
verify(time, times(2)).hiResClockMs();
Set<MetricName> expectedMetrics = new HashSet<>(Arrays.asList(
metrics.metricName("write-latency-avg", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-latency-max", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-rate", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-total", ShareCoordinatorMetrics.METRICS_GROUP)
));
expectedMetrics.forEach(metric -> assertTrue(metrics.metrics().containsKey(metric)));
}
@Test
public void testReadStateSuccess() throws ExecutionException, InterruptedException, TimeoutException {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
new ShareCoordinatorMetrics(),
Time.SYSTEM
);
service.startup(() -> 1);
String groupId = "group1";
Uuid topicId1 = Uuid.randomUuid();
int partition1 = 0;
Uuid topicId2 = Uuid.randomUuid();
int partition2 = 1;
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(groupId)
.setTopics(Arrays.asList(
new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(
new ReadShareGroupStateRequestData.PartitionData()
.setPartition(partition1)
.setLeaderEpoch(1)
)),
new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(
new ReadShareGroupStateRequestData.PartitionData()
.setPartition(partition2)
.setLeaderEpoch(1)
))
)
);
ReadShareGroupStateResponseData.ReadStateResult topicData1 = new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)
.setErrorCode(Errors.NONE.code())
.setStateEpoch(1)
.setStartOffset(0)
.setStateBatches(Collections.singletonList(new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0))
))
);
ReadShareGroupStateResponseData.ReadStateResult topicData2 = new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partition2)
.setErrorCode(Errors.NONE.code())
.setStateEpoch(1)
.setStartOffset(0)
.setStateBatches(Arrays.asList(
new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0),
new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(11)
.setLastOffset(20)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)
)))
);
when(runtime.scheduleReadOperation(
ArgumentMatchers.eq("read-share-group-state"),
ArgumentMatchers.eq(new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, 0)),
ArgumentMatchers.any()
))
.thenReturn(CompletableFuture.completedFuture(new ReadShareGroupStateResponseData()
.setResults(Collections.singletonList(topicData1))))
.thenReturn(CompletableFuture.completedFuture(new ReadShareGroupStateResponseData()
.setResults(Collections.singletonList(topicData2))));
CompletableFuture<ReadShareGroupStateResponseData> future = service.readState(
requestContext(ApiKeys.READ_SHARE_GROUP_STATE),
request
);
HashSet<ReadShareGroupStateResponseData.ReadStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<ReadShareGroupStateResponseData.ReadStateResult> expectedResult = new HashSet<>(Arrays.asList(
topicData1,
topicData2));
assertEquals(expectedResult, result);
}
@Test
public void testWriteStateValidationsError() throws ExecutionException, InterruptedException, TimeoutException {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
new ShareCoordinatorMetrics(),
Time.SYSTEM
);
service.startup(() -> 1);
String groupId = "group1";
Uuid topicId = Uuid.randomUuid();
int partition = 0;
// 1. Empty topicsData
assertEquals(new WriteShareGroupStateResponseData(),
service.writeState(
requestContext(ApiKeys.WRITE_SHARE_GROUP_STATE),
new WriteShareGroupStateRequestData().setGroupId(groupId)
).get(5, TimeUnit.SECONDS)
);
// 2. Empty partitionsData
assertEquals(new WriteShareGroupStateResponseData(),
service.writeState(
requestContext(ApiKeys.WRITE_SHARE_GROUP_STATE),
new WriteShareGroupStateRequestData().setGroupId(groupId).setTopics(Collections.singletonList(
new WriteShareGroupStateRequestData.WriteStateData().setTopicId(topicId)))
).get(5, TimeUnit.SECONDS)
);
// 3. Invalid groupId
assertEquals(new WriteShareGroupStateResponseData(),
service.writeState(
requestContext(ApiKeys.WRITE_SHARE_GROUP_STATE),
new WriteShareGroupStateRequestData().setGroupId(null).setTopics(Collections.singletonList(
new WriteShareGroupStateRequestData.WriteStateData().setTopicId(topicId).setPartitions(Collections.singletonList(
new WriteShareGroupStateRequestData.PartitionData().setPartition(partition)))))
).get(5, TimeUnit.SECONDS)
);
}
@Test
public void testReadStateValidationsError() throws ExecutionException, InterruptedException, TimeoutException {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
new ShareCoordinatorMetrics(),
Time.SYSTEM
);
service.startup(() -> 1);
String groupId = "group1";
Uuid topicId = Uuid.randomUuid();
int partition = 0;
// 1. Empty topicsData
assertEquals(new ReadShareGroupStateResponseData(),
service.readState(
requestContext(ApiKeys.READ_SHARE_GROUP_STATE),
new ReadShareGroupStateRequestData().setGroupId(groupId)
).get(5, TimeUnit.SECONDS)
);
// 2. Empty partitionsData
assertEquals(new ReadShareGroupStateResponseData(),
service.readState(
requestContext(ApiKeys.READ_SHARE_GROUP_STATE),
new ReadShareGroupStateRequestData().setGroupId(groupId).setTopics(Collections.singletonList(
new ReadShareGroupStateRequestData.ReadStateData().setTopicId(topicId)))
).get(5, TimeUnit.SECONDS)
);
// 3. Invalid groupId
assertEquals(new ReadShareGroupStateResponseData(),
service.readState(
requestContext(ApiKeys.READ_SHARE_GROUP_STATE),
new ReadShareGroupStateRequestData().setGroupId(null).setTopics(Collections.singletonList(
new ReadShareGroupStateRequestData.ReadStateData().setTopicId(topicId).setPartitions(Collections.singletonList(
new ReadShareGroupStateRequestData.PartitionData().setPartition(partition)))))
).get(5, TimeUnit.SECONDS)
);
}
@Test
public void testWriteStateWhenNotStarted() throws ExecutionException, InterruptedException, TimeoutException {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
new ShareCoordinatorMetrics(),
Time.SYSTEM
);
String groupId = "group1";
Uuid topicId1 = Uuid.randomUuid();
int partition1 = 0;
Uuid topicId2 = Uuid.randomUuid();
int partition2 = 1;
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(groupId)
.setTopics(Arrays.asList(
new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(
new WriteShareGroupStateRequestData.PartitionData()
.setPartition(partition1)
.setStartOffset(0)
.setStateEpoch(1)
.setLeaderEpoch(1)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0))
)
)),
new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(
new WriteShareGroupStateRequestData.PartitionData()
.setPartition(partition2)
.setStartOffset(0)
.setStateEpoch(1)
.setLeaderEpoch(1)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0))
)
))
)
);
CompletableFuture<WriteShareGroupStateResponseData> future = service.writeState(
requestContext(ApiKeys.WRITE_SHARE_GROUP_STATE),
request
);
HashSet<WriteShareGroupStateResponseData.WriteStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<WriteShareGroupStateResponseData.WriteStateResult> expectedResult = new HashSet<>(Arrays.asList(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition2)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available."))),
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available.")))));
assertEquals(expectedResult, result);
}
@Test
public void testReadStateWhenNotStarted() throws ExecutionException, InterruptedException, TimeoutException {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
new ShareCoordinatorMetrics(),
Time.SYSTEM
);
String groupId = "group1";
Uuid topicId1 = Uuid.randomUuid();
int partition1 = 0;
Uuid topicId2 = Uuid.randomUuid();
int partition2 = 1;
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(groupId)
.setTopics(Arrays.asList(
new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(
new ReadShareGroupStateRequestData.PartitionData()
.setPartition(partition1)
.setLeaderEpoch(1)
)),
new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(
new ReadShareGroupStateRequestData.PartitionData()
.setPartition(partition2)
.setLeaderEpoch(1)
))
)
);
CompletableFuture<ReadShareGroupStateResponseData> future = service.readState(
requestContext(ApiKeys.READ_SHARE_GROUP_STATE),
request
);
HashSet<ReadShareGroupStateResponseData.ReadStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<ReadShareGroupStateResponseData.ReadStateResult> expectedResult = new HashSet<>(Arrays.asList(
new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId2)
.setPartitions(Collections.singletonList(new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partition2)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available."))),
new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId1)
.setPartitions(Collections.singletonList(new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available.")))));
assertEquals(expectedResult, result);
}
@Test
public void testWriteFutureReturnsError() throws ExecutionException, InterruptedException, TimeoutException {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
new ShareCoordinatorMetrics(),
Time.SYSTEM
);
service.startup(() -> 1);
String groupId = "group1";
Uuid topicId = Uuid.randomUuid();
int partition = 0;
when(runtime.scheduleWriteOperation(any(), any(), any(), any()))
.thenReturn(FutureUtils.failedFuture(Errors.UNKNOWN_TOPIC_OR_PARTITION.exception()));
assertEquals(new WriteShareGroupStateResponseData()
.setResults(Collections.singletonList(new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId)
.setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Unable to write share group state: This server does not host this topic-partition."))))),
service.writeState(
requestContext(ApiKeys.WRITE_SHARE_GROUP_STATE),
new WriteShareGroupStateRequestData().setGroupId(groupId)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(topicId)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(partition)
.setLeaderEpoch(1)
.setStartOffset(1)
.setStateEpoch(1)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(2)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 1)))
))
))
).get(5, TimeUnit.SECONDS)
);
}
@Test
public void testReadFutureReturnsError() throws ExecutionException, InterruptedException, TimeoutException {
CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
ShareCoordinatorService service = new ShareCoordinatorService(
new LogContext(),
ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap()),
runtime,
new ShareCoordinatorMetrics(),
Time.SYSTEM
);
service.startup(() -> 1);
String groupId = "group1";
Uuid topicId = Uuid.randomUuid();
int partition = 0;
when(runtime.scheduleReadOperation(any(), any(), any()))
.thenReturn(FutureUtils.failedFuture(Errors.UNKNOWN_SERVER_ERROR.exception()));
assertEquals(new ReadShareGroupStateResponseData()
.setResults(Collections.singletonList(new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId)
.setPartitions(Collections.singletonList(new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partition)
.setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code())
.setErrorMessage("Unable to read share group state: The server experienced an unexpected error when processing the request."))))),
service.readState(
requestContext(ApiKeys.READ_SHARE_GROUP_STATE),
new ReadShareGroupStateRequestData().setGroupId(groupId)
.setTopics(Collections.singletonList(new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(topicId)
.setPartitions(Collections.singletonList(new ReadShareGroupStateRequestData.PartitionData()
.setPartition(partition)
.setLeaderEpoch(1)
))
))
).get(5, TimeUnit.SECONDS)
);
}
}

View File

@ -0,0 +1,800 @@
/*
* 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.coordinator.share;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.ReadShareGroupStateRequestData;
import org.apache.kafka.common.message.ReadShareGroupStateResponseData;
import org.apache.kafka.common.message.WriteShareGroupStateRequestData;
import org.apache.kafka.common.message.WriteShareGroupStateResponseData;
import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.ReadShareGroupStateResponse;
import org.apache.kafka.common.requests.WriteShareGroupStateResponse;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
import org.apache.kafka.coordinator.share.generated.ShareSnapshotKey;
import org.apache.kafka.coordinator.share.generated.ShareSnapshotValue;
import org.apache.kafka.coordinator.share.generated.ShareUpdateValue;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.TopicImage;
import org.apache.kafka.metadata.PartitionRegistration;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.config.ShareCoordinatorConfig;
import org.apache.kafka.server.group.share.SharePartitionKey;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
class ShareCoordinatorShardTest {
private static final String GROUP_ID = "group1";
private static final Uuid TOPIC_ID = Uuid.randomUuid();
private static final int PARTITION = 0;
public static class ShareCoordinatorShardBuilder {
private final LogContext logContext = new LogContext();
private ShareCoordinatorConfig config = null;
private CoordinatorMetrics coordinatorMetrics = mock(CoordinatorMetrics.class);
private CoordinatorMetricsShard metricsShard = mock(CoordinatorMetricsShard.class);
private final SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
private MetadataImage metadataImage = null;
private Map<String, String> configOverrides = new HashMap<>();
ShareCoordinatorShard build() {
if (metadataImage == null) metadataImage = mock(MetadataImage.class, RETURNS_DEEP_STUBS);
if (config == null) {
config = ShareCoordinatorConfigTest.createConfig(ShareCoordinatorConfigTest.testConfigMap(configOverrides));
}
ShareCoordinatorShard shard = new ShareCoordinatorShard(
logContext,
config,
coordinatorMetrics,
metricsShard,
snapshotRegistry
);
when(metadataImage.topics().getTopic((Uuid) any())).thenReturn(mock(TopicImage.class));
when(metadataImage.topics().getPartition(any(), anyInt())).thenReturn(mock(PartitionRegistration.class));
shard.onLoaded(metadataImage);
return shard;
}
public ShareCoordinatorShardBuilder setConfigOverrides(Map<String, String> configOverrides) {
this.configOverrides = configOverrides;
return this;
}
}
private void writeAndReplayDefaultRecord(ShareCoordinatorShard shard) {
writeAndReplayRecord(shard, 0);
}
private void writeAndReplayRecord(ShareCoordinatorShard shard, int leaderEpoch) {
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(leaderEpoch)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
}
@Test
public void testReplayWithShareSnapshot() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
long offset = 0;
long producerId = 0;
short producerEpoch = 0;
int leaderEpoch = 1;
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
CoordinatorRecord record1 = new CoordinatorRecord(
new ApiMessageAndVersion(
new ShareSnapshotKey()
.setGroupId(GROUP_ID)
.setTopicId(TOPIC_ID)
.setPartition(PARTITION),
(short) 0
),
new ApiMessageAndVersion(
new ShareSnapshotValue()
.setSnapshotEpoch(0)
.setStateEpoch(0)
.setLeaderEpoch(leaderEpoch)
.setStateBatches(Collections.singletonList(
new ShareSnapshotValue.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0))),
(short) 0
)
);
CoordinatorRecord record2 = new CoordinatorRecord(
new ApiMessageAndVersion(
new ShareSnapshotKey()
.setGroupId(GROUP_ID)
.setTopicId(TOPIC_ID)
.setPartition(PARTITION),
(short) 0
),
new ApiMessageAndVersion(
new ShareSnapshotValue()
.setSnapshotEpoch(1)
.setStateEpoch(1)
.setLeaderEpoch(leaderEpoch + 1)
.setStateBatches(Collections.singletonList(
new ShareSnapshotValue.StateBatch()
.setFirstOffset(11)
.setLastOffset(12)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0))),
(short) 0
)
);
// First replay should populate values in otherwise empty shareStateMap and leaderMap
shard.replay(offset, producerId, producerEpoch, record1);
assertEquals(groupOffset(record1.value().message()),
shard.getShareStateMapValue(shareCoordinatorKey));
assertEquals(leaderEpoch, shard.getLeaderMapValue(shareCoordinatorKey));
// Second replay should update the existing values in shareStateMap and leaderMap
shard.replay(offset + 1, producerId, producerEpoch, record2);
assertEquals(groupOffset(record2.value().message()), shard.getShareStateMapValue(shareCoordinatorKey));
assertEquals(leaderEpoch + 1, shard.getLeaderMapValue(shareCoordinatorKey));
}
@Test
public void testWriteStateSuccess() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0))
));
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0))
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey));
verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
}
@Test
public void testSubsequentWriteStateSnapshotEpochUpdatesSuccessfully() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
WriteShareGroupStateRequestData request2 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(11)
.setLastOffset(20)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request1);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0))
));
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(expectedRecords.get(0).value().message()),
shard.getShareStateMapValue(shareCoordinatorKey));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey));
result = shard.writeState(request2);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
// the snapshot epoch here will be 1 since this is a snapshot update record,
// and it refers to parent share snapshot
expectedRecords = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotUpdateRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request2.topics().get(0).partitions().get(0), 0)
));
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
ShareGroupOffset incrementalUpdate = groupOffset(expectedRecords.get(0).value().message());
ShareGroupOffset combinedState = shard.getShareStateMapValue(shareCoordinatorKey);
assertEquals(incrementalUpdate.snapshotEpoch(), combinedState.snapshotEpoch());
assertEquals(incrementalUpdate.leaderEpoch(), combinedState.leaderEpoch());
assertEquals(incrementalUpdate.startOffset(), combinedState.startOffset());
assertTrue(combinedState.stateBatchAsSet().containsAll(incrementalUpdate.stateBatchAsSet()));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey));
}
@Test
public void testWriteStateInvalidRequestData() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
int partition = -1;
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(partition)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request);
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toErrorResponseData(
TOPIC_ID, partition, Errors.INVALID_REQUEST, ShareCoordinatorShard.NEGATIVE_PARTITION_ID.getMessage());
List<CoordinatorRecord> expectedRecords = Collections.emptyList();
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
assertNull(shard.getShareStateMapValue(shareCoordinatorKey));
assertNull(shard.getLeaderMapValue(shareCoordinatorKey));
}
@Test
public void testWriteNullMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
shard.onLoaded(null);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(0)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request);
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toErrorResponseData(
TOPIC_ID, 0, Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.UNKNOWN_TOPIC_OR_PARTITION.message());
List<CoordinatorRecord> expectedRecords = Collections.emptyList();
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
assertNull(shard.getShareStateMapValue(shareCoordinatorKey));
assertNull(shard.getLeaderMapValue(shareCoordinatorKey));
}
@Test
public void testWriteStateFencedLeaderEpochError() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(5)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
WriteShareGroupStateRequestData request2 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(3) // lower leader epoch in the second request
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(11)
.setLastOffset(20)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request1);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0))
));
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(expectedRecords.get(0).value().message()),
shard.getShareStateMapValue(shareCoordinatorKey));
assertEquals(5, shard.getLeaderMapValue(shareCoordinatorKey));
result = shard.writeState(request2);
// Since the leader epoch in the second request was lower than the one in the first request, FENCED_LEADER_EPOCH error is expected
expectedData = WriteShareGroupStateResponse.toErrorResponseData(
TOPIC_ID, PARTITION, Errors.FENCED_LEADER_EPOCH, Errors.FENCED_LEADER_EPOCH.message());
expectedRecords = Collections.emptyList();
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
// No changes to the leaderMap
assertEquals(5, shard.getLeaderMapValue(shareCoordinatorKey));
}
@Test
public void testWriteStateFencedStateEpochError() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(1)
.setLeaderEpoch(5)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
WriteShareGroupStateRequestData request2 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(0) // lower state epoch in the second request
.setLeaderEpoch(5)
.setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(11)
.setLastOffset(20)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request1);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0))
));
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(expectedRecords.get(0).value().message()),
shard.getShareStateMapValue(shareCoordinatorKey));
assertEquals(5, shard.getLeaderMapValue(shareCoordinatorKey));
result = shard.writeState(request2);
// Since the leader epoch in the second request was lower than the one in the first request, FENCED_LEADER_EPOCH error is expected
expectedData = WriteShareGroupStateResponse.toErrorResponseData(
TOPIC_ID, PARTITION, Errors.FENCED_STATE_EPOCH, Errors.FENCED_STATE_EPOCH.message());
expectedRecords = Collections.emptyList();
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
// No changes to the stateEpochMap
assertEquals(1, shard.getStateEpochMapValue(shareCoordinatorKey));
}
@Test
public void testReadStateSuccess() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
SharePartitionKey coordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
writeAndReplayDefaultRecord(shard);
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new ReadShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setLeaderEpoch(1)))));
ReadShareGroupStateResponseData result = shard.readState(request, 0L);
assertEquals(ReadShareGroupStateResponse.toResponseData(
TOPIC_ID,
PARTITION,
0,
0,
Collections.singletonList(new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)
)
), result);
assertEquals(1, shard.getLeaderMapValue(coordinatorKey));
}
@Test
public void testReadStateInvalidRequestData() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
int partition = -1;
writeAndReplayDefaultRecord(shard);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new ReadShareGroupStateRequestData.PartitionData()
.setPartition(partition)
.setLeaderEpoch(5)))));
ReadShareGroupStateResponseData result = shard.readState(request, 0L);
ReadShareGroupStateResponseData expectedData = ReadShareGroupStateResponse.toErrorResponseData(
TOPIC_ID, partition, Errors.INVALID_REQUEST, ShareCoordinatorShard.NEGATIVE_PARTITION_ID.getMessage());
assertEquals(expectedData, result);
// Leader epoch should not be changed because the request failed
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey));
}
@Test
public void testReadNullMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
writeAndReplayDefaultRecord(shard);
shard.onLoaded(null);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new ReadShareGroupStateRequestData.PartitionData()
.setPartition(0)
.setLeaderEpoch(5)))));
ReadShareGroupStateResponseData result = shard.readState(request, 0L);
ReadShareGroupStateResponseData expectedData = ReadShareGroupStateResponse.toErrorResponseData(
TOPIC_ID, 0, Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.UNKNOWN_TOPIC_OR_PARTITION.message());
assertEquals(expectedData, result);
// Leader epoch should not be changed because the request failed
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey));
}
@Test
public void testReadStateFencedLeaderEpochError() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
int leaderEpoch = 5;
writeAndReplayRecord(shard, leaderEpoch); // leaderEpoch in the leaderMap will be 5
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new ReadShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setLeaderEpoch(3))))); // lower leaderEpoch than the one stored in leaderMap
ReadShareGroupStateResponseData result = shard.readState(request, 0L);
ReadShareGroupStateResponseData expectedData = ReadShareGroupStateResponse.toErrorResponseData(
TOPIC_ID,
PARTITION,
Errors.FENCED_LEADER_EPOCH,
Errors.FENCED_LEADER_EPOCH.message());
assertEquals(expectedData, result);
assertEquals(leaderEpoch, shard.getLeaderMapValue(shareCoordinatorKey));
}
@Test
public void testNonSequentialBatchUpdates() {
// startOffset: 100
// Batch1 {
// firstOffset: 100
// lastOffset: 109
// deliverState: Acquired
// deliverCount: 1
// }
// Batch2 {
// firstOffset: 110
// lastOffset: 119
// deliverState: Acquired
// deliverCount: 2
// }
// Batch3 {
// firstOffset: 120
// lastOffset: 129
// deliverState: Acquired
// deliverCount: 0
// }
//
// -Share leader acks batch 1 and sends the state of batch 1 to Share Coordinator.
// -Share leader advances startOffset to 110.
// -Share leader acks batch 3 and sends the new startOffset and the state of batch 3 to share coordinator.
// -Share coordinator writes the snapshot with startOffset 110 and batch 3.
// -batch2 should NOT be lost
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder()
.setConfigOverrides(Collections.singletonMap(ShareCoordinatorConfig.SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG, "0"))
.build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
// set initial state
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(100)
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(Arrays.asList(
new WriteShareGroupStateRequestData.StateBatch() //b1
.setFirstOffset(100)
.setLastOffset(109)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 1), //acquired
new WriteShareGroupStateRequestData.StateBatch() //b2
.setFirstOffset(110)
.setLastOffset(119)
.setDeliveryCount((short) 2)
.setDeliveryState((byte) 1), //acquired
new WriteShareGroupStateRequestData.StateBatch() //b3
.setFirstOffset(120)
.setLastOffset(129)
.setDeliveryCount((short) 0)
.setDeliveryState((byte) 1))) //acquired
))
));
CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0))
));
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0))
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey));
verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
// acknowledge b1
WriteShareGroupStateRequestData requestUpdateB1 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(-1)
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(Collections.singletonList(
new WriteShareGroupStateRequestData.StateBatch() //b1
.setFirstOffset(100)
.setLastOffset(109)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 2))) // acked
))
));
result = shard.writeState(requestUpdateB1);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
// ack batch 3 and move start offset
WriteShareGroupStateRequestData requestUpdateStartOffsetAndB3 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(Collections.singletonList(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(Collections.singletonList(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(110) // 100 -> 110
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(Collections.singletonList(
new WriteShareGroupStateRequestData.StateBatch() //b3
.setFirstOffset(120)
.setLastOffset(129)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 2))) //acked
))
));
result = shard.writeState(requestUpdateStartOffsetAndB3);
shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedDataFinal = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
ShareGroupOffset offsetFinal = new ShareGroupOffset.Builder()
.setStartOffset(110)
.setLeaderEpoch(0)
.setStateEpoch(0)
.setSnapshotEpoch(2) // since 2nd share snapshot
.setStateBatches(Arrays.asList(
new PersisterOffsetsStateBatch(110, 119, (byte) 1, (short) 2), // b2 not lost
new PersisterOffsetsStateBatch(120, 129, (byte) 2, (short) 1)
))
.build();
List<CoordinatorRecord> expectedRecordsFinal = Collections.singletonList(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, offsetFinal
));
assertEquals(expectedDataFinal, result.response());
assertEquals(expectedRecordsFinal, result.records());
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, offsetFinal
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey));
verify(shard.getMetricsShard(), times(3)).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
}
private static ShareGroupOffset groupOffset(ApiMessage record) {
if (record instanceof ShareSnapshotValue) {
return ShareGroupOffset.fromRecord((ShareSnapshotValue) record);
}
return ShareGroupOffset.fromRecord((ShareUpdateValue) record);
}
}

View File

@ -0,0 +1,77 @@
/*
* 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.coordinator.share.metrics;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashSet;
import static org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME;
import static org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class ShareCoordinatorMetricsTest {
@Test
public void testMetricNames() {
Metrics metrics = new Metrics();
HashSet<MetricName> expectedMetrics = new HashSet<>(Arrays.asList(
metrics.metricName("write-rate", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-total", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-latency-avg", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-latency-max", ShareCoordinatorMetrics.METRICS_GROUP)
));
ShareCoordinatorMetrics ignored = new ShareCoordinatorMetrics(metrics);
for (MetricName metricName : expectedMetrics) {
assertTrue(metrics.metrics().containsKey(metricName));
}
}
@Test
public void testGlobalSensors() {
MockTime time = new MockTime();
Metrics metrics = new Metrics(time);
ShareCoordinatorMetrics coordinatorMetrics = new ShareCoordinatorMetrics(metrics);
ShareCoordinatorMetricsShard shard = coordinatorMetrics.newMetricsShard(
new SnapshotRegistry(new LogContext()), new TopicPartition("__share_group_state", 0)
);
shard.record(SHARE_COORDINATOR_WRITE_SENSOR_NAME);
assertMetricValue(metrics, metrics.metricName("write-rate", ShareCoordinatorMetrics.METRICS_GROUP), 1.0 / 30); //sampled stats
assertMetricValue(metrics, metrics.metricName("write-total", ShareCoordinatorMetrics.METRICS_GROUP), 1.0);
shard.record(SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME, 20);
shard.record(SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME, 30);
assertMetricValue(metrics, metrics.metricName("write-latency-avg", ShareCoordinatorMetrics.METRICS_GROUP), 50.0 / 2);
assertMetricValue(metrics, metrics.metricName("write-latency-max", ShareCoordinatorMetrics.METRICS_GROUP), 30.0);
}
private void assertMetricValue(Metrics metrics, MetricName metricName, double val) {
assertEquals(val, metrics.metric(metricName).metricValue());
}
}