Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-12272: Fix commit-interval metrics #10103

Merged
merged 1 commit into from Feb 11, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -711,8 +711,9 @@ void runOnce() {
punctuateSensor.record(punctuateLatency / (double) punctuated, now);
}

final long beforeCommitMs = now;
final int committed = maybeCommit();
final long commitLatency = advanceNowAndComputeLatency();
final long commitLatency = Math.max(now - beforeCommitMs, 0);
totalCommitLatency += commitLatency;
if (committed > 0) {
commitSensor.record(commitLatency / (double) committed, now);
Expand Down
Expand Up @@ -20,6 +20,7 @@
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.InvalidOffsetException;
import org.apache.kafka.clients.consumer.MockConsumer;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
Expand Down Expand Up @@ -110,9 +111,11 @@
import static org.apache.kafka.common.utils.Utils.mkSet;
import static org.apache.kafka.streams.processor.internals.ClientUtils.getSharedAdminClientId;
import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME;
import static org.easymock.EasyMock.anyObject;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.expectLastCall;
import static org.easymock.EasyMock.mock;
import static org.easymock.EasyMock.niceMock;
import static org.easymock.EasyMock.verify;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.not;
Expand Down Expand Up @@ -779,6 +782,111 @@ int commit(final Collection<Task> tasksToCommit) {
assertTrue(committed.get());
}

@Test
public void shouldRecordCommitLatency() {
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
expect(consumer.poll(anyObject())).andStubReturn(new ConsumerRecords<>(Collections.emptyMap()));
final Task task = niceMock(Task.class);
expect(task.id()).andStubReturn(task1);
expect(task.inputPartitions()).andStubReturn(Collections.singleton(t1p1));
final ActiveTaskCreator activeTaskCreator = mock(ActiveTaskCreator.class);
expect(activeTaskCreator.createTasks(anyObject(), anyObject())).andStubReturn(Collections.singleton(task));
expect(activeTaskCreator.producerClientIds()).andStubReturn(Collections.singleton("producerClientId"));
EasyMock.replay(consumer, task, activeTaskCreator);

final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST);

final TaskManager taskManager = new TaskManager(
null,
null,
null,
activeTaskCreator,
null,
internalTopologyBuilder,
null,
null,
null
) {
@Override
int commit(final Collection<Task> tasksToCommit) {
mockTime.sleep(10L);
return 1;
}
};
taskManager.setMainConsumer(consumer);

final StreamThread thread = new StreamThread(
mockTime,
config,
null,
consumer,
consumer,
changelogReader,
null,
taskManager,
streamsMetrics,
internalTopologyBuilder,
CLIENT_ID,
new LogContext(""),
new AtomicInteger(),
new AtomicLong(Long.MAX_VALUE)
);
thread.updateThreadMetadata("adminClientId");
thread.setState(StreamThread.State.STARTING);

final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>();
activeTasks.put(task1, Collections.singleton(t1p1));
thread.taskManager().handleAssignment(activeTasks, emptyMap());
thread.rebalanceListener().onPartitionsAssigned(Collections.singleton(t1p1));

assertTrue(
Double.isNaN(
(Double) streamsMetrics.metrics().get(new MetricName(
"commit-latency-max",
"stream-thread-metrics",
"",
Collections.singletonMap("thread-id", CLIENT_ID))
).metricValue()
)
);
assertTrue(
Double.isNaN(
(Double) streamsMetrics.metrics().get(new MetricName(
"commit-latency-avg",
"stream-thread-metrics",
"",
Collections.singletonMap("thread-id", CLIENT_ID))
).metricValue()
)
);

thread.runOnce();

assertThat(
streamsMetrics.metrics().get(
new MetricName(
"commit-latency-max",
"stream-thread-metrics",
"",
Collections.singletonMap("thread-id", CLIENT_ID)
)
).metricValue(),
equalTo(10.0)
);
assertThat(
streamsMetrics.metrics().get(
new MetricName(
"commit-latency-avg",
"stream-thread-metrics",
"",
Collections.singletonMap("thread-id", CLIENT_ID)
)
).metricValue(),
equalTo(10.0)
);
}

@Test
public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEosDisabled() {
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
Expand Down Expand Up @@ -1228,26 +1336,20 @@ public void shouldReinitializeRevivedTasksInAnyState() {
@Override
public Processor<Object, Object> get() {
return new Processor<Object, Object>() {
private ProcessorContext context;

@Override
public void init(final ProcessorContext context) {
this.context = context;
}
public void init(final ProcessorContext context) {}

@Override
public void process(final Object key, final Object value) {
if (shouldThrow.get()) {
throw new TaskCorruptedException(singletonMap(task1, new HashSet<TopicPartition>(singleton(storeChangelogTopicPartition))));
throw new TaskCorruptedException(singletonMap(task1, new HashSet<>(singleton(storeChangelogTopicPartition))));
} else {
processed.set(true);
}
}

@Override
public void close() {

}
public void close() {}
};
}
}, "name");
Expand Down