diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 073e9f1bd563..1f0a67605c83 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -69,6 +69,11 @@ presto-thrift-connector + + com.fasterxml.jackson.core + jackson-annotations + + com.fasterxml.jackson.core jackson-core diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/event/DeterminismAnalysisDetails.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/event/DeterminismAnalysisDetails.java new file mode 100644 index 000000000000..77b4f3fcf673 --- /dev/null +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/event/DeterminismAnalysisDetails.java @@ -0,0 +1,63 @@ +/* + * 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.event; + +import com.facebook.airlift.event.client.EventField; +import com.facebook.airlift.event.client.EventType; +import com.facebook.presto.verifier.framework.LimitQueryDeterminismAnalysis; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.common.collect.ImmutableList; + +import javax.annotation.concurrent.Immutable; + +import java.util.List; +import java.util.Optional; + +@Immutable +@EventType("DeterminismAnalysisDetails") +public class DeterminismAnalysisDetails +{ + private final List runs; + private final String limitQueryAnalysis; + private final String limitQueryAnalysisQueryId; + + @JsonCreator + public DeterminismAnalysisDetails( + List runs, + LimitQueryDeterminismAnalysis limitQueryAnalysis, + Optional limitQueryAnalysisQueryId) + { + this.runs = ImmutableList.copyOf(runs); + this.limitQueryAnalysis = limitQueryAnalysis.name(); + this.limitQueryAnalysisQueryId = limitQueryAnalysisQueryId.orElse(null); + } + + @EventField + public List getRuns() + { + return runs; + } + + @EventField + public String getLimitQueryAnalysis() + { + return limitQueryAnalysis; + } + + @EventField + public String getLimitQueryAnalysisQueryId() + { + return limitQueryAnalysisQueryId; + } +} diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/event/DeterminismAnalysisRun.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/event/DeterminismAnalysisRun.java new file mode 100644 index 000000000000..9f60cfa6e51c --- /dev/null +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/event/DeterminismAnalysisRun.java @@ -0,0 +1,103 @@ +/* + * 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.event; + +import com.facebook.airlift.event.client.EventField; +import com.facebook.airlift.event.client.EventType; + +import javax.annotation.concurrent.Immutable; + +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +@Immutable +@EventType("DeterminismAnalysisRun") +public class DeterminismAnalysisRun +{ + private final String tableName; + private final String queryId; + private final String checksumQueryId; + + private DeterminismAnalysisRun( + Optional tableName, + Optional queryId, + Optional checksumQueryId) + { + this.tableName = tableName.orElse(null); + this.queryId = queryId.orElse(null); + this.checksumQueryId = checksumQueryId.orElse(null); + } + + @EventField + public String getTableName() + { + return tableName; + } + + @EventField + public String getQueryId() + { + return queryId; + } + + @EventField + public String getChecksumQueryId() + { + return checksumQueryId; + } + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder + { + private String tableName; + private String queryId; + private String checksumQueryId; + + private Builder() + { + } + + public Builder setTableName(String tableName) + { + checkState(this.tableName == null, "tableName is already set"); + this.tableName = requireNonNull(tableName, "tableName is null"); + return this; + } + + public Builder setQueryId(String queryId) + { + checkState(this.queryId == null, "queryId is already set"); + this.queryId = requireNonNull(queryId, "queryId is null"); + return this; + } + + public Builder setChecksumQueryId(String checksumQueryId) + { + checkState(this.checksumQueryId == null, "checksumQueryId is already set"); + this.checksumQueryId = requireNonNull(checksumQueryId, "checksumQueryId is null"); + return this; + } + + public DeterminismAnalysisRun build() + { + return new DeterminismAnalysisRun(Optional.ofNullable(tableName), Optional.ofNullable(queryId), Optional.ofNullable(checksumQueryId)); + } + } +} diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/event/VerifierQueryEvent.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/event/VerifierQueryEvent.java index 4358ac27dea5..fe86e1ca4d6b 100644 --- a/presto-verifier/src/main/java/com/facebook/presto/verifier/event/VerifierQueryEvent.java +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/event/VerifierQueryEvent.java @@ -48,6 +48,7 @@ public enum EventStatus private final Boolean deterministic; private final String determinismAnalysis; + private final DeterminismAnalysisDetails determinismAnalysisDetails; private final String resolveMessage; private final QueryInfo controlQueryInfo; @@ -66,6 +67,7 @@ public VerifierQueryEvent( EventStatus status, Optional skippedReason, Optional determinismAnalysis, + Optional determinismAnalysisDetails, Optional resolveMessage, Optional controlQueryInfo, Optional testQueryInfo, @@ -81,6 +83,7 @@ public VerifierQueryEvent( this.skippedReason = skippedReason.map(SkippedReason::name).orElse(null); this.deterministic = determinismAnalysis.filter(d -> !d.isUnknown()).map(DeterminismAnalysis::isDeterministic).orElse(null); this.determinismAnalysis = determinismAnalysis.map(DeterminismAnalysis::name).orElse(null); + this.determinismAnalysisDetails = determinismAnalysisDetails.orElse(null); this.resolveMessage = resolveMessage.orElse(null); this.controlQueryInfo = controlQueryInfo.orElse(null); this.testQueryInfo = testQueryInfo.orElse(null); @@ -109,6 +112,7 @@ public static VerifierQueryEvent skipped( Optional.empty(), Optional.empty(), Optional.empty(), + Optional.empty(), ImmutableList.of()); } @@ -155,6 +159,12 @@ public String getDeterminismAnalysis() return determinismAnalysis; } + @EventField + public DeterminismAnalysisDetails getDeterminismAnalysisDetails() + { + return determinismAnalysisDetails; + } + @EventField public String getResolveMessage() { diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/AbstractVerification.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/AbstractVerification.java index 352718672c38..24b7052fb08f 100644 --- a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/AbstractVerification.java +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/AbstractVerification.java @@ -18,6 +18,7 @@ import com.facebook.presto.sql.SqlFormatter; import com.facebook.presto.sql.tree.Statement; import com.facebook.presto.verifier.checksum.ChecksumResult; +import com.facebook.presto.verifier.event.DeterminismAnalysisDetails; import com.facebook.presto.verifier.event.QueryInfo; import com.facebook.presto.verifier.event.VerifierQueryEvent; import com.facebook.presto.verifier.event.VerifierQueryEvent.EventStatus; @@ -274,6 +275,12 @@ else if (skippedReason.isPresent()) { status, skippedReason, determinismAnalysis, + determinismAnalysis.isPresent() ? + Optional.of(new DeterminismAnalysisDetails( + verificationContext.getDeterminismAnalysisRuns(), + verificationContext.getLimitQueryAnalysis(), + verificationContext.getLimitQueryAnalysisQueryId())) : + Optional.empty(), resolveMessage, Optional.of(buildQueryInfo( sourceQuery.getControlConfiguration(), diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/DataVerification.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/DataVerification.java index e7a8b7254ee8..e6071d334cb9 100644 --- a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/DataVerification.java +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/DataVerification.java @@ -20,12 +20,14 @@ import com.facebook.presto.verifier.checksum.ChecksumResult; import com.facebook.presto.verifier.checksum.ChecksumValidator; import com.facebook.presto.verifier.checksum.ColumnMatchResult; +import com.facebook.presto.verifier.event.DeterminismAnalysisRun; import com.facebook.presto.verifier.framework.MatchResult.MatchType; import com.facebook.presto.verifier.prestoaction.PrestoAction; import com.facebook.presto.verifier.resolver.FailureResolverManager; import com.facebook.presto.verifier.rewrite.QueryRewriter; import com.google.common.collect.ImmutableMap; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; @@ -47,6 +49,7 @@ import static com.facebook.presto.verifier.framework.QueryStage.CHECKSUM; import static com.facebook.presto.verifier.framework.QueryStage.DESCRIBE; import static com.facebook.presto.verifier.framework.VerifierUtil.callWithQueryStatsConsumer; +import static com.facebook.presto.verifier.framework.VerifierUtil.runWithQueryStatsConsumer; import static com.google.common.collect.Iterables.getOnlyElement; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -99,28 +102,33 @@ public MatchResult verify(QueryBundle control, QueryBundle test) } @Override - protected DeterminismAnalysis analyzeDeterminism(QueryBundle control, ChecksumResult firstChecksum) + protected DeterminismAnalysis analyzeDeterminism(QueryBundle control, ChecksumResult controlChecksum) { List columns = getColumns(control.getTableName()); + List queryBundles = new ArrayList<>(); - QueryBundle secondRun = null; - QueryBundle thirdRun = null; try { - secondRun = getQueryRewriter().rewriteQuery(getSourceQuery().getControlQuery(), CONTROL); - setupAndRun(secondRun, true); - DeterminismAnalysis determinismAnalysis = matchResultToDeterminism(match(columns, columns, firstChecksum, computeChecksum(secondRun, columns).getResult())); - if (determinismAnalysis != DETERMINISTIC) { - return determinismAnalysis; + for (int i = 0; i < 2; i++) { + QueryBundle queryBundle = getQueryRewriter().rewriteQuery(getSourceQuery().getControlQuery(), CONTROL); + queryBundles.add(queryBundle); + DeterminismAnalysisRun.Builder run = getVerificationContext().startDeterminismAnalysisRun().setTableName(queryBundle.getTableName().toString()); + + runWithQueryStatsConsumer(() -> setupAndRun(queryBundle, true), stats -> run.setQueryId(stats.getQueryId())); + + Query checksumQuery = checksumValidator.generateChecksumQuery(queryBundle.getTableName(), columns); + ChecksumResult testChecksum = getOnlyElement(callWithQueryStatsConsumer( + () -> executeChecksumQuery(checksumQuery), + stats -> run.setChecksumQueryId(stats.getQueryId())).getResults()); + + DeterminismAnalysis determinismAnalysis = matchResultToDeterminism(match(columns, columns, controlChecksum, testChecksum)); + if (determinismAnalysis != DETERMINISTIC) { + return determinismAnalysis; + } } - thirdRun = getQueryRewriter().rewriteQuery(getSourceQuery().getControlQuery(), CONTROL); - setupAndRun(thirdRun, true); - determinismAnalysis = matchResultToDeterminism(match(columns, columns, firstChecksum, computeChecksum(thirdRun, columns).getResult())); - if (determinismAnalysis != DETERMINISTIC) { - return determinismAnalysis; - } + LimitQueryDeterminismAnalysis analysis = limitQueryDeterminismAnalyzer.analyze(control, controlChecksum.getRowCount(), getVerificationContext()); + getVerificationContext().setLimitQueryAnalysis(analysis); - LimitQueryDeterminismAnalysis analysis = limitQueryDeterminismAnalyzer.analyze(control, firstChecksum.getRowCount()); switch (analysis) { case NON_DETERMINISTIC: return NON_DETERMINISTIC_LIMIT_CLAUSE; @@ -140,8 +148,7 @@ protected DeterminismAnalysis analyzeDeterminism(QueryBundle control, ChecksumRe return ANALYSIS_FAILED; } finally { - teardownSafely(secondRun); - teardownSafely(thirdRun); + queryBundles.forEach(this::teardownSafely); } } @@ -208,46 +215,4 @@ private QueryResult executeChecksumQuery(Query query) { return getPrestoAction().execute(query, CHECKSUM, ChecksumResult::fromResultSet); } - - private ChecksumQueryAndResult computeChecksum(QueryBundle bundle, List columns) - { - Query checksumQuery = checksumValidator.generateChecksumQuery(bundle.getTableName(), columns); - QueryResult queryResult = getPrestoAction().execute( - checksumQuery, - CHECKSUM, - ChecksumResult::fromResultSet); - return new ChecksumQueryAndResult( - queryResult.getQueryStats().getQueryId(), - checksumQuery, - getOnlyElement(queryResult.getResults())); - } - - private class ChecksumQueryAndResult - { - private final String queryId; - private final Query query; - private final ChecksumResult result; - - public ChecksumQueryAndResult(String queryId, Query query, ChecksumResult result) - { - this.queryId = requireNonNull(queryId, "queryId is null"); - this.query = requireNonNull(query, "query is null"); - this.result = requireNonNull(result, "result is null"); - } - - public String getQueryId() - { - return queryId; - } - - public Query getQuery() - { - return query; - } - - public ChecksumResult getResult() - { - return result; - } - } } diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/LimitQueryDeterminismAnalyzer.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/LimitQueryDeterminismAnalyzer.java index f15a24d64458..74fdae65a164 100644 --- a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/LimitQueryDeterminismAnalyzer.java +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/LimitQueryDeterminismAnalyzer.java @@ -36,6 +36,7 @@ import static com.facebook.presto.verifier.framework.LimitQueryDeterminismAnalysis.NON_DETERMINISTIC; import static com.facebook.presto.verifier.framework.LimitQueryDeterminismAnalysis.NOT_RUN; import static com.facebook.presto.verifier.framework.QueryStage.DETERMINISM_ANALYSIS; +import static com.facebook.presto.verifier.framework.VerifierUtil.callWithQueryStatsConsumer; import static com.google.common.collect.Iterables.getOnlyElement; import static java.util.Objects.requireNonNull; @@ -50,7 +51,7 @@ public LimitQueryDeterminismAnalyzer(PrestoAction prestoAction, VerifierConfig v this.enabled = verifierConfig.isEnableLimitQueryDeterminismAnalyzer(); } - public LimitQueryDeterminismAnalysis analyze(QueryBundle control, long rowCount) + public LimitQueryDeterminismAnalysis analyze(QueryBundle control, long rowCount, VerificationContext verificationContext) { if (!enabled) { return NOT_RUN; @@ -121,8 +122,12 @@ else if (query.getQueryBody() instanceof QuerySpecification) { Query rowCountQuery = simpleQuery( new Select(false, ImmutableList.of(new SingleColumn(new FunctionCall(QualifiedName.of("count"), ImmutableList.of(new LongLiteral("1")))))), new TableSubquery(queryNoLimit)); - long rowCountNoLimit = getOnlyElement(prestoAction.execute(rowCountQuery, DETERMINISM_ANALYSIS, resultSet -> resultSet.getLong(1)).getResults()); + QueryResult result = callWithQueryStatsConsumer( + () -> prestoAction.execute(rowCountQuery, DETERMINISM_ANALYSIS, resultSet -> resultSet.getLong(1)), + stats -> verificationContext.setLimitQueryAnalysisQueryId(stats.getQueryId())); + + long rowCountNoLimit = getOnlyElement(result.getResults()); if (rowCountNoLimit > rowCount) { return NON_DETERMINISTIC; } diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerificationContext.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerificationContext.java index 080970e5e911..a32d817cf002 100644 --- a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerificationContext.java +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerificationContext.java @@ -13,12 +13,15 @@ */ package com.facebook.presto.verifier.framework; +import com.facebook.presto.verifier.event.DeterminismAnalysisRun; import com.facebook.presto.verifier.event.QueryFailure; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import java.util.List; import java.util.Optional; +import static com.facebook.presto.verifier.framework.LimitQueryDeterminismAnalysis.NOT_RUN; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; import static java.util.Objects.requireNonNull; @@ -30,6 +33,10 @@ public class VerificationContext private String testChecksumQueryId; private String testChecksumQuery; + private ImmutableList.Builder determinismAnalysisRuns = ImmutableList.builder(); + private LimitQueryDeterminismAnalysis limitQueryAnalysis; + private String limitQueryAnalysisQueryId; + private ImmutableSet.Builder queryExceptions = ImmutableSet.builder(); public Optional getControlChecksumQueryId() @@ -76,6 +83,42 @@ public void setTestChecksumQuery(String testChecksumQuery) this.testChecksumQuery = requireNonNull(testChecksumQuery, "testChecksumQuery is null"); } + public List getDeterminismAnalysisRuns() + { + return determinismAnalysisRuns.build().stream() + .map(DeterminismAnalysisRun.Builder::build) + .collect(toImmutableList()); + } + + public DeterminismAnalysisRun.Builder startDeterminismAnalysisRun() + { + DeterminismAnalysisRun.Builder run = DeterminismAnalysisRun.builder(); + determinismAnalysisRuns.add(run); + return run; + } + + public LimitQueryDeterminismAnalysis getLimitQueryAnalysis() + { + return limitQueryAnalysis == null ? NOT_RUN : limitQueryAnalysis; + } + + public void setLimitQueryAnalysis(LimitQueryDeterminismAnalysis limitQueryAnalysis) + { + checkState(this.limitQueryAnalysis == null, "limitQueryAnalysis is already set"); + this.limitQueryAnalysis = requireNonNull(limitQueryAnalysis, "limitQueryAnalysis is null"); + } + + public Optional getLimitQueryAnalysisQueryId() + { + return Optional.ofNullable(limitQueryAnalysisQueryId); + } + + public void setLimitQueryAnalysisQueryId(String limitQueryAnalysisQueryId) + { + checkState(this.limitQueryAnalysisQueryId == null, "limitQueryAnalysisQueryId is already set"); + this.limitQueryAnalysisQueryId = requireNonNull(limitQueryAnalysisQueryId, "limitQueryAnalysisQueryId is null"); + } + public void addException(QueryException exception) { queryExceptions.add(exception); diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerifierUtil.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerifierUtil.java index 84edc4d9c592..543063217cbc 100644 --- a/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerifierUtil.java +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/framework/VerifierUtil.java @@ -21,6 +21,7 @@ import java.util.function.Function; import static com.facebook.presto.sql.parser.ParsingOptions.DecimalLiteralTreatment.AS_DOUBLE; +import static com.google.common.base.Functions.identity; public class VerifierUtil { @@ -35,6 +36,11 @@ public static Identifier delimitedIdentifier(String name) return new Identifier(name, true); } + public static void runWithQueryStatsConsumer(Callable callable, Consumer queryStatsConsumer) + { + callWithQueryStatsConsumer(callable, identity(), queryStatsConsumer); + } + public static QueryResult callWithQueryStatsConsumer(Callable> callable, Consumer queryStatsConsumer) { return callWithQueryStatsConsumer(callable, QueryResult::getQueryStats, queryStatsConsumer); diff --git a/presto-verifier/src/test/java/com/facebook/presto/verifier/framework/TestDataVerification.java b/presto-verifier/src/test/java/com/facebook/presto/verifier/framework/TestDataVerification.java index 0c5948b67a60..61b81d9c7135 100644 --- a/presto-verifier/src/test/java/com/facebook/presto/verifier/framework/TestDataVerification.java +++ b/presto-verifier/src/test/java/com/facebook/presto/verifier/framework/TestDataVerification.java @@ -22,6 +22,7 @@ import com.facebook.presto.verifier.checksum.FloatingPointColumnValidator; import com.facebook.presto.verifier.checksum.OrderableArrayColumnValidator; import com.facebook.presto.verifier.checksum.SimpleColumnValidator; +import com.facebook.presto.verifier.event.DeterminismAnalysisRun; import com.facebook.presto.verifier.event.VerifierQueryEvent; import com.facebook.presto.verifier.event.VerifierQueryEvent.EventStatus; import com.facebook.presto.verifier.prestoaction.JdbcPrestoAction; @@ -231,6 +232,10 @@ public void testNonDeterministic() "Control 1 rows, Test 1 rows\n" + "Mismatched Columns:\n" + " _col0 \\(double\\): control\\(sum: .*\\) test\\(sum: 2.0\\) relative error: .*\n")); + + List runs = event.get().getDeterminismAnalysisDetails().getRuns(); + assertEquals(runs.size(), 1); + assertDeterminismAnalysisRun(runs.get(0)); } @Test @@ -252,6 +257,11 @@ public void testArrayOfRow() "Control 1 rows, Test 1 rows\n" + "Mismatched Columns:\n" + " _col0 \\(array\\(row\\(integer, varchar\\(1\\)\\)\\)\\): control\\(checksum: 71 b5 2f 7f 1e 9b a6 a4\\) test\\(checksum: b4 3c 7d 02 2b 14 77 12\\)\n")); + + List runs = event.get().getDeterminismAnalysisDetails().getRuns(); + assertEquals(runs.size(), 2); + assertDeterminismAnalysisRun(runs.get(0)); + assertDeterminismAnalysisRun(runs.get(1)); } @Test @@ -292,4 +302,11 @@ private void assertEvent( assertTrue(Pattern.compile(expectedErrorMessageRegex.get(), MULTILINE + DOTALL).matcher(event.getErrorMessage()).matches()); } } + + private void assertDeterminismAnalysisRun(DeterminismAnalysisRun run) + { + assertNotNull(run.getTableName()); + assertNotNull(run.getQueryId()); + assertNotNull(run.getChecksumQueryId()); + } } diff --git a/presto-verifier/src/test/java/com/facebook/presto/verifier/framework/TestLimitQueryDeterminismAnalyzer.java b/presto-verifier/src/test/java/com/facebook/presto/verifier/framework/TestLimitQueryDeterminismAnalyzer.java index ff9e811ba3c3..ba49006a56f8 100644 --- a/presto-verifier/src/test/java/com/facebook/presto/verifier/framework/TestLimitQueryDeterminismAnalyzer.java +++ b/presto-verifier/src/test/java/com/facebook/presto/verifier/framework/TestLimitQueryDeterminismAnalyzer.java @@ -39,6 +39,8 @@ import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; public class TestLimitQueryDeterminismAnalyzer { @@ -94,30 +96,30 @@ public void setup() public void testNotRun() { // Unsupported statement types - assertEquals(analyze("CREATE TABLE test (x varchar, ds varhcar) WITH (partitioned_by = ARRAY[\"ds\"])"), NOT_RUN); - assertEquals(analyze("SELECT * FROM source LIMIT 10"), NOT_RUN); + assertAnalysis("CREATE TABLE test (x varchar, ds varhcar) WITH (partitioned_by = ARRAY[\"ds\"])", NOT_RUN); + assertAnalysis("SELECT * FROM source LIMIT 10", NOT_RUN); // Order by clause - assertEquals(analyze("INSERT INTO test SELECT * FROM source UNION ALL SELECT * FROM source ORDER BY 1 LIMIT 1000"), NOT_RUN); - assertEquals(analyze("INSERT INTO test SELECT * FROM source ORDER BY 1 LIMIT 1000"), NOT_RUN); + assertAnalysis("INSERT INTO test SELECT * FROM source UNION ALL SELECT * FROM source ORDER BY 1 LIMIT 1000", NOT_RUN); + assertAnalysis("INSERT INTO test SELECT * FROM source ORDER BY 1 LIMIT 1000", NOT_RUN); // not outer limit clause - assertEquals(analyze("INSERT INTO test SELECT * FROM source UNION ALL SELECT * FROM source"), NOT_RUN); - assertEquals(analyze("INSERT INTO test SELECT * FROM source"), NOT_RUN); - assertEquals(analyze("INSERT INTO test SELECT * FROM (SELECT * FROM source LIMIT 1000)"), NOT_RUN); + assertAnalysis("INSERT INTO test SELECT * FROM source UNION ALL SELECT * FROM source", NOT_RUN); + assertAnalysis("INSERT INTO test SELECT * FROM source", NOT_RUN); + assertAnalysis("INSERT INTO test SELECT * FROM (SELECT * FROM source LIMIT 1000)", NOT_RUN); } @Test public void testNonDeterministic() { rowCount.set(1001); - assertEquals(analyze("INSERT INTO test SELECT * FROM source LIMIT 1000"), NON_DETERMINISTIC); + assertAnalysis("INSERT INTO test SELECT * FROM source LIMIT 1000", NON_DETERMINISTIC); assertRowCountQuery("SELECT count(1) FROM (SELECT * FROM source)"); - assertEquals(analyze("CREATE TABLE test AS (WITH f AS (select * from g) ((SELECT * FROM source UNION ALL SELECT * FROM source LIMIT 1000)))"), NON_DETERMINISTIC); + assertAnalysis("CREATE TABLE test AS (WITH f AS (select * from g) ((SELECT * FROM source UNION ALL SELECT * FROM source LIMIT 1000)))", NON_DETERMINISTIC); assertRowCountQuery("SELECT count(1) FROM (WITH f AS (select * from g) SELECT * FROM source UNION ALL SELECT * FROM source)"); - assertEquals(analyze("CREATE TABLE test AS (WITH f AS (select * from g) (SELECT * FROM source LIMIT 1000))"), NON_DETERMINISTIC); + assertAnalysis("CREATE TABLE test AS (WITH f AS (select * from g) (SELECT * FROM source LIMIT 1000))", NON_DETERMINISTIC); assertRowCountQuery("SELECT count(1) FROM (WITH f AS (select * from g) SELECT * FROM source)"); } @@ -125,26 +127,36 @@ public void testNonDeterministic() public void testDeterministic() { rowCount.set(1000); - assertEquals(analyze("INSERT INTO test SELECT * FROM source LIMIT 1000"), DETERMINISTIC); + assertAnalysis("INSERT INTO test SELECT * FROM source LIMIT 1000", DETERMINISTIC); } @Test public void testFailedDataChanged() { rowCount.set(999); - assertEquals(analyze("INSERT INTO test SELECT * FROM source LIMIT 1000"), FAILED_DATA_CHANGED); + assertAnalysis("INSERT INTO test SELECT * FROM source LIMIT 1000", FAILED_DATA_CHANGED); } - private LimitQueryDeterminismAnalysis analyze(String query) + private void assertAnalysis(String query, LimitQueryDeterminismAnalysis expectedAnalysis) { - return analyzer.analyze( + VerificationContext verificationContext = new VerificationContext(); + LimitQueryDeterminismAnalysis analysis = analyzer.analyze( new QueryBundle( TABLE_NAME, ImmutableList.of(), sqlParser.createStatement(query, PARSING_OPTIONS), ImmutableList.of(), CONTROL), - ROW_COUNT_WITH_LIMIT); + ROW_COUNT_WITH_LIMIT, + verificationContext); + + assertEquals(analysis, expectedAnalysis); + if (expectedAnalysis == NOT_RUN) { + assertFalse(verificationContext.getLimitQueryAnalysisQueryId().isPresent()); + } + else { + assertTrue(verificationContext.getLimitQueryAnalysisQueryId().isPresent()); + } } private void assertRowCountQuery(String expectedQuery)