-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Browse files
Browse the repository at this point in the history
- Loading branch information
Showing
18 changed files
with
1,012 additions
and
69 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
118 changes: 118 additions & 0 deletions
118
ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/KsqlTestException.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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()) | ||
); | ||
} | ||
|
||
} |
104 changes: 104 additions & 0 deletions
104
ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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(); | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.