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
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import com.facebook.presto.operator.TableFinishInfo;
import com.facebook.presto.operator.TaskStats;
import com.facebook.presto.server.BasicQueryInfo;
import com.facebook.presto.server.BasicQueryStats;
import com.facebook.presto.spi.ConnectorId;
import com.facebook.presto.spi.QueryId;
import com.facebook.presto.spi.eventlistener.OperatorStatistics;
Expand All @@ -57,6 +58,7 @@
import com.facebook.presto.spi.eventlistener.QueryInputMetadata;
import com.facebook.presto.spi.eventlistener.QueryMetadata;
import com.facebook.presto.spi.eventlistener.QueryOutputMetadata;
import com.facebook.presto.spi.eventlistener.QueryProgressEvent;
import com.facebook.presto.spi.eventlistener.QueryStatistics;
import com.facebook.presto.spi.eventlistener.QueryUpdatedEvent;
import com.facebook.presto.spi.eventlistener.ResourceDistribution;
Expand Down Expand Up @@ -173,6 +175,30 @@ public void queryUpdatedEvent(QueryInfo queryInfo)
eventListenerManager.queryUpdated(new QueryUpdatedEvent(createQueryMetadata(queryInfo)));
}

public void publishQueryProgressEvent(long monotonicallyIncreasingEventId, BasicQueryInfo queryInfo)
{
eventListenerManager.publishQueryProgress(new QueryProgressEvent(
monotonicallyIncreasingEventId,
new QueryMetadata(
queryInfo.getQueryId().toString(),
queryInfo.getSession().getTransactionId().map(TransactionId::toString),
queryInfo.getQuery(),
queryInfo.getQueryHash(),
queryInfo.getPreparedQuery(),
queryInfo.getState().toString(),
queryInfo.getSelf(),
Optional.empty(),
Optional.empty(),
Optional.empty(),
Optional.empty(),
ImmutableList.of(),
queryInfo.getSession().getTraceToken()),
createQueryStatistics(queryInfo),
createQueryContext(queryInfo.getSession(), queryInfo.getResourceGroupId()),
queryInfo.getQueryType(),
ofEpochMilli(queryInfo.getQueryStats().getCreateTime().getMillis())));
}

public void queryImmediateFailureEvent(BasicQueryInfo queryInfo, ExecutionFailureInfo failure)
{
eventListenerManager.queryCompleted(new QueryCompletedEvent(
Expand Down Expand Up @@ -431,6 +457,46 @@ private QueryStatistics createQueryStatistics(QueryInfo queryInfo)
queryStats.getRuntimeStats());
}

private QueryStatistics createQueryStatistics(BasicQueryInfo basicQueryInfo)
{
BasicQueryStats queryStats = basicQueryInfo.getQueryStats();

return new QueryStatistics(
ofMillis(queryStats.getTotalCpuTime().toMillis()),
ofMillis(0),
ofMillis(queryStats.getElapsedTime().toMillis()),
ofMillis(queryStats.getWaitingForPrerequisitesTime().toMillis()),
ofMillis(queryStats.getQueuedTime().toMillis()),
ofMillis(0),
ofMillis(0),
ofMillis(0),
ofMillis(0),
ofMillis(0),
Optional.of(ofMillis(0)),
ofMillis(queryStats.getExecutionTime().toMillis()),
queryStats.getPeakRunningTasks(),
queryStats.getPeakUserMemoryReservation().toBytes(),
queryStats.getPeakTotalMemoryReservation().toBytes(),
0,
0,
0,
0,
0,
queryStats.getRawInputDataSize().toBytes(),
queryStats.getRawInputPositions(),
0,
0,
0,
0,
0,
0,
queryStats.getCumulativeUserMemory(),
queryStats.getCumulativeTotalMemory(),
queryStats.getCompletedDrivers(),
false,
new RuntimeStats());
}

private QueryContext createQueryContext(SessionRepresentation session, Optional<ResourceGroupId> resourceGroup)
{
return new QueryContext(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,16 +14,21 @@
package com.facebook.presto.event;

import com.facebook.airlift.configuration.Config;
import com.facebook.airlift.configuration.ConfigDescription;
import io.airlift.units.DataSize;
import io.airlift.units.DataSize.Unit;
import io.airlift.units.Duration;
import io.airlift.units.MaxDataSize;
import io.airlift.units.MinDataSize;

import javax.validation.constraints.NotNull;

import static java.util.concurrent.TimeUnit.MINUTES;

public class QueryMonitorConfig
{
private DataSize maxOutputStageJsonSize = new DataSize(16, Unit.MEGABYTE);
private Duration queryProgressPublishInterval = new Duration(0, MINUTES);

@MinDataSize("1kB")
@MaxDataSize("1GB")
Expand All @@ -39,4 +44,17 @@ public QueryMonitorConfig setMaxOutputStageJsonSize(DataSize maxOutputStageJsonS
this.maxOutputStageJsonSize = maxOutputStageJsonSize;
return this;
}

public Duration getQueryProgressPublishInterval()
{
return queryProgressPublishInterval;
}

@Config("event.query-progress-publish-interval")
@ConfigDescription("How frequently to publish query progress events. 0 duration disables the publication of these events.")
public QueryMonitorConfig setQueryProgressPublishInterval(Duration queryProgressPublishInterval)
{
this.queryProgressPublishInterval = queryProgressPublishInterval;
return this;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* 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.event;

import com.facebook.presto.dispatcher.DispatchManager;
import com.facebook.presto.server.BasicQueryInfo;
import io.airlift.units.Duration;

import javax.annotation.PostConstruct;
import javax.annotation.PreDestroy;
import javax.annotation.concurrent.GuardedBy;
import javax.inject.Inject;

import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicLong;

import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;

public class QueryProgressMonitor
{
private final AtomicLong monotonicallyIncreasingEventId = new AtomicLong();

private final QueryMonitor queryMonitor;
private final DispatchManager dispatchManager;
private final Duration queryProgressPublishInterval;

@GuardedBy("this")
private ScheduledExecutorService queryProgressMonitorExecutor;

@Inject
public QueryProgressMonitor(
QueryMonitor queryMonitor,
DispatchManager dispatchManager,
QueryMonitorConfig queryMonitorConfig)
{
this.queryMonitor = requireNonNull(queryMonitor, "queryMonitor is null");
this.dispatchManager = requireNonNull(dispatchManager, "dispatchManager is null");
this.queryProgressPublishInterval = requireNonNull(queryMonitorConfig, "queryMonitorConfig is null").getQueryProgressPublishInterval();
}

@PostConstruct
public synchronized void start()
{
if (queryProgressPublishInterval.getValue() > 0) {
if (queryProgressMonitorExecutor == null) {
queryProgressMonitorExecutor = newSingleThreadScheduledExecutor(daemonThreadsNamed("query-progress-monitor-executor"));
}

queryProgressMonitorExecutor.scheduleWithFixedDelay(
this::publishQueryProgressEvent,
(long) queryProgressPublishInterval.getValue(),
(long) queryProgressPublishInterval.getValue(),
queryProgressPublishInterval.getUnit());
}
}

@PreDestroy
public synchronized void stop()
{
if (queryProgressMonitorExecutor != null) {
queryProgressMonitorExecutor.shutdown();
}
}

private void publishQueryProgressEvent()
{
for (BasicQueryInfo basicQueryInfo : dispatchManager.getQueries()) {
if (!basicQueryInfo.getState().isDone()) {
queryMonitor.publishQueryProgressEvent(monotonicallyIncreasingEventId.incrementAndGet(), basicQueryInfo);
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import com.facebook.presto.spi.eventlistener.EventListenerFactory;
import com.facebook.presto.spi.eventlistener.QueryCompletedEvent;
import com.facebook.presto.spi.eventlistener.QueryCreatedEvent;
import com.facebook.presto.spi.eventlistener.QueryProgressEvent;
import com.facebook.presto.spi.eventlistener.QueryUpdatedEvent;
import com.facebook.presto.spi.eventlistener.SplitCompletedEvent;
import com.google.common.annotations.VisibleForTesting;
Expand Down Expand Up @@ -115,6 +116,12 @@ public void queryUpdated(QueryUpdatedEvent queryUpdatedEvent)
.ifPresent(eventListener -> eventListener.queryUpdated(queryUpdatedEvent));
}

public void publishQueryProgress(QueryProgressEvent queryProgressEvent)
{
configuredEventListener.get()
.ifPresent(eventListener -> eventListener.publishQueryProgress(queryProgressEvent));
}

public void splitCompleted(SplitCompletedEvent splitCompletedEvent)
{
configuredEventListener.get()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import com.facebook.presto.dispatcher.LocalDispatchQueryFactory;
import com.facebook.presto.event.QueryMonitor;
import com.facebook.presto.event.QueryMonitorConfig;
import com.facebook.presto.event.QueryProgressMonitor;
import com.facebook.presto.execution.ClusterSizeMonitor;
import com.facebook.presto.execution.ExecutionFactoriesManager;
import com.facebook.presto.execution.ExplainAnalyzeContext;
Expand Down Expand Up @@ -163,6 +164,7 @@ protected void setup(Binder binder)
jsonCodecBinder(binder).bindJsonCodec(OperatorInfo.class);
configBinder(binder).bindConfig(QueryMonitorConfig.class);
binder.bind(QueryMonitor.class).in(Scopes.SINGLETON);
binder.bind(QueryProgressMonitor.class).in(Scopes.SINGLETON);

// query manager
jaxrsBinder(binder).bind(QueryResource.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import com.facebook.presto.spi.eventlistener.EventListenerFactory;
import com.facebook.presto.spi.eventlistener.QueryCompletedEvent;
import com.facebook.presto.spi.eventlistener.QueryCreatedEvent;
import com.facebook.presto.spi.eventlistener.QueryProgressEvent;
import com.facebook.presto.spi.eventlistener.QueryUpdatedEvent;
import com.facebook.presto.spi.eventlistener.SplitCompletedEvent;
import com.google.common.collect.ImmutableMap;
Expand Down Expand Up @@ -60,6 +61,14 @@ public void queryUpdated(QueryUpdatedEvent queryUpdatedEvent)
}
}

@Override
public void publishQueryProgress(QueryProgressEvent queryProgressEvent)
{
if (configuredEventListener.get().isPresent()) {
configuredEventListener.get().get().publishQueryProgress(queryProgressEvent);
}
}

@Override
public void splitCompleted(SplitCompletedEvent splitCompletedEvent)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import com.facebook.presto.spi.eventlistener.EventListenerFactory;
import com.facebook.presto.spi.eventlistener.QueryCompletedEvent;
import com.facebook.presto.spi.eventlistener.QueryCreatedEvent;
import com.facebook.presto.spi.eventlistener.QueryProgressEvent;
import com.facebook.presto.spi.eventlistener.QueryUpdatedEvent;
import com.facebook.presto.spi.eventlistener.SplitCompletedEvent;
import com.facebook.presto.spi.prerequisites.QueryPrerequisites;
Expand Down Expand Up @@ -522,6 +523,12 @@ public void queryUpdated(QueryUpdatedEvent queryUpdatedEvent)
fail("Query update events should not be created in this test");
}

@Override
public void publishQueryProgress(QueryProgressEvent queryProgressEvent)
{
fail("Query Progress events should not be created in this test");
}

@Override
public void queryCompleted(QueryCompletedEvent event)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,11 @@

import com.google.common.collect.ImmutableMap;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import org.testng.annotations.Test;

import java.util.Map;
import java.util.concurrent.TimeUnit;

import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertFullMapping;
import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults;
Expand All @@ -30,18 +32,21 @@ public class TestQueryMonitorConfig
public void testDefaults()
{
assertRecordedDefaults(recordDefaults(QueryMonitorConfig.class)
.setMaxOutputStageJsonSize(new DataSize(16, Unit.MEGABYTE)));
.setMaxOutputStageJsonSize(new DataSize(16, Unit.MEGABYTE))
.setQueryProgressPublishInterval(new Duration(0, TimeUnit.MINUTES)));
}

@Test
public void testExplicitPropertyMappings()
{
Map<String, String> properties = new ImmutableMap.Builder<String, String>()
.put("event.max-output-stage-size", "512kB")
.put("event.query-progress-publish-interval", "2m")
.build();

QueryMonitorConfig expected = new QueryMonitorConfig()
.setMaxOutputStageJsonSize(new DataSize(512, Unit.KILOBYTE));
.setMaxOutputStageJsonSize(new DataSize(512, Unit.KILOBYTE))
.setQueryProgressPublishInterval(new Duration(2, TimeUnit.MINUTES));

assertFullMapping(properties, expected);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,10 @@ default void queryUpdated(QueryUpdatedEvent queryUpdatedEvent)
{
}

default void publishQueryProgress(QueryProgressEvent queryProgressEvent)
{
}

default void queryCompleted(QueryCompletedEvent queryCompletedEvent)
{
}
Expand Down
Loading