[draft] Start publishing segment load metric from a separate monitor#18314
[draft] Start publishing segment load metric from a separate monitor#18314uds5501 wants to merge 14 commits intoapache:masterfrom
Conversation
1d22153 to
6d46baa
Compare
6d46baa to
f2a174a
Compare
kfaraz
left a comment
There was a problem hiding this comment.
Overall looks good. Left some minor suggestions.
|
Note - Something weird is happening with |
|
@uds5501 , where does the |
|
@uds5501 , after taking a look at the failure, it seems that the sub-class tests are failing because they override the |
kfaraz
left a comment
There was a problem hiding this comment.
LGTM. Only 1 blocking comment to de-duplicate segment counts.
| return RowKey.with(Dimension.DATASOURCE, segment.getDataSource()) | ||
| .with(Dimension.INTERVAL, String.valueOf(segment.getInterval())) | ||
| .and(Dimension.VERSION, segment.getVersion()); |
There was a problem hiding this comment.
Since we are now emitting an added/removed event per segment, rather than interval and version, we can just use the dimensions dataSource, server and description (which contains the full segment ID).
There was a problem hiding this comment.
Are you expecting a full message like {server_name} service added/removed {segment_id} segment in the description?
There was a problem hiding this comment.
No, just the segment id.
server should be a separate dimension on the row key
| private final ConcurrentHashMap<RowKey, Long> totalSegmentAddCount = new ConcurrentHashMap<>(); | ||
| private final ConcurrentHashMap<RowKey, Long> totalSegmentRemoveCount = new ConcurrentHashMap<>(); | ||
| @GuardedBy("totalSegmentAddCount") | ||
| private Map<RowKey, Long> previousSegmentAddCount = new HashMap<>(); | ||
| @GuardedBy("totalSegmentRemoveCount") | ||
| private Map<RowKey, Long> previousSegmentRemoveCount = new HashMap<>(); |
There was a problem hiding this comment.
Rather than this, we can just keep a list of events (added, removed).
When getSegmentsAdded is called, we just flush out that list.
The count for any given row key will always be 1 if we use segment ID and server in the dimensions.
There was a problem hiding this comment.
For now I have pushed a non event version to verify embedded tests.
I am thinking if a simple private final ConcurrentHashMap<RowKey, List<DiscoveryEvent>> discoveryEvents will suffice for now, where RowKey has dataSource and server name.
The DiscoveryEvent could be a simple enum with ADDED / REMOVED values. This map will be flushed on each getSegmentDiscoveryEvents() fetch.
There was a problem hiding this comment.
If you are adding a discovery event class, that can contain the RowKey itself. Then you don't really need the map. You could just have a list of events, which feels more natural.
Drawback is that on the monitor side, you would need to distinguish added and removed events. So you would need to expose this DiscoveryEvent class, which seems unnecessary.
See if you can just make do with a Boolean, or even keeping separate methods and lists for added/removed is fine.
There was a problem hiding this comment.
Ah, I see, yeah I think I can maintain separate added and removed RowKeys for now.
b870d54 to
e3a48f3
Compare
kfaraz
left a comment
There was a problem hiding this comment.
LGTM, left minor suggestions.
| event -> event.hasDimension(DruidMetrics.DATASOURCE, dataSource) | ||
| event -> event.hasMetricName("serverview/segment/added") | ||
| .hasDimension(DruidMetrics.DATASOURCE, dataSource) | ||
| .hasValue(1) |
There was a problem hiding this comment.
Nit: the value of this metric is always 1, so we can remove this condition.
| DUTY_GROUP("dutyGroup"), | ||
| DESCRIPTION("description"), | ||
| SERVER("server"); | ||
|
|
| .collect(Collectors.toList()); | ||
| } | ||
|
|
||
| private static RowKey getMetricKey(final DataSegment segment, DruidServerMetadata serverMetadata) |
There was a problem hiding this comment.
Nit: please either keep both args final or both non-final.
|
@uds5501 , I think we can close this off for now since the flakiness in the tests has already been addressed. |
Description
Create a
SegmentDiscoveryStatsMonitorthat takes care of segment discovery on broker for data sources. Adds a small POC in embedded tests as well.Key changed/added classes in this PR
BrokerServerViewBrokerSegmentStatsMonitorThis PR has: