Skip to content

Commit

Permalink
Adding support for pre and post queries for verifier
Browse files Browse the repository at this point in the history
  • Loading branch information
drudenko committed Oct 22, 2015
1 parent 9b7f08a commit 53d72f7
Show file tree
Hide file tree
Showing 7 changed files with 139 additions and 17 deletions.
14 changes: 14 additions & 0 deletions presto-docs/src/main/sphinx/release/release-0.124.rst
Expand Up @@ -6,3 +6,17 @@ General Changes
--------------- ---------------


* The :func:`approx_percentile` aggregation now also accepts an array of percentages * The :func:`approx_percentile` aggregation now also accepts an array of percentages

Verifier
--------

* Add support for pre and post control and test queries

If you're upgrading from 0.123, you need to alter your verifier_queries table

.. code-block:: sql
ALTER TABLE verifier_queries add column test_postqueries text null;
ALTER TABLE verifier_queries add column test_prequeries text null;
ALTER TABLE verifier_queries add column control_postqueries text null;
ALTER TABLE verifier_queries add column control_prequeries text null;
Expand Up @@ -177,15 +177,19 @@ private static List<QueryPair> applyOverrides(final VerifierConfig config, List<
Query test = new Query( Query test = new Query(
Optional.ofNullable(config.getTestCatalogOverride()).orElse(input.getTest().getCatalog()), Optional.ofNullable(config.getTestCatalogOverride()).orElse(input.getTest().getCatalog()),
Optional.ofNullable(config.getTestSchemaOverride()).orElse(input.getTest().getSchema()), Optional.ofNullable(config.getTestSchemaOverride()).orElse(input.getTest().getSchema()),
input.getTest().getPreQueries(),
input.getTest().getQuery(), input.getTest().getQuery(),
input.getTest().getPostQueries(),
Optional.ofNullable(config.getTestUsernameOverride()).orElse(input.getTest().getUsername()), Optional.ofNullable(config.getTestUsernameOverride()).orElse(input.getTest().getUsername()),
Optional.ofNullable(config.getTestPasswordOverride()).orElse( Optional.ofNullable(config.getTestPasswordOverride()).orElse(
Optional.ofNullable(input.getTest().getPassword()).orElse(null)), Optional.ofNullable(input.getTest().getPassword()).orElse(null)),
input.getTest().getSessionProperties()); input.getTest().getSessionProperties());
Query control = new Query( Query control = new Query(
Optional.ofNullable(config.getControlCatalogOverride()).orElse(input.getControl().getCatalog()), Optional.ofNullable(config.getControlCatalogOverride()).orElse(input.getControl().getCatalog()),
Optional.ofNullable(config.getControlSchemaOverride()).orElse(input.getControl().getSchema()), Optional.ofNullable(config.getControlSchemaOverride()).orElse(input.getControl().getSchema()),
input.getControl().getPreQueries(),
input.getControl().getQuery(), input.getControl().getQuery(),
input.getControl().getPostQueries(),
Optional.ofNullable(config.getControlUsernameOverride()).orElse(input.getControl().getUsername()), Optional.ofNullable(config.getControlUsernameOverride()).orElse(input.getControl().getUsername()),
Optional.ofNullable(config.getControlPasswordOverride()).orElse( Optional.ofNullable(config.getControlPasswordOverride()).orElse(
Optional.ofNullable(input.getControl().getPassword()).orElse(null)), Optional.ofNullable(input.getControl().getPassword()).orElse(null)),
Expand Down
Expand Up @@ -15,22 +15,29 @@


import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;


import java.util.List;
import java.util.Map; import java.util.Map;


import static com.facebook.presto.util.ImmutableCollectors.toImmutableList;

public class Query public class Query
{ {
private final String catalog; private final String catalog;
private final String schema; private final String schema;
private final List<String> preQueries;
private final String query; private final String query;
private final List<String> postQueries;
private final String username; private final String username;
private final String password; private final String password;
private final Map<String, String> sessionProperties; private final Map<String, String> sessionProperties;


public Query(String catalog, String schema, String query, String username, String password, Map<String, String> sessionProperties) public Query(String catalog, String schema, List<String> preQueries, String query, List<String> postQueries, String username, String password, Map<String, String> sessionProperties)
{ {
this.catalog = catalog; this.catalog = catalog;
this.schema = schema; this.schema = schema;
this.preQueries = preQueries.stream().map(Query::clean).collect(toImmutableList());
this.query = clean(query); this.query = clean(query);
this.postQueries = postQueries.stream().map(Query::clean).collect(toImmutableList());
this.username = username; this.username = username;
this.password = password; this.password = password;
this.sessionProperties = ImmutableMap.copyOf(sessionProperties); this.sessionProperties = ImmutableMap.copyOf(sessionProperties);
Expand All @@ -46,11 +53,21 @@ public String getSchema()
return schema; return schema;
} }


public List<String> getPreQueries()
{
return preQueries;
}

public String getQuery() public String getQuery()
{ {
return query; return query;
} }


public List<String> getPostQueries()
{
return postQueries;
}

public String getUsername() public String getUsername()
{ {
return username; return username;
Expand Down
Expand Up @@ -13,19 +13,22 @@
*/ */
package com.facebook.presto.verifier; package com.facebook.presto.verifier;


import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.airlift.json.JsonCodec; import io.airlift.json.JsonCodec;
import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.tweak.ResultSetMapper; import org.skife.jdbi.v2.tweak.ResultSetMapper;


import java.sql.ResultSet; import java.sql.ResultSet;
import java.sql.SQLException; import java.sql.SQLException;
import java.util.List;
import java.util.Map; import java.util.Map;


public class QueryPairMapper public class QueryPairMapper
implements ResultSetMapper<QueryPair> implements ResultSetMapper<QueryPair>
{ {
private static final JsonCodec<Map<String, String>> jsonCodec = JsonCodec.mapJsonCodec(String.class, String.class); private static final JsonCodec<Map<String, String>> propertiesJsonCodec = JsonCodec.mapJsonCodec(String.class, String.class);
private static final JsonCodec<List<String>> queriesJsonCodec = JsonCodec.listJsonCodec(String.class);


@Override @Override
public QueryPair map(int index, ResultSet resultSet, StatementContext statementContext) public QueryPair map(int index, ResultSet resultSet, StatementContext statementContext)
Expand All @@ -34,15 +37,23 @@ public QueryPair map(int index, ResultSet resultSet, StatementContext statementC
Map<String, String> sessionProperties = ImmutableMap.of(); Map<String, String> sessionProperties = ImmutableMap.of();
String json = resultSet.getString("session_properties_json"); String json = resultSet.getString("session_properties_json");
if (json != null) { if (json != null) {
sessionProperties = jsonCodec.fromJson(json); sessionProperties = propertiesJsonCodec.fromJson(json);
} }


return new QueryPair( return new QueryPair(
resultSet.getString("suite"), resultSet.getString("suite"),
resultSet.getString("name"), resultSet.getString("name"),
new Query(resultSet.getString("test_catalog"), resultSet.getString("test_schema"), resultSet.getString("test_query"), new Query(resultSet.getString("test_catalog"), resultSet.getString("test_schema"), fromJsonString(resultSet.getString("test_prequeries")), resultSet.getString("test_query"),
fromJsonString(resultSet.getString("test_postqueries")),
resultSet.getString("test_username"), resultSet.getString("test_password"), sessionProperties), resultSet.getString("test_username"), resultSet.getString("test_password"), sessionProperties),
new Query(resultSet.getString("control_catalog"), resultSet.getString("control_schema"), resultSet.getString("control_query"), new Query(resultSet.getString("control_catalog"), resultSet.getString("control_schema"), fromJsonString(resultSet.getString("control_prequeries")), resultSet.getString("control_query"),
fromJsonString(resultSet.getString("control_postqueries")),
resultSet.getString("control_username"), resultSet.getString("control_password"), sessionProperties)); resultSet.getString("control_username"), resultSet.getString("control_password"), sessionProperties));
} }

private static List<String> fromJsonString(String jsonString)
throws SQLException
{
return jsonString == null ? ImmutableList.of() : queriesJsonCodec.fromJson(jsonString);
}
} }
Expand Up @@ -24,7 +24,7 @@ public class QueryResult
{ {
public enum State public enum State
{ {
INVALID, FAILED, SUCCESS, TOO_MANY_ROWS, TIMEOUT INVALID, FAILED, SUCCESS, TOO_MANY_ROWS, TIMEOUT, FAILED_TO_SETUP, FAILED_TO_TEARDOWN
} }


private final State state; private final State state;
Expand Down
Expand Up @@ -47,6 +47,7 @@
import java.util.Map; import java.util.Map;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.regex.Pattern; import java.util.regex.Pattern;


import static com.facebook.presto.verifier.QueryResult.State; import static com.facebook.presto.verifier.QueryResult.State;
Expand Down Expand Up @@ -182,36 +183,61 @@ public boolean isDeterministic()


private boolean validate() private boolean validate()
{ {
controlResult = executeQuery(controlGateway, controlUsername, controlPassword, queryPair.getControl(), controlTimeout, sessionProperties); controlResult = executeQueryControl();


// query has too many rows. Consider blacklisting. // query has too many rows. Consider blacklisting.
if (controlResult.getState() == State.TOO_MANY_ROWS) { if (controlResult.getState() == State.TOO_MANY_ROWS) {
testResult = new QueryResult(State.INVALID, null, null, ImmutableList.<List<Object>>of()); testResult = new QueryResult(State.INVALID, null, null, ImmutableList.<List<Object>>of());
return false; return false;
} }
// query failed in the control // query failed in the control
else if (controlResult.getState() != State.SUCCESS) { if (controlResult.getState() != State.SUCCESS) {
testResult = new QueryResult(State.INVALID, null, null, ImmutableList.<List<Object>>of()); testResult = new QueryResult(State.INVALID, null, null, ImmutableList.<List<Object>>of());
return true; return true;
} }


testResult = executeQuery(testGateway, testUsername, testPassword, queryPair.getTest(), testTimeout, sessionProperties); testResult = executeQueryTest();


if ((controlResult.getState() != State.SUCCESS) || (testResult.getState() != State.SUCCESS)) { if (controlResult.getState() != State.SUCCESS || testResult.getState() != State.SUCCESS) {
return false; return false;
} }


if (!checkCorrectness) { if (!checkCorrectness) {
return true; return true;
} }


if (resultsMatch(controlResult, testResult, precision)) { return resultsMatch(controlResult, testResult, precision) || checkForDeterministicAndRerunTestQueriesIfNeeded();
return true; }

private QueryResult tearDown(Query query, Function<String, QueryResult> executor)
{
for (String postqueryString : query.getPostQueries()) {
QueryResult queryResult = executor.apply(postqueryString);
if (queryResult.getState() != State.SUCCESS) {
return new QueryResult(State.FAILED_TO_TEARDOWN, queryResult.getException(), queryResult.getDuration(), ImmutableList.<List<Object>>of());
}
} }


return new QueryResult(State.SUCCESS, null, null, ImmutableList.of());
}

private QueryResult setup(Query query, Function<String, QueryResult> executor)
{
for (String prequeryString : query.getPreQueries()) {
QueryResult queryResult = executor.apply(prequeryString);
if (queryResult.getState() != State.SUCCESS) {
return new QueryResult(State.FAILED_TO_SETUP, queryResult.getException(), queryResult.getDuration(), ImmutableList.<List<Object>>of());
}
}

return new QueryResult(State.SUCCESS, null, null, ImmutableList.of());
}

private boolean checkForDeterministicAndRerunTestQueriesIfNeeded()
{
// check if the control query is deterministic // check if the control query is deterministic
for (int i = 0; i < 3; i++) { for (int i = 0; i < 3; i++) {
QueryResult results = executeQuery(controlGateway, controlUsername, controlPassword, queryPair.getControl(), controlTimeout, sessionProperties); QueryResult results = executeQueryControl();
if (results.getState() != State.SUCCESS) { if (results.getState() != State.SUCCESS) {
return false; return false;
} }
Expand All @@ -225,7 +251,7 @@ else if (controlResult.getState() != State.SUCCESS) {
// Re-run the test query to confirm that the results don't match, in case there was caching on the test tier, // Re-run the test query to confirm that the results don't match, in case there was caching on the test tier,
// but require that it matches 3 times in a row to rule out a non-deterministic correctness bug. // but require that it matches 3 times in a row to rule out a non-deterministic correctness bug.
for (int i = 0; i < 3; i++) { for (int i = 0; i < 3; i++) {
testResult = executeQuery(testGateway, testUsername, testPassword, queryPair.getTest(), testTimeout, sessionProperties); testResult = executeQueryTest();
if (testResult.getState() != State.SUCCESS) { if (testResult.getState() != State.SUCCESS) {
return false; return false;
} }
Expand All @@ -238,6 +264,52 @@ else if (controlResult.getState() != State.SUCCESS) {
return true; return true;
} }


private QueryResult executeQueryTest()
{
Query query = queryPair.getTest();
QueryResult queryResult = new QueryResult(State.INVALID, null, null, ImmutableList.<List<Object>>of());
try {
// startup
queryResult = setup(query, testPrequery -> executeQuery(testGateway, testUsername, testPassword, queryPair.getTest(), testPrequery, testTimeout, sessionProperties));

// if startup is successful -> execute query
if (queryResult.getState() == State.SUCCESS) {
queryResult = executeQuery(testGateway, testUsername, testPassword, queryPair.getTest(), query.getQuery(), testTimeout, sessionProperties);
}
}
finally {
// teardown no matter what
QueryResult tearDownResult = tearDown(query, testPostquery -> executeQuery(testGateway, testUsername, testPassword, queryPair.getTest(), testPostquery, testTimeout, sessionProperties));

// if teardown is not successful the query fails
queryResult = tearDownResult.getState() == State.SUCCESS ? queryResult : tearDownResult;
}
return queryResult;
}

private QueryResult executeQueryControl()
{
Query query = queryPair.getControl();
QueryResult queryResult = new QueryResult(State.INVALID, null, null, ImmutableList.<List<Object>>of());
try {
// startup
queryResult = setup(query, controlPrequery -> executeQuery(controlGateway, controlUsername, controlPassword, queryPair.getControl(), controlPrequery, controlTimeout, sessionProperties));

// if startup is successful -> execute query
if (queryResult.getState() == State.SUCCESS) {
queryResult = executeQuery(controlGateway, controlUsername, controlPassword, queryPair.getControl(), query.getQuery(), controlTimeout, sessionProperties);
}
}
finally {
// teardown no matter what
QueryResult tearDownResult = tearDown(query, controlPostquery -> executeQuery(controlGateway, controlUsername, controlPassword, queryPair.getControl(), controlPostquery, controlTimeout, sessionProperties));

// if teardown is not successful the query fails
queryResult = tearDownResult.getState() == State.SUCCESS ? queryResult : tearDownResult;
}
return queryResult;
}

public QueryPair getQueryPair() public QueryPair getQueryPair()
{ {
return queryPair; return queryPair;
Expand All @@ -253,7 +325,7 @@ public QueryResult getTestResult()
return testResult; return testResult;
} }


private QueryResult executeQuery(String url, String username, String password, Query query, Duration timeout, Map<String, String> sessionProperties) private QueryResult executeQuery(String url, String username, String password, Query query, String sql, Duration timeout, Map<String, String> sessionProperties)
{ {
try (Connection connection = DriverManager.getConnection(url, username, password)) { try (Connection connection = DriverManager.getConnection(url, username, password)) {
trySetConnectionProperties(query, connection); trySetConnectionProperties(query, connection);
Expand All @@ -266,7 +338,6 @@ private QueryResult executeQuery(String url, String username, String password, Q
TimeLimiter limiter = new SimpleTimeLimiter(); TimeLimiter limiter = new SimpleTimeLimiter();
Stopwatch stopwatch = Stopwatch.createStarted(); Stopwatch stopwatch = Stopwatch.createStarted();
Statement limitedStatement = limiter.newProxy(statement, Statement.class, timeout.toMillis(), TimeUnit.MILLISECONDS); Statement limitedStatement = limiter.newProxy(statement, Statement.class, timeout.toMillis(), TimeUnit.MILLISECONDS);
String sql = query.getQuery();
if (explainOnly) { if (explainOnly) {
sql = "EXPLAIN " + sql; sql = "EXPLAIN " + sql;
} }
Expand Down Expand Up @@ -326,7 +397,8 @@ private void trySetConnectionProperties(Query query, Connection connection)


private Callable<List<List<Object>>> getResultSetConverter(final ResultSet resultSet) private Callable<List<List<Object>>> getResultSetConverter(final ResultSet resultSet)
{ {
return new Callable<List<List<Object>>>() { return new Callable<List<List<Object>>>()
{
@Override @Override
public List<List<Object>> call() public List<List<Object>> call()
throws Exception throws Exception
Expand Down
Expand Up @@ -27,12 +27,16 @@ public interface VerifierDao
", name\n" + ", name\n" +
", test_catalog\n" + ", test_catalog\n" +
", test_schema\n" + ", test_schema\n" +
", test_prequeries\n" +
", test_query\n" + ", test_query\n" +
", test_postqueries\n" +
", test_username\n" + ", test_username\n" +
", test_password\n" + ", test_password\n" +
", control_catalog\n" + ", control_catalog\n" +
", control_schema\n" + ", control_schema\n" +
", control_prequeries\n" +
", control_query\n" + ", control_query\n" +
", control_postqueries\n" +
", control_username\n" + ", control_username\n" +
", control_password\n" + ", control_password\n" +
", session_properties_json\n" + ", session_properties_json\n" +
Expand Down

0 comments on commit 53d72f7

Please sign in to comment.