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 @@ -20,26 +20,50 @@
import java.util.Set;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static java.util.Objects.requireNonNull;

public class VerificationContext
{
private final String sourceQueryName;
Copy link
Contributor

Choose a reason for hiding this comment

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

Just double check: is this the original query id? :)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Theoretically this could be anything provided in the xdb, but yes it is the original query ID for FB use cases.

private final String suite;

private final int resubmissionCount;
private final Set<QueryException> queryExceptions;

private VerificationContext(int resubmissionCount, Set<QueryException> queryExceptions)
private VerificationContext(
String sourceQueryName,
String suite,
int resubmissionCount,
Set<QueryException> queryExceptions)
{
this.sourceQueryName = requireNonNull(sourceQueryName, "sourceQueryName is null");
this.suite = requireNonNull(suite, "suite is null");
this.resubmissionCount = resubmissionCount;
this.queryExceptions = new HashSet<>(queryExceptions);
}

public static VerificationContext create()
public static VerificationContext create(String sourceQueryName, String suite)
{
return new VerificationContext(0, new HashSet<>());
return new VerificationContext(sourceQueryName, suite, 0, new HashSet<>());
}

public static VerificationContext createForResubmission(VerificationContext existing)
{
return new VerificationContext(existing.resubmissionCount + 1, existing.queryExceptions);
return new VerificationContext(
existing.sourceQueryName,
existing.suite,
existing.resubmissionCount + 1,
existing.queryExceptions);
}

public String getSourceQueryName()
{
return sourceQueryName;
}

public String getSuite()
{
return suite;
}

public int getResubmissionCount()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,8 @@ public Verification get(SourceQuery sourceQuery, Optional<VerificationContext> e
QueryType queryType = QueryType.of(sqlParser.createStatement(sourceQuery.getControlQuery(), PARSING_OPTIONS));
switch (queryType.getCategory()) {
case DATA_PRODUCING:
VerificationContext verificationContext = existingContext.map(VerificationContext::createForResubmission).orElseGet(VerificationContext::create);
VerificationContext verificationContext = existingContext.map(VerificationContext::createForResubmission)
.orElseGet(() -> VerificationContext.create(sourceQuery.getName(), sourceQuery.getSuite()));
QueryActions queryActions = queryActionsFactory.create(sourceQuery, verificationContext);
QueryRewriter queryRewriter = queryRewriterFactory.create(queryActions.getHelperAction());
DeterminismAnalyzer determinismAnalyzer = new DeterminismAnalyzer(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ public class VerifierConfig
private Optional<String> humanReadableEventLogFile = Optional.empty();

private String testId;
private Optional<String> testName = Optional.empty();
private int maxConcurrency = 10;
private int suiteRepetitions = 1;
private int queryRepetitions = 1;
Expand Down Expand Up @@ -144,14 +145,28 @@ public String getTestId()
return testId;
}

@ConfigDescription("A customizable string that will be logged with the results")
@ConfigDescription("A customizable string that will be passed into query client info and logged with the results")
@Config("test-id")
public VerifierConfig setTestId(String testId)
{
this.testId = testId;
return this;
}

@NotNull
public Optional<String> getTestName()
{
return testName;
}

@ConfigDescription("A customizable string that will be passed into query client info")
@Config("test-name")
public VerifierConfig setTestName(String testName)
{
this.testName = Optional.ofNullable(testName);
return this;
}

@Min(1)
public int getMaxConcurrency()
{
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* 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.prestoaction;

import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;

import java.util.Optional;

import static java.util.Objects.requireNonNull;

public class ClientInfo
{
private static final String CLIENT_INFO_TYPE = "VERIFIER";

private final String testId;
private final Optional<String> testName;
private final String sourceQueryName;
private final String suite;

public ClientInfo(String testId, Optional<String> testName, String sourceQueryName, String suite)
{
this.testId = requireNonNull(testId, "testId is null");
this.testName = requireNonNull(testName, "testName is null");
this.sourceQueryName = requireNonNull(sourceQueryName, "sourceQueryName is null");
this.suite = requireNonNull(suite, "suite is null");
}

@JsonProperty
public String getType()
{
return CLIENT_INFO_TYPE;
}

@JsonProperty
public String getTestId()
{
return testId;
}

@JsonProperty
public Optional<String> getTestName()
{
return testName;
}

@JsonProperty
public String getSourceQueryName()
{
return sourceQueryName;
}

@JsonProperty
public String getSuite()
{
return suite;
}

public String serialize()
{
try {
return new ObjectMapper().writeValueAsString(this);
Copy link
Contributor

@wenleix wenleix Aug 1, 2020

Choose a reason for hiding this comment

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

A common pattern in Presto code base is to have a private static final OBJECT_MAPPER in this class 😃 . But either way is fine :)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sorry, I missed this comment, will piggy-back into a future PR.

}
catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package com.facebook.presto.verifier.prestoaction;

import com.facebook.airlift.json.JsonCodec;
import com.facebook.presto.jdbc.PrestoConnection;
import com.facebook.presto.jdbc.PrestoStatement;
import com.facebook.presto.jdbc.QueryStats;
Expand All @@ -25,6 +26,7 @@
import com.facebook.presto.verifier.framework.QueryResult;
import com.facebook.presto.verifier.framework.QueryStage;
import com.facebook.presto.verifier.framework.VerificationContext;
import com.facebook.presto.verifier.framework.VerifierConfig;
import com.facebook.presto.verifier.retry.ForClusterConnection;
import com.facebook.presto.verifier.retry.ForPresto;
import com.facebook.presto.verifier.retry.RetryConfig;
Expand All @@ -41,6 +43,7 @@
import java.util.Optional;
import java.util.function.Consumer;

import static com.facebook.airlift.json.JsonCodec.jsonCodec;
import static com.facebook.presto.sql.SqlFormatter.formatSql;
import static com.facebook.presto.verifier.prestoaction.QueryActionUtil.mangleSessionProperties;
import static com.google.common.base.Preconditions.checkState;
Expand All @@ -50,14 +53,18 @@ public class JdbcPrestoAction
implements PrestoAction
{
public static final String QUERY_ACTION_TYPE = "presto-jdbc";
private static final JsonCodec<ClientInfo> CLIENT_INFO_JSON_CODEC = jsonCodec(ClientInfo.class);

private final SqlExceptionClassifier exceptionClassifier;
private final QueryConfiguration queryConfiguration;
private final VerificationContext verificationContext;

private final String jdbcUrl;
private final Duration queryTimeout;
private final Duration metadataTimeout;
private final Duration checksumTimeout;
private final String testId;
private final Optional<String> testName;

private final RetryDriver<QueryException> networkRetry;
private final RetryDriver<QueryException> prestoRetry;
Expand All @@ -70,15 +77,19 @@ public JdbcPrestoAction(
Duration metadataTimeout,
Duration checksumTimeout,
@ForClusterConnection RetryConfig networkRetryConfig,
@ForPresto RetryConfig prestoRetryConfig)
@ForPresto RetryConfig prestoRetryConfig,
VerifierConfig verifierConfig)
{
this.exceptionClassifier = requireNonNull(exceptionClassifier, "exceptionClassifier is null");
this.queryConfiguration = requireNonNull(queryConfiguration, "queryConfiguration is null");
this.verificationContext = requireNonNull(verificationContext, "verificationContext is null");

this.jdbcUrl = requireNonNull(prestoActionConfig.getJdbcUrl(), "jdbcUrl is null");
this.queryTimeout = requireNonNull(prestoActionConfig.getQueryTimeout(), "queryTimeout is null");
this.metadataTimeout = requireNonNull(metadataTimeout, "metadataTimeout is null");
this.checksumTimeout = requireNonNull(checksumTimeout, "checksumTimeout is null");
this.testId = requireNonNull(verifierConfig.getTestId(), "testId is null");
this.testName = requireNonNull(verifierConfig.getTestName(), "testName is null");

this.networkRetry = new RetryDriver<>(
networkRetryConfig,
Expand Down Expand Up @@ -116,8 +127,13 @@ private <T> T execute(Statement statement, QueryStage queryStage, StatementExecu
private <T> T executeOnce(Statement statement, QueryStage queryStage, StatementExecutor<T> statementExecutor)
{
String query = formatSql(statement, Optional.empty());
String clientInfo = new ClientInfo(
testId,
testName,
verificationContext.getSourceQueryName(),
verificationContext.getSuite()).serialize();

try (PrestoConnection connection = getConnection(queryStage)) {
try (PrestoConnection connection = getConnection(queryStage, clientInfo)) {
try (java.sql.Statement jdbcStatement = connection.createStatement()) {
PrestoStatement prestoStatement = jdbcStatement.unwrap(PrestoStatement.class);
prestoStatement.setProgressMonitor(statementExecutor.getProgressMonitor());
Expand All @@ -129,7 +145,7 @@ private <T> T executeOnce(Statement statement, QueryStage queryStage, StatementE
}
}

private PrestoConnection getConnection(QueryStage queryStage)
private PrestoConnection getConnection(QueryStage queryStage, String clientInfo)
throws SQLException
{
PrestoConnection connection = DriverManager.getConnection(
Expand All @@ -140,6 +156,7 @@ private PrestoConnection getConnection(QueryStage queryStage)

try {
connection.setClientInfo("ApplicationName", "verifier-test");
connection.setClientInfo("ClientInfo", clientInfo);
connection.setCatalog(queryConfiguration.getCatalog());
connection.setSchema(queryConfiguration.getSchema());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.facebook.presto.verifier.framework.QueryConfiguration;
import com.facebook.presto.verifier.framework.VerificationContext;
import com.facebook.presto.verifier.framework.VerifierConfig;
import com.facebook.presto.verifier.retry.RetryConfig;
import io.airlift.units.Duration;

Expand All @@ -29,21 +30,24 @@ public class JdbcPrestoActionFactory
private final RetryConfig prestoRetryConfig;
private final Duration metadataTimeout;
private final Duration checksumTimeout;
private final VerifierConfig verifierConfig;

public JdbcPrestoActionFactory(
SqlExceptionClassifier exceptionClassifier,
PrestoActionConfig prestoActionConfig,
RetryConfig networkRetryConfig,
RetryConfig prestoRetryConfig,
Duration metadataTimeout,
Duration checksumTimeout)
Duration checksumTimeout,
VerifierConfig verifierConfig)
{
this.exceptionClassifier = requireNonNull(exceptionClassifier, "exceptionClassifier is null");
this.prestoActionConfig = requireNonNull(prestoActionConfig, "prestoClusterConfig is null");
this.networkRetryConfig = requireNonNull(networkRetryConfig, "networkRetryConfig is null");
this.prestoRetryConfig = requireNonNull(prestoRetryConfig, "prestoRetryConfig is null");
this.metadataTimeout = requireNonNull(metadataTimeout, "metadataTimeout is null");
this.checksumTimeout = requireNonNull(checksumTimeout, "checksumTimeout is null");
this.verifierConfig = requireNonNull(verifierConfig, "verifierConfig is null");
}

@Override
Expand All @@ -57,6 +61,7 @@ public JdbcPrestoAction create(QueryConfiguration queryConfiguration, Verificati
metadataTimeout,
checksumTimeout,
networkRetryConfig,
prestoRetryConfig);
prestoRetryConfig,
verifierConfig);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.facebook.presto.verifier.annotation.ForControl;
import com.facebook.presto.verifier.annotation.ForHelper;
import com.facebook.presto.verifier.annotation.ForTest;
import com.facebook.presto.verifier.framework.VerifierConfig;
import com.facebook.presto.verifier.retry.ForClusterConnection;
import com.facebook.presto.verifier.retry.ForPresto;
import com.facebook.presto.verifier.retry.RetryConfig;
Expand Down Expand Up @@ -113,7 +114,8 @@ public PrestoActionFactory get()
injector.getInstance(Key.get(RetryConfig.class, ForClusterConnection.class)),
injector.getInstance(Key.get(RetryConfig.class, ForPresto.class)),
queryActionsConfig.getMetadataTimeout(),
queryActionsConfig.getChecksumTimeout());
queryActionsConfig.getChecksumTimeout(),
injector.getInstance(VerifierConfig.class));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ private Optional<VerifierQueryEvent> runVerification(String controlQuery, String
private Optional<VerifierQueryEvent> runVerification(SourceQuery sourceQuery, Optional<PrestoAction> mockPrestoAction, DeterminismAnalyzerConfig determinismAnalyzerConfig)
{
PrestoExceptionClassifier exceptionClassifier = PrestoExceptionClassifier.defaultBuilder().build();
VerificationContext verificationContext = VerificationContext.create();
VerificationContext verificationContext = VerificationContext.create(NAME, SUITE);
VerifierConfig verifierConfig = new VerifierConfig().setTestId(TEST_ID);
RetryConfig retryConfig = new RetryConfig();
QueryActionsConfig queryActionsConfig = new QueryActionsConfig();
Expand All @@ -125,7 +125,8 @@ private Optional<VerifierQueryEvent> runVerification(SourceQuery sourceQuery, Op
queryActionsConfig.getMetadataTimeout(),
queryActionsConfig.getChecksumTimeout(),
retryConfig,
retryConfig);
retryConfig,
verifierConfig);
});
QueryRewriter queryRewriter = new VerificationQueryRewriterFactory(
new SqlParser(new SqlParserOptions().allowIdentifierSymbol(COLON, AT_SIGN)),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@

public class TestDeterminismAnalyzer
{
private static final String SUITE = "test-suite";
private static final String NAME = "test-query";
private static final SqlParser sqlParser = new SqlParser(new SqlParserOptions().allowIdentifierSymbol(COLON, AT_SIGN));

@Test
Expand All @@ -62,7 +64,7 @@ private static boolean isMutableCatalogReferenced(DeterminismAnalyzer determinis
private static DeterminismAnalyzer createDeterminismAnalyzer(String mutableCatalogPattern)
{
QueryConfiguration configuration = new QueryConfiguration(CATALOG, SCHEMA, Optional.of("user"), Optional.empty(), Optional.empty());
VerificationContext verificationContext = VerificationContext.create();
VerificationContext verificationContext = VerificationContext.create(SUITE, NAME);
VerifierConfig verifierConfig = new VerifierConfig().setTestId("test-id");
RetryConfig retryConfig = new RetryConfig();
QueryActionsConfig queryActionsConfig = new QueryActionsConfig();
Expand All @@ -75,7 +77,8 @@ private static DeterminismAnalyzer createDeterminismAnalyzer(String mutableCatal
queryActionsConfig.getMetadataTimeout(),
queryActionsConfig.getChecksumTimeout(),
retryConfig,
retryConfig);
retryConfig,
verifierConfig);
QueryRewriter queryRewriter = new QueryRewriter(
sqlParser,
typeManager,
Expand Down
Loading