This commit is contained in:
Adithya Chakilam 2024-10-07 19:39:46 -05:00
parent 38ea363318
commit 84e2f10894
6 changed files with 28 additions and 95 deletions

View File

@ -262,8 +262,6 @@ batch ingestion emit the following metrics. These metrics are deltas for each em
|`ingest/pause/time`|Milliseconds spent by a task in a paused state without ingesting.|`dataSource`, `taskId`, `tags`| < 10 seconds|
|`ingest/handoff/time`|Total number of milliseconds taken to handoff a set of segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on the coordinator cycle time.|
|`task/autoScaler/requiredCount`|Count of required tasks based on the calculations of `lagBased` auto scaler.|`dataSource`, `stream`, `scalingSkipReason`|Depends on auto scaler config.|
|`task/supervisor/active/count`|Count of the active task groups for a supervisor.|`datasource`, `stream`|Depends on state of supervisor.|
|`task/supervisor/publishing/count`|Count of the publishing task groups for a supervisor.|`datasource`, `stream`|Depends on state of supervisor.|
If the JVM does not support CPU time measurement for the current thread, `ingest/merge/cpu` and `ingest/persists/cpu` will be 0.

View File

@ -343,4 +343,14 @@ public interface Task
{
return BroadcastDatasourceLoadingSpec.createFromContext(getContext(), BroadcastDatasourceLoadingSpec.ALL);
}
/**
* specifies the current status of the task.
*
* @return string
*/
default String status()
{
return "UNKONWN";
}
}

View File

@ -311,4 +311,11 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
{
return runnerSupplier.get();
}
@Override
@VisibleForTesting
public String status()
{
return getRunner().getStatus().name();
}
}

View File

@ -4362,12 +4362,6 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
spec.getMonitorSchedulerConfig().getEmissionDuration().getMillis(),
TimeUnit.MILLISECONDS
);
reportingExec.scheduleAtFixedRate(
this::emitTaskCount,
ioConfig.getStartDelay().getMillis(),
spec.getMonitorSchedulerConfig().getEmissionDuration().getMillis(),
TimeUnit.MILLISECONDS
);
}
/**
@ -4470,32 +4464,6 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
}
}
public void emitTaskCount()
{
try {
ServiceMetricEvent.Builder eventBuilder = ServiceMetricEvent.builder()
.setDimension(DruidMetrics.DATASOURCE, dataSource)
.setDimensionIfNotNull(
DruidMetrics.TAGS,
spec.getContextValue(DruidMetrics.TAGS)
)
.setDimension(
DruidMetrics.STREAM,
getIoConfig().getStream()
);
emitter.emit(eventBuilder.setMetric("task/supervisor/active/count", activelyReadingTaskGroups.size()));
emitter.emit(eventBuilder.setMetric("task/supervisor/publishing/count",
pendingCompletionTaskGroups.values()
.stream()
.filter(list -> !list.isEmpty())
.count()
));
}
catch (Exception e) {
log.warn(e, "Unable to publish active/publisihing task count");
}
}
protected void emitLag()
{
SupervisorStateManager.State basicState = stateManager.getSupervisorState().getBasicState();

View File

@ -1800,32 +1800,6 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
verifyAll();
}
@Test
public void testEmitTaskCounts() throws Exception
{
expectEmitterSupervisor(false);
CountDownLatch latch = new CountDownLatch(1);
TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor(
latch,
TestEmittingTestSeekableStreamSupervisor.TASK_COUNTS,
ImmutableMap.of("1", 100L, "2", 250L, "3", 500L),
ImmutableMap.of("1", 10000L, "2", 15000L, "3", 20000L)
);
supervisor.start();
Assert.assertTrue(supervisor.stateManager.isHealthy());
Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState());
Assert.assertEquals(BasicState.PENDING, supervisor.stateManager.getSupervisorState().getBasicState());
Assert.assertTrue(supervisor.stateManager.getExceptionEvents().isEmpty());
Assert.assertFalse(supervisor.stateManager.isAtLeastOneSuccessfulRun());
latch.await();
emitter.verifyEmitted("task/supervisor/active/count", 1);
emitter.verifyEmitted("task/supervisor/active/count", 1);
verifyAll();
}
@Test
public void testGetStats()
{
@ -2518,16 +2492,9 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
{
EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes();
DruidMonitorSchedulerConfig config = new DruidMonitorSchedulerConfig();
EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(config).times(3);
ScheduledExecutorService executorService = EasyMock.createMock(ScheduledExecutorService.class);
EasyMock.expect(executorService.scheduleWithFixedDelay(EasyMock.anyObject(), EasyMock.eq(86415000L), EasyMock.eq(300000L), EasyMock.eq(TimeUnit.MILLISECONDS))).andReturn(EasyMock.createMock(ScheduledFuture.class)).once();
EasyMock.expect(executorService.scheduleAtFixedRate(EasyMock.anyObject(), EasyMock.eq(86425000L), EasyMock.eq(config.getEmissionDuration().getMillis()), EasyMock.eq(TimeUnit.MILLISECONDS))).andReturn(EasyMock.createMock(ScheduledFuture.class)).times(2);
EasyMock.expect(executorService.scheduleAtFixedRate(
EasyMock.anyObject(),
EasyMock.eq(86400000L),
EasyMock.eq(config.getEmissionDuration().getMillis()),
EasyMock.eq(TimeUnit.MILLISECONDS)
)).andReturn(EasyMock.createMock(ScheduledFuture.class)).times(1);
EasyMock.replay(executorService, spec);
final BaseTestSeekableStreamSupervisor supervisor = new BaseTestSeekableStreamSupervisor()
@ -3027,7 +2994,6 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
private static final byte LAG = 0x01;
private static final byte NOTICE_QUEUE = 0x02;
private static final byte NOTICE_PROCESS = 0x04;
private static final byte TASK_COUNTS = 0x08;
TestEmittingTestSeekableStreamSupervisor(
@ -3089,16 +3055,6 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
latch.countDown();
}
@Override
public void emitTaskCount()
{
if ((metricFlag & TASK_COUNTS) == 0) {
return;
}
super.emitTaskCount();
latch.countDown();
}
@Override
public LagStats computeLagStats()
{
@ -3121,12 +3077,6 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
spec.getMonitorSchedulerConfig().getEmissionDuration().getMillis(),
TimeUnit.MILLISECONDS
);
reportingExec.scheduleAtFixedRate(
this::emitTaskCount,
ioConfig.getStartDelay().getMillis(),
spec.getMonitorSchedulerConfig().getEmissionDuration().getMillis(),
TimeUnit.MILLISECONDS
);
}
}

View File

@ -25,10 +25,8 @@ import com.github.rvesse.airline.annotations.Command;
import com.github.rvesse.airline.annotations.Option;
import com.github.rvesse.airline.annotations.restrictions.Required;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Binder;
import com.google.inject.Inject;
@ -141,6 +139,7 @@ import java.io.File;
import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
@ -303,19 +302,20 @@ public class CliPeon extends GuiceRunnable
@Named(ServiceStatusMonitor.HEARTBEAT_TAGS_BINDING)
public Supplier<Map<String, Object>> heartbeatDimensions(Task task)
{
ImmutableMap.Builder<String, Object> builder = ImmutableMap.builder();
builder.put(DruidMetrics.TASK_ID, task.getId());
builder.put(DruidMetrics.DATASOURCE, task.getDataSource());
builder.put(DruidMetrics.TASK_TYPE, task.getType());
builder.put(DruidMetrics.GROUP_ID, task.getGroupId());
Map<String, Object> map = new HashMap<>();
map.put(DruidMetrics.TASK_ID, task.getId());
map.put(DruidMetrics.DATASOURCE, task.getDataSource());
map.put(DruidMetrics.TASK_TYPE, task.getType());
map.put(DruidMetrics.GROUP_ID, task.getGroupId());
Map<String, Object> tags = task.getContextValue(DruidMetrics.TAGS);
if (tags != null && !tags.isEmpty()) {
builder.put(DruidMetrics.TAGS, tags);
map.put(DruidMetrics.TAGS, tags);
}
return Suppliers.ofInstance(
builder.build()
);
return () -> {
map.put(DruidMetrics.STATUS, task.status());
return map;
};
}
@Provides