From 3a0eaafb5e6e7d6a6b60fa27b685823070bca3b4 Mon Sep 17 00:00:00 2001 From: Eugene Date: Mon, 15 Nov 2021 12:16:36 +0200 Subject: [PATCH] =?UTF-8?q?=F0=9F=8E=89=20Source=20Mysql=20-=20added=20bas?= =?UTF-8?q?ic=20MySql=20performance=20tests=20(#7820)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * [ticket 7489, PR 7820] Source Mysql - added skeleton for MySql performance tests --- .../source/AbstractSourceConnectorTest.java | 31 +++ .../source/AbstractSourcePerformanceTest.java | 219 ++++++++++++++++++ .../connectors/source-mysql/README.md | 39 ++++ .../connectors/source-mysql/build.gradle | 5 + .../mysql/MySqlSourcePerformanceTest.java | 148 ++++++++++++ .../airbyte-performance-test-java.gradle | 53 +++++ docs/integrations/sources/mysql.md | 3 +- tools/bin/ci_performance_test.sh | 74 ++++++ 8 files changed, 571 insertions(+), 1 deletion(-) create mode 100644 airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourcePerformanceTest.java create mode 100644 airbyte-integrations/connectors/source-mysql/README.md create mode 100644 airbyte-integrations/connectors/source-mysql/src/test-performance/java/io/airbyte/integrations/source/mysql/MySqlSourcePerformanceTest.java create mode 100644 buildSrc/src/main/groovy/airbyte-performance-test-java.gradle create mode 100755 tools/bin/ci_performance_test.sh diff --git a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java index 65036c5e84ad..369293472be3 100644 --- a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java +++ b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourceConnectorTest.java @@ -16,6 +16,8 @@ import io.airbyte.config.WorkerSourceConfig; import io.airbyte.protocol.models.AirbyteCatalog; import io.airbyte.protocol.models.AirbyteMessage; +import io.airbyte.protocol.models.AirbyteMessage.Type; +import io.airbyte.protocol.models.AirbyteRecordMessage; import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; import io.airbyte.protocol.models.ConnectorSpecification; import io.airbyte.workers.DefaultCheckConnectionWorker; @@ -32,6 +34,8 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.Optional; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -155,4 +159,31 @@ protected List runRead(final ConfiguredAirbyteCatalog catalog, f return messages; } + protected Map runReadVerifyNumberOfReceivedMsgs(final ConfiguredAirbyteCatalog catalog, + final JsonNode state, + final Map mapOfExpectedRecordsCount) + throws Exception { + + final WorkerSourceConfig sourceConfig = new WorkerSourceConfig() + .withSourceConnectionConfiguration(getConfig()) + .withState(state == null ? null : new State().withState(state)) + .withCatalog(catalog); + + final AirbyteSource source = new DefaultAirbyteSource(new AirbyteIntegrationLauncher(JOB_ID, JOB_ATTEMPT, getImageName(), processFactory)); + source.start(sourceConfig, jobRoot); + + while (!source.isFinished()) { + Optional airbyteMessageOptional = source.attemptRead(); + if (airbyteMessageOptional.isPresent() && airbyteMessageOptional.get().getType().equals(Type.RECORD)) { + AirbyteMessage airbyteMessage = airbyteMessageOptional.get(); + AirbyteRecordMessage record = airbyteMessage.getRecord(); + + final String streamName = record.getStream(); + mapOfExpectedRecordsCount.put(streamName, mapOfExpectedRecordsCount.get(streamName) - 1); + } + } + source.close(); + return mapOfExpectedRecordsCount; + } + } diff --git a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourcePerformanceTest.java b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourcePerformanceTest.java new file mode 100644 index 000000000000..abdf8be8df57 --- /dev/null +++ b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/AbstractSourcePerformanceTest.java @@ -0,0 +1,219 @@ +/* + * Copyright (c) 2021 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.standardtest.source; + +import static org.junit.jupiter.api.Assertions.fail; + +import com.google.common.collect.Lists; +import io.airbyte.db.Database; +import io.airbyte.protocol.models.AirbyteStream; +import io.airbyte.protocol.models.CatalogHelpers; +import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.DestinationSyncMode; +import io.airbyte.protocol.models.Field; +import io.airbyte.protocol.models.JsonSchemaPrimitive; +import io.airbyte.protocol.models.SyncMode; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.StringJoiner; +import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This abstract class contains common helpers and boilerplate for comprehensively testing that all + * data types in a source can be read and handled correctly by the connector and within Airbyte's + * type system. + */ +public abstract class AbstractSourcePerformanceTest extends AbstractSourceConnectorTest { + + protected static final Logger c = LoggerFactory.getLogger(AbstractSourcePerformanceTest.class); + private static final String TEST_VALUE_TEMPLATE = "\"Some test value %s\""; + protected String databaseName = "test"; + + protected int numberOfColumns; // 240 is near the max value for varchar(8) type + // 200 is near the max value for 1 batch call,if need more - implement multiple batching for single + // stream + protected int numberOfDummyRecords; // 200; + protected int numberOfStreams; + + /** + * The column name will be used for a PK column in the test tables. Override it if default name is + * not valid for your source. + * + * @return Id column name + */ + protected String getIdColumnName() { + return "id"; + } + + /** + * The column name will be used for a test column in the test tables. Override it if default name is + * not valid for your source. + * + * @return Test column name + */ + protected String getTestColumnName() { + return "test_column"; + } + + /** + * The stream name template will be used for a test tables. Override it if default name is not valid + * for your source. + * + * @return Test steam name template + */ + protected String getTestStreamNameTemplate() { + return "test_%S"; + } + + /** + * Setup the test database. All tables and data described in the registered tests will be put there. + * + * @return configured test database + * @throws Exception - might throw any exception during initialization. + */ + protected abstract Database setupDatabase() throws Exception; + + /** + * Get a create table template for a DB + * + * @return a create tabple template, ex. "CREATE TABLE test.%s(id INTEGER PRIMARY KEY, %s)" + */ + protected abstract String getCreateTableTemplate(); + + /** + * Get a INSERT query template for a DB + * + * @return an INSERT into table query template, ex. "INSERT INTO test.%s (%s) VALUES %s" + */ + protected abstract String getInsertQueryTemplate(); + + /** + * Get a test field'stype that will be used in DB for table creation. + * + * @return a test's field type. Ex: varchar(8) + */ + protected abstract String getTestFieldType(); + + @Override + protected void setupEnvironment(final TestDestinationEnv environment) throws Exception { + // DO NOTHING. Mandatory to override. DB will be setup as part of each test + } + + /** + * Provide a source namespace. It's allocated place for table creation. It also known ask "Database + * Schema" or "Dataset" + * + * @return source name space + */ + protected abstract String getNameSpace(); + + protected void validateNumberOfReceivedMsgs(final Map checkStatusMap) { + // Iterate through all streams map and check for streams where + Map failedStreamsMap = checkStatusMap.entrySet().stream() + .filter(el -> el.getValue() != 0).collect(Collectors.toMap(Entry::getKey, Entry::getValue)); + + if (!failedStreamsMap.isEmpty()) { + fail("Non all messages were delivered. " + failedStreamsMap.toString()); + } + c.info("Finished all checks, no issues found for {} of streams", checkStatusMap.size()); + } + + protected Map prepareMapWithExpectedRecords(final int streamNumber, + final int expectedRecordsNumberInEachStream) { + Map resultMap = new HashMap<>(); // streamName&expected records in stream + + for (int currentStream = 0; currentStream < streamNumber; currentStream++) { + final String streamName = String.format(getTestStreamNameTemplate(), currentStream); + resultMap.put(streamName, expectedRecordsNumberInEachStream); + } + return resultMap; + } + + protected String prepareCreateTableQuery(final int numberOfColumns, + final String currentTableName) { + + StringJoiner sj = new StringJoiner(","); + for (int i = 0; i < numberOfColumns; i++) { + sj.add(String.format(" %s%s %s", getTestColumnName(), i, getTestFieldType())); + } + + return String.format(getCreateTableTemplate(), databaseName, currentTableName, sj.toString()); + } + + // ex. INSERT INTO test.test_1 (id, test_column0, test_column1) VALUES (101,"zzz0", "sss0"), ("102", + // "zzzz1", "sss1"); + protected String prepareInsertQueryTemplate(final int numberOfColumns, final int recordsNumber) { + + StringJoiner fieldsNames = new StringJoiner(","); + fieldsNames.add("id"); + + StringJoiner baseInsertQuery = new StringJoiner(","); + baseInsertQuery.add("%s"); + + for (int i = 0; i < numberOfColumns; i++) { + fieldsNames.add(getTestColumnName() + i); + baseInsertQuery.add(String.format(TEST_VALUE_TEMPLATE, i)); + } + + StringJoiner insertGroupValuesJoiner = new StringJoiner(","); + + for (int currentRecordNumber = 0; currentRecordNumber < recordsNumber; currentRecordNumber++) { + insertGroupValuesJoiner + .add("(" + String.format(baseInsertQuery.toString(), currentRecordNumber) + ")"); + } + + return String + .format(getInsertQueryTemplate(), databaseName, "%s", fieldsNames.toString(), + insertGroupValuesJoiner.toString()); + } + + /** + * Configures streams for all registered data type tests. + * + * @return configured catalog + */ + protected ConfiguredAirbyteCatalog getConfiguredCatalog() { + List streams = new ArrayList<>(); + + for (int currentStream = 0; currentStream < numberOfStreams; currentStream++) { + + // CREATE TABLE test.test_1_int(id INTEGER PRIMARY KEY, test_column int) + List fields = new ArrayList<>(); + + fields.add(Field.of(getIdColumnName(), JsonSchemaPrimitive.NUMBER)); + for (int currentColumnNumber = 0; + currentColumnNumber < numberOfColumns; + currentColumnNumber++) { + fields.add(Field.of(getTestColumnName() + currentColumnNumber, JsonSchemaPrimitive.STRING)); + } + + AirbyteStream airbyteStream = CatalogHelpers + .createAirbyteStream(String.format(getTestStreamNameTemplate(), currentStream), + getNameSpace(), fields) + .withSourceDefinedCursor(true) + .withSourceDefinedPrimaryKey(List.of(List.of(getIdColumnName()))) + .withSupportedSyncModes( + Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)); + + ConfiguredAirbyteStream configuredAirbyteStream = new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.INCREMENTAL) + .withCursorField(Lists.newArrayList(getIdColumnName())) + .withDestinationSyncMode(DestinationSyncMode.APPEND) + .withStream(airbyteStream); + + streams.add(configuredAirbyteStream); + + } + + return new ConfiguredAirbyteCatalog().withStreams(streams); + } + +} diff --git a/airbyte-integrations/connectors/source-mysql/README.md b/airbyte-integrations/connectors/source-mysql/README.md new file mode 100644 index 000000000000..f533c0207473 --- /dev/null +++ b/airbyte-integrations/connectors/source-mysql/README.md @@ -0,0 +1,39 @@ +# MySQL Source + +## Documentation +This is the repository for the MySQL only source connector in Java. +For information about how to use this connector within Airbyte, see [User Documentation](https://docs.airbyte.io/integrations/sources/mysql) + +## Local development + +#### Building via Gradle +From the Airbyte repository root, run: +``` +./gradlew :airbyte-integrations:connectors:source-mysql:build +``` + +### Locally running the connector docker image + +#### Build +Build the connector image via Gradle: +``` +./gradlew :airbyte-integrations:connectors:source-mysql:airbyteDocker +``` +When building via Gradle, the docker image name and tag, respectively, are the values of the `io.airbyte.name` and `io.airbyte.version` `LABEL`s in +the Dockerfile. + +## Testing +We use `JUnit` for Java tests. + +### Test Configuration +#### Acceptance Tests +To run acceptance and custom integration tests: +``` +./gradlew :airbyte-integrations:connectors:source-mysql:integrationTest +``` + +#### Performance Tests +To run performance tests: +``` +./gradlew :airbyte-integrations:connectors:source-mysql:performanceTest +``` \ No newline at end of file diff --git a/airbyte-integrations/connectors/source-mysql/build.gradle b/airbyte-integrations/connectors/source-mysql/build.gradle index 12f0ed5450eb..4345574db117 100644 --- a/airbyte-integrations/connectors/source-mysql/build.gradle +++ b/airbyte-integrations/connectors/source-mysql/build.gradle @@ -2,6 +2,7 @@ plugins { id 'application' id 'airbyte-docker' id 'airbyte-integration-test-java' + id 'airbyte-performance-test-java' } application { @@ -28,6 +29,10 @@ dependencies { integrationTestJavaImplementation project(':airbyte-integrations:bases:standard-source-test') integrationTestJavaImplementation project(':airbyte-integrations:connectors:source-mysql') + performanceTestJavaImplementation project(':airbyte-integrations:bases:standard-source-test') + performanceTestJavaImplementation project(':airbyte-integrations:connectors:source-mysql') + implementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs) integrationTestJavaImplementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs) + performanceTestJavaImplementation files(project(':airbyte-integrations:bases:base-java').airbyteDocker.outputs) } diff --git a/airbyte-integrations/connectors/source-mysql/src/test-performance/java/io/airbyte/integrations/source/mysql/MySqlSourcePerformanceTest.java b/airbyte-integrations/connectors/source-mysql/src/test-performance/java/io/airbyte/integrations/source/mysql/MySqlSourcePerformanceTest.java new file mode 100644 index 000000000000..98ffbf9fd65c --- /dev/null +++ b/airbyte-integrations/connectors/source-mysql/src/test-performance/java/io/airbyte/integrations/source/mysql/MySqlSourcePerformanceTest.java @@ -0,0 +1,148 @@ +/* + * Copyright (c) 2021 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.source.mysql; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableMap; +import io.airbyte.commons.json.Jsons; +import io.airbyte.db.Database; +import io.airbyte.db.Databases; +import io.airbyte.integrations.source.mysql.MySqlSource.ReplicationMethod; +import io.airbyte.integrations.standardtest.source.AbstractSourcePerformanceTest; +import io.airbyte.integrations.standardtest.source.TestDestinationEnv; +import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; +import java.util.Map; +import org.jooq.SQLDialect; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.MySQLContainer; + +public class MySqlSourcePerformanceTest extends AbstractSourcePerformanceTest { + + private MySQLContainer container; + private JsonNode config; + private static final String CREATE_DB_TABLE_TEMPLATE = "CREATE TABLE %s.%s(id INTEGER PRIMARY KEY, %s)"; + private static final String INSERT_INTO_DB_TABLE_QUERY_TEMPLATE = "INSERT INTO %s.%s (%s) VALUES %s"; + private static final String TEST_DB_FIELD_TYPE = "varchar(8)"; + + @Override + protected JsonNode getConfig() { + return config; + } + + @Override + protected void tearDown(final TestDestinationEnv testEnv) { + container.close(); + } + + @Override + protected String getImageName() { + return "airbyte/source-mysql:dev"; + } + + @Override + protected Database setupDatabase() throws Exception { + container = new MySQLContainer<>("mysql:8.0"); + container.start(); + + config = Jsons.jsonNode(ImmutableMap.builder() + .put("host", container.getHost()) + .put("port", container.getFirstMappedPort()) + .put("database", container.getDatabaseName()) + .put("username", container.getUsername()) + .put("password", container.getPassword()) + .put("replication_method", ReplicationMethod.STANDARD) + .build()); + + final Database database = Databases.createDatabase( + config.get("username").asText(), + config.get("password").asText(), + String.format("jdbc:mysql://%s:%s/%s", + config.get("host").asText(), + config.get("port").asText(), + config.get("database").asText()), + "com.mysql.cj.jdbc.Driver", + SQLDialect.MYSQL, + "zeroDateTimeBehavior=convertToNull"); + + super.databaseName = container.getDatabaseName(); + + // It disable strict mode in the DB and allows to insert specific values. + // For example, it's possible to insert date with zero values "2021-00-00" + database.query(ctx -> ctx.execute("SET @@sql_mode=''")); + + return database; + } + + @Override + protected String getCreateTableTemplate() { + return CREATE_DB_TABLE_TEMPLATE; + } + + @Override + protected String getInsertQueryTemplate() { + return INSERT_INTO_DB_TABLE_QUERY_TEMPLATE; + } + + @Override + protected String getTestFieldType() { + return TEST_DB_FIELD_TYPE; + } + + @Override + protected String getNameSpace() { + return container.getDatabaseName(); + } + + /** + * Creates all tables and insert data described in the registered data type tests. + * + * @throws Exception might raise exception if configuration goes wrong or tables creation/insert + * scripts failed. + */ + private void setupDatabaseInternal() throws Exception { + final Database database = setupDatabase(); + + database.query(ctx -> { + String insertQueryTemplate = prepareInsertQueryTemplate(numberOfColumns, + numberOfDummyRecords); + + for (int currentSteamNumber = 0; currentSteamNumber < numberOfStreams; currentSteamNumber++) { + // CREATE TABLE test.test_1_int(id INTEGER PRIMARY KEY, test_column int) + + String currentTableName = String.format(getTestStreamNameTemplate(), currentSteamNumber); + + ctx.fetch(prepareCreateTableQuery(numberOfColumns, currentTableName)); + ctx.fetch(String.format(insertQueryTemplate, currentTableName)); + + c.info("Finished processing for stream " + currentSteamNumber); + } + return null; + }); + + database.close(); + } + + /** + * The test checks that connector can fetch prepared data without failure. + */ + @Test + public void test1000Streams() throws Exception { + numberOfColumns = 240; // 240 is near the max value for varchar(8) type + // 200 is near the max value for 1 batch call,if need more - implement multiple batching for single + // stream + numberOfDummyRecords = 20; // 200; + numberOfStreams = 1000; + + setupDatabaseInternal(); + + final ConfiguredAirbyteCatalog catalog = getConfiguredCatalog(); + final Map mapOfExpectedRecordsCount = prepareMapWithExpectedRecords( + numberOfStreams, numberOfDummyRecords); + final Map checkStatusMap = + runReadVerifyNumberOfReceivedMsgs(catalog, null, mapOfExpectedRecordsCount); + validateNumberOfReceivedMsgs(checkStatusMap); + } + +} diff --git a/buildSrc/src/main/groovy/airbyte-performance-test-java.gradle b/buildSrc/src/main/groovy/airbyte-performance-test-java.gradle new file mode 100644 index 000000000000..99d4b93125ff --- /dev/null +++ b/buildSrc/src/main/groovy/airbyte-performance-test-java.gradle @@ -0,0 +1,53 @@ +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.tasks.testing.Test + +class AirbytePerformanceTestJavaPlugin implements Plugin { + void apply(Project project) { + project.sourceSets { + performanceTestJava { + java { + srcDir 'src/test-performance/java' + } + resources { + srcDir 'src/test-performance/resources' + } + } + } + project.test.dependsOn('compilePerformanceTestJavaJava') + + project.configurations { + performanceTestJavaImplementation.extendsFrom testImplementation + performanceTestJavaRuntimeOnly.extendsFrom testRuntimeOnly + } + + project.task('performanceTestJava', type: Test) { + testClassesDirs += project.sourceSets.performanceTestJava.output.classesDirs + classpath += project.sourceSets.performanceTestJava.runtimeClasspath + + useJUnitPlatform() + testLogging() { + events "passed", "failed" + exceptionFormat "full" + showStandardStreams = true + } + + outputs.upToDateWhen { false } + + if(project.hasProperty('airbyteDocker')) { + dependsOn project.airbyteDocker + } + + maxHeapSize = '3g' + + mustRunAfter project.test + } + + // make sure we create the performanceTest task once in case a standard source test was already initialized + if(!project.hasProperty('performanceTest')) { + project.task('performanceTest') + } + + project.performanceTest.dependsOn(project.performanceTestJava) + } +} diff --git a/docs/integrations/sources/mysql.md b/docs/integrations/sources/mysql.md index d502cd5f6de3..05047709b442 100644 --- a/docs/integrations/sources/mysql.md +++ b/docs/integrations/sources/mysql.md @@ -180,7 +180,8 @@ If you do not see a type in this list, assume that it is coerced into a string. | Version | Date | Pull Request | Subject | | :--- | :--- | :--- | :--- | -| 0.4.9 | 2021-11-02 | [7559](https://github.com/airbytehq/airbyte/pull/7559) | Correctly process large unsigned short integer values which may fall outside java's `Short` data type capability| +| 0.4.10| 2021-11-15 | [7820](https://github.com/airbytehq/airbyte/pull/7820) | Added basic performance test | +| 0.4.9 | 2021-11-02 | [7559](https://github.com/airbytehq/airbyte/pull/7559) | Correctly process large unsigned short integer values which may fall outside java's `Short` data type capability | | 0.4.8 | 2021-09-16 | [6093](https://github.com/airbytehq/airbyte/pull/6093) | Improve reliability of processing various data types like decimals, dates, datetime, binary, and text | | 0.4.7 | 2021-09-30 | [6585](https://github.com/airbytehq/airbyte/pull/6585) | Improved SSH Tunnel key generation steps | | 0.4.6 | 2021-09-29 | [6510](https://github.com/airbytehq/airbyte/pull/6510) | Support SSL connection | diff --git a/tools/bin/ci_performance_test.sh b/tools/bin/ci_performance_test.sh new file mode 100755 index 000000000000..50ff997978b3 --- /dev/null +++ b/tools/bin/ci_performance_test.sh @@ -0,0 +1,74 @@ +#!/usr/bin/env bash + +set -e + +. tools/lib/lib.sh + +# runs performance tests for an performance name + +connector="$1" +all_performance_tests=$(./gradlew performanceTest --dry-run | grep 'performanceTest SKIPPED' | cut -d: -f 4) +run() { +if [[ "$connector" == "all" ]] ; then + echo "Running: ./gradlew --no-daemon --scan performanceTest" + ./gradlew --no-daemon --scan performanceTest +else + if [[ "$connector" == *"base-normalization"* ]]; then + selected_performance_test="base-normalization" + performanceTestCommand="$(_to_gradle_path "airbyte-integrations/bases/base-normalization" performanceTest)" + export SUB_BUILD="CONNECTORS_BASE" + # avoid schema conflicts when multiple tests for normalization are run concurrently + export RANDOM_TEST_SCHEMA="true" + ./gradlew --no-daemon --scan airbyteDocker + elif [[ "$connector" == *"bases"* ]]; then + connector_name=$(echo $connector | cut -d / -f 2) + selected_performance_test=$(echo "$all_performance_tests" | grep "^$connector_name$" || echo "") + performanceTestCommand="$(_to_gradle_path "airbyte-integrations/$connector" performanceTest)" + export SUB_BUILD="CONNECTORS_BASE" + elif [[ "$connector" == *"connectors"* ]]; then + connector_name=$(echo $connector | cut -d / -f 2) + selected_performance_test=$(echo "$all_performance_tests" | grep "^$connector_name$" || echo "") + performanceTestCommand="$(_to_gradle_path "airbyte-integrations/$connector" performanceTest)" + else + selected_performance_test=$(echo "$all_performance_tests" | grep "^$connector$" || echo "") + performanceTestCommand=":airbyte-integrations:connectors:$connector:performanceTest" + fi + if [ -n "$selected_performance_test" ] ; then + echo "Running: ./gradlew --no-daemon --scan $performanceTestCommand" + ./gradlew --no-daemon --scan "$performanceTestCommand" + else + echo "Connector '$connector' not found..." + return 1 + fi +fi +} + +# Copy command output to extract gradle scan link. +run | tee build.out +# return status of "run" command, not "tee" +# https://tldp.org/LDP/abs/html/internalvariables.html#PIPESTATUSREF +run_status=${PIPESTATUS[0]} + +test_performance $run_status == "0" || { + # Build failed + link=$(cat build.out | grep -a -A1 "Publishing build scan..." | tail -n1 | tr -d "\n") + # Save gradle scan link to github GRADLE_SCAN_LINK variable for next job. + # https://docs.github.com/en/actions/reference/workflow-commands-for-github-actions#setting-an-environment-variable + echo "GRADLE_SCAN_LINK=$link" >> $GITHUB_ENV + exit $run_status +} + +# Build successed +coverage_report=`sed -n '/^[ \t]*-\+ coverage: /,/TOTAL /p' build.out` + +if ! test_performance -z "$coverage_report" +then + echo "PYTHON_UNITTEST_COVERAGE_REPORT<> $GITHUB_ENV + echo "Python tests coverage:" >> $GITHUB_ENV + echo '```' >> $GITHUB_ENV + echo "$coverage_report" >> $GITHUB_ENV + echo '```' >> $GITHUB_ENV + echo "EOF" >> $GITHUB_ENV +else + echo "PYTHON_UNITTEST_COVERAGE_REPORT=No Python unittests run" >> $GITHUB_ENV +fi