Skip to content

Commit

Permalink
feat: introduce the sql-based testing tool (YATT) (#6051)
Browse files Browse the repository at this point in the history
  • Loading branch information
agavra authored Aug 19, 2020
1 parent 994c902 commit 33e71c3
Show file tree
Hide file tree
Showing 18 changed files with 1,012 additions and 69 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ void sendRecord(
}

@VisibleForTesting
InsertValuesExecutor(
public InsertValuesExecutor(
final boolean canBeDisabledByConfig,
final RecordProducer producer
) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -234,13 +234,25 @@ public TransientQueryMetadata executeQuery(
}
}

@Override
public void close() {
allLiveQueries.forEach(QueryMetadata::stop);
/**
* @param closeQueries whether or not to clean up the local state for any running queries
*/
public void close(final boolean closeQueries) {
if (closeQueries) {
allLiveQueries.forEach(QueryMetadata::close);
} else {
allLiveQueries.forEach(QueryMetadata::stop);
}

engineMetrics.close();
aggregateMetricsCollector.shutdown();
}

@Override
public void close() {
close(false);
}

/**
* Determines if a statement is executable by the engine.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,10 @@ public ExecutionStep<?> getPhysicalPlan() {
return physicalPlan;
}

public DataSource getSink() {
return sinkDataSource;
}

@VisibleForTesting
Optional<MaterializationProvider> getMaterializationProvider() {
return materializationProvider;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public abstract class QueryMetadata {
private final KafkaStreamsBuilder kafkaStreamsBuilder;
private final Map<String, Object> streamsProperties;
private final Map<String, Object> overriddenProperties;
private final Consumer<QueryMetadata> closeCallback;
private Consumer<QueryMetadata> closeCallback;
private final Set<SourceName> sourceNames;
private final LogicalSchema logicalSchema;
private final Long closeTimeout;
Expand Down Expand Up @@ -142,6 +142,10 @@ public void setQueryStateListener(final QueryStateListener queryStateListener) {
queryStateListener.onChange(kafkaStreams.state(), kafkaStreams.state());
}

public void closeAndThen(final Consumer<QueryMetadata> andThen) {
this.closeCallback = closeCallback.andThen(andThen);
}

private void uncaughtHandler(final Thread t, final Throwable e) {
LOG.error("Unhandled exception caught in streams thread {}.", t.getName(), e);
final QueryError queryError =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
/*
* Copyright 2020 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.test;

import io.confluent.ksql.parser.KsqlParser.ParsedStatement;
import io.confluent.ksql.parser.NodeLocation;
import io.confluent.ksql.parser.SqlFormatter;
import io.confluent.ksql.parser.tree.AssertStatement;
import io.confluent.ksql.test.model.LocationWithinFile;
import io.confluent.ksql.test.parser.TestDirective;
import io.confluent.ksql.test.parser.TestStatement;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.ParserUtil;
import java.nio.file.Path;
import java.util.Objects;
import java.util.Optional;

/**
* Indicates a test exception as well as where it occurred. All sql-driven
* tests should throw this exception at the top-level if possible in order
* to automatically populate the statement that produced it as well as the
* location in the file.
*/
public class KsqlTestException extends KsqlException {

public KsqlTestException(
final TestStatement statement,
final Path file,
final Throwable cause
) {
super(getMessage(statement, cause.getMessage(), file), cause);
}

public KsqlTestException(
final TestStatement statement,
final Path file,
final String message
) {
super(getMessage(statement, message, file));
}

private static String getMessage(
final TestStatement stmt,
final String message,
final Path file
) {
return stmt.apply(
parsed -> engineMessage(parsed, message, file),
assertStatement -> assertMessage(assertStatement, message, file),
directive -> directiveMessage(directive, message, file)
);
}

private static String engineMessage(
final ParsedStatement parsedStatement,
final String message,
final Path file
) {
final Optional<NodeLocation> loc = ParserUtil.getLocation(
parsedStatement.getStatement());

return String.format(
"Test failure for statement `%s` (%s):%n\t%s%n\t%s",
parsedStatement.getStatementText(),
loc.map(NodeLocation::toString).orElse("unknown"),
message,
new LocationWithinFile(
file,
loc.map(NodeLocation::getLineNumber).orElse(1))
);
}

private static String assertMessage(
final AssertStatement assertStatement,
final String message,
final Path file
) {
return String.format(
"Test failure for assert `%s` (%s):%n\t%s%n\t%s",
SqlFormatter.formatSql(assertStatement),
assertStatement.getLocation().map(Objects::toString).orElse("unknown"),
message,
new LocationWithinFile(
file,
assertStatement.getLocation().map(NodeLocation::getLineNumber).orElse(1))
);
}

private static String directiveMessage(
final TestDirective directive,
final String message,
final Path file
) {
return String.format(
"Test failure during directive evaluation `%s` (%s):%n\t%s%n\t%s",
directive,
directive.getLocation(),
message,
new LocationWithinFile(
file,
directive.getLocation().getLineNumber())
);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* Copyright 2020 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.test.driver;

import io.confluent.ksql.GenericRow;
import io.confluent.ksql.KsqlExecutionContext;
import io.confluent.ksql.engine.generic.GenericRecordFactory;
import io.confluent.ksql.engine.generic.KsqlGenericRecord;
import io.confluent.ksql.metastore.model.DataSource;
import io.confluent.ksql.parser.AssertTable;
import io.confluent.ksql.parser.tree.AssertStream;
import io.confluent.ksql.parser.tree.AssertValues;
import io.confluent.ksql.parser.tree.InsertValues;
import io.confluent.ksql.schema.ksql.SystemColumns;
import io.confluent.ksql.util.KsqlConfig;
import io.confluent.ksql.util.KsqlException;
import java.util.Iterator;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.streams.test.TestRecord;

/**
* {@code AssertExecutor} handles the assertion statements for the Sql-based
* testing tool.
*/
public final class AssertExecutor {

private AssertExecutor() {
}

public static void assertValues(
final KsqlExecutionContext engine,
final KsqlConfig config,
final AssertValues assertValues,
final TestDriverPipeline driverPipeline
) {
final InsertValues values = assertValues.getStatement();
final boolean compareTimestamp = values
.getColumns()
.stream()
.anyMatch(SystemColumns.ROWTIME_NAME::equals);

final DataSource dataSource = engine.getMetaStore().getSource(values.getTarget());
final KsqlGenericRecord expected = new GenericRecordFactory(
config, engine.getMetaStore(), System::currentTimeMillis
).build(
values.getColumns(),
values.getValues(),
dataSource.getSchema(),
dataSource.getDataSourceType()
);

final Iterator<TestRecord<Struct, GenericRow>> records = driverPipeline
.getRecordsForTopic(dataSource.getKafkaTopicName());
if (!records.hasNext()) {
throw new KsqlException(
String.format(
"Expected another record (%s) for %s but already read all records: %s",
expected,
dataSource.getName(),
driverPipeline.getAllRecordsForTopic(dataSource.getKafkaTopicName())
)
);
}

final TestRecord<Struct, GenericRow> actualTestRecord = records.next();
final KsqlGenericRecord actual = KsqlGenericRecord.of(
actualTestRecord.key(),
actualTestRecord.value(),
compareTimestamp ? actualTestRecord.timestamp() : expected.ts
);

if (!actual.equals(expected)) {
throw new KsqlException(
String.format(
"Expected record does not match actual. Expected: %s vs. Actual: %s",
expected,
actual
)
);
}
}

public static void assertStream(final AssertStream assertStatement) {
throw new UnsupportedOperationException();
}

public static void assertTable(final AssertTable assertStatement) {
throw new UnsupportedOperationException();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -111,13 +111,18 @@ public TopicInfo(
private final ListMultimap<String, Input> inputs;
private final ListMultimap<String, Output> outputs;
private final ListMultimap<String, TestRecord<Struct, GenericRow>> outputCache;
private final Map<String, Iterator<TestRecord<Struct, GenericRow>>> outputIterators;

// this map indexes into the outputCache to track which records we've already
// read - we don't need to worry about concurrent modification while iterating
// because appends only happen at the end of the outputCache
private final Map<String, Integer> assertPositions;

public TestDriverPipeline() {
inputs = ArrayListMultimap.create();
outputs = ArrayListMultimap.create();
outputCache = ArrayListMultimap.create();
outputIterators = new HashMap<>();

assertPositions = new HashMap<>();
}

public void addDriver(
Expand Down Expand Up @@ -182,7 +187,7 @@ private void pipeInput(
for (final Input input : inputs) {
input.topic.pipeInput(key, value, timestampMs);

// handle the fallout of piping in a record (propegation)
// handle the fallout of piping in a record (propagation)
for (final Output receiver : input.receivers) {
for (final TestRecord<Struct, GenericRow> record : receiver.topic.readRecordsToList()) {
outputCache.put(receiver.name, record);
Expand All @@ -207,7 +212,19 @@ public List<TestRecord<Struct, GenericRow>> getAllRecordsForTopic(final String t
}

public Iterator<TestRecord<Struct, GenericRow>> getRecordsForTopic(final String topic) {
return outputIterators.computeIfAbsent(topic, name -> outputCache.get(topic).iterator());
return new Iterator<TestRecord<Struct, GenericRow>>() {
@Override
public boolean hasNext() {
final int idx = assertPositions.getOrDefault(topic, 0);
return outputCache.get(topic).size() > idx;
}

@Override
public TestRecord<Struct, GenericRow> next() {
final int idx = assertPositions.merge(topic, 0, (old, zero) -> old + 1);
return outputCache.get(topic).get(idx);
}
};
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

import com.google.common.collect.ImmutableList;
import io.confluent.ksql.test.tools.Test;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.stream.Stream;
Expand All @@ -34,7 +35,7 @@ public interface TestLoader<T extends Test> {
// mvn test -pl ksql-engine -Dtest=QueryTranslationTest -Dksql.test.files=test1.json,test2,json
String KSQL_TEST_FILES = "ksql.test.files";

Stream<T> load();
Stream<T> load() throws IOException;

static List<String> getWhiteList() {
final String ksqlTestFiles = System.getProperty(KSQL_TEST_FILES, "").trim();
Expand Down
Loading

0 comments on commit 33e71c3

Please sign in to comment.