Skip to content

Commit

Permalink
Store cpu and wall time for queries run in verifier
Browse files Browse the repository at this point in the history
For INSERT queries the real query is run at "pre" phase, while the main
test query is merely a check sum computation. Storing the total time
spent on running all test or control related queries will make it less
confusing.
  • Loading branch information
arhimondr committed Apr 29, 2022
1 parent 8be7fee commit 5e28b5b
Show file tree
Hide file tree
Showing 5 changed files with 91 additions and 6 deletions.
Expand Up @@ -16,11 +16,13 @@
import io.airlift.event.client.AbstractEventClient;
import io.airlift.json.JsonCodec;

import javax.annotation.Nullable;
import javax.annotation.PostConstruct;
import javax.inject.Inject;

import java.util.List;
import java.util.Optional;
import java.util.OptionalDouble;

import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -58,12 +60,24 @@ protected <T> void postEvent(T event)
queryEvent.getTestSetupQueryIds().isEmpty() ? Optional.empty() : Optional.of(codec.toJson(queryEvent.getTestSetupQueryIds())),
Optional.ofNullable(queryEvent.getTestQueryId()),
queryEvent.getTestTeardownQueryIds().isEmpty() ? Optional.empty() : Optional.of(codec.toJson(queryEvent.getTestTeardownQueryIds())),
toOptionalDouble(queryEvent.getTestCpuTimeSecs()),
toOptionalDouble(queryEvent.getTestWallTimeSecs()),
Optional.ofNullable(queryEvent.getControlCatalog()),
Optional.ofNullable(queryEvent.getControlSchema()),
queryEvent.getControlSetupQueryIds().isEmpty() ? Optional.empty() : Optional.of(codec.toJson(queryEvent.getControlSetupQueryIds())),
Optional.ofNullable(queryEvent.getControlQueryId()),
queryEvent.getControlTeardownQueryIds().isEmpty() ? Optional.empty() : Optional.of(codec.toJson(queryEvent.getControlTeardownQueryIds())),
toOptionalDouble(queryEvent.getControlCpuTimeSecs()),
toOptionalDouble(queryEvent.getControlWallTimeSecs()),
Optional.ofNullable(queryEvent.getErrorMessage()));
dao.store(entity);
}

private static OptionalDouble toOptionalDouble(@Nullable Double value)
{
if (value == null) {
return OptionalDouble.empty();
}
return OptionalDouble.of(value);
}
}
Expand Up @@ -21,6 +21,8 @@
import io.trino.spi.ErrorCode;
import io.trino.spi.TrinoException;

import javax.annotation.Nullable;

import java.io.Closeable;
import java.io.IOException;
import java.io.PrintStream;
Expand All @@ -30,15 +32,19 @@
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.OptionalDouble;
import java.util.Set;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Stream;

import static com.google.common.base.Throwables.getStackTraceAsString;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.Streams.concat;
import static io.trino.spi.StandardErrorCode.PAGE_TRANSPORT_TIMEOUT;
import static io.trino.spi.StandardErrorCode.REMOTE_TASK_MISMATCH;
import static io.trino.spi.StandardErrorCode.TOO_MANY_REQUESTS_FAILED;
Expand Down Expand Up @@ -237,6 +243,9 @@ private VerifierQueryEvent buildEvent(Validator validator)
}
}

Stream<QueryResult> controlQueries = concat(Stream.of(control), validator.getControlPreQueryResults().stream(), validator.getControlPostQueryResults().stream());
Stream<QueryResult> testQueries = concat(Stream.of(test), validator.getTestPreQueryResults().stream(), validator.getTestPostQueryResults().stream());

return new VerifierQueryEvent(
queryPair.getSuite(),
config.getRunId(),
Expand All @@ -257,8 +266,8 @@ private VerifierQueryEvent buildEvent(Validator validator)
.map(QueryResult::getQueryId)
.filter(Objects::nonNull)
.collect(toImmutableList()),
optionalDurationToSeconds(test.getCpuTime()),
optionalDurationToSeconds(test.getWallTime()),
getTotalDurationInSeconds(testQueries, QueryResult::getCpuTime),
getTotalDurationInSeconds(testQueries, QueryResult::getWallTime),
queryPair.getControl().getCatalog(),
queryPair.getControl().getSchema(),
queryPair.getControl().getPreQueries(),
Expand All @@ -273,14 +282,23 @@ private VerifierQueryEvent buildEvent(Validator validator)
.map(QueryResult::getQueryId)
.filter(Objects::nonNull)
.collect(toImmutableList()),
optionalDurationToSeconds(control.getCpuTime()),
optionalDurationToSeconds(control.getWallTime()),
getTotalDurationInSeconds(controlQueries, QueryResult::getCpuTime),
getTotalDurationInSeconds(controlQueries, QueryResult::getWallTime),
errorMessage);
}

private static Double optionalDurationToSeconds(Duration duration)
@Nullable
private static Double getTotalDurationInSeconds(Stream<QueryResult> queries, Function<QueryResult, Duration> metric)
{
return duration != null ? duration.convertTo(SECONDS).getValue() : null;
OptionalDouble result = queries
.map(metric)
.filter(Objects::nonNull)
.mapToDouble(duration -> duration.getValue(SECONDS))
.reduce(Double::sum);
if (result.isEmpty()) {
return null;
}
return result.getAsDouble();
}

private static <T> T takeUnchecked(CompletionService<T> completionService)
Expand Down
Expand Up @@ -30,11 +30,15 @@ public interface VerifierQueryEventDao
" test_setup_query_ids_json VARCHAR(255) NULL,\n" +
" test_query_id VARCHAR(255) NULL,\n" +
" test_teardown_query_ids_json VARCHAR(255) NULL,\n" +
" test_cpu_time_seconds DOUBLE NULL,\n" +
" test_wall_time_seconds DOUBLE NULL,\n" +
" control_catalog VARCHAR(255) NULL,\n" +
" control_schema VARCHAR(255) NULL,\n" +
" control_setup_query_ids_json VARCHAR(255) NULL,\n" +
" control_query_id VARCHAR(255) NULL,\n" +
" control_teardown_query_ids_json VARCHAR(255) NULL,\n" +
" control_cpu_time_seconds DOUBLE NULL,\n" +
" control_wall_time_seconds DOUBLE NULL,\n" +
" error_message TEXT NULL,\n" +
" PRIMARY KEY (id),\n" +
" INDEX run_id_name_index(run_id, name)\n" +
Expand All @@ -52,11 +56,15 @@ public interface VerifierQueryEventDao
" test_setup_query_ids_json,\n" +
" test_query_id,\n" +
" test_teardown_query_ids_json,\n" +
" test_cpu_time_seconds,\n" +
" test_wall_time_seconds,\n" +
" control_catalog,\n" +
" control_schema,\n" +
" control_setup_query_ids_json,\n" +
" control_query_id,\n" +
" control_teardown_query_ids_json,\n" +
" control_cpu_time_seconds,\n" +
" control_wall_time_seconds,\n" +
" error_message\n" +
")\n" +
"VALUES (\n" +
Expand All @@ -70,11 +78,15 @@ public interface VerifierQueryEventDao
" :testSetupQueryIdsJson,\n" +
" :testQueryId,\n" +
" :testTeardownQueryIdsJson,\n" +
" :testCpuTimeSeconds,\n" +
" :testWallTimeSeconds,\n" +
" :controlCatalog,\n" +
" :controlSchema,\n" +
" :controlSetupQueryIdsJson,\n" +
" :controlQueryId,\n" +
" :controlTeardownQueryIdsJson,\n" +
" :controlCpuTimeSeconds,\n" +
" :controlWallTimeSeconds,\n" +
" :errorMessage\n" +
")")
void store(@BindBean VerifierQueryEventEntity entity);
Expand Down
Expand Up @@ -14,6 +14,7 @@
package io.trino.verifier;

import java.util.Optional;
import java.util.OptionalDouble;

import static java.util.Objects.requireNonNull;

Expand All @@ -30,12 +31,16 @@ public class VerifierQueryEventEntity
private final Optional<String> testSetupQueryIdsJson;
private final Optional<String> testQueryId;
private final Optional<String> testTeardownQueryIdsJson;
private final OptionalDouble testCpuTimeSeconds;
private final OptionalDouble testWallTimeSeconds;

private final Optional<String> controlCatalog;
private final Optional<String> controlSchema;
private final Optional<String> controlSetupQueryIdsJson;
private final Optional<String> controlQueryId;
private final Optional<String> controlTeardownQueryIdsJson;
private final OptionalDouble controlCpuTimeSeconds;
private final OptionalDouble controlWallTimeSeconds;

private final Optional<String> errorMessage;

Expand All @@ -50,11 +55,15 @@ public VerifierQueryEventEntity(
Optional<String> testSetupQueryIdsJson,
Optional<String> testQueryId,
Optional<String> testTeardownQueryIdsJson,
OptionalDouble testCpuTimeSeconds,
OptionalDouble testWallTimeSeconds,
Optional<String> controlCatalog,
Optional<String> controlSchema,
Optional<String> controlSetupQueryIdsJson,
Optional<String> controlQueryId,
Optional<String> controlTeardownQueryIdsJson,
OptionalDouble controlCpuTimeSeconds,
OptionalDouble controlWallTimeSeconds,
Optional<String> errorMessage)
{
this.suite = requireNonNull(suite, "suite is null");
Expand All @@ -67,11 +76,15 @@ public VerifierQueryEventEntity(
this.testSetupQueryIdsJson = requireNonNull(testSetupQueryIdsJson, "testSetupQueryIdsJson is null");
this.testQueryId = requireNonNull(testQueryId, "testQueryId is null");
this.testTeardownQueryIdsJson = requireNonNull(testTeardownQueryIdsJson, "testTeardownQueryIdsJson is null");
this.testCpuTimeSeconds = requireNonNull(testCpuTimeSeconds, "testCpuTimeSeconds is null");
this.testWallTimeSeconds = requireNonNull(testWallTimeSeconds, "testWallTimeSeconds is null");
this.controlCatalog = requireNonNull(controlCatalog, "controlCatalog is null");
this.controlSchema = requireNonNull(controlSchema, "controlSchema is null");
this.controlSetupQueryIdsJson = requireNonNull(controlSetupQueryIdsJson, "controlSetupQueryIdsJson is null");
this.controlQueryId = requireNonNull(controlQueryId, "controlQueryId is null");
this.controlTeardownQueryIdsJson = requireNonNull(controlTeardownQueryIdsJson, "controlTeardownQueryIdsJson is null");
this.controlCpuTimeSeconds = requireNonNull(controlCpuTimeSeconds, "controlCpuTimeSeconds is null");
this.controlWallTimeSeconds = requireNonNull(controlWallTimeSeconds, "controlWallTimeSeconds is null");
this.errorMessage = requireNonNull(errorMessage, "errorMessage is null");
}

Expand Down Expand Up @@ -125,6 +138,16 @@ public Optional<String> getTestTeardownQueryIdsJson()
return testTeardownQueryIdsJson;
}

public OptionalDouble getTestCpuTimeSeconds()
{
return testCpuTimeSeconds;
}

public OptionalDouble getTestWallTimeSeconds()
{
return testWallTimeSeconds;
}

public Optional<String> getControlCatalog()
{
return controlCatalog;
Expand All @@ -150,6 +173,16 @@ public Optional<String> getControlTeardownQueryIdsJson()
return controlTeardownQueryIdsJson;
}

public OptionalDouble getControlCpuTimeSeconds()
{
return controlCpuTimeSeconds;
}

public OptionalDouble getControlWallTimeSeconds()
{
return controlWallTimeSeconds;
}

public Optional<String> getErrorMessage()
{
return errorMessage;
Expand Down
Expand Up @@ -153,11 +153,15 @@ public void testFull()
assertEquals(resultSet.getString("test_setup_query_ids_json"), codec.toJson(FULL_EVENT.getTestSetupQueryIds()));
assertEquals(resultSet.getString("test_query_id"), "TEST_QUERY_ID");
assertEquals(resultSet.getString("test_teardown_query_ids_json"), codec.toJson(FULL_EVENT.getTestTeardownQueryIds()));
assertEquals(resultSet.getDouble("test_cpu_time_seconds"), 1.1);
assertEquals(resultSet.getDouble("test_wall_time_seconds"), 2.2);
assertEquals(resultSet.getString("control_catalog"), "controlcatalog");
assertEquals(resultSet.getString("control_schema"), "controlschema");
assertEquals(resultSet.getString("control_setup_query_ids_json"), codec.toJson(FULL_EVENT.getControlSetupQueryIds()));
assertEquals(resultSet.getString("control_query_id"), "CONTROL_QUERY_ID");
assertEquals(resultSet.getString("control_teardown_query_ids_json"), codec.toJson(FULL_EVENT.getControlTeardownQueryIds()));
assertEquals(resultSet.getDouble("control_cpu_time_seconds"), 3.3);
assertEquals(resultSet.getDouble("control_wall_time_seconds"), 4.4);
assertEquals(resultSet.getString("error_message"), "error message");
assertFalse(resultSet.next());
}
Expand Down Expand Up @@ -186,11 +190,15 @@ public void testMinimal()
assertNull(resultSet.getString("test_setup_query_ids_json"));
assertNull(resultSet.getString("test_query_id"));
assertNull(resultSet.getString("test_teardown_query_ids_json"));
assertNull(resultSet.getObject("test_cpu_time_seconds"));
assertNull(resultSet.getObject("test_wall_time_seconds"));
assertNull(resultSet.getString("control_catalog"));
assertNull(resultSet.getString("control_schema"));
assertNull(resultSet.getString("control_setup_query_ids_json"));
assertNull(resultSet.getString("control_query_id"));
assertNull(resultSet.getString("control_teardown_query_ids_json"));
assertNull(resultSet.getObject("control_cpu_time_seconds"));
assertNull(resultSet.getObject("control_wall_time_seconds"));
assertNull(resultSet.getString("error_message"));
assertFalse(resultSet.next());
}
Expand Down

0 comments on commit 5e28b5b

Please sign in to comment.