Run integration test queries once (#10564)

* Run integration test queries once

* missed a few
This commit is contained in:
Suneet Saldanha 2020-11-09 17:34:27 -08:00 committed by GitHub
parent 83667172ce
commit cd231d8511
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 55 additions and 60 deletions

View File

@ -65,23 +65,23 @@ public abstract class AbstractTestQueryHelper<QueryResultType extends AbstractQu
public abstract String getQueryURL(String schemeAndHost); public abstract String getQueryURL(String schemeAndHost);
public void testQueriesFromFile(String filePath, int timesToRun) throws Exception public void testQueriesFromFile(String filePath) throws Exception
{ {
testQueriesFromFile(getQueryURL(broker), filePath, timesToRun); testQueriesFromFile(getQueryURL(broker), filePath);
testQueriesFromFile(getQueryURL(brokerTLS), filePath, timesToRun); testQueriesFromFile(getQueryURL(brokerTLS), filePath);
testQueriesFromFile(getQueryURL(router), filePath, timesToRun); testQueriesFromFile(getQueryURL(router), filePath);
testQueriesFromFile(getQueryURL(routerTLS), filePath, timesToRun); testQueriesFromFile(getQueryURL(routerTLS), filePath);
} }
public void testQueriesFromString(String str, int timesToRun) throws Exception public void testQueriesFromString(String str) throws Exception
{ {
testQueriesFromString(getQueryURL(broker), str, timesToRun); testQueriesFromString(getQueryURL(broker), str);
testQueriesFromString(getQueryURL(brokerTLS), str, timesToRun); testQueriesFromString(getQueryURL(brokerTLS), str);
testQueriesFromString(getQueryURL(router), str, timesToRun); testQueriesFromString(getQueryURL(router), str);
testQueriesFromString(getQueryURL(routerTLS), str, timesToRun); testQueriesFromString(getQueryURL(routerTLS), str);
} }
public void testQueriesFromFile(String url, String filePath, int timesToRun) throws Exception public void testQueriesFromFile(String url, String filePath) throws Exception
{ {
LOG.info("Starting query tests for [%s]", filePath); LOG.info("Starting query tests for [%s]", filePath);
List<QueryResultType> queries = List<QueryResultType> queries =
@ -92,10 +92,10 @@ public abstract class AbstractTestQueryHelper<QueryResultType extends AbstractQu
} }
); );
testQueries(url, queries, timesToRun); testQueries(url, queries);
} }
public void testQueriesFromString(String url, String str, int timesToRun) throws Exception public void testQueriesFromString(String url, String str) throws Exception
{ {
LOG.info("Starting query tests using\n%s", str); LOG.info("Starting query tests using\n%s", str);
List<QueryResultType> queries = List<QueryResultType> queries =
@ -105,14 +105,12 @@ public abstract class AbstractTestQueryHelper<QueryResultType extends AbstractQu
{ {
} }
); );
testQueries(url, queries, timesToRun); testQueries(url, queries);
} }
private void testQueries(String url, List<QueryResultType> queries, int timesToRun) throws Exception private void testQueries(String url, List<QueryResultType> queries) throws Exception
{ {
LOG.info("Running queries, url [%s]", url); LOG.info("Running queries, url [%s]", url);
for (int i = 0; i < timesToRun; i++) {
LOG.info("Starting Iteration %d", i);
boolean failed = false; boolean failed = false;
for (QueryResultType queryWithResult : queries) { for (QueryResultType queryWithResult : queries) {
@ -135,7 +133,6 @@ public abstract class AbstractTestQueryHelper<QueryResultType extends AbstractQu
throw new ISE("one or more queries failed"); throw new ISE("one or more queries failed");
} }
} }
}
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public int countRows(String dataSource, String interval) public int countRows(String dataSource, String interval)

View File

@ -309,7 +309,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
fullDatasourceName fullDatasourceName
); );
queryHelper.testQueriesFromString(queryResponseTemplate, 2); queryHelper.testQueriesFromString(queryResponseTemplate);
} }
private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest) throws Exception private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest) throws Exception

View File

@ -130,11 +130,11 @@ public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
submitTaskAndWait(taskSpec, fullDatasourceName, waitForNewVersion, waitForSegmentsToLoad); submitTaskAndWait(taskSpec, fullDatasourceName, waitForNewVersion, waitForSegmentsToLoad);
if (runTestQueries) { if (runTestQueries) {
doTestQuery(dataSource, queryFilePath, 2); doTestQuery(dataSource, queryFilePath);
} }
} }
protected void doTestQuery(String dataSource, String queryFilePath, int timesToRun) protected void doTestQuery(String dataSource, String queryFilePath)
{ {
try { try {
String queryResponseTemplate; String queryResponseTemplate;
@ -151,7 +151,7 @@ public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
"%%DATASOURCE%%", "%%DATASOURCE%%",
dataSource + config.getExtraDatasourceNameSuffix() dataSource + config.getExtraDatasourceNameSuffix()
); );
queryHelper.testQueriesFromString(queryResponseTemplate, timesToRun); queryHelper.testQueriesFromString(queryResponseTemplate);
} }
catch (Exception e) { catch (Exception e) {
@ -212,7 +212,7 @@ public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
fullReindexDatasourceName fullReindexDatasourceName
); );
queryHelper.testQueriesFromString(queryResponseTemplate, 2); queryHelper.testQueriesFromString(queryResponseTemplate);
// verify excluded dimension is not reIndexed // verify excluded dimension is not reIndexed
final List<String> dimensions = clientInfoResourceTestClient.getDimensions( final List<String> dimensions = clientInfoResourceTestClient.getDimensions(
fullReindexDatasourceName, fullReindexDatasourceName,
@ -241,7 +241,7 @@ public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
submitTaskAndWait(taskSpec, fullDatasourceName, false, true); submitTaskAndWait(taskSpec, fullDatasourceName, false, true);
try { try {
sqlQueryHelper.testQueriesFromFile(queryFilePath, 2); sqlQueryHelper.testQueriesFromFile(queryFilePath);
} }
catch (Exception e) { catch (Exception e) {
LOG.error(e, "Error while testing"); LOG.error(e, "Error while testing");

View File

@ -145,7 +145,7 @@ public abstract class AbstractITRealtimeIndexTaskTest extends AbstractIndexerTes
// and some on historical. Which it is depends on where in the minute we were // and some on historical. Which it is depends on where in the minute we were
// when we started posting events. // when we started posting events.
try { try {
this.queryHelper.testQueriesFromString(getRouterURL(), queryStr, 2); this.queryHelper.testQueriesFromString(getRouterURL(), queryStr);
} }
catch (Exception e) { catch (Exception e) {
throw new RuntimeException(e); throw new RuntimeException(e);
@ -164,7 +164,7 @@ public abstract class AbstractITRealtimeIndexTaskTest extends AbstractIndexerTes
); );
// queries should be answered by historical // queries should be answered by historical
this.queryHelper.testQueriesFromString(getRouterURL(), queryStr, 2); this.queryHelper.testQueriesFromString(getRouterURL(), queryStr);
} }
catch (Exception e) { catch (Exception e) {
throw new RuntimeException(e); throw new RuntimeException(e);

View File

@ -482,7 +482,7 @@ public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest
final String querySpec = generatedTestConfig.getStreamQueryPropsTransform() final String querySpec = generatedTestConfig.getStreamQueryPropsTransform()
.apply(getResourceAsString(QUERIES_FILE)); .apply(getResourceAsString(QUERIES_FILE));
// this query will probably be answered from the indexing tasks but possibly from 2 historical segments / 2 indexing // this query will probably be answered from the indexing tasks but possibly from 2 historical segments / 2 indexing
this.queryHelper.testQueriesFromString(querySpec, 2); this.queryHelper.testQueriesFromString(querySpec);
LOG.info("Shutting down supervisor"); LOG.info("Shutting down supervisor");
indexer.shutdownSupervisor(generatedTestConfig.getSupervisorId()); indexer.shutdownSupervisor(generatedTestConfig.getSupervisorId());
// Clear supervisor ID to not shutdown again. // Clear supervisor ID to not shutdown again.
@ -503,7 +503,7 @@ public abstract class AbstractStreamIndexingTest extends AbstractIndexerTest
); );
// this query will be answered by at least 1 historical segment, most likely 2, and possibly up to all 4 // this query will be answered by at least 1 historical segment, most likely 2, and possibly up to all 4
this.queryHelper.testQueriesFromString(querySpec, 2); this.queryHelper.testQueriesFromString(querySpec);
} }
long getSumOfEventSequence(int numEvents) long getSumOfEventSequence(int numEvents)

View File

@ -98,17 +98,17 @@ public class ITAppendBatchIndexTest extends AbstractITBatchIndexTest
// Submit initial ingestion task // Submit initial ingestion task
submitIngestionTaskAndVerify(indexDatasource, partitionsSpecList.get(0), false); submitIngestionTaskAndVerify(indexDatasource, partitionsSpecList.get(0), false);
verifySegmentsCountAndLoaded(indexDatasource, expectedSegmentCountList.get(0)); verifySegmentsCountAndLoaded(indexDatasource, expectedSegmentCountList.get(0));
doTestQuery(indexDatasource, INDEX_QUERIES_INITIAL_INGESTION_RESOURCE, 2); doTestQuery(indexDatasource, INDEX_QUERIES_INITIAL_INGESTION_RESOURCE);
// Submit append ingestion task // Submit append ingestion task
submitIngestionTaskAndVerify(indexDatasource, partitionsSpecList.get(1), true); submitIngestionTaskAndVerify(indexDatasource, partitionsSpecList.get(1), true);
verifySegmentsCountAndLoaded(indexDatasource, expectedSegmentCountList.get(1)); verifySegmentsCountAndLoaded(indexDatasource, expectedSegmentCountList.get(1));
doTestQuery(indexDatasource, INDEX_QUERIES_POST_APPEND_PRE_COMPACT_RESOURCE, 2); doTestQuery(indexDatasource, INDEX_QUERIES_POST_APPEND_PRE_COMPACT_RESOURCE);
// Submit compaction task // Submit compaction task
compactData(indexDatasource, COMPACTION_TASK); compactData(indexDatasource, COMPACTION_TASK);
// Verification post compaction // Verification post compaction
verifySegmentsCountAndLoaded(indexDatasource, expectedSegmentCountList.get(2)); verifySegmentsCountAndLoaded(indexDatasource, expectedSegmentCountList.get(2));
verifySegmentsCompacted(indexDatasource, expectedSegmentCountList.get(2)); verifySegmentsCompacted(indexDatasource, expectedSegmentCountList.get(2));
doTestQuery(indexDatasource, INDEX_QUERIES_POST_APPEND_POST_COMPACT_RESOURCE, 2); doTestQuery(indexDatasource, INDEX_QUERIES_POST_APPEND_POST_COMPACT_RESOURCE);
} }
} }

View File

@ -97,12 +97,12 @@ public class ITCompactionTaskTest extends AbstractIndexerTest
); );
queryHelper.testQueriesFromString(queryResponseTemplate, 2); queryHelper.testQueriesFromString(queryResponseTemplate);
compactData(); compactData();
// 4 segments across 2 days, compacted into 2 new segments (6 total) // 4 segments across 2 days, compacted into 2 new segments (6 total)
checkCompactionFinished(6); checkCompactionFinished(6);
queryHelper.testQueriesFromString(queryResponseTemplate, 2); queryHelper.testQueriesFromString(queryResponseTemplate);
checkCompactionIntervals(intervalsBeforeCompaction); checkCompactionIntervals(intervalsBeforeCompaction);
} }

View File

@ -89,7 +89,7 @@ public class ITNestedQueryPushDownTest extends AbstractIndexerTest
fullDatasourceName fullDatasourceName
); );
queryHelper.testQueriesFromString(queryResponseTemplate, 2); queryHelper.testQueriesFromString(queryResponseTemplate);
} }
catch (Exception e) { catch (Exception e) {
LOG.error(e, "Error while testing"); LOG.error(e, "Error while testing");

View File

@ -95,8 +95,7 @@ public class ITBroadcastJoinQueryTest extends AbstractIndexerTest
replaceJoinTemplate( replaceJoinTemplate(
getResourceAsString(BROADCAST_JOIN_METADATA_QUERIES_RESOURCE), getResourceAsString(BROADCAST_JOIN_METADATA_QUERIES_RESOURCE),
BROADCAST_JOIN_DATASOURCE BROADCAST_JOIN_DATASOURCE
), )
1
); );
return true; return true;
} }
@ -110,8 +109,7 @@ public class ITBroadcastJoinQueryTest extends AbstractIndexerTest
// now do some queries // now do some queries
queryHelper.testQueriesFromString( queryHelper.testQueriesFromString(
queryHelper.getQueryURL(config.getRouterUrl()), queryHelper.getQueryURL(config.getRouterUrl()),
replaceJoinTemplate(getResourceAsString(BROADCAST_JOIN_QUERIES_RESOURCE), BROADCAST_JOIN_DATASOURCE), replaceJoinTemplate(getResourceAsString(BROADCAST_JOIN_QUERIES_RESOURCE), BROADCAST_JOIN_DATASOURCE)
1
); );
} }
finally { finally {

View File

@ -63,7 +63,7 @@ public class ITSystemTableQueryTest
public void testSystemTableQueries() public void testSystemTableQueries()
{ {
try { try {
this.queryHelper.testQueriesFromFile(SYSTEM_QUERIES_RESOURCE, 2); this.queryHelper.testQueriesFromFile(SYSTEM_QUERIES_RESOURCE);
} }
catch (Exception e) { catch (Exception e) {
throw new RuntimeException(e); throw new RuntimeException(e);

View File

@ -52,7 +52,7 @@ public class ITTwitterQueryTest
@Test @Test
public void testTwitterQueriesFromFile() throws Exception public void testTwitterQueriesFromFile() throws Exception
{ {
queryHelper.testQueriesFromFile(TWITTER_QUERIES_RESOURCE, 2); queryHelper.testQueriesFromFile(TWITTER_QUERIES_RESOURCE);
} }
} }

View File

@ -151,7 +151,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest
fullDatasourceName fullDatasourceName
); );
this.queryHelper.testQueriesFromString(queryResponseTemplate, 2); this.queryHelper.testQueriesFromString(queryResponseTemplate);
// wait for the task to complete // wait for the task to complete
for (int i = 0; i < numTasks; i++) { for (int i = 0; i < numTasks; i++) {
@ -169,7 +169,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest
); );
} }
// run queries on historical nodes // run queries on historical nodes
this.queryHelper.testQueriesFromString(queryResponseTemplate, 2); this.queryHelper.testQueriesFromString(queryResponseTemplate);
} }
catch (Throwable e) { catch (Throwable e) {

View File

@ -79,7 +79,7 @@ public class ITWikipediaQueryTest
@Test @Test
public void testWikipediaQueriesFromFile() throws Exception public void testWikipediaQueriesFromFile() throws Exception
{ {
queryHelper.testQueriesFromFile(WIKIPEDIA_QUERIES_RESOURCE, 2); queryHelper.testQueriesFromFile(WIKIPEDIA_QUERIES_RESOURCE);
} }
@Test @Test