Skip to content

Commit

Permalink
feat: add the number of merge buffers used for druid emitter
Browse files Browse the repository at this point in the history
Signed-off-by: TessaIO <ahmedgrati1999@gmail.com>
  • Loading branch information
TessaIO committed Apr 27, 2024
1 parent 9aef8e0 commit 846f7e7
Show file tree
Hide file tree
Showing 7 changed files with 58 additions and 3 deletions.
3 changes: 3 additions & 0 deletions docs/operations/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/success/count`|Number of queries successfully processed.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/merge/buffersUsed`|number of merge buffers allocated to broker while performing groupBy merge queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/segments/count`|This metric is not enabled by default. See the `QueryMetrics` Interface for reference regarding enabling this metric. Number of segments that will be touched by the query. In the broker, it makes a plan to distribute the query to realtime tasks and historicals based on a snapshot of segment distribution state. If there are some segments moved after this snapshot is created, certain historicals and realtime tasks can report those segments as missing to the broker. The broker will resend the query to the new servers that serve those segments after move. In this case, those segments can be counted more than once in this metric.||Varies|
Expand Down Expand Up @@ -106,6 +107,7 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/success/count`|Number of queries successfully processed.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/merge/buffersUsed`|number of merge buffers allocated to historical while performing groupBy merge queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `QueryCountStatsMonitor` module is included.||

Expand All @@ -123,6 +125,7 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/merge/buffersUsed`|number of merge buffers allocated to historical while performing groupBy merge queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||

### Jetty

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,4 +49,9 @@ public interface BlockingPool<T>
* @return count of pending requests
*/
long getPendingRequests();

/**
* @return number of buffers used/polled from the pool at that time.
*/
int getUsedBufferCount();
}
Original file line number Diff line number Diff line change
Expand Up @@ -212,4 +212,10 @@ private void offer(T theObject)
lock.unlock();
}
}

@Override
public int getUsedBufferCount()
{
return maxSize - objects.size();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -61,4 +61,10 @@ public long getPendingRequests()
{
return 0;
}

@Override
public int getUsedBufferCount()
{
return 0;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
package org.apache.druid.query;

import com.google.common.base.Supplier;
import org.apache.druid.collections.DefaultBlockingPool;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;

public class MetricsEmittingMergingBlockingPool<T> extends DefaultBlockingPool<T>
implements ExecutorServiceMonitor.MetricEmitter{

public MetricsEmittingMergingBlockingPool(
Supplier<T> generator,
int limit,
ExecutorServiceMonitor executorServiceMonitor
) {
super(generator, limit);
executorServiceMonitor.add(this);
}

@Override
public void emitMetrics(ServiceEmitter emitter, ServiceMetricEvent.Builder metricBuilder)
{
emitter.emit(metricBuilder.build("query/merge/buffersUsed", getUsedBufferCount()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -147,4 +147,10 @@ public Collection<RuntimeException> getOutstandingExceptionsCreated()
{
return takenFromMap.values();
}

@Override
public int getUsedBufferCount()
{
return Integer.MAX_VALUE;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -126,12 +126,16 @@ public NonBlockingPool<ByteBuffer> getIntermediateResultsPool(DruidProcessingCon
@Provides
@LazySingleton
@Merging
public BlockingPool<ByteBuffer> getMergeBufferPool(DruidProcessingConfig config)
public BlockingPool<ByteBuffer> getMergeBufferPool(
DruidProcessingConfig config,
ExecutorServiceMonitor executorServiceMonitor
)
{
verifyDirectMemory(config);
return new DefaultBlockingPool<>(
return new MetricsEmittingMergingBlockingPool<>(
new OffheapBufferGenerator("result merging", config.intermediateComputeSizeBytes()),
config.getNumMergeBuffers()
config.getNumMergeBuffers(),
executorServiceMonitor
);
}

Expand Down

0 comments on commit 846f7e7

Please sign in to comment.