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 @@ -19,8 +19,6 @@
import com.facebook.presto.client.FailureInfo;
import com.facebook.presto.client.QueryError;
import com.facebook.presto.client.QueryResults;
import com.facebook.presto.client.StageStats;
import com.facebook.presto.client.StatementStats;
import com.facebook.presto.common.Page;
import com.facebook.presto.common.block.BlockEncodingSerde;
import com.facebook.presto.common.type.BooleanType;
Expand All @@ -30,11 +28,7 @@
import com.facebook.presto.execution.QueryInfo;
import com.facebook.presto.execution.QueryManager;
import com.facebook.presto.execution.QueryState;
import com.facebook.presto.execution.QueryStats;
import com.facebook.presto.execution.StageExecutionInfo;
import com.facebook.presto.execution.StageExecutionStats;
import com.facebook.presto.execution.StageInfo;
import com.facebook.presto.execution.TaskInfo;
import com.facebook.presto.execution.buffer.PagesSerdeFactory;
import com.facebook.presto.operator.ExchangeClient;
import com.facebook.presto.spi.ErrorCode;
Expand All @@ -61,7 +55,6 @@
import javax.ws.rs.core.UriInfo;

import java.net.URI;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
Expand All @@ -76,6 +69,7 @@
import static com.facebook.presto.SystemSessionProperties.getTargetResultSize;
import static com.facebook.presto.SystemSessionProperties.isExchangeCompressionEnabled;
import static com.facebook.presto.execution.QueryState.FAILED;
import static com.facebook.presto.server.protocol.QueryResourceUtil.toStatementStats;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static com.facebook.presto.util.Failures.toFailure;
import static com.google.common.base.MoreObjects.firstNonNull;
Expand Down Expand Up @@ -523,90 +517,6 @@ private synchronized URI createNextResultsUri(String scheme, UriInfo uriInfo, lo
return uri.build();
}

private static StatementStats toStatementStats(QueryInfo queryInfo)
{
QueryStats queryStats = queryInfo.getQueryStats();
StageInfo outputStage = queryInfo.getOutputStage().orElse(null);

return StatementStats.builder()
.setState(queryInfo.getState().toString())
.setQueued(queryInfo.getState() == QueryState.QUEUED)
.setScheduled(queryInfo.isScheduled())
.setNodes(globalUniqueNodes(outputStage).size())
.setTotalSplits(queryStats.getTotalDrivers())
.setQueuedSplits(queryStats.getQueuedDrivers())
.setRunningSplits(queryStats.getRunningDrivers() + queryStats.getBlockedDrivers())
.setCompletedSplits(queryStats.getCompletedDrivers())
.setCpuTimeMillis(queryStats.getTotalCpuTime().toMillis())
.setWallTimeMillis(queryStats.getTotalScheduledTime().toMillis())
.setQueuedTimeMillis(queryStats.getQueuedTime().toMillis())
.setElapsedTimeMillis(queryStats.getElapsedTime().toMillis())
.setProcessedRows(queryStats.getRawInputPositions())
.setProcessedBytes(queryStats.getRawInputDataSize().toBytes())
.setPeakMemoryBytes(queryStats.getPeakUserMemoryReservation().toBytes())
.setPeakTotalMemoryBytes(queryStats.getPeakTotalMemoryReservation().toBytes())
.setPeakTaskTotalMemoryBytes(queryStats.getPeakTaskTotalMemory().toBytes())
.setSpilledBytes(queryStats.getSpilledDataSize().toBytes())
.setRootStage(toStageStats(outputStage))
.build();
}

private static StageStats toStageStats(StageInfo stageInfo)
{
if (stageInfo == null) {
return null;
}

StageExecutionInfo currentStageExecutionInfo = stageInfo.getLatestAttemptExecutionInfo();
StageExecutionStats stageExecutionStats = currentStageExecutionInfo.getStats();

ImmutableList.Builder<StageStats> subStages = ImmutableList.builder();
for (StageInfo subStage : stageInfo.getSubStages()) {
subStages.add(toStageStats(subStage));
}

Set<String> uniqueNodes = new HashSet<>();
for (TaskInfo task : currentStageExecutionInfo.getTasks()) {
// todo add nodeId to TaskInfo
URI uri = task.getTaskStatus().getSelf();
uniqueNodes.add(uri.getHost() + ":" + uri.getPort());
}

return StageStats.builder()
.setStageId(String.valueOf(stageInfo.getStageId().getId()))
.setState(currentStageExecutionInfo.getState().toString())
.setDone(currentStageExecutionInfo.getState().isDone())
.setNodes(uniqueNodes.size())
.setTotalSplits(stageExecutionStats.getTotalDrivers())
.setQueuedSplits(stageExecutionStats.getQueuedDrivers())
.setRunningSplits(stageExecutionStats.getRunningDrivers() + stageExecutionStats.getBlockedDrivers())
.setCompletedSplits(stageExecutionStats.getCompletedDrivers())
.setCpuTimeMillis(stageExecutionStats.getTotalCpuTime().toMillis())
.setWallTimeMillis(stageExecutionStats.getTotalScheduledTime().toMillis())
.setProcessedRows(stageExecutionStats.getRawInputPositions())
.setProcessedBytes(stageExecutionStats.getRawInputDataSize().toBytes())
.setSubStages(subStages.build())
.build();
}

private static Set<String> globalUniqueNodes(StageInfo stageInfo)
{
if (stageInfo == null) {
return ImmutableSet.of();
}
ImmutableSet.Builder<String> nodes = ImmutableSet.builder();
for (TaskInfo task : stageInfo.getLatestAttemptExecutionInfo().getTasks()) {
// todo add nodeId to TaskInfo
URI uri = task.getTaskStatus().getSelf();
nodes.add(uri.getHost() + ":" + uri.getPort());
}

for (StageInfo subStage : stageInfo.getSubStages()) {
nodes.addAll(globalUniqueNodes(subStage));
}
return nodes.build();
}

private static URI findCancelableLeafStage(QueryInfo queryInfo)
{
// if query is running, find the leaf-most running stage
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,12 +14,26 @@
package com.facebook.presto.server.protocol;

import com.facebook.presto.client.QueryResults;
import com.facebook.presto.client.StageStats;
import com.facebook.presto.client.StatementStats;
import com.facebook.presto.execution.QueryInfo;
import com.facebook.presto.execution.QueryState;
import com.facebook.presto.execution.QueryStats;
import com.facebook.presto.execution.StageExecutionInfo;
import com.facebook.presto.execution.StageExecutionStats;
import com.facebook.presto.execution.StageInfo;
import com.facebook.presto.execution.TaskInfo;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;

import javax.ws.rs.core.Response;

import java.io.UnsupportedEncodingException;
import java.net.URI;
import java.net.URLEncoder;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;

import static com.facebook.presto.client.PrestoHeaders.PRESTO_ADDED_PREPARE;
import static com.facebook.presto.client.PrestoHeaders.PRESTO_CLEAR_SESSION;
Expand Down Expand Up @@ -79,6 +93,34 @@ public static Response toResponse(Query query, QueryResults queryResults)
return response.build();
}

public static StatementStats toStatementStats(QueryInfo queryInfo)
{
QueryStats queryStats = queryInfo.getQueryStats();
StageInfo outputStage = queryInfo.getOutputStage().orElse(null);

return StatementStats.builder()
.setState(queryInfo.getState().toString())
.setQueued(queryInfo.getState() == QueryState.QUEUED)
.setScheduled(queryInfo.isScheduled())
.setNodes(globalUniqueNodes(outputStage).size())
.setTotalSplits(queryStats.getTotalDrivers())
.setQueuedSplits(queryStats.getQueuedDrivers())
.setRunningSplits(queryStats.getRunningDrivers() + queryStats.getBlockedDrivers())
.setCompletedSplits(queryStats.getCompletedDrivers())
.setCpuTimeMillis(queryStats.getTotalCpuTime().toMillis())
.setWallTimeMillis(queryStats.getTotalScheduledTime().toMillis())
.setQueuedTimeMillis(queryStats.getQueuedTime().toMillis())
.setElapsedTimeMillis(queryStats.getElapsedTime().toMillis())
.setProcessedRows(queryStats.getRawInputPositions())
.setProcessedBytes(queryStats.getRawInputDataSize().toBytes())
.setPeakMemoryBytes(queryStats.getPeakUserMemoryReservation().toBytes())
.setPeakTotalMemoryBytes(queryStats.getPeakTotalMemoryReservation().toBytes())
.setPeakTaskTotalMemoryBytes(queryStats.getPeakTaskTotalMemory().toBytes())
.setSpilledBytes(queryStats.getSpilledDataSize().toBytes())
.setRootStage(toStageStats(outputStage))
.build();
}

private static String urlEncode(String value)
{
try {
Expand All @@ -88,4 +130,60 @@ private static String urlEncode(String value)
throw new AssertionError(e);
}
}

private static Set<String> globalUniqueNodes(StageInfo stageInfo)
{
if (stageInfo == null) {
return ImmutableSet.of();
}
ImmutableSet.Builder<String> nodes = ImmutableSet.builder();
for (TaskInfo task : stageInfo.getLatestAttemptExecutionInfo().getTasks()) {
// todo add nodeId to TaskInfo
URI uri = task.getTaskStatus().getSelf();
nodes.add(uri.getHost() + ":" + uri.getPort());
}

for (StageInfo subStage : stageInfo.getSubStages()) {
nodes.addAll(globalUniqueNodes(subStage));
}
return nodes.build();
}

private static StageStats toStageStats(StageInfo stageInfo)
{
if (stageInfo == null) {
return null;
}

StageExecutionInfo currentStageExecutionInfo = stageInfo.getLatestAttemptExecutionInfo();
StageExecutionStats stageExecutionStats = currentStageExecutionInfo.getStats();

ImmutableList.Builder<StageStats> subStages = ImmutableList.builder();
for (StageInfo subStage : stageInfo.getSubStages()) {
subStages.add(toStageStats(subStage));
}

Set<String> uniqueNodes = new HashSet<>();
for (TaskInfo task : currentStageExecutionInfo.getTasks()) {
// todo add nodeId to TaskInfo
URI uri = task.getTaskStatus().getSelf();
uniqueNodes.add(uri.getHost() + ":" + uri.getPort());
}

return StageStats.builder()
.setStageId(String.valueOf(stageInfo.getStageId().getId()))
.setState(currentStageExecutionInfo.getState().toString())
.setDone(currentStageExecutionInfo.getState().isDone())
.setNodes(uniqueNodes.size())
.setTotalSplits(stageExecutionStats.getTotalDrivers())
.setQueuedSplits(stageExecutionStats.getQueuedDrivers())
.setRunningSplits(stageExecutionStats.getRunningDrivers() + stageExecutionStats.getBlockedDrivers())
.setCompletedSplits(stageExecutionStats.getCompletedDrivers())
.setCpuTimeMillis(stageExecutionStats.getTotalCpuTime().toMillis())
.setWallTimeMillis(stageExecutionStats.getTotalScheduledTime().toMillis())
.setProcessedRows(stageExecutionStats.getRawInputPositions())
.setProcessedBytes(stageExecutionStats.getRawInputDataSize().toBytes())
.setSubStages(subStages.build())
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,12 @@ public Set<String> getCustomEventClientTypes()
return ImmutableSet.of();
}

@Override
public Set<String> getCustomQueryActionTypes()
{
return ImmutableSet.of();
}

@Override
public List<Class<? extends Predicate<SourceQuery>>> getCustomQueryFilterClasses()
{
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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.verifier.annotation;

import javax.inject.Qualifier;

import java.lang.annotation.Retention;
import java.lang.annotation.Target;

import static java.lang.annotation.ElementType.FIELD;
import static java.lang.annotation.ElementType.METHOD;
import static java.lang.annotation.ElementType.PARAMETER;
import static java.lang.annotation.RetentionPolicy.RUNTIME;

@Retention(RUNTIME)
@Target({FIELD, PARAMETER, METHOD})
@Qualifier
public @interface ForHelper
{
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,13 +43,14 @@ public class QueryInfo
private final Double wallTimeSecs;
private final Long peakTotalMemoryBytes;
private final Long peakTaskTotalMemoryBytes;
private final QueryStatsEvent queryStats;

public QueryInfo(
String catalog,
String schema,
String originalQuery)
{
this(catalog, schema, originalQuery, Optional.empty(), ImmutableList.of(), ImmutableList.of(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty());
this(catalog, schema, originalQuery, Optional.empty(), ImmutableList.of(), ImmutableList.of(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty());
}

public QueryInfo(
Expand All @@ -67,7 +68,8 @@ public QueryInfo(
Optional<Double> cpuTimeSecs,
Optional<Double> wallTimeSecs,
Optional<Long> peakTotalMemoryBytes,
Optional<Long> peakTaskTotalMemoryBytes)
Optional<Long> peakTaskTotalMemoryBytes,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wow. why not using OptionalLong/OptionalDouble? -- this is irrelevant to this PR though

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since we have everything in the QueryStatsEvent, we'll eventually deprecate those fields.

Optional<QueryStatsEvent> queryStats)
{
this.catalog = requireNonNull(catalog, "catalog is null");
this.schema = requireNonNull(schema, "schema is null");
Expand All @@ -84,6 +86,7 @@ public QueryInfo(
this.wallTimeSecs = wallTimeSecs.orElse(null);
this.peakTotalMemoryBytes = peakTotalMemoryBytes.orElse(null);
this.peakTaskTotalMemoryBytes = peakTaskTotalMemoryBytes.orElse(null);
this.queryStats = queryStats.orElse(null);
}

@EventField
Expand Down Expand Up @@ -175,4 +178,10 @@ public Long getPeakTaskTotalMemoryBytes()
{
return peakTaskTotalMemoryBytes;
}

@EventField
public QueryStatsEvent getQueryStats()
{
return queryStats;
}
}
Loading