Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
17 changes: 0 additions & 17 deletions presto-main/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -308,23 +308,6 @@
<artifactId>reactor-netty-http</artifactId>
</dependency>

<dependency>
<groupId>io.micrometer</groupId>
<artifactId>micrometer-core</artifactId>
<version>1.11.0</version>
<exclusions>
<exclusion>
<groupId>org.hdrhistogram</groupId>
<artifactId>HdrHistogram</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>io.micrometer</groupId>
<artifactId>micrometer-registry-jmx</artifactId>
<version>1.11.0</version>
</dependency>

<dependency>
<groupId>io.projectreactor</groupId>
<artifactId>reactor-core</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,8 @@
import com.facebook.presto.server.protocol.QueryBlockingRateLimiter;
import com.facebook.presto.server.protocol.QueuedStatementResource;
import com.facebook.presto.server.protocol.RetryCircuitBreaker;
import com.facebook.presto.server.remotetask.HttpClientConnectionPoolStats;
import com.facebook.presto.server.remotetask.HttpClientStats;
import com.facebook.presto.server.remotetask.HttpRemoteTaskFactory;
import com.facebook.presto.server.remotetask.ReactorNettyHttpClient;
import com.facebook.presto.server.remotetask.ReactorNettyHttpClientConfig;
Expand Down Expand Up @@ -277,6 +279,10 @@ protected void setup(Binder binder)
ReactorNettyHttpClientConfig reactorNettyHttpClientConfig = buildConfigObject(ReactorNettyHttpClientConfig.class);
if (reactorNettyHttpClientConfig.isReactorNettyHttpClientEnabled()) {
binder.bind(ReactorNettyHttpClient.class).in(Scopes.SINGLETON);
binder.bind(HttpClientStats.class).in(Scopes.SINGLETON);
newExporter(binder).export(HttpClientStats.class).withGeneratedName();
binder.bind(HttpClientConnectionPoolStats.class).in(Scopes.SINGLETON);
newExporter(binder).export(HttpClientConnectionPoolStats.class).withGeneratedName();
binder.bind(HttpClient.class).annotatedWith(ForScheduler.class).to(ReactorNettyHttpClient.class);
}
else {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
/*
* Licensed 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 com.facebook.presto.server.remotetask;

import com.facebook.airlift.stats.DistributionStat;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import org.weakref.jmx.Managed;
import org.weakref.jmx.Nested;
import reactor.netty.resources.ConnectionPoolMetrics;
import reactor.netty.resources.ConnectionProvider;

import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;

@Singleton
public class HttpClientConnectionPoolStats
implements ConnectionProvider.MeterRegistrar
{
private final ConcurrentHashMap<String, ConnectionPoolMetrics> poolMetrics = new ConcurrentHashMap<>();

private final DistributionStat activeConnections = new DistributionStat();
private final DistributionStat totalConnections = new DistributionStat();
private final DistributionStat idleConnections = new DistributionStat();
private final DistributionStat pendingAcquires = new DistributionStat();
private final DistributionStat maxConnections = new DistributionStat();
private final DistributionStat maxPendingAcquires = new DistributionStat();

@Inject
public HttpClientConnectionPoolStats()
{
scheduleStatsExport();
}

@Override
public void registerMetrics(String poolName, String id, SocketAddress remoteAddress, ConnectionPoolMetrics metrics)
{
poolMetrics.put(createPoolKey(poolName, remoteAddress), metrics);
}

private static String createPoolKey(String poolName, SocketAddress remoteAddress)
{
return poolName + ":" + formatSocketAddress(remoteAddress);
}

private static String formatSocketAddress(SocketAddress socketAddress)
{
if (socketAddress != null) {
if (socketAddress instanceof InetSocketAddress) {
InetSocketAddress address = (InetSocketAddress) socketAddress;
return address.getHostString().replace(".", "_");
Comment thread
NikhilCollooru marked this conversation as resolved.
}
else {
return socketAddress.toString().replace(".", "_");
}
}
return "UNKNOWN";
}

private void scheduleStatsExport()
{
Executors.newSingleThreadScheduledExecutor()
.scheduleAtFixedRate(
() -> {
for (ConnectionPoolMetrics metrics : poolMetrics.values()) {
activeConnections.add(metrics.acquiredSize());
totalConnections.add(metrics.allocatedSize());
idleConnections.add(metrics.idleSize());
pendingAcquires.add(metrics.pendingAcquireSize());
maxConnections.add(metrics.maxAllocatedSize());
maxPendingAcquires.add(metrics.maxPendingAcquireSize());
}
},
0,
1,
TimeUnit.SECONDS);
}

@Managed
@Nested
public DistributionStat getActiveConnections()
{
return activeConnections;
}

@Managed
@Nested
public DistributionStat getTotalConnections()
{
return totalConnections;
}

@Managed
@Nested
public DistributionStat getIdleConnections()
{
return idleConnections;
}

@Managed
@Nested
public DistributionStat getPendingAcquires()
{
return pendingAcquires;
}

@Managed
@Nested
public DistributionStat getMaxConnections()
{
return maxConnections;
}

@Managed
@Nested
public DistributionStat getMaxPendingAcquires()
{
return maxPendingAcquires;
}
}
Loading
Loading