mirror of https://github.com/apache/kafka.git
KAFKA-13036: Replace EasyMock and PowerMock with Mockito for RocksDBMetricsRecorderTest (#12459)
Changes: - Migrate to Mockito - Add more assertive checks using verify - Minor indentation fixes Reviewers: Dalibor Plavcic <dalibor.os@proton.me>, Bruno Cadonna <cadonna@apache.org>
This commit is contained in:
parent
fe99262fe2
commit
140faf9f2b
|
@ -23,11 +23,12 @@ import org.apache.kafka.streams.StreamsConfig;
|
|||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
|
||||
import org.apache.kafka.streams.state.internals.metrics.RocksDBMetrics.RocksDBMetricContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.powermock.core.classloader.annotations.PrepareForTest;
|
||||
import org.powermock.modules.junit4.PowerMockRunner;
|
||||
import org.mockito.MockedStatic;
|
||||
import org.mockito.junit.MockitoJUnitRunner;
|
||||
import org.rocksdb.Cache;
|
||||
import org.rocksdb.HistogramData;
|
||||
import org.rocksdb.HistogramType;
|
||||
|
@ -36,22 +37,22 @@ import org.rocksdb.Statistics;
|
|||
import org.rocksdb.StatsLevel;
|
||||
import org.rocksdb.TickerType;
|
||||
|
||||
import static org.easymock.EasyMock.anyObject;
|
||||
import static org.easymock.EasyMock.eq;
|
||||
import static org.easymock.EasyMock.expect;
|
||||
import static org.easymock.EasyMock.mock;
|
||||
import static org.easymock.EasyMock.niceMock;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
import static org.powermock.api.easymock.PowerMock.reset;
|
||||
import static org.powermock.api.easymock.PowerMock.createMock;
|
||||
import static org.powermock.api.easymock.PowerMock.mockStatic;
|
||||
import static org.powermock.api.easymock.PowerMock.replay;
|
||||
import static org.powermock.api.easymock.PowerMock.verify;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.ArgumentMatchers.isA;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.mockStatic;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.verifyNoInteractions;
|
||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@RunWith(PowerMockRunner.class)
|
||||
@PrepareForTest({RocksDBMetrics.class, Sensor.class})
|
||||
@RunWith(MockitoJUnitRunner.StrictStubs.class)
|
||||
public class RocksDBMetricsRecorderTest {
|
||||
private final static String METRICS_SCOPE = "metrics-scope";
|
||||
private final static TaskId TASK_ID1 = new TaskId(0, 0);
|
||||
|
@ -70,53 +71,92 @@ public class RocksDBMetricsRecorderTest {
|
|||
private final Statistics statisticsToAdd2 = mock(Statistics.class);
|
||||
private final Statistics statisticsToAdd3 = mock(Statistics.class);
|
||||
|
||||
private final Sensor bytesWrittenToDatabaseSensor = createMock(Sensor.class);
|
||||
private final Sensor bytesReadFromDatabaseSensor = createMock(Sensor.class);
|
||||
private final Sensor memtableBytesFlushedSensor = createMock(Sensor.class);
|
||||
private final Sensor memtableHitRatioSensor = createMock(Sensor.class);
|
||||
private final Sensor memtableAvgFlushTimeSensor = createMock(Sensor.class);
|
||||
private final Sensor memtableMinFlushTimeSensor = createMock(Sensor.class);
|
||||
private final Sensor memtableMaxFlushTimeSensor = createMock(Sensor.class);
|
||||
private final Sensor writeStallDurationSensor = createMock(Sensor.class);
|
||||
private final Sensor blockCacheDataHitRatioSensor = createMock(Sensor.class);
|
||||
private final Sensor blockCacheIndexHitRatioSensor = createMock(Sensor.class);
|
||||
private final Sensor blockCacheFilterHitRatioSensor = createMock(Sensor.class);
|
||||
private final Sensor bytesReadDuringCompactionSensor = createMock(Sensor.class);
|
||||
private final Sensor bytesWrittenDuringCompactionSensor = createMock(Sensor.class);
|
||||
private final Sensor compactionTimeAvgSensor = createMock(Sensor.class);
|
||||
private final Sensor compactionTimeMinSensor = createMock(Sensor.class);
|
||||
private final Sensor compactionTimeMaxSensor = createMock(Sensor.class);
|
||||
private final Sensor numberOfOpenFilesSensor = createMock(Sensor.class);
|
||||
private final Sensor numberOfFileErrorsSensor = createMock(Sensor.class);
|
||||
private final Sensor bytesWrittenToDatabaseSensor = mock(Sensor.class);
|
||||
private final Sensor bytesReadFromDatabaseSensor = mock(Sensor.class);
|
||||
private final Sensor memtableBytesFlushedSensor = mock(Sensor.class);
|
||||
private final Sensor memtableHitRatioSensor = mock(Sensor.class);
|
||||
private final Sensor memtableAvgFlushTimeSensor = mock(Sensor.class);
|
||||
private final Sensor memtableMinFlushTimeSensor = mock(Sensor.class);
|
||||
private final Sensor memtableMaxFlushTimeSensor = mock(Sensor.class);
|
||||
private final Sensor writeStallDurationSensor = mock(Sensor.class);
|
||||
private final Sensor blockCacheDataHitRatioSensor = mock(Sensor.class);
|
||||
private final Sensor blockCacheIndexHitRatioSensor = mock(Sensor.class);
|
||||
private final Sensor blockCacheFilterHitRatioSensor = mock(Sensor.class);
|
||||
private final Sensor bytesReadDuringCompactionSensor = mock(Sensor.class);
|
||||
private final Sensor bytesWrittenDuringCompactionSensor = mock(Sensor.class);
|
||||
private final Sensor compactionTimeAvgSensor = mock(Sensor.class);
|
||||
private final Sensor compactionTimeMinSensor = mock(Sensor.class);
|
||||
private final Sensor compactionTimeMaxSensor = mock(Sensor.class);
|
||||
private final Sensor numberOfOpenFilesSensor = mock(Sensor.class);
|
||||
private final Sensor numberOfFileErrorsSensor = mock(Sensor.class);
|
||||
|
||||
private final StreamsMetricsImpl streamsMetrics = niceMock(StreamsMetricsImpl.class);
|
||||
private final StreamsMetricsImpl streamsMetrics = mock(StreamsMetricsImpl.class);
|
||||
private final RocksDBMetricsRecordingTrigger recordingTrigger = mock(RocksDBMetricsRecordingTrigger.class);
|
||||
|
||||
private final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME);
|
||||
private final RocksDBMetricContext metricsContext = new RocksDBMetricContext(TASK_ID1.toString(), METRICS_SCOPE, STORE_NAME);
|
||||
|
||||
private MockedStatic<RocksDBMetrics> dbMetrics;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
setUpMetricsStubMock();
|
||||
expect(streamsMetrics.rocksDBMetricsRecordingTrigger()).andStubReturn(recordingTrigger);
|
||||
replay(streamsMetrics);
|
||||
setUpMetricsMock();
|
||||
when(streamsMetrics.rocksDBMetricsRecordingTrigger()).thenReturn(recordingTrigger);
|
||||
recorder.init(streamsMetrics, TASK_ID1);
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanUpMocks() {
|
||||
dbMetrics.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldInitMetricsRecorder() {
|
||||
setUpMetricsMock();
|
||||
|
||||
recorder.init(streamsMetrics, TASK_ID1);
|
||||
|
||||
verify(RocksDBMetrics.class);
|
||||
dbMetrics.verify(() -> RocksDBMetrics.bytesWrittenToDatabaseSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.bytesReadFromDatabaseSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.memtableBytesFlushedSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.memtableHitRatioSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.memtableAvgFlushTimeSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.memtableMinFlushTimeSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.memtableMaxFlushTimeSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.writeStallDurationSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.blockCacheDataHitRatioSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.blockCacheIndexHitRatioSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.blockCacheFilterHitRatioSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.bytesWrittenDuringCompactionSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.bytesReadDuringCompactionSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.compactionTimeAvgSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.compactionTimeMinSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.compactionTimeMaxSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.numberOfOpenFilesSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.numberOfFileErrorsSensor(any(), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addNumImmutableMemTableMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addCurSizeActiveMemTable(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addCurSizeAllMemTables(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addSizeAllMemTables(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addNumEntriesActiveMemTableMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addNumEntriesImmMemTablesMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addNumDeletesActiveMemTableMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addNumDeletesImmMemTablesMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addMemTableFlushPending(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addNumRunningFlushesMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addCompactionPendingMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addNumRunningCompactionsMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addEstimatePendingCompactionBytesMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addTotalSstFilesSizeMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addLiveSstFilesSizeMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addNumLiveVersionMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addBlockCacheCapacityMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addBlockCacheUsageMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addBlockCachePinnedUsageMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addEstimateNumKeysMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addEstimateTableReadersMemMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
dbMetrics.verify(() -> RocksDBMetrics.addBackgroundErrorsMetric(eq(streamsMetrics), eq(metricsContext), any()));
|
||||
assertThat(recorder.taskId(), is(TASK_ID1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldThrowIfMetricRecorderIsReInitialisedWithDifferentTask() {
|
||||
setUpMetricsStubMock();
|
||||
recorder.init(streamsMetrics, TASK_ID1);
|
||||
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> recorder.init(streamsMetrics, TASK_ID2)
|
||||
|
@ -125,9 +165,6 @@ public class RocksDBMetricsRecorderTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfMetricRecorderIsReInitialisedWithDifferentStreamsMetrics() {
|
||||
setUpMetricsStubMock();
|
||||
recorder.init(streamsMetrics, TASK_ID1);
|
||||
|
||||
assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> recorder.init(
|
||||
|
@ -139,21 +176,14 @@ public class RocksDBMetricsRecorderTest {
|
|||
|
||||
@Test
|
||||
public void shouldSetStatsLevelToExceptDetailedTimersWhenValueProvidersWithStatisticsAreAdded() {
|
||||
statisticsToAdd1.setStatsLevel(StatsLevel.EXCEPT_DETAILED_TIMERS);
|
||||
replay(statisticsToAdd1);
|
||||
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
|
||||
verify(statisticsToAdd1);
|
||||
verify(statisticsToAdd1).setStatsLevel(StatsLevel.EXCEPT_DETAILED_TIMERS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotSetStatsLevelToExceptDetailedTimersWhenValueProvidersWithoutStatisticsAreAdded() {
|
||||
replay(statisticsToAdd1);
|
||||
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, null);
|
||||
|
||||
verify(statisticsToAdd1);
|
||||
verifyNoInteractions(statisticsToAdd1);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -292,79 +322,63 @@ public class RocksDBMetricsRecorderTest {
|
|||
|
||||
@Test
|
||||
public void shouldAddItselfToRecordingTriggerWhenFirstValueProvidersAreAddedToNewlyCreatedRecorder() {
|
||||
recordingTrigger.addMetricsRecorder(recorder);
|
||||
replay(recordingTrigger);
|
||||
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
|
||||
verify(recordingTrigger);
|
||||
verify(recordingTrigger).addMetricsRecorder(recorder);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldAddItselfToRecordingTriggerWhenFirstValueProvidersAreAddedAfterLastValueProvidersWereRemoved() {
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
recorder.removeValueProviders(SEGMENT_STORE_NAME_1);
|
||||
reset(recordingTrigger);
|
||||
recordingTrigger.addMetricsRecorder(recorder);
|
||||
replay(recordingTrigger);
|
||||
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_2, dbToAdd2, cacheToAdd2, statisticsToAdd2);
|
||||
|
||||
verify(recordingTrigger);
|
||||
verify(recordingTrigger, times(2)).addMetricsRecorder(recorder);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAddItselfToRecordingTriggerWhenNotEmpty2() {
|
||||
public void shouldNotAddItselfToRecordingTriggerWhenNotEmpty() {
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
reset(recordingTrigger);
|
||||
replay(recordingTrigger);
|
||||
|
||||
verify(recordingTrigger).addMetricsRecorder(recorder);
|
||||
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_2, dbToAdd2, cacheToAdd2, statisticsToAdd2);
|
||||
|
||||
verify(recordingTrigger);
|
||||
verifyNoMoreInteractions(recordingTrigger);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldCloseStatisticsWhenValueProvidersAreRemoved() {
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
reset(statisticsToAdd1);
|
||||
statisticsToAdd1.close();
|
||||
replay(statisticsToAdd1);
|
||||
|
||||
recorder.removeValueProviders(SEGMENT_STORE_NAME_1);
|
||||
|
||||
verify(statisticsToAdd1);
|
||||
verify(statisticsToAdd1).close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotCloseStatisticsWhenValueProvidersWithoutStatisticsAreRemoved() {
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, null);
|
||||
reset(statisticsToAdd1);
|
||||
replay(statisticsToAdd1);
|
||||
|
||||
recorder.removeValueProviders(SEGMENT_STORE_NAME_1);
|
||||
|
||||
verify(statisticsToAdd1);
|
||||
verify(statisticsToAdd1, never()).close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldRemoveItselfFromRecordingTriggerWhenLastValueProvidersAreRemoved() {
|
||||
public void shouldNotRemoveItselfFromRecordingTriggerWhenAtLeastOneValueProviderIsPresent() {
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_2, dbToAdd2, cacheToAdd2, statisticsToAdd2);
|
||||
reset(recordingTrigger);
|
||||
replay(recordingTrigger);
|
||||
|
||||
recorder.removeValueProviders(SEGMENT_STORE_NAME_1);
|
||||
|
||||
verify(recordingTrigger);
|
||||
verify(recordingTrigger, never()).removeMetricsRecorder(recorder);
|
||||
}
|
||||
|
||||
reset(recordingTrigger);
|
||||
recordingTrigger.removeMetricsRecorder(recorder);
|
||||
replay(recordingTrigger);
|
||||
@Test
|
||||
public void shouldRemoveItselfFromRecordingTriggerWhenAllValueProvidersAreRemoved() {
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_2, dbToAdd2, cacheToAdd2, statisticsToAdd2);
|
||||
|
||||
recorder.removeValueProviders(SEGMENT_STORE_NAME_1);
|
||||
recorder.removeValueProviders(SEGMENT_STORE_NAME_2);
|
||||
|
||||
verify(recordingTrigger);
|
||||
verify(recordingTrigger).removeMetricsRecorder(recorder);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -381,157 +395,115 @@ public class RocksDBMetricsRecorderTest {
|
|||
public void shouldRecordStatisticsBasedMetrics() {
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_2, dbToAdd2, cacheToAdd2, statisticsToAdd2);
|
||||
reset(statisticsToAdd1);
|
||||
reset(statisticsToAdd2);
|
||||
final long now = 0L;
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BYTES_WRITTEN)).andReturn(1L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BYTES_WRITTEN)).andReturn(2L);
|
||||
bytesWrittenToDatabaseSensor.record(1 + 2, 0L);
|
||||
replay(bytesWrittenToDatabaseSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.BYTES_WRITTEN)).thenReturn(1L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.BYTES_WRITTEN)).thenReturn(2L);
|
||||
final double expectedBytesWrittenToDatabaseSensor = 1 + 2;
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BYTES_READ)).andReturn(2L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BYTES_READ)).andReturn(3L);
|
||||
bytesReadFromDatabaseSensor.record(2 + 3, 0L);
|
||||
replay(bytesReadFromDatabaseSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.BYTES_READ)).thenReturn(2L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.BYTES_READ)).thenReturn(3L);
|
||||
final double expectedBytesReadFromDatabaseSensor = 2 + 3;
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.FLUSH_WRITE_BYTES)).andReturn(3L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.FLUSH_WRITE_BYTES)).andReturn(4L);
|
||||
memtableBytesFlushedSensor.record(3 + 4, 0L);
|
||||
replay(memtableBytesFlushedSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.FLUSH_WRITE_BYTES)).thenReturn(3L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.FLUSH_WRITE_BYTES)).thenReturn(4L);
|
||||
final double expectedMemtableBytesFlushedSensor = 3 + 4;
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.MEMTABLE_HIT)).andReturn(1L);
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.MEMTABLE_MISS)).andReturn(2L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.MEMTABLE_HIT)).andReturn(3L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.MEMTABLE_MISS)).andReturn(4L);
|
||||
memtableHitRatioSensor.record((double) 4 / (4 + 6), 0L);
|
||||
replay(memtableHitRatioSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.MEMTABLE_HIT)).thenReturn(1L);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.MEMTABLE_MISS)).thenReturn(2L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.MEMTABLE_HIT)).thenReturn(3L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.MEMTABLE_MISS)).thenReturn(4L);
|
||||
final double expectedMemtableHitRatioSensorRecord = (double) 4 / (4 + 6);
|
||||
|
||||
final HistogramData memtableFlushTimeData1 = new HistogramData(0.0, 0.0, 0.0, 0.0, 0.0, 16.0, 2L, 10L, 3.0);
|
||||
final HistogramData memtableFlushTimeData2 = new HistogramData(0.0, 0.0, 0.0, 0.0, 0.0, 20.0, 4L, 8L, 10.0);
|
||||
expect(statisticsToAdd1.getHistogramData(HistogramType.FLUSH_TIME)).andReturn(memtableFlushTimeData1);
|
||||
expect(statisticsToAdd2.getHistogramData(HistogramType.FLUSH_TIME)).andReturn(memtableFlushTimeData2);
|
||||
memtableAvgFlushTimeSensor.record((double) (10 + 8) / (2 + 4), 0L);
|
||||
replay(memtableAvgFlushTimeSensor);
|
||||
memtableMinFlushTimeSensor.record(3.0, 0L);
|
||||
replay(memtableMinFlushTimeSensor);
|
||||
memtableMaxFlushTimeSensor.record(20.0, 0L);
|
||||
replay(memtableMaxFlushTimeSensor);
|
||||
when(statisticsToAdd1.getHistogramData(HistogramType.FLUSH_TIME)).thenReturn(memtableFlushTimeData1);
|
||||
when(statisticsToAdd2.getHistogramData(HistogramType.FLUSH_TIME)).thenReturn(memtableFlushTimeData2);
|
||||
final double expectedMemtableAvgFlushTimeSensor = (double) (10 + 8) / (2 + 4);
|
||||
final double expectedMemtableMinFlushTimeSensor = 3.0d;
|
||||
final double expectedMemtableMaxFlushTimeSensor = 20.0d;
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.STALL_MICROS)).andReturn(4L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.STALL_MICROS)).andReturn(5L);
|
||||
writeStallDurationSensor.record(4 + 5, 0L);
|
||||
replay(writeStallDurationSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.STALL_MICROS)).thenReturn(4L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.STALL_MICROS)).thenReturn(5L);
|
||||
final double expectedWriteStallDurationSensor = 4 + 5;
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_HIT)).andReturn(5L);
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_MISS)).andReturn(4L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_HIT)).andReturn(3L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_MISS)).andReturn(2L);
|
||||
blockCacheDataHitRatioSensor.record((double) 8 / (8 + 6), 0L);
|
||||
replay(blockCacheDataHitRatioSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_HIT)).thenReturn(5L);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_MISS)).thenReturn(4L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_HIT)).thenReturn(3L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_DATA_MISS)).thenReturn(2L);
|
||||
final double expectedBlockCacheDataHitRatioSensor = (double) 8 / (8 + 6);
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_HIT)).andReturn(4L);
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_MISS)).andReturn(2L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_HIT)).andReturn(2L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_MISS)).andReturn(4L);
|
||||
blockCacheIndexHitRatioSensor.record((double) 6 / (6 + 6), 0L);
|
||||
replay(blockCacheIndexHitRatioSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_HIT)).thenReturn(4L);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_MISS)).thenReturn(2L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_HIT)).thenReturn(2L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_INDEX_MISS)).thenReturn(4L);
|
||||
final double expectedBlockCacheIndexHitRatioSensor = (double) 6 / (6 + 6);
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_HIT)).andReturn(2L);
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_MISS)).andReturn(4L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_HIT)).andReturn(3L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_MISS)).andReturn(5L);
|
||||
blockCacheFilterHitRatioSensor.record((double) 5 / (5 + 9), 0L);
|
||||
replay(blockCacheFilterHitRatioSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_HIT)).thenReturn(2L);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_MISS)).thenReturn(4L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_HIT)).thenReturn(3L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.BLOCK_CACHE_FILTER_MISS)).thenReturn(5L);
|
||||
final double expectedBlockCacheFilterHitRatioSensor = (double) 5 / (5 + 9);
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.COMPACT_WRITE_BYTES)).andReturn(2L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.COMPACT_WRITE_BYTES)).andReturn(4L);
|
||||
bytesWrittenDuringCompactionSensor.record(2 + 4, 0L);
|
||||
replay(bytesWrittenDuringCompactionSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.COMPACT_WRITE_BYTES)).thenReturn(2L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.COMPACT_WRITE_BYTES)).thenReturn(4L);
|
||||
final double expectedBytesWrittenDuringCompactionSensor = 2 + 4;
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.COMPACT_READ_BYTES)).andReturn(5L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.COMPACT_READ_BYTES)).andReturn(6L);
|
||||
bytesReadDuringCompactionSensor.record(5 + 6, 0L);
|
||||
replay(bytesReadDuringCompactionSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.COMPACT_READ_BYTES)).thenReturn(5L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.COMPACT_READ_BYTES)).thenReturn(6L);
|
||||
final double expectedBytesReadDuringCompactionSensor = 5 + 6;
|
||||
|
||||
final HistogramData compactionTimeData1 = new HistogramData(0.0, 0.0, 0.0, 0.0, 0.0, 16.0, 2L, 8L, 6.0);
|
||||
final HistogramData compactionTimeData2 = new HistogramData(0.0, 0.0, 0.0, 0.0, 0.0, 24.0, 2L, 8L, 4.0);
|
||||
expect(statisticsToAdd1.getHistogramData(HistogramType.COMPACTION_TIME)).andReturn(compactionTimeData1);
|
||||
expect(statisticsToAdd2.getHistogramData(HistogramType.COMPACTION_TIME)).andReturn(compactionTimeData2);
|
||||
compactionTimeAvgSensor.record((double) (8 + 8) / (2 + 2), 0L);
|
||||
replay(compactionTimeAvgSensor);
|
||||
compactionTimeMinSensor.record(4.0, 0L);
|
||||
replay(compactionTimeMinSensor);
|
||||
compactionTimeMaxSensor.record(24.0, 0L);
|
||||
replay(compactionTimeMaxSensor);
|
||||
when(statisticsToAdd1.getHistogramData(HistogramType.COMPACTION_TIME)).thenReturn(compactionTimeData1);
|
||||
when(statisticsToAdd2.getHistogramData(HistogramType.COMPACTION_TIME)).thenReturn(compactionTimeData2);
|
||||
final double expectedCompactionTimeAvgSensor = (double) (8 + 8) / (2 + 2);
|
||||
final double expectedCompactionTimeMinSensor = 4.0;
|
||||
final double expectedCompactionTimeMaxSensor = 24.0;
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_OPENS)).andReturn(5L);
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_CLOSES)).andReturn(3L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_OPENS)).andReturn(7L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_CLOSES)).andReturn(4L);
|
||||
numberOfOpenFilesSensor.record((5 + 7) - (3 + 4), 0L);
|
||||
replay(numberOfOpenFilesSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_OPENS)).thenReturn(5L);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_CLOSES)).thenReturn(3L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_OPENS)).thenReturn(7L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_CLOSES)).thenReturn(4L);
|
||||
final double expectedNumberOfOpenFilesSensor = (5 + 7) - (3 + 4);
|
||||
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_ERRORS)).andReturn(34L);
|
||||
expect(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_ERRORS)).andReturn(11L);
|
||||
numberOfFileErrorsSensor.record(11 + 34, 0L);
|
||||
replay(numberOfFileErrorsSensor);
|
||||
when(statisticsToAdd1.getAndResetTickerCount(TickerType.NO_FILE_ERRORS)).thenReturn(34L);
|
||||
when(statisticsToAdd2.getAndResetTickerCount(TickerType.NO_FILE_ERRORS)).thenReturn(11L);
|
||||
final double expectedNumberOfFileErrorsSensor = 11 + 34;
|
||||
|
||||
replay(statisticsToAdd1);
|
||||
replay(statisticsToAdd2);
|
||||
recorder.record(now);
|
||||
|
||||
recorder.record(0L);
|
||||
|
||||
verify(statisticsToAdd1);
|
||||
verify(statisticsToAdd2);
|
||||
verify(
|
||||
bytesWrittenToDatabaseSensor,
|
||||
bytesReadFromDatabaseSensor,
|
||||
memtableBytesFlushedSensor,
|
||||
memtableHitRatioSensor,
|
||||
memtableAvgFlushTimeSensor,
|
||||
memtableMinFlushTimeSensor,
|
||||
memtableMaxFlushTimeSensor,
|
||||
writeStallDurationSensor,
|
||||
blockCacheDataHitRatioSensor,
|
||||
blockCacheIndexHitRatioSensor,
|
||||
blockCacheFilterHitRatioSensor,
|
||||
bytesWrittenDuringCompactionSensor,
|
||||
bytesReadDuringCompactionSensor,
|
||||
compactionTimeAvgSensor,
|
||||
compactionTimeMinSensor,
|
||||
compactionTimeMaxSensor,
|
||||
numberOfOpenFilesSensor,
|
||||
numberOfFileErrorsSensor
|
||||
);
|
||||
verify(statisticsToAdd1, times(17)).getAndResetTickerCount(isA(TickerType.class));
|
||||
verify(statisticsToAdd2, times(17)).getAndResetTickerCount(isA(TickerType.class));
|
||||
verify(statisticsToAdd1, times(2)).getHistogramData(isA(HistogramType.class));
|
||||
verify(statisticsToAdd2, times(2)).getHistogramData(isA(HistogramType.class));
|
||||
verify(bytesWrittenToDatabaseSensor).record(expectedBytesWrittenToDatabaseSensor, now);
|
||||
verify(bytesReadFromDatabaseSensor).record(expectedBytesReadFromDatabaseSensor, now);
|
||||
verify(memtableBytesFlushedSensor).record(expectedMemtableBytesFlushedSensor, now);
|
||||
verify(memtableHitRatioSensor).record(expectedMemtableHitRatioSensorRecord, now);
|
||||
verify(memtableAvgFlushTimeSensor).record(expectedMemtableAvgFlushTimeSensor, now);
|
||||
verify(memtableMinFlushTimeSensor).record(expectedMemtableMinFlushTimeSensor, now);
|
||||
verify(memtableMaxFlushTimeSensor).record(expectedMemtableMaxFlushTimeSensor, now);
|
||||
verify(writeStallDurationSensor).record(expectedWriteStallDurationSensor, now);
|
||||
verify(blockCacheDataHitRatioSensor).record(expectedBlockCacheDataHitRatioSensor, now);
|
||||
verify(blockCacheIndexHitRatioSensor).record(expectedBlockCacheIndexHitRatioSensor, now);
|
||||
verify(blockCacheFilterHitRatioSensor).record(expectedBlockCacheFilterHitRatioSensor, now);
|
||||
verify(bytesWrittenDuringCompactionSensor).record(expectedBytesWrittenDuringCompactionSensor, now);
|
||||
verify(bytesReadDuringCompactionSensor).record(expectedBytesReadDuringCompactionSensor, now);
|
||||
verify(compactionTimeAvgSensor).record(expectedCompactionTimeAvgSensor, now);
|
||||
verify(compactionTimeMinSensor).record(expectedCompactionTimeMinSensor, now);
|
||||
verify(compactionTimeMaxSensor).record(expectedCompactionTimeMaxSensor, now);
|
||||
verify(numberOfOpenFilesSensor).record(expectedNumberOfOpenFilesSensor, now);
|
||||
verify(numberOfFileErrorsSensor).record(expectedNumberOfFileErrorsSensor, now);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotRecordStatisticsBasedMetricsIfStatisticsIsNull() {
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, null);
|
||||
replay(
|
||||
bytesWrittenToDatabaseSensor,
|
||||
bytesReadFromDatabaseSensor,
|
||||
memtableBytesFlushedSensor,
|
||||
memtableHitRatioSensor,
|
||||
memtableAvgFlushTimeSensor,
|
||||
memtableMinFlushTimeSensor,
|
||||
memtableMaxFlushTimeSensor,
|
||||
writeStallDurationSensor,
|
||||
blockCacheDataHitRatioSensor,
|
||||
blockCacheIndexHitRatioSensor,
|
||||
blockCacheFilterHitRatioSensor,
|
||||
bytesWrittenDuringCompactionSensor,
|
||||
bytesReadDuringCompactionSensor,
|
||||
compactionTimeAvgSensor,
|
||||
compactionTimeMinSensor,
|
||||
compactionTimeMaxSensor,
|
||||
numberOfOpenFilesSensor,
|
||||
numberOfFileErrorsSensor
|
||||
);
|
||||
|
||||
recorder.record(0L);
|
||||
|
||||
verify(
|
||||
verifyNoInteractions(
|
||||
bytesWrittenToDatabaseSensor,
|
||||
bytesReadFromDatabaseSensor,
|
||||
memtableBytesFlushedSensor,
|
||||
|
@ -555,173 +527,68 @@ public class RocksDBMetricsRecorderTest {
|
|||
|
||||
@Test
|
||||
public void shouldCorrectlyHandleHitRatioRecordingsWithZeroHitsAndMisses() {
|
||||
reset(statisticsToAdd1);
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
expect(statisticsToAdd1.getHistogramData(anyObject())).andStubReturn(new HistogramData(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0L, 0L, 0.0));
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(anyObject())).andStubReturn(0L);
|
||||
replay(statisticsToAdd1);
|
||||
memtableHitRatioSensor.record(0, 0L);
|
||||
blockCacheDataHitRatioSensor.record(0, 0L);
|
||||
blockCacheIndexHitRatioSensor.record(0, 0L);
|
||||
blockCacheFilterHitRatioSensor.record(0, 0L);
|
||||
replay(memtableHitRatioSensor);
|
||||
replay(blockCacheDataHitRatioSensor);
|
||||
replay(blockCacheIndexHitRatioSensor);
|
||||
replay(blockCacheFilterHitRatioSensor);
|
||||
when(statisticsToAdd1.getHistogramData(any())).thenReturn(new HistogramData(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0L, 0L, 0.0));
|
||||
when(statisticsToAdd1.getAndResetTickerCount(any())).thenReturn(0L);
|
||||
|
||||
recorder.record(0L);
|
||||
|
||||
verify(memtableHitRatioSensor);
|
||||
verify(blockCacheDataHitRatioSensor);
|
||||
verify(blockCacheIndexHitRatioSensor);
|
||||
verify(blockCacheFilterHitRatioSensor);
|
||||
verify(memtableHitRatioSensor).record(0d, 0L);
|
||||
verify(blockCacheDataHitRatioSensor).record(0d, 0L);
|
||||
verify(blockCacheIndexHitRatioSensor).record(0d, 0L);
|
||||
verify(blockCacheFilterHitRatioSensor).record(0d, 0L);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldCorrectlyHandleAvgRecordingsWithZeroSumAndCount() {
|
||||
reset(statisticsToAdd1);
|
||||
recorder.addValueProviders(SEGMENT_STORE_NAME_1, dbToAdd1, cacheToAdd1, statisticsToAdd1);
|
||||
expect(statisticsToAdd1.getHistogramData(anyObject())).andStubReturn(new HistogramData(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0L, 0L, 0.0));
|
||||
expect(statisticsToAdd1.getAndResetTickerCount(anyObject())).andStubReturn(0L);
|
||||
replay(statisticsToAdd1);
|
||||
memtableAvgFlushTimeSensor.record(0, 0L);
|
||||
compactionTimeAvgSensor.record(0, 0L);
|
||||
replay(memtableAvgFlushTimeSensor);
|
||||
replay(compactionTimeAvgSensor);
|
||||
final long now = 0L;
|
||||
when(statisticsToAdd1.getHistogramData(any())).thenReturn(new HistogramData(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0L, 0L, 0.0));
|
||||
when(statisticsToAdd1.getAndResetTickerCount(any())).thenReturn(0L);
|
||||
|
||||
recorder.record(0L);
|
||||
recorder.record(now);
|
||||
|
||||
verify(memtableAvgFlushTimeSensor);
|
||||
verify(compactionTimeAvgSensor);
|
||||
verify(compactionTimeAvgSensor).record(0d, now);
|
||||
verify(memtableAvgFlushTimeSensor).record(0d, now);
|
||||
}
|
||||
|
||||
private void setUpMetricsMock() {
|
||||
mockStatic(RocksDBMetrics.class);
|
||||
final RocksDBMetricContext metricsContext =
|
||||
new RocksDBMetricContext(TASK_ID1.toString(), METRICS_SCOPE, STORE_NAME);
|
||||
expect(RocksDBMetrics.bytesWrittenToDatabaseSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(bytesWrittenToDatabaseSensor);
|
||||
expect(RocksDBMetrics.bytesReadFromDatabaseSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(bytesReadFromDatabaseSensor);
|
||||
expect(RocksDBMetrics.memtableBytesFlushedSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(memtableBytesFlushedSensor);
|
||||
expect(RocksDBMetrics.memtableHitRatioSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(memtableHitRatioSensor);
|
||||
expect(RocksDBMetrics.memtableAvgFlushTimeSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(memtableAvgFlushTimeSensor);
|
||||
expect(RocksDBMetrics.memtableMinFlushTimeSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(memtableMinFlushTimeSensor);
|
||||
expect(RocksDBMetrics.memtableMaxFlushTimeSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(memtableMaxFlushTimeSensor);
|
||||
expect(RocksDBMetrics.writeStallDurationSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(writeStallDurationSensor);
|
||||
expect(RocksDBMetrics.blockCacheDataHitRatioSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(blockCacheDataHitRatioSensor);
|
||||
expect(RocksDBMetrics.blockCacheIndexHitRatioSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(blockCacheIndexHitRatioSensor);
|
||||
expect(RocksDBMetrics.blockCacheFilterHitRatioSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(blockCacheFilterHitRatioSensor);
|
||||
expect(RocksDBMetrics.bytesWrittenDuringCompactionSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(bytesWrittenDuringCompactionSensor);
|
||||
expect(RocksDBMetrics.bytesReadDuringCompactionSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(bytesReadDuringCompactionSensor);
|
||||
expect(RocksDBMetrics.compactionTimeAvgSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(compactionTimeAvgSensor);
|
||||
expect(RocksDBMetrics.compactionTimeMinSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(compactionTimeMinSensor);
|
||||
expect(RocksDBMetrics.compactionTimeMaxSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(compactionTimeMaxSensor);
|
||||
expect(RocksDBMetrics.numberOfOpenFilesSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(numberOfOpenFilesSensor);
|
||||
expect(RocksDBMetrics.numberOfFileErrorsSensor(eq(streamsMetrics), eq(metricsContext)))
|
||||
.andReturn(numberOfFileErrorsSensor);
|
||||
RocksDBMetrics.addNumImmutableMemTableMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addCurSizeActiveMemTable(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addCurSizeAllMemTables(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addSizeAllMemTables(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumEntriesActiveMemTableMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumEntriesImmMemTablesMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumDeletesActiveMemTableMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumDeletesImmMemTablesMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addMemTableFlushPending(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumRunningFlushesMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addCompactionPendingMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumRunningCompactionsMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addEstimatePendingCompactionBytesMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addTotalSstFilesSizeMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addLiveSstFilesSizeMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumLiveVersionMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addBlockCacheCapacityMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addBlockCacheUsageMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addBlockCachePinnedUsageMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addEstimateNumKeysMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addEstimateTableReadersMemMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addBackgroundErrorsMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
replay(RocksDBMetrics.class);
|
||||
}
|
||||
|
||||
private void setUpMetricsStubMock() {
|
||||
mockStatic(RocksDBMetrics.class);
|
||||
final RocksDBMetricContext metricsContext =
|
||||
new RocksDBMetricContext(TASK_ID1.toString(), METRICS_SCOPE, STORE_NAME);
|
||||
expect(RocksDBMetrics.bytesWrittenToDatabaseSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(bytesWrittenToDatabaseSensor);
|
||||
expect(RocksDBMetrics.bytesReadFromDatabaseSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(bytesReadFromDatabaseSensor);
|
||||
expect(RocksDBMetrics.memtableBytesFlushedSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(memtableBytesFlushedSensor);
|
||||
expect(RocksDBMetrics.memtableHitRatioSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(memtableHitRatioSensor);
|
||||
expect(RocksDBMetrics.memtableAvgFlushTimeSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(memtableAvgFlushTimeSensor);
|
||||
expect(RocksDBMetrics.memtableMinFlushTimeSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(memtableMinFlushTimeSensor);
|
||||
expect(RocksDBMetrics.memtableMaxFlushTimeSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(memtableMaxFlushTimeSensor);
|
||||
expect(RocksDBMetrics.writeStallDurationSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(writeStallDurationSensor);
|
||||
expect(RocksDBMetrics.blockCacheDataHitRatioSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(blockCacheDataHitRatioSensor);
|
||||
expect(RocksDBMetrics.blockCacheIndexHitRatioSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(blockCacheIndexHitRatioSensor);
|
||||
expect(RocksDBMetrics.blockCacheFilterHitRatioSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(blockCacheFilterHitRatioSensor);
|
||||
expect(RocksDBMetrics.bytesWrittenDuringCompactionSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(bytesWrittenDuringCompactionSensor);
|
||||
expect(RocksDBMetrics.bytesReadDuringCompactionSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(bytesReadDuringCompactionSensor);
|
||||
expect(RocksDBMetrics.compactionTimeAvgSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(compactionTimeAvgSensor);
|
||||
expect(RocksDBMetrics.compactionTimeMinSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(compactionTimeMinSensor);
|
||||
expect(RocksDBMetrics.compactionTimeMaxSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(compactionTimeMaxSensor);
|
||||
expect(RocksDBMetrics.numberOfOpenFilesSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(numberOfOpenFilesSensor);
|
||||
expect(RocksDBMetrics.numberOfFileErrorsSensor(streamsMetrics, metricsContext))
|
||||
.andStubReturn(numberOfFileErrorsSensor);
|
||||
RocksDBMetrics.addNumImmutableMemTableMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addCurSizeActiveMemTable(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addCurSizeAllMemTables(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addSizeAllMemTables(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumEntriesActiveMemTableMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumEntriesImmMemTablesMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumDeletesActiveMemTableMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumDeletesImmMemTablesMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addMemTableFlushPending(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumRunningFlushesMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addCompactionPendingMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumRunningCompactionsMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addEstimatePendingCompactionBytesMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addTotalSstFilesSizeMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addLiveSstFilesSizeMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addNumLiveVersionMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addBlockCacheCapacityMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addBlockCacheUsageMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addBlockCachePinnedUsageMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addEstimateNumKeysMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addEstimateTableReadersMemMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
RocksDBMetrics.addBackgroundErrorsMetric(eq(streamsMetrics), eq(metricsContext), anyObject());
|
||||
replay(RocksDBMetrics.class);
|
||||
dbMetrics = mockStatic(RocksDBMetrics.class);
|
||||
dbMetrics.when(() -> RocksDBMetrics.bytesWrittenToDatabaseSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(bytesWrittenToDatabaseSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.bytesReadFromDatabaseSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(bytesReadFromDatabaseSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.memtableBytesFlushedSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(memtableBytesFlushedSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.memtableHitRatioSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(memtableHitRatioSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.memtableAvgFlushTimeSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(memtableAvgFlushTimeSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.memtableMinFlushTimeSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(memtableMinFlushTimeSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.memtableMaxFlushTimeSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(memtableMaxFlushTimeSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.writeStallDurationSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(writeStallDurationSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.blockCacheDataHitRatioSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(blockCacheDataHitRatioSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.blockCacheIndexHitRatioSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(blockCacheIndexHitRatioSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.blockCacheFilterHitRatioSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(blockCacheFilterHitRatioSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.bytesWrittenDuringCompactionSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(bytesWrittenDuringCompactionSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.bytesReadDuringCompactionSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(bytesReadDuringCompactionSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.compactionTimeAvgSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(compactionTimeAvgSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.compactionTimeMinSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(compactionTimeMinSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.compactionTimeMaxSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(compactionTimeMaxSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.numberOfOpenFilesSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(numberOfOpenFilesSensor);
|
||||
dbMetrics.when(() -> RocksDBMetrics.numberOfFileErrorsSensor(streamsMetrics, metricsContext))
|
||||
.thenReturn(numberOfFileErrorsSensor);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue