mirror of https://github.com/apache/druid.git
Unregisters `RealtimeMetricsMonitor`, `TaskRealtimeMetricsMonitor` on Indexers after task completion (#12743)
Few indexing tasks register RealtimeMetricsMonitor or TaskRealtimeMetricsMonitor with the process’s MonitorScheduler when they start. These monitors never unregister themselves (they always return true, they'd need to return false to unregister). Each of these monitors emits a set of metrics once every druid.monitoring.emissionPeriod. As a result, after executing several tasks for a while, Indexer emits metrics of these tasks even after they're long gone. Proposed Solution Since one should be able to obtain the last round of ingestion metrics after the task unregisters the monitor, introducing lastRoundMetricsToBePushed variable to keep track of the same and overriding the AbstractMonitor.monitor method in RealtimeMetricsMonitor, TaskRealtimeMetricsMonitor to implement the new logic.
This commit is contained in:
parent
75045970cd
commit
cc1ff56ca5
|
@ -41,6 +41,11 @@ public abstract class AbstractMonitor implements Monitor
|
|||
started = false;
|
||||
}
|
||||
|
||||
public boolean isStarted()
|
||||
{
|
||||
return started;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean monitor(ServiceEmitter emitter)
|
||||
{
|
||||
|
|
|
@ -50,6 +50,8 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor
|
|||
private FireDepartmentMetrics previousFireDepartmentMetrics;
|
||||
private RowIngestionMetersTotals previousRowIngestionMetersTotals;
|
||||
|
||||
private volatile boolean lastRoundMetricsToBePushed = false;
|
||||
|
||||
public TaskRealtimeMetricsMonitor(
|
||||
FireDepartment fireDepartment,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
|
@ -63,6 +65,27 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor
|
|||
previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start()
|
||||
{
|
||||
super.start();
|
||||
lastRoundMetricsToBePushed = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean monitor(ServiceEmitter emitter)
|
||||
{
|
||||
if (isStarted()) {
|
||||
return doMonitor(emitter);
|
||||
} else if (lastRoundMetricsToBePushed) {
|
||||
// Run one more time even if the monitor was removed, in case there's some extra data to flush
|
||||
lastRoundMetricsToBePushed = false;
|
||||
return doMonitor(emitter);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doMonitor(ServiceEmitter emitter)
|
||||
{
|
||||
|
|
|
@ -850,14 +850,12 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
|||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null);
|
||||
FireDepartmentMetrics buildSegmentsFireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
|
||||
if (toolbox.getMonitorScheduler() != null) {
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(
|
||||
this,
|
||||
fireDepartmentForMetrics,
|
||||
buildSegmentsMeters
|
||||
);
|
||||
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
|
||||
}
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(
|
||||
this,
|
||||
fireDepartmentForMetrics,
|
||||
buildSegmentsMeters
|
||||
);
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
final PartitionsSpec partitionsSpec = partitionAnalysis.getPartitionsSpec();
|
||||
final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
||||
|
@ -1046,6 +1044,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
|||
} else {
|
||||
appenderator.close();
|
||||
}
|
||||
toolbox.removeMonitor(metricsMonitor);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -166,12 +166,11 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
buildSegmentsMeters = toolbox.getRowIngestionMetersFactory().createRowIngestionMeters();
|
||||
|
||||
toolbox.addMonitor(
|
||||
new RealtimeMetricsMonitor(
|
||||
Collections.singletonList(fireDepartmentForMetrics),
|
||||
Collections.singletonMap(DruidMetrics.TASK_ID, new String[]{getId()})
|
||||
)
|
||||
RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor(
|
||||
Collections.singletonList(fireDepartmentForMetrics),
|
||||
Collections.singletonMap(DruidMetrics.TASK_ID, new String[]{getId()})
|
||||
);
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
||||
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
|
||||
|
@ -231,6 +230,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||
} else {
|
||||
appenderator.close();
|
||||
}
|
||||
toolbox.removeMonitor(metricsMonitor);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -361,12 +361,11 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
|
|||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null);
|
||||
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
|
||||
toolbox.addMonitor(
|
||||
new RealtimeMetricsMonitor(
|
||||
Collections.singletonList(fireDepartmentForMetrics),
|
||||
Collections.singletonMap(DruidMetrics.TASK_ID, new String[]{getId()})
|
||||
)
|
||||
RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor(
|
||||
Collections.singletonList(fireDepartmentForMetrics),
|
||||
Collections.singletonMap(DruidMetrics.TASK_ID, new String[]{getId()})
|
||||
);
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
||||
final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
|
||||
|
@ -483,6 +482,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
|
|||
} else {
|
||||
appenderator.close();
|
||||
}
|
||||
toolbox.removeMonitor(metricsMonitor);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -60,6 +60,7 @@ import org.apache.druid.indexing.common.actions.CheckPointDataSourceMetadataActi
|
|||
import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction;
|
||||
import org.apache.druid.indexing.common.actions.SegmentLockAcquireAction;
|
||||
import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
|
||||
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
|
||||
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
||||
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
|
||||
import org.apache.druid.indexing.input.InputRowSchemas;
|
||||
|
@ -393,7 +394,8 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
null
|
||||
);
|
||||
this.fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
toolbox.addMonitor(TaskRealtimeMetricsMonitorBuilder.build(task, fireDepartmentForMetrics, rowIngestionMeters));
|
||||
TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(task, fireDepartmentForMetrics, rowIngestionMeters);
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
final String lookupTier = task.getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER);
|
||||
final LookupNodeService lookupNodeService = lookupTier == null ?
|
||||
|
@ -900,7 +902,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
driver.close();
|
||||
}
|
||||
toolbox.getChatHandlerProvider().unregister(task.getId());
|
||||
|
||||
toolbox.removeMonitor(metricsMonitor);
|
||||
if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) {
|
||||
toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode);
|
||||
toolbox.getDataSegmentServerAnnouncer().unannounce();
|
||||
|
|
|
@ -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.druid.indexing.common.stats;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.segment.incremental.NoopRowIngestionMeters;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
public class TaskRealtimeMetricsMonitorTest
|
||||
{
|
||||
private ServiceEmitter emitter;
|
||||
private FireDepartment fireDepartment;
|
||||
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
emitter = EasyMock.mock(ServiceEmitter.class);
|
||||
fireDepartment = EasyMock.mock(FireDepartment.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLastRoundMetricsEmission()
|
||||
{
|
||||
FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
||||
RowIngestionMeters rowIngestionMeters = new NoopRowIngestionMeters();
|
||||
DataSchema schema = new DataSchema("dataSource", null, null, null, null, null, null, null);
|
||||
EasyMock.expect(fireDepartment.getMetrics()).andReturn(metrics);
|
||||
EasyMock.expectLastCall().times(2);
|
||||
EasyMock.expect(fireDepartment.getDataSchema()).andReturn(schema);
|
||||
EasyMock.expectLastCall().times(2);
|
||||
EasyMock.replay(fireDepartment);
|
||||
|
||||
TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor(fireDepartment, rowIngestionMeters, ImmutableMap.of());
|
||||
|
||||
Assert.assertFalse(monitor.isStarted());
|
||||
boolean zerothRound = monitor.monitor(emitter);
|
||||
monitor.start();
|
||||
Assert.assertTrue(monitor.isStarted());
|
||||
boolean firstRound = monitor.monitor(emitter);
|
||||
monitor.stop();
|
||||
Assert.assertFalse(monitor.isStarted());
|
||||
boolean secondRound = monitor.monitor(emitter);
|
||||
boolean thirdRound = monitor.monitor(emitter);
|
||||
|
||||
Assert.assertFalse(zerothRound);
|
||||
Assert.assertTrue(firstRound && secondRound);
|
||||
Assert.assertFalse(thirdRound);
|
||||
|
||||
EasyMock.verify(fireDepartment);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -46,6 +46,8 @@ public class RealtimeMetricsMonitor extends AbstractMonitor
|
|||
private final List<FireDepartment> fireDepartments;
|
||||
private final Map<String, String[]> dimensions;
|
||||
|
||||
private volatile boolean lastRoundMetricsToBePushed = false;
|
||||
|
||||
@Inject
|
||||
public RealtimeMetricsMonitor(List<FireDepartment> fireDepartments)
|
||||
{
|
||||
|
@ -59,6 +61,27 @@ public class RealtimeMetricsMonitor extends AbstractMonitor
|
|||
this.dimensions = ImmutableMap.copyOf(dimensions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start()
|
||||
{
|
||||
super.start();
|
||||
lastRoundMetricsToBePushed = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean monitor(ServiceEmitter emitter)
|
||||
{
|
||||
if (isStarted()) {
|
||||
return doMonitor(emitter);
|
||||
} else if (lastRoundMetricsToBePushed) {
|
||||
// Run one more time even if the monitor was removed, in case there's some extra data to flush
|
||||
lastRoundMetricsToBePushed = false;
|
||||
return doMonitor(emitter);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doMonitor(ServiceEmitter emitter)
|
||||
{
|
||||
|
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* 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.druid.segment.realtime;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
public class RealtimeMetricsMonitorTest
|
||||
{
|
||||
private ServiceEmitter emitter;
|
||||
private FireDepartment fireDepartment;
|
||||
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
emitter = EasyMock.mock(ServiceEmitter.class);
|
||||
fireDepartment = EasyMock.mock(FireDepartment.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLastRoundMetricsEmission()
|
||||
{
|
||||
FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
||||
DataSchema schema = new DataSchema("dataSource", null, null, null, null, null, null, null);
|
||||
EasyMock.expect(fireDepartment.getMetrics()).andReturn(metrics);
|
||||
EasyMock.expectLastCall().times(2);
|
||||
EasyMock.expect(fireDepartment.getDataSchema()).andReturn(schema);
|
||||
EasyMock.expectLastCall().times(2);
|
||||
EasyMock.replay(fireDepartment);
|
||||
|
||||
RealtimeMetricsMonitor monitor = new RealtimeMetricsMonitor(ImmutableList.of(fireDepartment));
|
||||
Assert.assertFalse(monitor.isStarted());
|
||||
boolean zerothRound = monitor.monitor(emitter);
|
||||
monitor.start();
|
||||
Assert.assertTrue(monitor.isStarted());
|
||||
boolean firstRound = monitor.monitor(emitter);
|
||||
monitor.stop();
|
||||
Assert.assertFalse(monitor.isStarted());
|
||||
boolean secondRound = monitor.monitor(emitter);
|
||||
boolean thirdRound = monitor.monitor(emitter);
|
||||
|
||||
Assert.assertFalse(zerothRound);
|
||||
Assert.assertTrue(firstRound && secondRound);
|
||||
Assert.assertFalse(thirdRound);
|
||||
EasyMock.verify(fireDepartment);
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue