Skip to content

Commit

Permalink
Unify fields in QueryInfo and BasicQueryInfo
Browse files Browse the repository at this point in the history
This also introduces BasicQueryStats, which is the lightweight version
of QueryStats. This reduces duplication by allowing client side code
to assume that both basic and full query info and stats objects have
the required fields to render the Web UI.
  • Loading branch information
Raghav Sethi committed Oct 10, 2016
1 parent b1a6a67 commit 03071f8
Show file tree
Hide file tree
Showing 6 changed files with 435 additions and 265 deletions.
182 changes: 22 additions & 160 deletions presto-main/src/main/java/com/facebook/presto/server/BasicQueryInfo.java
Expand Up @@ -16,78 +16,49 @@
import com.facebook.presto.SessionRepresentation;
import com.facebook.presto.execution.QueryInfo;
import com.facebook.presto.execution.QueryState;
import com.facebook.presto.operator.BlockedReason;
import com.facebook.presto.spi.ErrorCode;
import com.facebook.presto.spi.ErrorType;
import com.facebook.presto.spi.QueryId;
import com.facebook.presto.spi.memory.MemoryPoolId;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableSet;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import org.joda.time.DateTime;

import javax.annotation.Nullable;
import javax.annotation.concurrent.Immutable;

import java.net.URI;
import java.util.Set;

import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
import static java.util.Objects.requireNonNull;

/**
* Lightweight version of QueryInfo. Parts of the web UI depend on the fields
* being named consistently across these classes.
*/
@Immutable
public class BasicQueryInfo
{
private final QueryId queryId;
private final SessionRepresentation session;
private final QueryState state;
private final MemoryPoolId memoryPool;
private final ErrorType errorType;
private final ErrorCode errorCode;
private final boolean scheduled;
private final boolean fullyBlocked;
private final Set<BlockedReason> blockedReasons;
private final URI self;
private final String query;
private final Duration elapsedTime;
private final Duration executionTime;
private final Duration cpuTime;
private final DateTime endTime;
private final DateTime createTime;
private final DataSize currentMemory;
private final DataSize peakMemory;
private final double cumulativeMemory;
private final int runningDrivers;
private final int queuedDrivers;
private final int completedDrivers;
private final int totalDrivers;
private final BasicQueryStats queryStats;
private final ErrorType errorType;
private final ErrorCode errorCode;

public BasicQueryInfo(
QueryId queryId,
SessionRepresentation session,
QueryState state,
MemoryPoolId memoryPool,
ErrorType errorType,
ErrorCode errorCode,
boolean scheduled,
boolean fullyBlocked,
Set<BlockedReason> blockedReasons,
URI self,
String query,
Duration elapsedTime,
Duration executionTime,
Duration cpuTime,
DateTime endTime,
DateTime createTime,
DataSize currentMemory,
DataSize peakMemory,
double cumulativeMemory,
int runningDrivers,
int queuedDrivers,
int completedDrivers,
int totalDrivers)
BasicQueryStats queryStats,
ErrorType errorType,
ErrorCode errorCode)
{
this.queryId = requireNonNull(queryId, "queryId is null");
this.session = requireNonNull(session, "session is null");
Expand All @@ -96,27 +67,9 @@ public BasicQueryInfo(
this.errorType = errorType;
this.errorCode = errorCode;
this.scheduled = scheduled;
this.fullyBlocked = fullyBlocked;
this.blockedReasons = ImmutableSet.copyOf(requireNonNull(blockedReasons, "blockedReasons is null"));
this.self = requireNonNull(self, "self is null");
this.query = requireNonNull(query, "query is null");
this.elapsedTime = elapsedTime;
this.executionTime = executionTime;
this.cpuTime = cpuTime;
this.endTime = endTime;
this.createTime = createTime;
this.currentMemory = currentMemory;
this.peakMemory = peakMemory;
this.cumulativeMemory = cumulativeMemory;

checkArgument(runningDrivers >= 0, "runningDrivers is less than zero");
this.runningDrivers = runningDrivers;
checkArgument(queuedDrivers >= 0, "queuedDrivers is less than zero");
this.queuedDrivers = queuedDrivers;
checkArgument(completedDrivers >= 0, "completedDrivers is less than zero");
this.completedDrivers = completedDrivers;
checkArgument(totalDrivers >= 0, "totalDrivers is less than zero");
this.totalDrivers = totalDrivers;
this.queryStats = requireNonNull(queryStats, "queryStats is null");
}

public BasicQueryInfo(QueryInfo queryInfo)
Expand All @@ -125,25 +78,12 @@ public BasicQueryInfo(QueryInfo queryInfo)
queryInfo.getSession(),
queryInfo.getState(),
queryInfo.getMemoryPool(),
queryInfo.getErrorType(),
queryInfo.getErrorCode(),
queryInfo.isScheduled(),
queryInfo.getQueryStats().isFullyBlocked(),
queryInfo.getQueryStats().getBlockedReasons(),
queryInfo.getSelf(),
queryInfo.getQuery(),
queryInfo.getQueryStats().getElapsedTime(),
queryInfo.getQueryStats().getExecutionTime(),
queryInfo.getQueryStats().getTotalCpuTime(),
queryInfo.getQueryStats().getEndTime(),
queryInfo.getQueryStats().getCreateTime(),
queryInfo.getQueryStats().getTotalMemoryReservation(),
queryInfo.getQueryStats().getPeakMemoryReservation(),
queryInfo.getQueryStats().getCumulativeMemory(),
queryInfo.getQueryStats().getRunningDrivers(),
queryInfo.getQueryStats().getQueuedDrivers(),
queryInfo.getQueryStats().getCompletedDrivers(),
queryInfo.getQueryStats().getTotalDrivers());
new BasicQueryStats(queryInfo.getQueryStats()),
queryInfo.getErrorType(),
queryInfo.getErrorCode());
}

@JsonProperty
Expand All @@ -170,38 +110,12 @@ public MemoryPoolId getMemoryPool()
return memoryPool;
}

@Nullable
@JsonProperty
public ErrorType getErrorType()
{
return errorType;
}

@Nullable
@JsonProperty
public ErrorCode getErrorCode()
{
return errorCode;
}

@JsonProperty
public boolean isScheduled()
{
return scheduled;
}

@JsonProperty
public boolean isFullyBlocked()
{
return fullyBlocked;
}

@JsonProperty
public Set<BlockedReason> getBlockedReasons()
{
return blockedReasons;
}

@JsonProperty
public URI getSelf()
{
Expand All @@ -215,75 +129,23 @@ public String getQuery()
}

@JsonProperty
public long getExecutionTimeMillis()
{
return executionTime.toMillis();
}

@JsonProperty
public long getCpuTimeMillis()
{
return cpuTime.toMillis();
}

@JsonProperty
public long getElapsedTimeMillis()
{
return elapsedTime.toMillis();
}

@JsonProperty
public DateTime getEndTime()
{
return endTime;
}

@JsonProperty
public int getRunningDrivers()
{
return runningDrivers;
}

@JsonProperty
public int getQueuedDrivers()
{
return queuedDrivers;
}

@JsonProperty
public int getTotalDrivers()
public BasicQueryStats getQueryStats()
{
return totalDrivers;
}

@JsonProperty
public int getCompletedDrivers()
{
return completedDrivers;
}

@JsonProperty
public DateTime getCreateTime()
{
return createTime;
}

@JsonProperty
public double getCumulativeMemory()
{
return cumulativeMemory;
return queryStats;
}

@Nullable
@JsonProperty
public long getCurrentMemoryBytes()
public ErrorType getErrorType()
{
return currentMemory.toBytes();
return errorType;
}

@Nullable
@JsonProperty
public long getPeakMemoryBytes()
public ErrorCode getErrorCode()
{
return peakMemory.toBytes();
return errorCode;
}

@Override
Expand Down

0 comments on commit 03071f8

Please sign in to comment.