mirror of https://github.com/apache/kafka.git
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:
parent
92672d1df8
commit
821c10157d
|
@ -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
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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;
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 = {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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 = {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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]),
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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)) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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) {
|
||||
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue