Skip to content

Commit

Permalink
[BACKPORT] feat: add the number of merge buffers used for druid emitt…
Browse files Browse the repository at this point in the history
…er (#49) (#226)

* feat: add the number of merge buffers used for druid emitter (#49)

Signed-off-by: TessaIO <[email protected]>
Co-authored-by: TessaIO <[email protected]>

* Update setMetric (#51)

* Improve metrics emit (#52)

* Fix checkstyle (#53)

---------

Signed-off-by: TessaIO <[email protected]>
Co-authored-by: TessaIO <[email protected]>
  • Loading branch information
pagrawal10 and TessaIO authored Sep 23, 2024
1 parent 66f7801 commit a2aa518
Show file tree
Hide file tree
Showing 8 changed files with 89 additions and 4 deletions.
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
Expand Up @@ -292,4 +292,14 @@ public void testConcurrentTakeBatchClose() throws ExecutionException, Interrupte
r2.forEach(ReferenceCountingResourceHolder::close);
Assert.assertEquals(pool.maxSize(), pool.getPoolSize());
}

@Test(timeout = 60_000L)
public void testGetUsedBufferCount()
{
final List<ReferenceCountingResourceHolder<Integer>> holder = pool.takeBatch(6, 100L);
Assert.assertNotNull(holder);
Assert.assertEquals(6, pool.getUsedBufferCount());
holder.forEach(ReferenceCountingResourceHolder::close);
Assert.assertEquals(0, pool.getUsedBufferCount());
}
}
4 changes: 4 additions & 0 deletions docs/operations/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ Metrics may have additional dimensions beyond those listed above.
|`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 used up to merge the results of group by 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 All @@ -81,6 +82,7 @@ Metrics may have additional dimensions beyond those listed above.
|`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 used up to merge the results of group by 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 @@ -98,6 +100,8 @@ Metrics may have additional dimensions beyond those listed above.
|`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 used up to merge the results of group by 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.||

### Jetty

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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.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()));
emitter.emit(metricBuilder.build("query/merge/totalBuffers", maxSize()));
}
}
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 takenFromMap.size();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import org.apache.druid.client.cache.CachePopulatorStats;
import org.apache.druid.client.cache.ForegroundCachePopulator;
import org.apache.druid.collections.BlockingPool;
import org.apache.druid.collections.DefaultBlockingPool;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.StupidPool;
import org.apache.druid.guice.annotations.Global;
Expand All @@ -44,6 +43,7 @@
import org.apache.druid.offheap.OffheapBufferGenerator;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.ExecutorServiceMonitor;
import org.apache.druid.query.MetricsEmittingMergingBlockingPool;
import org.apache.druid.query.MetricsEmittingQueryProcessingPool;
import org.apache.druid.query.PrioritizedExecutorService;
import org.apache.druid.query.QueryProcessingPool;
Expand Down Expand Up @@ -126,12 +126,13 @@ 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 a2aa518

Please sign in to comment.