From d9577fa3b4c3ecfd1ed9689d9fc6cc4cf24ded74 Mon Sep 17 00:00:00 2001 From: Chris Riccomini Date: Tue, 1 Nov 2016 20:58:20 -0700 Subject: [PATCH 001/190] Fix Docker integration test to create retriever.tar --- kcbq-connector/test/integrationtest.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/kcbq-connector/test/integrationtest.sh b/kcbq-connector/test/integrationtest.sh index 03aaf7c0a..a65ae7038 100755 --- a/kcbq-connector/test/integrationtest.sh +++ b/kcbq-connector/test/integrationtest.sh @@ -233,6 +233,7 @@ CONNECT_DOCKER_IMAGE='kcbq/connect' CONNECT_DOCKER_NAME='kcbq_test_connect' cp "$BASE_DIR"/../../bin/tar/kcbq-connector-*-confluent-dist.tar "$DOCKER_DIR/connect/kcbq.tar" +cp "$BASE_DIR"/../../bin/tar/kcbq-connector-*-confluent-dist.tar "$DOCKER_DIR/connect/retriever.tar" cp "$KCBQ_TEST_KEYFILE" "$DOCKER_DIR/connect/key.json" if ! dockerimageexists "$CONNECT_DOCKER_IMAGE"; then From 30f29c57bfa21fdeae6e26abe6660531681f9de6 Mon Sep 17 00:00:00 2001 From: Hongyi Wang Date: Fri, 27 Apr 2018 13:42:09 -0700 Subject: [PATCH 002/190] release version 1.1.0 --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 5d7b7a41b..c0e9c4b03 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,3 +1,3 @@ group=com.wepay.kcbq sourceCompatibility=1.8 -version=1.1.0-SNAPSHOT +version=1.1.0 From bc20a83492e666a2234d6bbc65e87994025f3c4c Mon Sep 17 00:00:00 2001 From: Joy Gao Date: Mon, 30 Jul 2018 14:05:27 -0700 Subject: [PATCH 003/190] Merge pull request #119 from jgao54/close-executor Add executor shutdown when BigQuerySinkTask is stopped --- .../connect/bigquery/BigQuerySinkTask.java | 41 +++++++------ .../bigquery/BigQuerySinkTaskTest.java | 59 +++++++++++++------ 2 files changed, 63 insertions(+), 37 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index edec55aaa..f72098ebb 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -21,46 +21,36 @@ import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; import com.google.cloud.bigquery.TableId; - +import com.google.common.annotations.VisibleForTesting; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; - import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; - import com.wepay.kafka.connect.bigquery.convert.RecordConverter; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; - import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; - import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.utils.TopicToTableResolver; import com.wepay.kafka.connect.bigquery.utils.Version; - import com.wepay.kafka.connect.bigquery.write.batch.KCBQThreadPoolExecutor; import com.wepay.kafka.connect.bigquery.write.batch.TableWriter; import com.wepay.kafka.connect.bigquery.write.row.AdaptiveBigQueryWriter; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; import com.wepay.kafka.connect.bigquery.write.row.SimpleBigQueryWriter; - +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.consumer.OffsetAndMetadata; - import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; - import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - -import java.util.Set; -import java.util.concurrent.LinkedBlockingQueue; - /** * A {@link SinkTask} used to translate Kafka Connect {@link SinkRecord SinkRecords} into BigQuery * {@link RowToInsert RowToInserts} and subsequently write them to BigQuery. @@ -79,6 +69,7 @@ public class BigQuerySinkTask extends SinkTask { private TopicPartitionManager topicPartitionManager; private KCBQThreadPoolExecutor executor; + private static final int EXECUTOR_SHUTDOWN_TIMEOUT_SEC = 30; public BigQuerySinkTask() { testBigQuery = null; @@ -146,6 +137,7 @@ private RowToInsert getRecordRow(SinkRecord record) { @Override public void put(Collection records) { + logger.info("Putting {} records in the sink.", records.size()); // create tableWriters Map tableWriterBuilders = new HashMap<>(); @@ -241,7 +233,20 @@ public void start(Map properties) { @Override public void stop() { - logger.trace("task.stop()"); + try { + executor.shutdown(); + executor.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT_SEC, TimeUnit.SECONDS); + } catch (InterruptedException e) { + logger.warn("{} active threads are still executing tasks {}s after shutdown is signaled.", + executor.getActiveCount(), EXECUTOR_SHUTDOWN_TIMEOUT_SEC); + } finally { + logger.trace("task.stop()"); + } + } + + @VisibleForTesting + int getTaskThreadsActiveCount() { + return executor.getActiveCount(); } @Override diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 579616266..d58a8b076 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -18,16 +18,6 @@ */ -import static junit.framework.TestCase.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryError; import com.google.cloud.bigquery.BigQueryException; @@ -42,7 +32,10 @@ import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; -import com.wepay.kafka.connect.bigquery.retrieve.MemorySchemaRetriever; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.RejectedExecutionException; + import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; @@ -50,15 +43,20 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; - import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; import org.mockito.Captor; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class BigQuerySinkTaskTest { private static SinkTaskPropertiesFactory propertiesFactory; @@ -384,11 +382,34 @@ public void testVersion() { assertNotNull(new BigQuerySinkTask().version()); } - // Doesn't do anything at the moment, but having this here will encourage tests to be written if - // the stop() method ever does anything significant - @Test + // Existing tasks should succeed upon stop is called. New tasks should be rejected once task is stopped. + @Test(expected = RejectedExecutionException.class) public void testStop() { - new BigQuerySinkTask().stop(); + final String dataset = "scratch"; + final String topic = "test_topic"; + + Map properties = propertiesFactory.getProperties(); + properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); + properties.put(BigQuerySinkConfig.DATASETS_CONFIG, String.format(".*=%s", dataset)); + + BigQuery bigQuery = mock(BigQuery.class); + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + InsertAllResponse insertAllResponse = mock(InsertAllResponse.class); + + when(bigQuery.insertAll(anyObject())).thenReturn(insertAllResponse); + when(insertAllResponse.hasErrors()).thenReturn(false); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, null); + testTask.initialize(sinkTaskContext); + testTask.start(properties); + testTask.put(Collections.singletonList(spoofSinkRecord(topic))); + + assertEquals(1, testTask.getTaskThreadsActiveCount()); + testTask.stop(); + assertEquals(0, testTask.getTaskThreadsActiveCount()); + verify(bigQuery, times(1)).insertAll(any(InsertAllRequest.class)); + + testTask.put(Collections.singletonList(spoofSinkRecord(topic))); } /** From ebb575027e7e9ca80cd7be0ba012f3fa0e396e8c Mon Sep 17 00:00:00 2001 From: nicolasguyomar Date: Thu, 20 Jun 2019 19:26:03 +0200 Subject: [PATCH 004/190] Prevent context update after/while rebalancing (#161) This call might have been needed before, but it is no longer required --- .../kafka/connect/bigquery/BigQuerySinkTask.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index f72098ebb..b06dff5d6 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -89,21 +89,10 @@ public void flush(Map offsets) { } catch (InterruptedException err) { throw new ConnectException("Interrupted while waiting for write tasks to complete.", err); } - updateOffsets(offsets); topicPartitionManager.resumeAll(); } - /** - * This really doesn't do much and I'm not totally clear on whether or not I need it. - * But, in the interest of maintaining old functionality, here we are. - */ - private void updateOffsets(Map offsets) { - for (Map.Entry offsetEntry : offsets.entrySet()) { - context.offset(offsetEntry.getKey(), offsetEntry.getValue().offset()); - } - } - private PartitionedTableId getRecordTable(SinkRecord record) { TableId baseTableId = topicsToBaseTableIds.get(record.topic()); From 96846d6fcc848fc23ca4cef574cfa601f3454fe2 Mon Sep 17 00:00:00 2001 From: Chris Riccomini Date: Wed, 26 Jun 2019 16:57:15 -0700 Subject: [PATCH 005/190] release version 1.1.2 --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index c0e9c4b03..301b5832f 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,3 +1,3 @@ group=com.wepay.kcbq sourceCompatibility=1.8 -version=1.1.0 +version=1.1.2 From 4514a2feb307c77fd23f23d2715d086a28094a7e Mon Sep 17 00:00:00 2001 From: Chris Riccomini Date: Mon, 1 Jul 2019 11:23:23 -0700 Subject: [PATCH 006/190] release version 1.2.0 --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 5d7b7a41b..711f84905 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,3 +1,3 @@ group=com.wepay.kcbq sourceCompatibility=1.8 -version=1.1.0-SNAPSHOT +version=1.2.0 From e77c40564e1c0f5f20836576e4b2dc9b16e5f322 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 2 Jun 2020 11:54:06 -0700 Subject: [PATCH 007/190] GH-264: Add support for upsert/delete via periodic merge flushes --- .../bigquery/BigQuerySinkConnector.java | 4 +- .../connect/bigquery/BigQuerySinkTask.java | 176 +++++++-- .../kafka/connect/bigquery/MergeQueries.java | 336 ++++++++++++++++++ .../kafka/connect/bigquery/SchemaManager.java | 262 ++++++++++++-- .../bigquery/config/BigQuerySinkConfig.java | 174 +++++++-- .../config/BigQuerySinkTaskConfig.java | 12 +- .../convert/BigQueryRecordConverter.java | 28 +- .../convert/BigQuerySchemaConverter.java | 1 + .../bigquery/utils/FieldNameSanitizer.java | 1 + .../bigquery/utils/SinkRecordConverter.java | 119 +++++-- .../write/batch/GCSBatchTableWriter.java | 19 +- .../write/batch/KCBQThreadPoolExecutor.java | 12 +- .../bigquery/write/batch/MergeBatches.java | 332 +++++++++++++++++ .../bigquery/write/batch/TableWriter.java | 40 ++- .../write/batch/TableWriterBuilder.java | 4 +- .../write/row/AdaptiveBigQueryWriter.java | 8 +- .../write/row/UpsertDeleteBigQueryWriter.java | 94 +++++ .../bigquery/BigQuerySinkConnectorTest.java | 4 +- .../bigquery/BigQuerySinkTaskTest.java | 2 +- .../bigquery/SinkTaskPropertiesFactory.java | 1 + .../config/BigQuerySinkConfigTest.java | 3 - 21 files changed, 1461 insertions(+), 171 deletions(-) create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index 2981adb8a..a8ecaf562 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -20,6 +20,8 @@ import com.google.cloud.bigquery.BigQuery; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; + +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import com.wepay.kafka.connect.bigquery.utils.Version; @@ -106,12 +108,12 @@ public List> taskConfigs(int maxTasks) { logger.trace("connector.taskConfigs()"); List> taskConfigs = new ArrayList<>(); for (int i = 0; i < maxTasks; i++) { - // Copy configProperties so that tasks can't interfere with each others' configurations HashMap taskConfig = new HashMap<>(configProperties); if (i == 0 && !config.getList(BigQuerySinkConfig.ENABLE_BATCH_CONFIG).isEmpty()) { // if batch loading is enabled, configure first task to do the GCS -> BQ loading taskConfig.put(GCS_BQ_TASK_CONFIG_KEY, "true"); } + taskConfig.put(BigQuerySinkTaskConfig.TASK_ID_CONFIG, Integer.toString(i)); taskConfigs.add(taskConfig); } return taskConfigs; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index b338e27b0..96679b5f3 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -28,7 +28,6 @@ import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; -import com.wepay.kafka.connect.bigquery.convert.RecordConverter; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import com.wepay.kafka.connect.bigquery.utils.SinkRecordConverter; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; @@ -38,12 +37,14 @@ import com.wepay.kafka.connect.bigquery.utils.Version; import com.wepay.kafka.connect.bigquery.write.batch.GCSBatchTableWriter; import com.wepay.kafka.connect.bigquery.write.batch.KCBQThreadPoolExecutor; +import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; import com.wepay.kafka.connect.bigquery.write.batch.TableWriter; import com.wepay.kafka.connect.bigquery.write.batch.TableWriterBuilder; import com.wepay.kafka.connect.bigquery.write.row.AdaptiveBigQueryWriter; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; import com.wepay.kafka.connect.bigquery.write.row.GCSToBQWriter; import com.wepay.kafka.connect.bigquery.write.row.SimpleBigQueryWriter; +import com.wepay.kafka.connect.bigquery.write.row.UpsertDeleteBigQueryWriter; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; @@ -66,6 +67,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; /** * A {@link SinkTask} used to translate Kafka Connect {@link SinkRecord SinkRecords} into BigQuery @@ -74,6 +76,8 @@ public class BigQuerySinkTask extends SinkTask { private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkTask.class); + private AtomicReference bigQuery; + private AtomicReference schemaManager; private SchemaRetriever schemaRetriever; private BigQueryWriter bigQueryWriter; private GCSToBQWriter gcsToBQWriter; @@ -83,6 +87,10 @@ public class BigQuerySinkTask extends SinkTask { private boolean useMessageTimeDatePartitioning; private boolean usePartitionDecorator; private boolean sanitize; + private boolean upsertDelete; + private MergeBatches mergeBatches; + private MergeQueries mergeQueries; + private long mergeRecordsThreshold; private TopicPartitionManager topicPartitionManager; @@ -94,7 +102,7 @@ public class BigQuerySinkTask extends SinkTask { private final SchemaManager testSchemaManager; private final UUID uuid = UUID.randomUUID(); - private ScheduledExecutorService gcsLoadExecutor; + private ScheduledExecutorService loadExecutor; /** * Create a new BigquerySinkTask. @@ -137,11 +145,23 @@ private void maybeEnsureExistingTable(TableId table) { BigQuery bigQuery = getBigQuery(); if (bigQuery.getTable(table) == null && !config.getBoolean(config.TABLE_CREATE_CONFIG)) { throw new BigQueryConnectException("Table '" + table + "' does not exist. " + - "You may want to enable auto table creation by setting " + config.TABLE_CREATE_CONFIG - + "=true in the properties file"); + "You may want to enable auto table creation by setting " + config.TABLE_CREATE_CONFIG + + "=true in the properties file"); } } + @Override + public Map preCommit(Map offsets) { + if (upsertDelete) { + Map result = mergeBatches.latestOffsets(); + checkQueueSize(); + return result; + } + + flush(offsets); + return offsets; + } + private PartitionedTableId getRecordTable(SinkRecord record) { String tableName; String dataset = config.getString(config.DEFAULT_DATASET_CONFIG); @@ -161,11 +181,16 @@ private PartitionedTableId getRecordTable(SinkRecord record) { tableName = FieldNameSanitizer.sanitizeName(tableName); } TableId baseTableId = TableId.of(dataset, tableName); - maybeEnsureExistingTable(baseTableId); + if (upsertDelete) { + TableId intermediateTableId = mergeBatches.intermediateTableFor(baseTableId); + // If upsert/delete is enabled, we want to stream into a non-partitioned intermediate table + return new PartitionedTableId.Builder(intermediateTableId).build(); + } else { + maybeEnsureExistingTable(baseTableId); + } PartitionedTableId.Builder builder = new PartitionedTableId.Builder(baseTableId); if (usePartitionDecorator) { - if (useMessageTimeDatePartitioning) { if (record.timestampType() == TimestampType.NO_TIMESTAMP_TYPE) { throw new ConnectException( @@ -182,7 +207,12 @@ private PartitionedTableId getRecordTable(SinkRecord record) { @Override public void put(Collection records) { - logger.info("Putting {} records in the sink.", records.size()); + if (upsertDelete) { + // Periodically poll for errors here instead of doing a stop-the-world check in flush() + executor.maybeThrowEncounteredErrors(); + } + + logger.debug("Putting {} records in the sink.", records.size()); // create tableWriters Map tableWriterBuilders = new HashMap<>(); @@ -206,12 +236,17 @@ public void put(Collection records) { gcsBlobName, recordConverter); } else { - tableWriterBuilder = + TableWriter.Builder simpleTableWriterBuilder = new TableWriter.Builder(bigQueryWriter, table, recordConverter); + if (upsertDelete) { + simpleTableWriterBuilder.onFinish(rows -> + mergeBatches.onRowWrites(table.getBaseTableId(), rows)); + } + tableWriterBuilder = simpleTableWriterBuilder; } tableWriterBuilders.put(table, tableWriterBuilder); } - tableWriterBuilders.get(table).addRow(record); + tableWriterBuilders.get(table).addRow(record, table.getBaseTableId()); } } @@ -221,6 +256,18 @@ public void put(Collection records) { } // check if we should pause topics + checkQueueSize(); + } + + // Important: this method is only safe to call during put(), flush(), or preCommit(); otherwise, + // a ConcurrentModificationException may be triggered if the Connect framework is in the middle of + // a method invocation on the consumer for this task. This becomes especially likely if all topics + // have been paused as the framework will most likely be in the middle of a poll for that consumer + // which, because all of its topics have been paused, will not return until it's time for the next + // offset commit. Invoking context.requestCommit() won't wake up the consumer in that case, so we + // really have no choice but to wait for the framework to call a method on this task that implies + // that it's safe to pause or resume partitions on the consumer. + private void checkQueueSize() { long queueSoftLimit = config.getLong(BigQuerySinkTaskConfig.QUEUE_SIZE_CONFIG); if (queueSoftLimit != -1) { int currentQueueSize = executor.getQueue().size(); @@ -237,16 +284,24 @@ private BigQuery getBigQuery() { if (testBigQuery != null) { return testBigQuery; } + return bigQuery.updateAndGet(bq -> bq != null ? bq : newBigQuery()); + } + + private BigQuery newBigQuery() { String projectName = config.getString(config.PROJECT_CONFIG); String keyFile = config.getKeyFile(); String keySource = config.getString(config.KEY_SOURCE_CONFIG); return new BigQueryHelper().setKeySource(keySource).connect(projectName, keyFile); } - private SchemaManager getSchemaManager(BigQuery bigQuery) { + private SchemaManager getSchemaManager() { if (testSchemaManager != null) { return testSchemaManager; } + return schemaManager.updateAndGet(sm -> sm != null ? sm : newSchemaManager()); + } + + private SchemaManager newSchemaManager() { schemaRetriever = config.getSchemaRetriever(); SchemaConverter schemaConverter = config.getSchemaConverter(); @@ -256,8 +311,10 @@ private SchemaManager getSchemaManager(BigQuery bigQuery) { Optional> clusteringFieldName = config.getClusteringPartitionFieldName(); boolean allowNewBQFields = config.getBoolean(config.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); boolean allowReqFieldRelaxation = config.getBoolean(config.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); - return new SchemaManager(schemaRetriever, schemaConverter, bigQuery, allowNewBQFields, allowReqFieldRelaxation, kafkaKeyFieldName, - kafkaDataFieldName, timestampPartitionFieldName, clusteringFieldName); + return new SchemaManager(schemaRetriever, schemaConverter, getBigQuery(), + allowNewBQFields, allowReqFieldRelaxation, + kafkaKeyFieldName, kafkaDataFieldName, + timestampPartitionFieldName, clusteringFieldName); } private BigQueryWriter getBigQueryWriter() { @@ -267,9 +324,16 @@ private BigQueryWriter getBigQueryWriter() { int retry = config.getInt(config.BIGQUERY_RETRY_CONFIG); long retryWait = config.getLong(config.BIGQUERY_RETRY_WAIT_CONFIG); BigQuery bigQuery = getBigQuery(); - if (autoCreateTables || allowNewBigQueryFields || allowRequiredFieldRelaxation ) { + if (upsertDelete) { + return new UpsertDeleteBigQueryWriter(bigQuery, + getSchemaManager(), + retry, + retryWait, + autoCreateTables, + mergeBatches.intermediateToDestinationTables()); + } else if (autoCreateTables || allowNewBigQueryFields || allowRequiredFieldRelaxation) { return new AdaptiveBigQueryWriter(bigQuery, - getSchemaManager(bigQuery), + getSchemaManager(), retry, retryWait, autoCreateTables); @@ -296,7 +360,7 @@ private GCSToBQWriter getGcsWriter() { boolean autoCreateTables = config.getBoolean(config.TABLE_CREATE_CONFIG); // schemaManager shall only be needed for creating table hence do not fetch instance if not // needed. - SchemaManager schemaManager = autoCreateTables ? getSchemaManager(bigQuery) : null; + SchemaManager schemaManager = autoCreateTables ? getSchemaManager() : null; return new GCSToBQWriter(getGcs(), bigQuery, schemaManager, @@ -305,11 +369,8 @@ private GCSToBQWriter getGcsWriter() { autoCreateTables); } - private SinkRecordConverter getConverter(BigQuerySinkConfig config) { - return new SinkRecordConverter(config.getRecordConverter(), - config.getBoolean(config.SANITIZE_FIELD_NAME_CONFIG), - config.getKafkaKeyFieldName(), - config.getKafkaDataFieldName()); + private SinkRecordConverter getConverter(BigQuerySinkTaskConfig config) { + return new SinkRecordConverter(config, mergeBatches, mergeQueries); } @Override @@ -325,6 +386,22 @@ public void start(Map properties) { err ); } + upsertDelete = config.getBoolean(config.UPSERT_ENABLED_CONFIG) + || config.getBoolean(config.DELETE_ENABLED_CONFIG); + + bigQuery = new AtomicReference<>(); + schemaManager = new AtomicReference<>(); + + if (upsertDelete) { + String intermediateTableSuffix = String.format("_%s_%d_%s_%d", + config.getString(config.INTERMEDIATE_TABLE_SUFFIX_CONFIG), + config.getInt(config.TASK_ID_CONFIG), + uuid, + Instant.now().toEpochMilli() + ); + mergeBatches = new MergeBatches(intermediateTableSuffix); + mergeRecordsThreshold = config.getLong(config.MERGE_RECORDS_THRESHOLD_CONFIG); + } bigQueryWriter = getBigQueryWriter(); gcsToBQWriter = getGcsWriter(); @@ -339,12 +416,16 @@ public void start(Map properties) { config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG); if (hasGCSBQTask) { startGCSToBQLoadTask(); + } else if (upsertDelete) { + mergeQueries = + new MergeQueries(config, mergeBatches, executor, getBigQuery(), getSchemaManager(), context); + maybeStartMergeFlushTask(); } } private void startGCSToBQLoadTask() { logger.info("Attempting to start GCS Load Executor."); - gcsLoadExecutor = Executors.newScheduledThreadPool(1); + loadExecutor = Executors.newScheduledThreadPool(1); String bucketName = config.getString(config.GCS_BUCKET_NAME_CONFIG); Storage gcs = getGcs(); // get the bucket, or create it if it does not exist. @@ -362,29 +443,50 @@ private void startGCSToBQLoadTask() { GCSToBQLoadRunnable loadRunnable = new GCSToBQLoadRunnable(getBigQuery(), bucket); int intervalSec = config.getInt(BigQuerySinkConfig.BATCH_LOAD_INTERVAL_SEC_CONFIG); - gcsLoadExecutor.scheduleAtFixedRate(loadRunnable, intervalSec, intervalSec, TimeUnit.SECONDS); + loadExecutor.scheduleAtFixedRate(loadRunnable, intervalSec, intervalSec, TimeUnit.SECONDS); + } + + private void maybeStartMergeFlushTask() { + long intervalMs = config.getLong(config.MERGE_INTERVAL_MS_CONFIG); + if (intervalMs == -1) { + logger.info("{} is set to -1; periodic merge flushes are disabled", config.MERGE_INTERVAL_MS_CONFIG); + return; + } + logger.info("Attempting to start upsert/delete load executor"); + loadExecutor = Executors.newScheduledThreadPool(1); + loadExecutor.scheduleAtFixedRate( + mergeQueries::mergeFlushAll, intervalMs, intervalMs, TimeUnit.MILLISECONDS); } @Override public void stop() { try { - executor.shutdown(); - executor.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT_SEC, TimeUnit.SECONDS); - if (gcsLoadExecutor != null) { - try { - logger.info("Attempting to shut down GCS Load Executor."); - gcsLoadExecutor.shutdown(); - gcsLoadExecutor.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT_SEC, TimeUnit.SECONDS); - } catch (InterruptedException ex) { - logger.warn("Could not shut down GCS Load Executor within {}s.", - EXECUTOR_SHUTDOWN_TIMEOUT_SEC); - } + if (upsertDelete) { + mergeBatches.intermediateTables().forEach(table -> { + logger.debug("Deleting intermediate table {}", table); + getBigQuery().delete(table); + }); } - } catch (InterruptedException ex) { - logger.warn("{} active threads are still executing tasks {}s after shutdown is signaled.", - executor.getActiveCount(), EXECUTOR_SHUTDOWN_TIMEOUT_SEC); } finally { - logger.trace("task.stop()"); + try { + executor.shutdown(); + executor.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT_SEC, TimeUnit.SECONDS); + if (loadExecutor != null) { + try { + logger.info("Attempting to shut down load executor."); + loadExecutor.shutdown(); + loadExecutor.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT_SEC, TimeUnit.SECONDS); + } catch (InterruptedException ex) { + logger.warn("Could not shut down load executor within {}s.", + EXECUTOR_SHUTDOWN_TIMEOUT_SEC); + } + } + } catch (InterruptedException ex) { + logger.warn("{} active threads are still executing tasks {}s after shutdown is signaled.", + executor.getActiveCount(), EXECUTOR_SHUTDOWN_TIMEOUT_SEC); + } finally { + logger.trace("task.stop()"); + } } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java new file mode 100644 index 000000000..df9272af7 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java @@ -0,0 +1,336 @@ +package com.wepay.kafka.connect.bigquery; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.FieldList; +import com.google.cloud.bigquery.QueryJobConfiguration; +import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.TableId; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; +import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class MergeQueries { + public static final String INTERMEDIATE_TABLE_KEY_FIELD_NAME = "key"; + public static final String INTERMEDIATE_TABLE_VALUE_FIELD_NAME = "value"; + public static final String INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME = "partitionTime"; + public static final String INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD = "batchNumber"; + + private static final Logger logger = LoggerFactory.getLogger(MergeQueries.class); + + private final String keyFieldName; + private final boolean usePartitionDecorator; + private final boolean upsertEnabled; + private final boolean deleteEnabled; + private final MergeBatches mergeBatches; + private final ExecutorService executor; + private final BigQuery bigQuery; + private final SchemaManager schemaManager; + private final SinkTaskContext context; + + public MergeQueries(BigQuerySinkTaskConfig config, + MergeBatches mergeBatches, + ExecutorService executor, + BigQuery bigQuery, + SchemaManager schemaManager, + SinkTaskContext context) { + this.keyFieldName = config.getKafkaKeyFieldName().orElseThrow(() -> + new ConnectException("Kafka key field must be configured when upsert/delete is enabled") + ); + this.usePartitionDecorator = config.getBoolean(config.BIGQUERY_PARTITION_DECORATOR_CONFIG); + this.upsertEnabled = config.getBoolean(config.UPSERT_ENABLED_CONFIG); + this.deleteEnabled = config.getBoolean(config.DELETE_ENABLED_CONFIG); + + this.mergeBatches = mergeBatches; + this.executor = executor; + this.bigQuery = bigQuery; + this.schemaManager = schemaManager; + this.context = context; + } + + public void mergeFlushAll() { + logger.debug("Triggering merge flush for all tables"); + mergeBatches.intermediateTables().forEach(this::mergeFlush); + } + + public void mergeFlush(TableId intermediateTable) { + final TableId destinationTable = mergeBatches.destinationTableFor(intermediateTable); + final int batchNumber = mergeBatches.incrementBatch(intermediateTable); + logger.trace("Triggering merge flush from intermediate table {} to destination table {} for batch {}", + intermediateTable, destinationTable, batchNumber); + + executor.submit(() -> { + // If there are rows to flush in this batch, flush them + if (mergeBatches.prepareToFlush(intermediateTable, batchNumber)) { + try { + logger.debug("Running merge query on batch {} from intermediate table {}", + batchNumber, intermediateTable); + String mergeFlushQuery = mergeFlushQuery(intermediateTable, destinationTable, batchNumber); + logger.trace(mergeFlushQuery); + bigQuery.query(QueryJobConfiguration.of(mergeFlushQuery)); + logger.trace("Merge from intermediate table {} to destination table {} completed", + intermediateTable, destinationTable); + } catch (Throwable t) { + logger.warn("Failed on merge flush from intermediate table {} to destination table {}", + intermediateTable, destinationTable, t); + throw new ConnectException( + String.format("Failed to perform merge flush from intermediate table %s to destination table %s", + intermediateTable, + destinationTable), + t); + } + + logger.debug("Recording flush success for batch {} from {}", + batchNumber, intermediateTable); + mergeBatches.recordSuccessfulFlush(intermediateTable, batchNumber); + + // Commit those offsets ASAP + context.requestCommit(); + + logger.info("Completed merge flush of batch {} from {} to {}", + batchNumber, intermediateTable, destinationTable); + } + + // After, regardless of whether we flushed or not, clean up old batches from the intermediate + // table. Some rows may be several batches old but still in the table if they were in the + // streaming buffer during the last purge. + try { + logger.trace("Clearing batches from {} on back from intermediate table {}", batchNumber, intermediateTable); + String tableClearQuery = clearBatchQuery(intermediateTable, batchNumber); + logger.trace(tableClearQuery); + bigQuery.query(QueryJobConfiguration.of(tableClearQuery)); + } catch (Throwable t) { + logger.error("Failed to clear old batches from intermediate table {}", intermediateTable, t); + throw new ConnectException( + String.format("Failed to clear old batches from intermediate table %s", + intermediateTable), + t); + } + }); + } + + /* + + upsert+delete: + + MERGE ``.`` + USING ( + SELECT * FROM ( + SELECT ARRAY_AGG( + x ORDER BY partitionTime DESC LIMIT 1 + )[OFFSET(0)] src + FROM ``.`` x + WHERE batchNumber= + GROUP BY key.[, key....] + ) + ) + ON ``.=`src`.key + WHEN MATCHED AND `src`.value IS NOT NULL + THEN UPDATE SET =`src`.value.[, =`src`.value....] + WHEN MATCHED AND `src`.value IS NULL + THEN DELETE + WHEN NOT MATCHED AND `src`.value IS NOT NULL + THEN INSERT (, _PARTITIONTIME, [, ]) + VALUES ( + `src`.key, + CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), + `src`.value.[, `src`.value....] + ); + + + delete only: + + MERGE ``.`` + USING ( + SELECT * FROM ( + SELECT ARRAY_AGG( + x ORDER BY partitionTime DESC LIMIT 1 + )[OFFSET(0)] src + FROM ``.`` x + WHERE batchNumber= + GROUP BY key.[, key....] + ) + ) + ON ``.=`src`.key AND `src`.value IS NULL + WHEN MATCHED + THEN DELETE + WHEN NOT MATCHED + THEN INSERT (, _PARTITIONTIME, [, ]) + VALUES ( + `src`.key, + CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), + `src`.value.[, `src`.value....] + ); + + + upsert only: + + MERGE ``.`` + USING ( + SELECT * FROM ( + SELECT ARRAY_AGG( + x ORDER BY partitionTime DESC LIMIT 1 + )[OFFSET(0)] src + FROM ``.`` x + WHERE batchNumber= + GROUP BY key.[, key....] + ) + ) + ON ``.=`src`.key + WHEN MATCHED + THEN UPDATE SET =`src`.value.[, ...] + WHEN NOT MATCHED + THEN INSERT ([, `src`.value....] + ); + + */ + private String mergeFlushQuery(TableId intermediateTable, TableId destinationTable, int batchNumber) { + Schema intermediateSchema = schemaManager.cachedSchema(intermediateTable); + + String srcKey = INTERMEDIATE_TABLE_KEY_FIELD_NAME; + + List keyFields = listFields( + intermediateSchema.getFields().get(keyFieldName).getSubFields(), + srcKey + "." + ); + List dstValueFields = intermediateSchema.getFields().get(INTERMEDIATE_TABLE_VALUE_FIELD_NAME).getSubFields() + .stream() + .map(Field::getName) + .collect(Collectors.toList()); + + List srcValueFields = dstValueFields.stream() + .map(field -> "`src`." + INTERMEDIATE_TABLE_VALUE_FIELD_NAME + "." + field) + .collect(Collectors.toList()); + List updateValues = dstValueFields.stream() + .map(field -> field + "=`src`." + INTERMEDIATE_TABLE_VALUE_FIELD_NAME + "." + field) + .collect(Collectors.toList()); + + String partitionTimeField = usePartitionDecorator ? "_PARTITIONTIME, " : ""; + String partitionTimeValue = usePartitionDecorator + ? "CAST(CAST(DATE(`src`." + INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME + ") AS DATE) AS TIMESTAMP), " + : ""; + + String dst = destinationTable.getTable(); + + StringBuilder keysMatch = new StringBuilder("`").append(dst).append("`.").append(keyFieldName).append("=`src`.").append(srcKey); + + StringBuilder mergeOpening = new StringBuilder("MERGE `").append(destinationTable.getDataset()).append("`.`").append(destinationTable.getTable()).append("` ") + .append("USING (") + .append("SELECT * FROM (") + .append("SELECT ARRAY_AGG(") + .append("x ORDER BY ").append(INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME).append(" DESC LIMIT 1") + .append(")[OFFSET(0)] src ") + .append("FROM `").append(intermediateTable.getDataset()).append("`.`").append(intermediateTable.getTable()).append("` x ") + .append("WHERE ").append(INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD).append("=").append(batchNumber).append(" ") + .append("GROUP BY ").append(String.join(", ", keyFields)) + .append(")") + .append(") "); + + StringBuilder insertClause = new StringBuilder("THEN INSERT (") + .append(keyFieldName).append(", ") + .append(partitionTimeField) + .append(String.join(", ", dstValueFields)) + .append(") ") + .append("VALUES (") + .append("`src`.").append(srcKey).append(", ") + .append(partitionTimeValue) + .append(String.join(", ", srcValueFields)) + .append(")"); + + StringBuilder updateClause = new StringBuilder("THEN UPDATE SET ") + .append(String.join(", ", updateValues)); + + StringBuilder valueIs = new StringBuilder("`src`.").append(INTERMEDIATE_TABLE_VALUE_FIELD_NAME).append(" IS "); + + if (upsertEnabled && deleteEnabled) { + // Delete rows with null values, and upsert all others + return mergeOpening + .append("ON ").append(keysMatch).append(" ") + .append("WHEN MATCHED AND ").append(valueIs).append("NOT NULL ") + .append(updateClause).append(" ") + .append("WHEN MATCHED AND ").append(valueIs).append("NULL ") + .append("THEN DELETE ") + .append("WHEN NOT MATCHED AND ").append(valueIs).append("NOT NULL ") + .append(insertClause) + .append(";") + .toString(); + } else if (deleteEnabled) { + // Delete rows with null values, and insert all others + return mergeOpening + .append("ON ").append(keysMatch).append(" ") + .append("AND ").append(valueIs).append("NULL ") + .append("WHEN MATCHED ") + .append("THEN DELETE ") + .append("WHEN NOT MATCHED ") + .append(insertClause) + .append(";") + .toString(); + } else if (upsertEnabled) { + // Assume all rows have non-null values and upsert them all + return mergeOpening + .append("ON ").append(keysMatch).append(" ") + .append("WHEN MATCHED") + .append(updateClause).append(" ") + .append("WHEN NOT MATCHED") + .append(insertClause) + .append(";") + .toString(); + } else { + throw new IllegalStateException("At least one of upsert or delete must be enabled for merge flushing to occur."); + } + } + + // DELETE FROM `` WHERE batchNumber <= AND _PARTITIONTIME IS NOT NULL; + private static String clearBatchQuery(TableId intermediateTable, int batchNumber) { + return new StringBuilder("DELETE FROM `").append(intermediateTable.getDataset()).append("`.`").append(intermediateTable.getTable()).append("` ") + .append("WHERE ") + .append(INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD).append(" <= ").append(batchNumber).append(" ") + // Use this clause to filter out rows that are still in the streaming buffer, which should + // not be subjected to UPDATE or DELETE operations or the query will FAIL + .append("AND _PARTITIONTIME IS NOT NULL") + .append(";") + .toString(); + } + + private static List listFields(FieldList keyFields, String prefix) { + return keyFields.stream() + .flatMap(field -> { + String fieldName = prefix + field.getName(); + FieldList subFields = field.getSubFields(); + if (subFields == null) { + return Stream.of(fieldName); + } + return listFields(subFields, fieldName + ".").stream(); + }).collect(Collectors.toList()); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index e10839b65..b0968759b 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -30,11 +30,14 @@ import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; /** * Class for managing Schemas of BigQuery tables (creating and updating). */ public class SchemaManager { + private static final Logger logger = LoggerFactory.getLogger(SchemaManager.class); private final SchemaRetriever schemaRetriever; @@ -46,6 +49,10 @@ public class SchemaManager { private final Optional kafkaDataFieldName; private final Optional timestampPartitionFieldName; private final Optional> clusteringFieldName; + private final boolean intermediateTables; + private final ConcurrentMap tableCreateLocks; + private final ConcurrentMap tableUpdateLocks; + private final ConcurrentMap schemaCache; /** * @param schemaRetriever Used to determine the Kafka Connect Schema that should be used for a @@ -55,9 +62,14 @@ public class SchemaManager { * @param allowNewBQFields If set to true, allows new fields to be added to BigQuery Schema. * @param allowBQRequiredFieldRelaxation If set to true, allows changing field mode from REQUIRED to NULLABLE * @param kafkaKeyFieldName The name of kafka key field to be used in BigQuery. - * If set to null, Kafka Key Field will not be included in BigQuery. + * If set to null, Kafka Key Field will not be included in BigQuery. * @param kafkaDataFieldName The name of kafka data field to be used in BigQuery. * If set to null, Kafka Data Field will not be included in BigQuery. + * @param timestampPartitionFieldName The name of the field to use for column-based time + * partitioning in BigQuery. + * If set to null, ingestion time-based partitioning will be + * used instead. + * @param clusteringFieldName */ public SchemaManager( SchemaRetriever schemaRetriever, @@ -69,6 +81,36 @@ public SchemaManager( Optional kafkaDataFieldName, Optional timestampPartitionFieldName, Optional> clusteringFieldName) { + this( + schemaRetriever, + schemaConverter, + bigQuery, + allowNewBQFields, + allowBQRequiredFieldRelaxation, + kafkaKeyFieldName, + kafkaDataFieldName, + timestampPartitionFieldName, + clusteringFieldName, + false, + new ConcurrentHashMap<>(), + new ConcurrentHashMap<>(), + new ConcurrentHashMap<>()); + } + + private SchemaManager( + SchemaRetriever schemaRetriever, + SchemaConverter schemaConverter, + BigQuery bigQuery, + boolean allowNewBQFields, + boolean allowBQRequiredFieldRelaxation, + Optional kafkaKeyFieldName, + Optional kafkaDataFieldName, + Optional timestampPartitionFieldName, + Optional> clusteringFieldName, + boolean intermediateTables, + ConcurrentMap tableCreateLocks, + ConcurrentMap tableUpdateLocks, + ConcurrentMap schemaCache) { this.schemaRetriever = schemaRetriever; this.schemaConverter = schemaConverter; this.bigQuery = bigQuery; @@ -78,6 +120,59 @@ public SchemaManager( this.kafkaDataFieldName = kafkaDataFieldName; this.timestampPartitionFieldName = timestampPartitionFieldName; this.clusteringFieldName = clusteringFieldName; + this.intermediateTables = intermediateTables; + this.tableCreateLocks = tableCreateLocks; + this.tableUpdateLocks = tableUpdateLocks; + this.schemaCache = schemaCache; + } + + public SchemaManager forIntermediateTables() { + return new SchemaManager( + schemaRetriever, + schemaConverter, + bigQuery, + allowNewBQFields, + allowBQRequiredFieldRelaxation, + kafkaKeyFieldName, + kafkaDataFieldName, + timestampPartitionFieldName, + clusteringFieldName, + true, + tableCreateLocks, + tableUpdateLocks, + schemaCache + ); + } + + /** + * Fetch the most recent schema for the given table, assuming it has been created and/or updated + * over the lifetime of this schema manager. + * @param table the table to fetch the schema for; may be null + * @return the latest schema for that table; may be null if the table does not exist or has not + * been created or updated by this schema manager + */ + public com.google.cloud.bigquery.Schema cachedSchema(TableId table) { + return schemaCache.get(table); + } + + /** + * Create a new table in BigQuery, if it doesn't already exist. Otherwise, update the existing + * table to use the most-current schema. + * @param table The BigQuery table to create, + * @param records The sink records used to determine the schema. + */ + public void createOrUpdateTable(TableId table, Set records) { + synchronized (lock(tableCreateLocks, table)) { + if (bigQuery.getTable(table) == null) { + logger.debug("{} doesn't exist; creating instead of updating", table(table)); + createTable(table, records); + return; + } + } + + // Table already existed; attempt to update instead + logger.debug("{} already exists; updating instead of creating", table(table)); + updateSchema(table, records); } /** @@ -86,8 +181,27 @@ public SchemaManager( * @param records The sink records used to determine the schema. */ public void createTable(TableId table, Set records) { + synchronized (lock(tableCreateLocks, table)) { + if (schemaCache.containsKey(table)) { + // Table already exists; noop + logger.debug("Skipping create of {} as it should already exist or appear very soon", table(table)); + return; + } TableInfo tableInfo = getTableInfo(table, records); - bigQuery.create(tableInfo); + logger.info("Attempting to create {} with schema {}", + table(table), tableInfo.getDefinition().getSchema()); + try { + bigQuery.create(tableInfo); + logger.debug("Successfully created {}", table(table)); + schemaCache.put(table, tableInfo.getDefinition().getSchema()); + } catch (BigQueryException e) { + if (e.getCode() == 409) { + logger.debug("Failed to create {} as it already exists (possibly created by another task)", table(table)); + com.google.cloud.bigquery.Schema schema = bigQuery.getTable(table).getDefinition().getSchema(); + schemaCache.put(table, schema); + } + } + } } /** @@ -96,10 +210,25 @@ public void createTable(TableId table, Set records) { * @param records The sink records used to update the schema. */ public void updateSchema(TableId table, Set records) { - TableInfo tableInfo = getTableInfo(table, records); - logger.info("Attempting to update table `{}` with schema {}", - table, tableInfo.getDefinition().getSchema()); - bigQuery.update(tableInfo); + synchronized (tableUpdateLocks.computeIfAbsent(table, t -> new Object())) { + TableInfo tableInfo = getTableInfo(table, records); + + if (!schemaCache.containsKey(table)) { + logger.debug("Reading schema for {}", table(table)); + schemaCache.put(table, bigQuery.getTable(table).getDefinition().getSchema()); + } + + if (!schemaCache.get(table).equals(tableInfo.getDefinition().getSchema())) { + logger.info("Attempting to update {} with schema {}", + table(table), tableInfo.getDefinition().getSchema()); + bigQuery.update(tableInfo); + logger.debug("Successfully updated {}", table(table)); + schemaCache.put(table, tableInfo.getDefinition().getSchema()); + } else { + logger.debug("Skipping update of {} since current schema should be compatible", table(table)); + } + } + } /** @@ -211,46 +340,119 @@ private String getUnionizedTableDescription(Set records) { // package private for testing. TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema bigQuerySchema, String tableDescription) { - TimePartitioning timePartitioning = TimePartitioning.of(Type.DAY); - if (timestampPartitionFieldName.isPresent()) { - timePartitioning = timePartitioning.toBuilder().setField(timestampPartitionFieldName.get()).build(); - } - StandardTableDefinition.Builder builder = StandardTableDefinition.newBuilder() - .setSchema(bigQuerySchema) - .setTimePartitioning(timePartitioning); - - if (timestampPartitionFieldName.isPresent() && clusteringFieldName.isPresent()) { - Clustering clustering = Clustering.newBuilder() - .setFields(clusteringFieldName.get()) - .build(); - builder.setClustering(clustering); + .setSchema(bigQuerySchema); + + if (intermediateTables) { + // Shameful hack: make the table ingestion time-partitioned here so that the _PARTITIONTIME + // pseudocolumn can be queried to filter out rows that are still in the streaming buffer + builder.setTimePartitioning(TimePartitioning.of(Type.DAY)); + } else { + TimePartitioning timePartitioning = TimePartitioning.of(Type.DAY); + if (timestampPartitionFieldName.isPresent()) { + timePartitioning = timePartitioning.toBuilder().setField(timestampPartitionFieldName.get()).build(); + } + + builder.setTimePartitioning(timePartitioning); + + if (timestampPartitionFieldName.isPresent() && clusteringFieldName.isPresent()) { + Clustering clustering = Clustering.newBuilder() + .setFields(clusteringFieldName.get()) + .build(); + builder.setClustering(clustering); + } } StandardTableDefinition tableDefinition = builder.build(); TableInfo.Builder tableInfoBuilder = TableInfo.newBuilder(table, tableDefinition); - if (tableDescription != null) { + if (intermediateTables) { + tableInfoBuilder.setDescription("Temporary table"); + } else if (tableDescription != null) { tableInfoBuilder.setDescription(tableDescription); } + return tableInfoBuilder.build(); } private com.google.cloud.bigquery.Schema getBigQuerySchema(Schema kafkaKeySchema, Schema kafkaValueSchema) { - List allFields = new ArrayList<> (); com.google.cloud.bigquery.Schema valueSchema = schemaConverter.convertSchema(kafkaValueSchema); - allFields.addAll(valueSchema.getFields()); - if (kafkaKeyFieldName.isPresent()) { - com.google.cloud.bigquery.Schema keySchema = schemaConverter.convertSchema(kafkaKeySchema); - Field kafkaKeyField = Field.newBuilder(kafkaKeyFieldName.get(), LegacySQLTypeName.RECORD, keySchema.getFields()) - .setMode(Field.Mode.NULLABLE).build(); - allFields.add(kafkaKeyField); + + List schemaFields = intermediateTables + ? getIntermediateSchemaFields(valueSchema, kafkaKeySchema) + : getRegularSchemaFields(valueSchema, kafkaKeySchema); + + return com.google.cloud.bigquery.Schema.of(schemaFields); + } + + private List getIntermediateSchemaFields(com.google.cloud.bigquery.Schema valueSchema, Schema kafkaKeySchema) { + if (kafkaKeySchema == null) { + throw new BigQueryConnectException(String.format( + "Cannot create intermediate table without specifying a value for '%s'", + BigQuerySinkConfig.KAFKA_KEY_FIELD_NAME_CONFIG + )); } + + List result = new ArrayList<>(); + + List valueFields = new ArrayList<>(valueSchema.getFields()); if (kafkaDataFieldName.isPresent()) { - Field kafkaDataField = KafkaDataBuilder.buildKafkaDataField(kafkaDataFieldName.get()); - allFields.add(kafkaDataField); + Field kafkaDataField = KafkaDataBuilder.buildKafkaDataField(kafkaDataFieldName.get()); + valueFields.add(kafkaDataField); } - return com.google.cloud.bigquery.Schema.of(allFields); + + // Wrap the sink record value (and possibly also its Kafka data) in a struct in order to support deletes + Field wrappedValueField = Field + .newBuilder(MergeQueries.INTERMEDIATE_TABLE_VALUE_FIELD_NAME, LegacySQLTypeName.RECORD, valueFields.toArray(new Field[0])) + .setMode(Field.Mode.NULLABLE) + .build(); + result.add(wrappedValueField); + + com.google.cloud.bigquery.Schema keySchema = schemaConverter.convertSchema(kafkaKeySchema); + Field kafkaKeyField = Field.newBuilder(MergeQueries.INTERMEDIATE_TABLE_KEY_FIELD_NAME, LegacySQLTypeName.RECORD, keySchema.getFields()) + .setMode(Field.Mode.REQUIRED).build(); + result.add(kafkaKeyField); + + Field partitionTimeField = Field + .newBuilder(MergeQueries.INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME, LegacySQLTypeName.TIMESTAMP) + .setMode(Field.Mode.NULLABLE) + .build(); + result.add(partitionTimeField); + + Field batchNumberField = Field + .newBuilder(MergeQueries.INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD, LegacySQLTypeName.INTEGER) + .setMode(Field.Mode.REQUIRED) + .build(); + result.add(batchNumberField); + + return result; + } + + private List getRegularSchemaFields(com.google.cloud.bigquery.Schema valueSchema, Schema kafkaKeySchema) { + List result = new ArrayList<>(valueSchema.getFields()); + + if (kafkaDataFieldName.isPresent()) { + Field kafkaDataField = KafkaDataBuilder.buildKafkaDataField(kafkaDataFieldName.get()); + result.add(kafkaDataField); + } + + if (kafkaKeyFieldName.isPresent()) { + com.google.cloud.bigquery.Schema keySchema = schemaConverter.convertSchema(kafkaKeySchema); + Field kafkaKeyField = Field.newBuilder(kafkaKeyFieldName.get(), LegacySQLTypeName.RECORD, keySchema.getFields()) + .setMode(Field.Mode.NULLABLE).build(); + result.add(kafkaKeyField); + } + + return result; + } + + private String table(TableId table) { + return (intermediateTables ? "intermediate " : "") + + "table " + + table; } + private Object lock(ConcurrentMap locks, TableId table) { + return locks.computeIfAbsent(table, t -> new Object()); + } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index e1ab964c8..8cfc9c272 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -51,7 +51,6 @@ * Base class for connector and task configs; contains properties shared between the two of them. */ public class BigQuerySinkConfig extends AbstractConfig { - private static final ConfigDef config; private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkConfig.class); // Values taken from https://github.com/apache/kafka/blob/1.1.1/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java#L33 @@ -239,8 +238,61 @@ public class BigQuerySinkConfig extends AbstractConfig { private static final String ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_DOC = "If true, fields in BigQuery Schema can be changed from REQUIRED to NULLABLE"; - static { - config = new ConfigDef() + public static final String UPSERT_ENABLED_CONFIG = "upsertEnabled"; + private static final ConfigDef.Type UPSERT_ENABLED_TYPE = ConfigDef.Type.BOOLEAN; + public static final boolean UPSERT_ENABLED_DEFAULT = false; + private static final ConfigDef.Importance UPSERT_ENABLED_IMPORTANCE = ConfigDef.Importance.LOW; + private static final String UPSERT_ENABLED_DOC = + "Enable upsert functionality on the connector through the use of record keys, intermediate " + + "tables, and periodic merge flushes. Row-matching will be performed based on the contents " + + "of record keys."; + + public static final String DELETE_ENABLED_CONFIG = "deleteEnabled"; + private static final ConfigDef.Type DELETE_ENABLED_TYPE = ConfigDef.Type.BOOLEAN; + public static final boolean DELETE_ENABLED_DEFAULT = false; + private static final ConfigDef.Importance DELETE_ENABLED_IMPORTANCE = ConfigDef.Importance.LOW; + private static final String DELETE_ENABLED_DOC = + "Enable delete functionality on the connector through the use of record keys, intermediate " + + "tables, and periodic merge flushes. A delete will be performed when a record with a null " + + "value (i.e., a tombstone record) is read."; + + public static final String INTERMEDIATE_TABLE_SUFFIX_CONFIG = "intermediateTableSuffix"; + private static final ConfigDef.Type INTERMEDIATE_TABLE_SUFFIX_TYPE = ConfigDef.Type.STRING; + public static final String INTERMEDIATE_TABLE_SUFFIX_DEFAULT = "tmp"; + private static final ConfigDef.Validator INTERMEDIATE_TABLE_SUFFIX_VALIDATOR = new ConfigDef.NonEmptyString(); + private static final ConfigDef.Importance INTERMEDIATE_TTABLE_SUFFIX_IMPORTANCE = ConfigDef.Importance.LOW; + private static final String INTERMEDIATE_TABLE_SUFFIX_DOC = + "A suffix that will be appended to the names of destination tables to create the names for " + + "the corresponding intermediate tables. Multiple intermediate tables may be created for a " + + "single destination table, but their names will always start with the name of the " + + "destination table, followed by this suffix, and possibly followed by an additional " + + "suffix."; + + public static final String MERGE_INTERVAL_MS_CONFIG = "mergeIntervalMs"; + private static final ConfigDef.Type MERGE_INTERVAL_MS_TYPE = ConfigDef.Type.LONG; + public static final long MERGE_INTERVAL_MS_DEFAULT = 60_000L; + private static final ConfigDef.Validator MERGE_INTERVAL_MS_VALIDATOR = ConfigDef.Range.atLeast(-1); + private static final ConfigDef.Importance MERGE_INTERVAL_MS_IMPORTANCE = ConfigDef.Importance.LOW; + private static final String MERGE_INTERVAL_MS_DOC = + "How often (in milliseconds) to perform a merge flush, if upsert/delete is enabled. Can be " + + "set to -1 to disable periodic flushing."; + + public static final String MERGE_RECORDS_THRESHOLD_CONFIG = "mergeRecordsThreshold"; + private static final ConfigDef.Type MERGE_RECORDS_THRESHOLD_TYPE = ConfigDef.Type.LONG; + public static final long MERGE_RECORDS_THRESHOLD_DEFAULT = -1; + private static final ConfigDef.Validator MERGE_RECORDS_THRESHOLD_VALIDATOR = ConfigDef.Range.atLeast(-1); + private static final ConfigDef.Importance MERGE_RECORDS_THRESHOLD_IMPORTANCE = ConfigDef.Importance.LOW; + private static final String MERGE_RECORDS_THRESHOLD_DOC = + "How many records to write to an intermediate table before performing a merge flush, if " + + "upsert/delete is enabled. Can be set to -1 to disable record count-based flushing."; + + /** + * Return a ConfigDef object used to define this config's fields. + * + * @return A ConfigDef object used to define this config's fields. + */ + public static ConfigDef getConfig() { + return new ConfigDef() .define( TOPICS_CONFIG, TOPICS_TYPE, @@ -382,25 +434,85 @@ public class BigQuerySinkConfig extends AbstractConfig { ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_DEFAULT, ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_IMPORTANCE, ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_DOC + ).define( + UPSERT_ENABLED_CONFIG, + UPSERT_ENABLED_TYPE, + UPSERT_ENABLED_DEFAULT, + UPSERT_ENABLED_IMPORTANCE, + UPSERT_ENABLED_DOC + ).define( + DELETE_ENABLED_CONFIG, + DELETE_ENABLED_TYPE, + DELETE_ENABLED_DEFAULT, + DELETE_ENABLED_IMPORTANCE, + DELETE_ENABLED_DOC + ).define( + INTERMEDIATE_TABLE_SUFFIX_CONFIG, + INTERMEDIATE_TABLE_SUFFIX_TYPE, + INTERMEDIATE_TABLE_SUFFIX_DEFAULT, + INTERMEDIATE_TABLE_SUFFIX_VALIDATOR, + INTERMEDIATE_TTABLE_SUFFIX_IMPORTANCE, + INTERMEDIATE_TABLE_SUFFIX_DOC + ).define( + MERGE_INTERVAL_MS_CONFIG, + MERGE_INTERVAL_MS_TYPE, + MERGE_INTERVAL_MS_DEFAULT, + MERGE_INTERVAL_MS_VALIDATOR, + MERGE_INTERVAL_MS_IMPORTANCE, + MERGE_INTERVAL_MS_DOC + ).define( + MERGE_RECORDS_THRESHOLD_CONFIG, + MERGE_RECORDS_THRESHOLD_TYPE, + MERGE_RECORDS_THRESHOLD_DEFAULT, + MERGE_RECORDS_THRESHOLD_VALIDATOR, + MERGE_RECORDS_THRESHOLD_IMPORTANCE, + MERGE_RECORDS_THRESHOLD_DOC ); } - /** - * Throw an exception if the passed-in properties do not constitute a valid sink. - * @param props sink configuration properties - */ - public static void validate(Map props) { - final boolean hasTopicsConfig = hasTopicsConfig(props); - final boolean hasTopicsRegexConfig = hasTopicsRegexConfig(props); - - if (hasTopicsConfig && hasTopicsRegexConfig) { - throw new ConfigException(TOPICS_CONFIG + " and " + TOPICS_REGEX_CONFIG + - " are mutually exclusive options, but both are set."); - } - - if (!hasTopicsConfig && !hasTopicsRegexConfig) { - throw new ConfigException("Must configure one of " + - TOPICS_CONFIG + " or " + TOPICS_REGEX_CONFIG); - } + + /** + * Throw an exception if the passed-in properties do not constitute a valid sink. + * @param props sink configuration properties + */ + public static void validate(Map props) { + final boolean hasTopicsConfig = hasTopicsConfig(props); + final boolean hasTopicsRegexConfig = hasTopicsRegexConfig(props); + + if (hasTopicsConfig && hasTopicsRegexConfig) { + throw new ConfigException(TOPICS_CONFIG + " and " + TOPICS_REGEX_CONFIG + + " are mutually exclusive options, but both are set."); + } + + if (!hasTopicsConfig && !hasTopicsRegexConfig) { + throw new ConfigException("Must configure one of " + + TOPICS_CONFIG + " or " + TOPICS_REGEX_CONFIG); + } + + if (upsertDeleteEnabled(props)) { + String mergeIntervalStr = Optional.ofNullable(props.get(MERGE_INTERVAL_MS_CONFIG)) + .map(String::trim) + .orElse(Long.toString(MERGE_INTERVAL_MS_DEFAULT)); + String mergeRecordsThresholdStr = Optional.ofNullable(props.get(MERGE_RECORDS_THRESHOLD_CONFIG)) + .map(String::trim) + .orElse(Long.toString(MERGE_RECORDS_THRESHOLD_DEFAULT)); + if ("-1".equals(mergeIntervalStr) && "-1".equals(mergeRecordsThresholdStr)) { + throw new ConfigException(MERGE_INTERVAL_MS_CONFIG + " and " + + MERGE_RECORDS_THRESHOLD_CONFIG + " cannot both be -1"); + } + + if ("0".equals(mergeIntervalStr)) { + throw new ConfigException(MERGE_INTERVAL_MS_CONFIG, mergeIntervalStr, "cannot be zero"); + } + if ("0".equals(mergeRecordsThresholdStr)) { + throw new ConfigException(MERGE_RECORDS_THRESHOLD_CONFIG, mergeRecordsThresholdStr, "cannot be zero"); + } + + String kafkaKeyFieldStr = props.get(KAFKA_KEY_FIELD_NAME_CONFIG); + if (kafkaKeyFieldStr == null || kafkaKeyFieldStr.trim().isEmpty()) { + throw new ConfigException(KAFKA_KEY_FIELD_NAME_CONFIG + " must be specified when " + + UPSERT_ENABLED_CONFIG + " and/or " + DELETE_ENABLED_CONFIG + " are set to true"); + } + } } public static boolean hasTopicsConfig(Map props) { @@ -413,6 +525,13 @@ public static boolean hasTopicsRegexConfig(Map props) { return topicsRegexStr != null && !topicsRegexStr.trim().isEmpty(); } + public static boolean upsertDeleteEnabled(Map props) { + String upsertStr = props.get(UPSERT_ENABLED_CONFIG); + String deleteStr = props.get(DELETE_ENABLED_CONFIG); + return Boolean.TRUE.toString().equalsIgnoreCase(upsertStr) + || Boolean.TRUE.toString().equalsIgnoreCase(deleteStr); + } + /** * Returns the keyfile */ @@ -516,6 +635,10 @@ public Optional getKafkaDataFieldName() { return Optional.ofNullable(getString(KAFKA_DATA_FIELD_NAME_CONFIG)); } + public boolean isUpsertDeleteEnabled() { + return getBoolean(UPSERT_ENABLED_CONFIG) || getBoolean(DELETE_ENABLED_CONFIG); + } + /** * Verifies that a bucket is specified if GCS batch loading is enabled. * @throws ConfigException Exception thrown if no bucket is specified and batch loading is on. @@ -550,22 +673,13 @@ private void checkBigQuerySchemaUpdateConfigs() { } } - /** - * Return the ConfigDef object used to define this config's fields. - * - * @return The ConfigDef object used to define this config's fields. - */ - public static ConfigDef getConfig() { - return config; - } - protected BigQuerySinkConfig(ConfigDef config, Map properties) { super(config, properties); verifyBucketSpecified(); } public BigQuerySinkConfig(Map properties) { - super(config, properties); + super(getConfig(), properties); verifyBucketSpecified(); checkAutoCreateTables(); checkBigQuerySchemaUpdateConfigs(); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java index 9b395700b..1ba973dc7 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java @@ -55,7 +55,7 @@ public class BigQuerySinkTaskConfig extends BigQuerySinkConfig { "The maximum size (or -1 for no maximum size) of the worker queue for bigQuery write " + "requests before all topics are paused. This is a soft limit; the size of the queue can " + "go over this before topics are paused. All topics will be resumed once a flush is " - + "requested or the size of the queue drops under half of the maximum size."; + + "triggered or the size of the queue drops under half of the maximum size."; public static final String BIGQUERY_RETRY_CONFIG = "bigQueryRetry"; private static final ConfigDef.Type BIGQUERY_RETRY_TYPE = ConfigDef.Type.INT; @@ -120,6 +120,11 @@ public class BigQuerySinkTaskConfig extends BigQuerySinkConfig { private static final String BIGQUERY_CLUSTERING_FIELD_NAMES_DOC = "List of fields on which data should be clustered by in BigQuery, separated by commas"; + public static final String TASK_ID_CONFIG = "taskId"; + private static final ConfigDef.Type TASK_ID_TYPE = ConfigDef.Type.INT; + public static final ConfigDef.Importance TASK_ID_IMPORTANCE = ConfigDef.Importance.LOW; + private static final String TASK_ID_DOC = "A unique for each task created by the connector"; + static { config = BigQuerySinkConfig.getConfig() .define( @@ -174,6 +179,11 @@ public class BigQuerySinkTaskConfig extends BigQuerySinkConfig { BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT, BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE, BIGQUERY_CLUSTERING_FIELD_NAMES_DOC + ).define( + TASK_ID_CONFIG, + TASK_ID_TYPE, + TASK_ID_IMPORTANCE, + TASK_ID_DOC ); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverter.java index 835935bd8..816e74b4e 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverter.java @@ -47,12 +47,12 @@ */ public class BigQueryRecordConverter implements RecordConverter> { - private static final Set BASIC_TYPES = new HashSet( + private static final Set> BASIC_TYPES = new HashSet<>( Arrays.asList( Boolean.class, Character.class, Byte.class, Short.class, Integer.class, Long.class, Float.class, Double.class, String.class) ); - private boolean shouldConvertSpecialDouble; + private final boolean shouldConvertSpecialDouble; static { // force registration @@ -72,6 +72,7 @@ public BigQueryRecordConverter(boolean shouldConvertDoubleSpecial) { * @param recordType The type of the record to convert, either value or key. * @return The result BigQuery row content. */ + @SuppressWarnings("unchecked") public Map convertRecord(SinkRecord record, KafkaSchemaRecordType recordType) { Schema kafkaConnectSchema = recordType == KafkaSchemaRecordType.KEY ? record.keySchema() : record.valueSchema(); Object kafkaConnectStruct = recordType == KafkaSchemaRecordType.KEY ? record.key() : record.value(); @@ -89,6 +90,7 @@ public Map convertRecord(SinkRecord record, KafkaSchemaRecordTyp return convertStruct(kafkaConnectStruct, kafkaConnectSchema); } + @SuppressWarnings("unchecked") private Object convertSchemalessRecord(Object value) { if (value == null) { return null; @@ -103,10 +105,9 @@ private Object convertSchemalessRecord(Object value) { return convertBytes(value); } if (value instanceof List) { - return - ((List) value).stream().map( - v -> convertSchemalessRecord(v) - ).collect(Collectors.toList()); + return ((List) value).stream() + .map(this::convertSchemalessRecord) + .collect(Collectors.toList()); } if (value instanceof Map) { return @@ -128,7 +129,6 @@ private Object convertSchemalessRecord(Object value) { " found in schemaless record data. Can't convert record to bigQuery format"); } - @SuppressWarnings("unchecked") private Object convertObject(Object kafkaConnectObject, Schema kafkaConnectSchema) { if (kafkaConnectObject == null) { if (kafkaConnectSchema.isOptional()) { @@ -152,22 +152,16 @@ private Object convertObject(Object kafkaConnectObject, Schema kafkaConnectSchem return convertStruct(kafkaConnectObject, kafkaConnectSchema); case BYTES: return convertBytes(kafkaConnectObject); - case BOOLEAN: - return (Boolean) kafkaConnectObject; - case FLOAT32: - return (Float) kafkaConnectObject; case FLOAT64: return convertDouble((Double)kafkaConnectObject); + case BOOLEAN: + case FLOAT32: case INT8: - return (Byte) kafkaConnectObject; case INT16: - return (Short) kafkaConnectObject; case INT32: - return (Integer) kafkaConnectObject; case INT64: - return (Long) kafkaConnectObject; case STRING: - return (String) kafkaConnectObject; + return kafkaConnectObject; default: throw new ConversionConnectException("Unrecognized schema type: " + kafkaConnectSchemaType); } @@ -214,7 +208,7 @@ private List> convertMap(Object kafkaConnectObject, Schema kafkaConnectValueSchema = kafkaConnectSchema.valueSchema(); List> bigQueryEntryList = new ArrayList<>(); Map kafkaConnectMap = (Map) kafkaConnectObject; - for (Map.Entry kafkaConnectMapEntry : kafkaConnectMap.entrySet()) { + for (Map.Entry kafkaConnectMapEntry : kafkaConnectMap.entrySet()) { Map bigQueryEntry = new HashMap<>(); Object bigQueryKey = convertObject( kafkaConnectMapEntry.getKey(), diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java index 175d6f884..a48c3d2eb 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java @@ -97,6 +97,7 @@ public BigQuerySchemaConverter(boolean allFieldsNullable) { * existing one. */ public com.google.cloud.bigquery.Schema convertSchema(Schema kafkaConnectSchema) { + // TODO: Permit non-struct keys if (kafkaConnectSchema.type() != Schema.Type.STRUCT) { throw new ConversionConnectException("Top-level Kafka Connect schema must be of type 'struct'"); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizer.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizer.java index 09aeb70c2..a55399331 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizer.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizer.java @@ -20,6 +20,7 @@ public static String sanitizeName(String name) { // letters, numbers, and underscores. // Note: a.b and a/b will have the same value after sanitization which will cause Duplicate key // Exception. + @SuppressWarnings("unchecked") public static Map replaceInvalidKeys(Map map) { return map.entrySet().stream().collect(Collectors.toMap( (entry) -> sanitizeName(entry.getKey()), diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java index 96cd2144b..4af836142 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java @@ -18,11 +18,20 @@ */ import com.google.cloud.bigquery.InsertAllRequest; +import com.google.cloud.bigquery.TableId; +import com.wepay.kafka.connect.bigquery.MergeQueries; import com.wepay.kafka.connect.bigquery.api.KafkaSchemaRecordType; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.convert.KafkaDataBuilder; import com.wepay.kafka.connect.bigquery.convert.RecordConverter; +import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -30,36 +39,104 @@ * A class for converting a {@link SinkRecord SinkRecord} to {@link InsertAllRequest.RowToInsert BigQuery row} */ public class SinkRecordConverter { + private static final Logger logger = LoggerFactory.getLogger(SinkRecordConverter.class); + + private final BigQuerySinkTaskConfig config; + private final MergeBatches mergeBatches; + private final MergeQueries mergeQueries; + private final RecordConverter> recordConverter; - private final boolean sanitizeFieldName; - private final Optional kafkaKeyFieldName; - private final Optional kafkaDataFieldName; - - public SinkRecordConverter(RecordConverter> recordConverter, boolean sanitizeFieldName, Optional kafkaKeyFieldName, Optional kafkaDataFieldName) { - this.recordConverter = recordConverter; - this.sanitizeFieldName = sanitizeFieldName; - this.kafkaKeyFieldName = kafkaKeyFieldName; - this.kafkaDataFieldName = kafkaDataFieldName; + private final long mergeRecordsThreshold; + private final boolean useMessageTimeDatePartitioning; + private final boolean usePartitionDecorator; + + + public SinkRecordConverter(BigQuerySinkTaskConfig config, + MergeBatches mergeBatches, MergeQueries mergeQueries) { + this.config = config; + this.mergeBatches = mergeBatches; + this.mergeQueries = mergeQueries; + + this.recordConverter = config.getRecordConverter(); + this.mergeRecordsThreshold = config.getLong(config.MERGE_RECORDS_THRESHOLD_CONFIG); + this.useMessageTimeDatePartitioning = + config.getBoolean(config.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG); + this.usePartitionDecorator = + config.getBoolean(config.BIGQUERY_PARTITION_DECORATOR_CONFIG); + + } + + public InsertAllRequest.RowToInsert getRecordRow(SinkRecord record, TableId table) { + Map convertedRecord = config.isUpsertDeleteEnabled() + ? getUpsertDeleteRow(record, table) + : getRegularRow(record); + + Map result = config.getBoolean(config.SANITIZE_FIELD_NAME_CONFIG) + ? FieldNameSanitizer.replaceInvalidKeys(convertedRecord) + : convertedRecord; + + return InsertAllRequest.RowToInsert.of(getRowId(record), result); } - public InsertAllRequest.RowToInsert getRecordRow(SinkRecord record) { - Map convertedRecord = recordConverter.convertRecord(record, KafkaSchemaRecordType.VALUE); - if (kafkaKeyFieldName.isPresent()) { - convertedRecord.put(kafkaKeyFieldName.get(), recordConverter.convertRecord(record, KafkaSchemaRecordType.KEY)); + private Map getUpsertDeleteRow(SinkRecord record, TableId table) { + // Unconditionally allow tombstone records if delete is enabled. + Map convertedValue = config.getBoolean(config.DELETE_ENABLED_CONFIG) && record.value() == null + ? null + : recordConverter.convertRecord(record, KafkaSchemaRecordType.VALUE); + + Map convertedKey = recordConverter.convertRecord(record, KafkaSchemaRecordType.KEY); + + if (convertedValue != null) { + config.getKafkaDataFieldName().ifPresent( + fieldName -> convertedValue.put(fieldName, KafkaDataBuilder.buildKafkaDataRecord(record)) + ); } - if (kafkaDataFieldName.isPresent()) { - convertedRecord.put(kafkaDataFieldName.get(), KafkaDataBuilder.buildKafkaDataRecord(record)); + + Map result = new HashMap<>(); + long totalBatchSize = mergeBatches.addToBatch(record, table, result); + if (mergeRecordsThreshold != -1 && totalBatchSize >= mergeRecordsThreshold) { + logger.debug("Triggering merge flush for table {} since the size of its current batch has " + + "exceeded the configured threshold of {}}", + table, mergeRecordsThreshold); + mergeQueries.mergeFlush(table); } - if (sanitizeFieldName) { - convertedRecord = FieldNameSanitizer.replaceInvalidKeys(convertedRecord); + + result.put(MergeQueries.INTERMEDIATE_TABLE_KEY_FIELD_NAME, convertedKey); + result.put(MergeQueries.INTERMEDIATE_TABLE_VALUE_FIELD_NAME, convertedValue); + if (usePartitionDecorator && useMessageTimeDatePartitioning) { + if (record.timestampType() == TimestampType.NO_TIMESTAMP_TYPE) { + throw new ConnectException( + "Message has no timestamp type, cannot use message timestamp to partition."); + } + result.put(MergeQueries.INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME, record.timestamp()); + } else { + // Provide a value for this column even if it's not used for partitioning in the destination + // table, so that it can be used to deduplicate rows during merge flushes + result.put(MergeQueries.INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME, System.currentTimeMillis() / 1000); } - return InsertAllRequest.RowToInsert.of(getRowId(record), convertedRecord); + + return result; + } + + private Map getRegularRow(SinkRecord record) { + Map result = recordConverter.convertRecord(record, KafkaSchemaRecordType.VALUE); + + config.getKafkaDataFieldName().ifPresent( + fieldName -> result.put(fieldName, KafkaDataBuilder.buildKafkaDataRecord(record)) + ); + + config.getKafkaKeyFieldName().ifPresent(fieldName -> { + Map keyData = recordConverter.convertRecord(record, KafkaSchemaRecordType.KEY); + result.put(fieldName, keyData); + }); + + return result; } private String getRowId(SinkRecord record) { return String.format("%s-%d-%d", - record.topic(), - record.kafkaPartition(), - record.kafkaOffset()); + record.topic(), + record.kafkaPartition(), + record.kafkaOffset()); } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/GCSBatchTableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/GCSBatchTableWriter.java index 45d12e1dd..d876bc2d8 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/GCSBatchTableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/GCSBatchTableWriter.java @@ -47,7 +47,7 @@ public class GCSBatchTableWriter implements Runnable { private final String bucketName; private final String blobName; - private SortedMap rows; + private final SortedMap rows; private final GCSToBQWriter writer; /** @@ -90,7 +90,7 @@ public static class Builder implements TableWriterBuilder { private String blobName; private final TableId tableId; - private SortedMap rows; + private final SortedMap rows; private final SinkRecordConverter recordConverter; private final GCSToBQWriter writer; @@ -119,19 +119,12 @@ public Builder(GCSToBQWriter writer, this.writer = writer; } - public Builder setBlobName(String blobName) { - this.blobName = blobName; - return this; - } - - /** - * Adds a record to the builder. - * @param record the row to add - */ - public void addRow(SinkRecord record) { - rows.put(record, recordConverter.getRecordRow(record)); + @Override + public void addRow(SinkRecord record, TableId table) { + rows.put(record, recordConverter.getRecordRow(record, table)); } + @Override public GCSBatchTableWriter build() { return new GCSBatchTableWriter(rows, writer, tableId, bucketName, blobName); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java index 637ac2ea4..9e128cdaa 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java @@ -91,11 +91,21 @@ public void awaitCurrentTasks() throws InterruptedException, BigQueryConnectExce execute(new CountDownRunnable(countDownLatch)); } countDownLatch.await(); + maybeThrowEncounteredErrors(); + } + + /** + * Immediately throw an exception if any unrecoverable errors were encountered by any of the write + * tasks. + * + * @throws BigQueryConnectException if any of the tasks failed. + */ + public void maybeThrowEncounteredErrors() { if (encounteredErrors.size() > 0) { String errorString = createErrorString(encounteredErrors); encounteredErrors.clear(); throw new BigQueryConnectException("Some write threads encountered unrecoverable errors: " - + errorString + "; See logs for more detail"); + + errorString + "; See logs for more detail"); } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java new file mode 100644 index 000000000..92558a251 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java @@ -0,0 +1,332 @@ +package com.wepay.kafka.connect.bigquery.write.batch; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + + +import com.google.cloud.bigquery.InsertAllRequest; +import com.google.cloud.bigquery.TableId; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; +import com.google.common.collect.Maps; +import com.wepay.kafka.connect.bigquery.MergeQueries; +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +public class MergeBatches { + private static final Logger logger = LoggerFactory.getLogger(MergeBatches.class); + private static final long STREAMING_BUFFER_AVAILABILITY_WAIT_MS = 10_000L; + + private final String intermediateTableSuffix; + private final BiMap intermediateToDestinationTables; + private final ConcurrentMap batchNumbers; + private final ConcurrentMap> batches; + private final Map offsets; + + public MergeBatches(String intermediateTableSuffix) { + this.intermediateTableSuffix = intermediateTableSuffix; + + this.intermediateToDestinationTables = Maps.synchronizedBiMap(HashBiMap.create()); + this.batchNumbers = new ConcurrentHashMap<>(); + this.batches = new ConcurrentHashMap<>(); + this.offsets = new HashMap<>(); + } + + /** + * Get the latest safe-to-commit offsets for every topic partition that has had at least one + * record make its way to a destination table. + * @return the offsets map which can be used in + * {@link org.apache.kafka.connect.sink.SinkTask#preCommit(Map)}; never null + */ + public Map latestOffsets() { + synchronized (offsets) { + return offsets.entrySet().stream().collect(Collectors.toMap( + Map.Entry::getKey, + entry -> new OffsetAndMetadata(entry.getValue()) + )); + } + } + + /** + * @return a thread-safe map from intermediate tables to destination tables; never null + */ + public Map intermediateToDestinationTables() { + return intermediateToDestinationTables; + } + + /** + * @return a collection of all currently-in-use intermediate tables; never null + */ + public Collection intermediateTables() { + return intermediateToDestinationTables.keySet(); + } + + /** + * Get the intermediate table for a given destination table, computing a new one if necessary + * @param destinationTable the destination table to fetch an intermediate table for + * @return the {@link TableId} of the intermediate table; never null + */ + public TableId intermediateTableFor(TableId destinationTable) { + return intermediateToDestinationTables.inverse() + .computeIfAbsent(destinationTable, this::newIntermediateTable); + } + + private TableId newIntermediateTable(TableId destinationTable) { + String tableName = FieldNameSanitizer.sanitizeName( + destinationTable.getTable() + intermediateTableSuffix + ); + TableId result = TableId.of( + destinationTable.getDataset(), + tableName + ); + + batchNumbers.put(result, new AtomicInteger()); + batches.put(result, new ConcurrentHashMap<>()); + + return result; } + + public TableId destinationTableFor(TableId intermediateTable) { + return intermediateToDestinationTables.get(intermediateTable); + } + + /** + * Find a batch number for the record, insert that number into the converted value, record the + * offset for that record, and return the total size of that batch. + * @param record the record for the batch + * @param intermediateTable the intermediate table the record will be streamed into + * @param convertedRecord the converted record that will be passed to the BigQuery client + * @return the total number of records in the batch that this record is added to + */ + public long addToBatch(SinkRecord record, TableId intermediateTable, Map convertedRecord) { + AtomicInteger batchCount = batchNumbers.get(intermediateTable); + // Synchronize here to ensure that the batch number isn't bumped in the middle of this method. + // On its own, that wouldn't be such a bad thing, but since a merge flush is supposed to + // immediately follow that bump, it might cause some trouble if we want to add this row to the + // batch but a merge flush on that batch has already started. This way, either the batch number + // is bumped before we add the row to the batch (in which case, the row is added to the fresh + // batch), or the row is added to the batch before preparation for the flush takes place and it + // is safely counted and tracked there. + synchronized (batchCount) { + int batchNumber = batchCount.get(); + convertedRecord.put(MergeQueries.INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD, batchNumber); + + Batch batch = batches.get(intermediateTable).computeIfAbsent(batchNumber, n -> new Batch()); + batch.recordOffsetFor(record); + + long pendingBatchSize = batch.increment(); + logger.trace("Added record to batch {} for intermediate table {}; {} rows are currently pending", + batchNumber, intermediateTable, pendingBatchSize); + return batch.total(); + } + } + + /** + * Record a successful write of a list of rows to the given intermediate table, and decrease the + * pending record counts for every applicable batch accordingly. + * @param intermediateTable the intermediate table + * @param rows the rows + */ + public void onRowWrites(TableId intermediateTable, Collection rows) { + Map rowsByBatch = rows.stream().collect(Collectors.groupingBy( + row -> (Integer) row.getContent().get(MergeQueries.INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD), + Collectors.counting() + )); + + rowsByBatch.forEach((batchNumber, batchSize) -> { + Batch batch = batch(intermediateTable, batchNumber); + synchronized (batch) { + long remainder = batch.recordWrites(batchSize); + batch.notifyAll(); + logger.trace("Notified merge flush executor of successful write of {} rows " + + "for batch {} for intermediate table {}; {} rows remaining", + batchSize, batchNumber, intermediateTable, remainder); + } + }); + } + + /** + * Increment the batch number for the given table, and return the old batch number. + * @param intermediateTable the table whose batch number should be incremented + * @return the batch number for the table, pre-increment + */ + public int incrementBatch(TableId intermediateTable) { + AtomicInteger batchCount = batchNumbers.get(intermediateTable); + // See addToBatch for an explanation of the synchronization here + synchronized (batchCount) { + return batchCount.getAndIncrement(); + } + } + + /** + * Prepare to merge the batch for the given table, by ensuring that all prior batches for that + * table have completed and that all rows for the batch itself have been written. + * @param intermediateTable the table for the batch + * @param batchNumber the batch number to prepare to flush + * @return whether a flush is necessary (will be false if no rows were present in the given batch) + */ + public boolean prepareToFlush(TableId intermediateTable, int batchNumber) { + final ConcurrentMap allBatchesForTable = batches.get(intermediateTable); + if (batchNumber != 0) { + final int priorBatchNumber = batchNumber - 1; + synchronized (allBatchesForTable) { + logger.debug("Ensuring batch {} is completed for intermediate table {} before flushing batch {}", + priorBatchNumber, intermediateTable, batchNumber); + while (allBatchesForTable.containsKey(priorBatchNumber)) { + try { + allBatchesForTable.wait(); + } catch (InterruptedException e) { + logger.warn("Interrupted while waiting for batch {} to complete for intermediate table {}", + batchNumber, intermediateTable); + throw new ConnectException(String.format( + "Interrupted while waiting for batch %d to complete for intermediate tabld %s", + batchNumber, intermediateTable + )); + } + } + } + } else { + logger.debug("Flushing first batch for intermediate table {}", intermediateTable); + } + + final Batch currentBatch = allBatchesForTable.get(batchNumber); + if (currentBatch == null) { + logger.trace("No rows to write in batch {} for intermediate table {}", batchNumber, intermediateTable); + return false; + } + + synchronized (currentBatch) { + logger.debug("{} rows currently remaining for batch {} for intermediate table {}", + currentBatch.pending(), batchNumber, intermediateTable); + while (currentBatch.pending() != 0) { + logger.trace("Waiting for all rows for batch {} from intermediate table {} to be written before flushing; {} remaining", + batchNumber, intermediateTable, currentBatch.pending()); + try { + currentBatch.wait(); + } catch (InterruptedException e) { + logger.warn("Interrupted while waiting for all rows for batch {} from intermediate table {} to be written", + batchNumber, intermediateTable); + throw new ConnectException(String.format( + "Interrupted while waiting for all rows for batch %d from intermediate table %s to be written", + batchNumber, intermediateTable + )); + } + } + } + + try { + logger.trace( + "Waiting {} seconds before running merge query on batch {} from intermediate table {} " + + "in order to ensure that all rows are available in the streaming buffer", + STREAMING_BUFFER_AVAILABILITY_WAIT_MS, batchNumber, intermediateTable); + Thread.sleep(STREAMING_BUFFER_AVAILABILITY_WAIT_MS); + } catch (InterruptedException e) { + logger.warn("Interrupted while waiting before merge flushing batch {} for intermediate table {}", + batchNumber, intermediateTable); + throw new ConnectException(String.format( + "Interrupted while waiting before merge flushing batch %d for intermediate table %s", + batchNumber, intermediateTable + )); + } + return true; + } + + /** + * Record a successful merge flush of all of the rows for the given batch in the intermediate + * table, alert any waiting merge flushes that are predicated on the completion of this merge + * flush, and marke the offsets for all of those rows as safe to commit. + * @param intermediateTable the source of the merge flush + * @param batchNumber the batch for the merge flush + */ + public void recordSuccessfulFlush(TableId intermediateTable, int batchNumber) { + logger.trace("Successfully merge flushed batch {} for intermediate table {}", + batchNumber, intermediateTable); + final ConcurrentMap allBatchesForTable = batches.get(intermediateTable); + Batch batch = allBatchesForTable.remove(batchNumber); + + synchronized (allBatchesForTable) { + allBatchesForTable.notifyAll(); + } + + synchronized (offsets) { + offsets.putAll(batch.offsets()); + } + } + + private Batch batch(TableId intermediateTable, int batchNumber) { + return batches.get(intermediateTable).get(batchNumber); + } + + private static class Batch { + private final AtomicLong pending; + private final AtomicLong total; + private final Map offsets; + + public Batch() { + this.total = new AtomicLong(); + this.pending = new AtomicLong(); + this.offsets = new HashMap<>(); + } + + public long pending() { + return pending.get(); + } + + public long total() { + return total.get(); + } + + public Map offsets() { + return offsets; + } + + public void recordOffsetFor(SinkRecord record) { + offsets.put( + new TopicPartition(record.topic(), record.kafkaPartition()), + // Use the offset of the record plus one here since that'll be the offset that we'll + // resume at if/when this record is the last-committed record and then the task is + // restarted + record.kafkaOffset() + 1); + } + + /** + * Increment the total and pending number of records, and return the number of pending records + * @return the number of pending records for this batch + */ + public long increment() { + total.incrementAndGet(); + return pending.incrementAndGet(); + } + + public long recordWrites(long numRows) { + return pending.addAndGet(-numRows); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index dee011142..e8747f851 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -21,6 +21,7 @@ import com.google.cloud.bigquery.BigQueryException; import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; +import com.google.cloud.bigquery.TableId; import com.wepay.kafka.connect.bigquery.utils.SinkRecordConverter; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; @@ -31,11 +32,14 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; +import java.util.Objects; +import java.util.function.Consumer; /** * Simple Table Writer that attempts to write all the rows it is given at once. @@ -50,18 +54,23 @@ public class TableWriter implements Runnable { private final BigQueryWriter writer; private final PartitionedTableId table; private final SortedMap rows; + private final Consumer> onFinish; /** * @param writer the {@link BigQueryWriter} to use. * @param table the BigQuery table to write to. * @param rows the rows to write. + * @param onFinish a callback to invoke after all rows have been written successfully, which is + * called with all the rows written by the writer */ public TableWriter(BigQueryWriter writer, PartitionedTableId table, - SortedMap rows) { + SortedMap rows, + Consumer> onFinish) { this.writer = writer; this.table = table; this.rows = rows; + this.onFinish = onFinish; } @Override @@ -106,6 +115,7 @@ public void run() { logger.debug(logMessage, rows.size(), successCount, failureCount); } + onFinish.accept(rows.values()); } private static int getNewBatchSize(int currentBatchSize) { @@ -152,6 +162,7 @@ public static class Builder implements TableWriterBuilder { private SortedMap rows; private SinkRecordConverter recordConverter; + private Consumer> onFinish; /** * @param writer the BigQueryWriter to use @@ -165,22 +176,31 @@ public Builder(BigQueryWriter writer, PartitionedTableId table, SinkRecordConver this.rows = new TreeMap<>(Comparator.comparing(SinkRecord::kafkaPartition) .thenComparing(SinkRecord::kafkaOffset)); this.recordConverter = recordConverter; + + this.onFinish = null; } - /** - * Add a record to the builder. - * @param record the row to add - */ - public void addRow(SinkRecord record) { - rows.put(record, recordConverter.getRecordRow(record)); + @Override + public void addRow(SinkRecord record, TableId table) { + rows.put(record, recordConverter.getRecordRow(record, table)); } /** - * Create a {@link TableWriter} from this builder. - * @return a TableWriter containing the given writer, table, topic, and all added rows. + * Specify a callback to be invoked after all rows have been written. The callback will be + * invoked with the full list of rows written by this table writer. + * @param onFinish the callback to invoke; may not be null + * @throws IllegalStateException if invoked more than once on a single builder instance */ + public void onFinish(Consumer> onFinish) { + if (this.onFinish != null) { + throw new IllegalStateException("Cannot overwrite existing finish callback"); + } + this.onFinish = Objects.requireNonNull(onFinish, "Finish callback cannot be null"); + } + + @Override public TableWriter build() { - return new TableWriter(writer, table, rows); + return new TableWriter(writer, table, rows, onFinish != null ? onFinish : n -> { }); } } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriterBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriterBuilder.java index 556ae98e3..cbd9ba8b8 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriterBuilder.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriterBuilder.java @@ -18,6 +18,7 @@ */ +import com.google.cloud.bigquery.TableId; import org.apache.kafka.connect.sink.SinkRecord; /** @@ -28,8 +29,9 @@ public interface TableWriterBuilder { /** * Add a record to the builder. * @param sinkRecord the row to add. + * @param table the table the row will be written to. */ - void addRow(SinkRecord sinkRecord); + void addRow(SinkRecord sinkRecord, TableId table); /** * Create a {@link TableWriter} from this builder. diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java index 10c30babc..e3b970459 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java @@ -47,7 +47,7 @@ public class AdaptiveBigQueryWriter extends BigQueryWriter { private static final Logger logger = LoggerFactory.getLogger(AdaptiveBigQueryWriter.class); // The maximum number of retries we will attempt to write rows after creating a table or updating a BQ table schema. - private static final int RETRY_LIMIT = 5; + private static final int RETRY_LIMIT = 10; // Wait for about 30s between each retry since both creating table and updating schema take up to 2~3 minutes to take effect. private static final int RETRY_WAIT_TIME = 30000; @@ -60,6 +60,7 @@ public class AdaptiveBigQueryWriter extends BigQueryWriter { * @param schemaManager Used to update BigQuery tables. * @param retry How many retries to make in the event of a 500/503 error. * @param retryWait How long to wait in between retries. + * @param autoCreateTables Whether tables should be automatically created */ public AdaptiveBigQueryWriter(BigQuery bigQuery, SchemaManager schemaManager, @@ -149,7 +150,7 @@ && onlyContainsInvalidSchemaErrors(writeResponse.getInsertErrors())) { return new HashMap<>(); } - private void attemptSchemaUpdate(PartitionedTableId tableId, Set records) { + protected void attemptSchemaUpdate(PartitionedTableId tableId, Set records) { try { schemaManager.updateSchema(tableId.getBaseTableId(), records); } catch (BigQueryException exception) { @@ -158,9 +159,8 @@ private void attemptSchemaUpdate(PartitionedTableId tableId, Set rec } } - private void attemptTableCreate(TableId tableId, Set records) { + protected void attemptTableCreate(TableId tableId, Set records) { try { - logger.info("Table {} does not exist, auto-creating table", tableId); schemaManager.createTable(tableId, records); } catch (BigQueryException exception) { throw new BigQueryConnectException( diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java new file mode 100644 index 000000000..20e30226d --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java @@ -0,0 +1,94 @@ +package com.wepay.kafka.connect.bigquery.write.row; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryException; +import com.google.cloud.bigquery.TableId; +import com.wepay.kafka.connect.bigquery.SchemaManager; +import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; +import org.apache.kafka.connect.sink.SinkRecord; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Future; + +public class UpsertDeleteBigQueryWriter extends AdaptiveBigQueryWriter { + + private final SchemaManager schemaManager; + private final boolean autoCreateTables; + private final Map intermediateToDestinationTables; + + /** + * @param bigQuery Used to send write requests to BigQuery. + * @param schemaManager Used to update BigQuery tables. + * @param retry How many retries to make in the event of a 500/503 error. + * @param retryWait How long to wait in between retries. + * @param autoCreateTables Whether destination tables should be automatically created + * @param intermediateToDestinationTables A mapping used to determine the destination table for + * given intermediate tables; used for create/update + * operations in order to propagate them to the destination + * table + */ + public UpsertDeleteBigQueryWriter(BigQuery bigQuery, + SchemaManager schemaManager, + int retry, + long retryWait, + boolean autoCreateTables, + Map intermediateToDestinationTables) { + // Hardcode autoCreateTables to true in the superclass so that intermediate tables will be + // automatically created + // The super class will handle all of the logic for writing to, creating, and updating + // intermediate tables; this class will handle logic for creating/updating the destination table + super(bigQuery, schemaManager.forIntermediateTables(), retry, retryWait, true); + this.schemaManager = schemaManager; + this.autoCreateTables = autoCreateTables; + this.intermediateToDestinationTables = intermediateToDestinationTables; + } + + @Override + protected void attemptSchemaUpdate(PartitionedTableId tableId, Set records) { + // Update the intermediate table here... + super.attemptSchemaUpdate(tableId, records); + try { + // ... and update the destination table here + schemaManager.updateSchema(intermediateToDestinationTables.get(tableId.getBaseTableId()), records); + } catch (BigQueryException exception) { + throw new BigQueryConnectException( + "Failed to update destination table schema for: " + tableId.getBaseTableId(), exception); + } + } + + @Override + protected void attemptTableCreate(TableId tableId, Set records) { + // Create the intermediate table here... + super.attemptTableCreate(tableId, records); + if (autoCreateTables) { + try { + // ... and create or update the destination table here, if it doesn't already exist and auto + // table creation is enabled + schemaManager.createOrUpdateTable(intermediateToDestinationTables.get(tableId), records); + } catch (BigQueryException exception) { + throw new BigQueryConnectException( + "Failed to create table " + tableId, exception); + } + } + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java index 1da4bafe1..026e989ba 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java @@ -38,6 +38,7 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import org.apache.kafka.common.config.ConfigException; @@ -102,6 +103,7 @@ public void testTaskConfigs() { List> taskConfigs = testConnector.taskConfigs(i); assertEquals(i, taskConfigs.size()); for (int j = 0; j < i; j++) { + expectedProperties.put(BigQuerySinkTaskConfig.TASK_ID_CONFIG, Integer.toString(j)); assertEquals( "Connector properties should match task configs", expectedProperties, @@ -127,7 +129,7 @@ public void testTaskConfigs() { @Test public void testConfig() { - assertEquals(BigQuerySinkConfig.getConfig(), new BigQuerySinkConnector().config()); + assertNotNull(new BigQuerySinkConnector().config()); } // Make sure that a config exception is properly translated into a SinkConfigConnectException diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 358a3c8ab..b24e7f40a 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -314,7 +314,7 @@ public void testPutWhenPartitioningOnMessageTimeWhenNoTimestampType() { TimestampType.NO_TIMESTAMP_TYPE, null))); } - // It's important that the buffer be completely wiped after a call to flush, since any execption + // It's important that the buffer be completely wiped after a call to flush, since any exception // thrown during flush causes Kafka Connect to not commit the offsets for any records sent to the // task since the last flush @Test diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java index 8e7b139af..aec3ed471 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java @@ -29,6 +29,7 @@ public Map getProperties() { Map properties = super.getProperties(); properties.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "false"); + properties.put(BigQuerySinkTaskConfig.TASK_ID_CONFIG, "4"); return properties; } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index 6e916fc0a..e76fc192d 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -18,9 +18,7 @@ */ -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import com.wepay.kafka.connect.bigquery.SinkPropertiesFactory; @@ -31,7 +29,6 @@ import org.junit.Before; import org.junit.Test; -import java.util.HashMap; import java.util.Map; public class BigQuerySinkConfigTest { From c8509b50ba145fcdbe31e75297504afec68c2e81 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 16 Jun 2020 13:23:11 -0700 Subject: [PATCH 008/190] GH-264: Update existing unit tests --- .../bigquery/config/BigQuerySinkConfig.java | 2 -- .../bigquery/config/BigQuerySinkTaskConfig.java | 16 ++++++++-------- .../connect/bigquery/SinkPropertiesFactory.java | 1 + .../bigquery/write/row/BigQueryWriterTest.java | 1 + .../bigquery/write/row/GCSToBQWriterTest.java | 1 + 5 files changed, 11 insertions(+), 10 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 8cfc9c272..949305965 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -51,8 +51,6 @@ * Base class for connector and task configs; contains properties shared between the two of them. */ public class BigQuerySinkConfig extends AbstractConfig { - private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkConfig.class); - // Values taken from https://github.com/apache/kafka/blob/1.1.1/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java#L33 public static final String TOPICS_CONFIG = SinkConnector.TOPICS_CONFIG; private static final ConfigDef.Type TOPICS_TYPE = ConfigDef.Type.LIST; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java index 1ba973dc7..e9e35bc3b 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java @@ -32,7 +32,6 @@ * Class for task-specific configuration properties. */ public class BigQuerySinkTaskConfig extends BigQuerySinkConfig { - private static final ConfigDef config; private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkTaskConfig.class); public static final String THREAD_POOL_SIZE_CONFIG = "threadPoolSize"; @@ -125,8 +124,13 @@ public class BigQuerySinkTaskConfig extends BigQuerySinkConfig { public static final ConfigDef.Importance TASK_ID_IMPORTANCE = ConfigDef.Importance.LOW; private static final String TASK_ID_DOC = "A unique for each task created by the connector"; - static { - config = BigQuerySinkConfig.getConfig() + /** + * Return a ConfigDef object used to define this config's fields. + * + * @return A ConfigDef object used to define this config's fields. + */ + public static ConfigDef getConfig() { + return BigQuerySinkConfig.getConfig() .define( THREAD_POOL_SIZE_CONFIG, THREAD_POOL_SIZE_TYPE, @@ -251,15 +255,11 @@ private void checkClusteringConfigs() { } } - public static ConfigDef getConfig() { - return config; - } - /** * @param properties A Map detailing configuration properties and their respective values. */ public BigQuerySinkTaskConfig(Map properties) { - super(config, properties); + super(getConfig(), properties); checkSchemaUpdates(); checkPartitionConfigs(); checkClusteringConfigs(); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java index 9e20cc73a..9198bb8c2 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java @@ -19,6 +19,7 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import java.util.HashMap; import java.util.Map; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index d1d4d9477..d1ab8b77b 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -298,6 +298,7 @@ private Map makeProperties(String bigqueryRetry, Map properties = propertiesFactory.getProperties(); properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_CONFIG, bigqueryRetry); properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_WAIT_CONFIG, bigqueryRetryWait); + properties.put(BigQuerySinkTaskConfig.TASK_ID_CONFIG, "6"); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, dataset); return properties; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java index 435328b55..bce4ff9bf 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java @@ -163,6 +163,7 @@ private Map makeProperties(String bigqueryRetry, Map properties = propertiesFactory.getProperties(); properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_CONFIG, bigqueryRetry); properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_WAIT_CONFIG, bigqueryRetryWait); + properties.put(BigQuerySinkTaskConfig.TASK_ID_CONFIG, "9"); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, dataset); // gcs config From 5901eb066b1c2594068609e86e24657e1dae1093 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 17 Jun 2020 14:50:00 -0700 Subject: [PATCH 009/190] GH-264: Address some review comments, add unit tests --- .../connect/bigquery/BigQuerySinkTask.java | 7 +- .../kafka/connect/bigquery/MergeQueries.java | 141 +++++---- .../kafka/connect/bigquery/SchemaManager.java | 3 +- .../bigquery/config/BigQuerySinkConfig.java | 104 ++++--- .../bigquery/utils/SinkRecordConverter.java | 7 +- .../bigquery/write/batch/MergeBatches.java | 19 +- .../bigquery/BigQuerySinkTaskTest.java | 157 ++++++++-- .../connect/bigquery/MergeQueriesTest.java | 272 ++++++++++++++++++ 8 files changed, 569 insertions(+), 141 deletions(-) create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 96679b5f3..ca30f382c 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -90,7 +90,6 @@ public class BigQuerySinkTask extends SinkTask { private boolean upsertDelete; private MergeBatches mergeBatches; private MergeQueries mergeQueries; - private long mergeRecordsThreshold; private TopicPartitionManager topicPartitionManager; @@ -218,7 +217,7 @@ public void put(Collection records) { Map tableWriterBuilders = new HashMap<>(); for (SinkRecord record : records) { - if (record.value() != null) { + if (record.value() != null || config.getBoolean(config.DELETE_ENABLED_CONFIG)) { PartitionedTableId table = getRecordTable(record); if (!tableWriterBuilders.containsKey(table)) { TableWriterBuilder tableWriterBuilder; @@ -400,12 +399,10 @@ public void start(Map properties) { Instant.now().toEpochMilli() ); mergeBatches = new MergeBatches(intermediateTableSuffix); - mergeRecordsThreshold = config.getLong(config.MERGE_RECORDS_THRESHOLD_CONFIG); } bigQueryWriter = getBigQueryWriter(); gcsToBQWriter = getGcsWriter(); - recordConverter = getConverter(config); executor = new KCBQThreadPoolExecutor(config, new LinkedBlockingQueue<>()); topicPartitionManager = new TopicPartitionManager(); useMessageTimeDatePartitioning = @@ -421,6 +418,8 @@ public void start(Map properties) { new MergeQueries(config, mergeBatches, executor, getBigQuery(), getSchemaManager(), context); maybeStartMergeFlushTask(); } + + recordConverter = getConverter(config); } private void startGCSToBQLoadTask() { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java index df9272af7..168e3e1b9 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java @@ -24,7 +24,9 @@ import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.bigquery.Schema; import com.google.cloud.bigquery.TableId; +import com.google.common.annotations.VisibleForTesting; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; +import com.wepay.kafka.connect.bigquery.write.batch.KCBQThreadPoolExecutor; import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkTaskContext; @@ -32,7 +34,6 @@ import org.slf4j.LoggerFactory; import java.util.List; -import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -45,28 +46,50 @@ public class MergeQueries { private static final Logger logger = LoggerFactory.getLogger(MergeQueries.class); private final String keyFieldName; - private final boolean usePartitionDecorator; + private final boolean insertPartitionTime; private final boolean upsertEnabled; private final boolean deleteEnabled; private final MergeBatches mergeBatches; - private final ExecutorService executor; + private final KCBQThreadPoolExecutor executor; private final BigQuery bigQuery; private final SchemaManager schemaManager; private final SinkTaskContext context; public MergeQueries(BigQuerySinkTaskConfig config, MergeBatches mergeBatches, - ExecutorService executor, + KCBQThreadPoolExecutor executor, BigQuery bigQuery, SchemaManager schemaManager, SinkTaskContext context) { - this.keyFieldName = config.getKafkaKeyFieldName().orElseThrow(() -> - new ConnectException("Kafka key field must be configured when upsert/delete is enabled") + this( + config.getKafkaKeyFieldName().orElseThrow(() -> + new ConnectException("Kafka key field must be configured when upsert/delete is enabled") + ), + config.getBoolean(config.BIGQUERY_PARTITION_DECORATOR_CONFIG), + config.getBoolean(config.UPSERT_ENABLED_CONFIG), + config.getBoolean(config.DELETE_ENABLED_CONFIG), + mergeBatches, + executor, + bigQuery, + schemaManager, + context ); - this.usePartitionDecorator = config.getBoolean(config.BIGQUERY_PARTITION_DECORATOR_CONFIG); - this.upsertEnabled = config.getBoolean(config.UPSERT_ENABLED_CONFIG); - this.deleteEnabled = config.getBoolean(config.DELETE_ENABLED_CONFIG); + } + @VisibleForTesting + MergeQueries(String keyFieldName, + boolean insertPartitionTime, + boolean upsertEnabled, + boolean deleteEnabled, + MergeBatches mergeBatches, + KCBQThreadPoolExecutor executor, + BigQuery bigQuery, + SchemaManager schemaManager, + SinkTaskContext context) { + this.keyFieldName = keyFieldName; + this.insertPartitionTime = insertPartitionTime; + this.upsertEnabled = upsertEnabled; + this.deleteEnabled = deleteEnabled; this.mergeBatches = mergeBatches; this.executor = executor; this.bigQuery = bigQuery; @@ -85,56 +108,50 @@ public void mergeFlush(TableId intermediateTable) { logger.trace("Triggering merge flush from intermediate table {} to destination table {} for batch {}", intermediateTable, destinationTable, batchNumber); - executor.submit(() -> { - // If there are rows to flush in this batch, flush them - if (mergeBatches.prepareToFlush(intermediateTable, batchNumber)) { - try { - logger.debug("Running merge query on batch {} from intermediate table {}", - batchNumber, intermediateTable); - String mergeFlushQuery = mergeFlushQuery(intermediateTable, destinationTable, batchNumber); - logger.trace(mergeFlushQuery); - bigQuery.query(QueryJobConfiguration.of(mergeFlushQuery)); - logger.trace("Merge from intermediate table {} to destination table {} completed", - intermediateTable, destinationTable); - } catch (Throwable t) { - logger.warn("Failed on merge flush from intermediate table {} to destination table {}", - intermediateTable, destinationTable, t); - throw new ConnectException( - String.format("Failed to perform merge flush from intermediate table %s to destination table %s", - intermediateTable, - destinationTable), - t); - } - - logger.debug("Recording flush success for batch {} from {}", - batchNumber, intermediateTable); - mergeBatches.recordSuccessfulFlush(intermediateTable, batchNumber); - - // Commit those offsets ASAP - context.requestCommit(); - - logger.info("Completed merge flush of batch {} from {} to {}", - batchNumber, intermediateTable, destinationTable); - } - - // After, regardless of whether we flushed or not, clean up old batches from the intermediate - // table. Some rows may be several batches old but still in the table if they were in the - // streaming buffer during the last purge. + executor.execute(() -> { try { - logger.trace("Clearing batches from {} on back from intermediate table {}", batchNumber, intermediateTable); - String tableClearQuery = clearBatchQuery(intermediateTable, batchNumber); - logger.trace(tableClearQuery); - bigQuery.query(QueryJobConfiguration.of(tableClearQuery)); - } catch (Throwable t) { - logger.error("Failed to clear old batches from intermediate table {}", intermediateTable, t); - throw new ConnectException( - String.format("Failed to clear old batches from intermediate table %s", - intermediateTable), - t); + mergeFlush(intermediateTable, destinationTable, batchNumber); + } catch (InterruptedException e) { + throw new ConnectException(String.format( + "Interrupted while performing merge flush of batch %d from %s to %s", + batchNumber, intermediateTable, destinationTable)); } }); } + private void mergeFlush( + TableId intermediateTable, TableId destinationTable, int batchNumber + ) throws InterruptedException{ + // If there are rows to flush in this batch, flush them + if (mergeBatches.prepareToFlush(intermediateTable, batchNumber)) { + logger.debug("Running merge query on batch {} from intermediate table {}", + batchNumber, intermediateTable); + String mergeFlushQuery = mergeFlushQuery(intermediateTable, destinationTable, batchNumber); + logger.trace(mergeFlushQuery); + bigQuery.query(QueryJobConfiguration.of(mergeFlushQuery)); + logger.trace("Merge from intermediate table {} to destination table {} completed", + intermediateTable, destinationTable); + + logger.debug("Recording flush success for batch {} from {}", + batchNumber, intermediateTable); + mergeBatches.recordSuccessfulFlush(intermediateTable, batchNumber); + + // Commit those offsets ASAP + context.requestCommit(); + + logger.info("Completed merge flush of batch {} from {} to {}", + batchNumber, intermediateTable, destinationTable); + } + + // After, regardless of whether we flushed or not, clean up old batches from the intermediate + // table. Some rows may be several batches old but still in the table if they were in the + // streaming buffer during the last purge. + logger.trace("Clearing batches from {} on back from intermediate table {}", batchNumber, intermediateTable); + String batchClearQuery = batchClearQuery(intermediateTable, batchNumber); + logger.trace(batchClearQuery); + bigQuery.query(QueryJobConfiguration.of(batchClearQuery)); + } + /* upsert+delete: @@ -214,13 +231,14 @@ THEN INSERT ( keyFields = listFields( - intermediateSchema.getFields().get(keyFieldName).getSubFields(), + intermediateSchema.getFields().get(srcKey).getSubFields(), srcKey + "." ); List dstValueFields = intermediateSchema.getFields().get(INTERMEDIATE_TABLE_VALUE_FIELD_NAME).getSubFields() @@ -235,8 +253,8 @@ private String mergeFlushQuery(TableId intermediateTable, TableId destinationTab .map(field -> field + "=`src`." + INTERMEDIATE_TABLE_VALUE_FIELD_NAME + "." + field) .collect(Collectors.toList()); - String partitionTimeField = usePartitionDecorator ? "_PARTITIONTIME, " : ""; - String partitionTimeValue = usePartitionDecorator + String partitionTimeField = insertPartitionTime ? "_PARTITIONTIME, " : ""; + String partitionTimeValue = insertPartitionTime ? "CAST(CAST(DATE(`src`." + INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME + ") AS DATE) AS TIMESTAMP), " : ""; @@ -299,9 +317,9 @@ private String mergeFlushQuery(TableId intermediateTable, TableId destinationTab // Assume all rows have non-null values and upsert them all return mergeOpening .append("ON ").append(keysMatch).append(" ") - .append("WHEN MATCHED") + .append("WHEN MATCHED ") .append(updateClause).append(" ") - .append("WHEN NOT MATCHED") + .append("WHEN NOT MATCHED ") .append(insertClause) .append(";") .toString(); @@ -311,7 +329,8 @@ private String mergeFlushQuery(TableId intermediateTable, TableId destinationTab } // DELETE FROM `` WHERE batchNumber <= AND _PARTITIONTIME IS NOT NULL; - private static String clearBatchQuery(TableId intermediateTable, int batchNumber) { + @VisibleForTesting + static String batchClearQuery(TableId intermediateTable, int batchNumber) { return new StringBuilder("DELETE FROM `").append(intermediateTable.getDataset()).append("`.`").append(intermediateTable.getTable()).append("` ") .append("WHERE ") .append(INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD).append(" <= ").append(batchNumber).append(" ") diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index b0968759b..fb49450aa 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -410,7 +410,8 @@ private List getIntermediateSchemaFields(com.google.cloud.bigquery.Schema com.google.cloud.bigquery.Schema keySchema = schemaConverter.convertSchema(kafkaKeySchema); Field kafkaKeyField = Field.newBuilder(MergeQueries.INTERMEDIATE_TABLE_KEY_FIELD_NAME, LegacySQLTypeName.RECORD, keySchema.getFields()) - .setMode(Field.Mode.REQUIRED).build(); + .setMode(Field.Mode.REQUIRED) + .build(); result.add(kafkaKeyField); Field partitionTimeField = Field diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 949305965..0a2dc8775 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -45,6 +45,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import java.util.Optional; /** @@ -473,62 +476,71 @@ public static ConfigDef getConfig() { * @param props sink configuration properties */ public static void validate(Map props) { - final boolean hasTopicsConfig = hasTopicsConfig(props); - final boolean hasTopicsRegexConfig = hasTopicsRegexConfig(props); + final boolean hasTopicsConfig = hasTopicsConfig(props); + final boolean hasTopicsRegexConfig = hasTopicsRegexConfig(props); - if (hasTopicsConfig && hasTopicsRegexConfig) { - throw new ConfigException(TOPICS_CONFIG + " and " + TOPICS_REGEX_CONFIG + - " are mutually exclusive options, but both are set."); + if (hasTopicsConfig && hasTopicsRegexConfig) { + throw new ConfigException(TOPICS_CONFIG + " and " + TOPICS_REGEX_CONFIG + + " are mutually exclusive options, but both are set."); + } + + if (!hasTopicsConfig && !hasTopicsRegexConfig) { + throw new ConfigException("Must configure one of " + + TOPICS_CONFIG + " or " + TOPICS_REGEX_CONFIG); + } + + if (upsertDeleteEnabled(props)) { + if (gcsBatchLoadingEnabled(props)) { + throw new ConfigException("Cannot enable both upsert/delete and GCS batch loading"); } - if (!hasTopicsConfig && !hasTopicsRegexConfig) { - throw new ConfigException("Must configure one of " + - TOPICS_CONFIG + " or " + TOPICS_REGEX_CONFIG); + String mergeIntervalStr = Optional.ofNullable(props.get(MERGE_INTERVAL_MS_CONFIG)) + .map(String::trim) + .orElse(Long.toString(MERGE_INTERVAL_MS_DEFAULT)); + String mergeRecordsThresholdStr = Optional.ofNullable(props.get(MERGE_RECORDS_THRESHOLD_CONFIG)) + .map(String::trim) + .orElse(Long.toString(MERGE_RECORDS_THRESHOLD_DEFAULT)); + if ("-1".equals(mergeIntervalStr) && "-1".equals(mergeRecordsThresholdStr)) { + throw new ConfigException(MERGE_INTERVAL_MS_CONFIG + " and " + + MERGE_RECORDS_THRESHOLD_CONFIG + " cannot both be -1"); } - if (upsertDeleteEnabled(props)) { - String mergeIntervalStr = Optional.ofNullable(props.get(MERGE_INTERVAL_MS_CONFIG)) - .map(String::trim) - .orElse(Long.toString(MERGE_INTERVAL_MS_DEFAULT)); - String mergeRecordsThresholdStr = Optional.ofNullable(props.get(MERGE_RECORDS_THRESHOLD_CONFIG)) - .map(String::trim) - .orElse(Long.toString(MERGE_RECORDS_THRESHOLD_DEFAULT)); - if ("-1".equals(mergeIntervalStr) && "-1".equals(mergeRecordsThresholdStr)) { - throw new ConfigException(MERGE_INTERVAL_MS_CONFIG + " and " - + MERGE_RECORDS_THRESHOLD_CONFIG + " cannot both be -1"); - } - - if ("0".equals(mergeIntervalStr)) { - throw new ConfigException(MERGE_INTERVAL_MS_CONFIG, mergeIntervalStr, "cannot be zero"); - } - if ("0".equals(mergeRecordsThresholdStr)) { - throw new ConfigException(MERGE_RECORDS_THRESHOLD_CONFIG, mergeRecordsThresholdStr, "cannot be zero"); - } - - String kafkaKeyFieldStr = props.get(KAFKA_KEY_FIELD_NAME_CONFIG); - if (kafkaKeyFieldStr == null || kafkaKeyFieldStr.trim().isEmpty()) { - throw new ConfigException(KAFKA_KEY_FIELD_NAME_CONFIG + " must be specified when " - + UPSERT_ENABLED_CONFIG + " and/or " + DELETE_ENABLED_CONFIG + " are set to true"); - } + if ("0".equals(mergeIntervalStr)) { + throw new ConfigException(MERGE_INTERVAL_MS_CONFIG, mergeIntervalStr, "cannot be zero"); + } + if ("0".equals(mergeRecordsThresholdStr)) { + throw new ConfigException(MERGE_RECORDS_THRESHOLD_CONFIG, mergeRecordsThresholdStr, "cannot be zero"); } - } - public static boolean hasTopicsConfig(Map props) { - String topicsStr = props.get(TOPICS_CONFIG); - return topicsStr != null && !topicsStr.trim().isEmpty(); + String kafkaKeyFieldStr = props.get(KAFKA_KEY_FIELD_NAME_CONFIG); + if (kafkaKeyFieldStr == null || kafkaKeyFieldStr.trim().isEmpty()) { + throw new ConfigException(KAFKA_KEY_FIELD_NAME_CONFIG + " must be specified when " + + UPSERT_ENABLED_CONFIG + " and/or " + DELETE_ENABLED_CONFIG + " are set to true"); + } } + } - public static boolean hasTopicsRegexConfig(Map props) { - String topicsRegexStr = props.get(TOPICS_REGEX_CONFIG); - return topicsRegexStr != null && !topicsRegexStr.trim().isEmpty(); - } + public static boolean hasTopicsConfig(Map props) { + String topicsStr = props.get(TOPICS_CONFIG); + return topicsStr != null && !topicsStr.trim().isEmpty(); + } - public static boolean upsertDeleteEnabled(Map props) { - String upsertStr = props.get(UPSERT_ENABLED_CONFIG); - String deleteStr = props.get(DELETE_ENABLED_CONFIG); - return Boolean.TRUE.toString().equalsIgnoreCase(upsertStr) - || Boolean.TRUE.toString().equalsIgnoreCase(deleteStr); - } + public static boolean hasTopicsRegexConfig(Map props) { + String topicsRegexStr = props.get(TOPICS_REGEX_CONFIG); + return topicsRegexStr != null && !topicsRegexStr.trim().isEmpty(); + } + + public static boolean upsertDeleteEnabled(Map props) { + String upsertStr = props.get(UPSERT_ENABLED_CONFIG); + String deleteStr = props.get(DELETE_ENABLED_CONFIG); + return Boolean.TRUE.toString().equalsIgnoreCase(upsertStr) + || Boolean.TRUE.toString().equalsIgnoreCase(deleteStr); + } + + public static boolean gcsBatchLoadingEnabled(Map props) { + String batchLoadStr = props.get(ENABLE_BATCH_CONFIG); + return batchLoadStr != null && !batchLoadStr.isEmpty(); + } /** * Returns the keyfile diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java index 4af836142..0cd9ac389 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java @@ -84,8 +84,6 @@ private Map getUpsertDeleteRow(SinkRecord record, TableId table) ? null : recordConverter.convertRecord(record, KafkaSchemaRecordType.VALUE); - Map convertedKey = recordConverter.convertRecord(record, KafkaSchemaRecordType.KEY); - if (convertedValue != null) { config.getKafkaDataFieldName().ifPresent( fieldName -> convertedValue.put(fieldName, KafkaDataBuilder.buildKafkaDataRecord(record)) @@ -101,6 +99,11 @@ private Map getUpsertDeleteRow(SinkRecord record, TableId table) mergeQueries.mergeFlush(table); } + Map convertedKey = recordConverter.convertRecord(record, KafkaSchemaRecordType.KEY); + if (convertedKey == null) { + throw new ConnectException("Record keys must be non-null when upsert/delete is enabled"); + } + result.put(MergeQueries.INTERMEDIATE_TABLE_KEY_FIELD_NAME, convertedKey); result.put(MergeQueries.INTERMEDIATE_TABLE_VALUE_FIELD_NAME, convertedValue); if (usePartitionDecorator && useMessageTimeDatePartitioning) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java index 92558a251..28eabc3a3 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java @@ -20,6 +20,7 @@ import com.google.cloud.bigquery.InsertAllRequest; import com.google.cloud.bigquery.TableId; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.Maps; @@ -45,12 +46,24 @@ public class MergeBatches { private static final Logger logger = LoggerFactory.getLogger(MergeBatches.class); private static final long STREAMING_BUFFER_AVAILABILITY_WAIT_MS = 10_000L; + private static long streamingBufferAvailabilityWaitMs = STREAMING_BUFFER_AVAILABILITY_WAIT_MS; + private final String intermediateTableSuffix; private final BiMap intermediateToDestinationTables; private final ConcurrentMap batchNumbers; private final ConcurrentMap> batches; private final Map offsets; + @VisibleForTesting + public static void setStreamingBufferAvailabilityWait(long waitMs) { + streamingBufferAvailabilityWaitMs = waitMs; + } + + @VisibleForTesting + public static void resetStreamingBufferAvailabilityWait() { + streamingBufferAvailabilityWaitMs = STREAMING_BUFFER_AVAILABILITY_WAIT_MS; + } + public MergeBatches(String intermediateTableSuffix) { this.intermediateTableSuffix = intermediateTableSuffix; @@ -243,10 +256,10 @@ public boolean prepareToFlush(TableId intermediateTable, int batchNumber) { try { logger.trace( - "Waiting {} seconds before running merge query on batch {} from intermediate table {} " + "Waiting {}ms before running merge query on batch {} from intermediate table {} " + "in order to ensure that all rows are available in the streaming buffer", - STREAMING_BUFFER_AVAILABILITY_WAIT_MS, batchNumber, intermediateTable); - Thread.sleep(STREAMING_BUFFER_AVAILABILITY_WAIT_MS); + streamingBufferAvailabilityWaitMs, batchNumber, intermediateTable); + Thread.sleep(streamingBufferAvailabilityWaitMs); } catch (InterruptedException e) { logger.warn("Interrupted while waiting before merge flushing batch {} for intermediate table {}", batchNumber, intermediateTable); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index b24e7f40a..51646dcc3 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyObject; @@ -31,10 +32,12 @@ import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryError; import com.google.cloud.bigquery.BigQueryException; +import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.InsertAllRequest; import com.google.cloud.bigquery.InsertAllResponse; import com.google.cloud.bigquery.Table; -import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.LegacySQLTypeName; +import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.storage.Storage; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; @@ -42,6 +45,7 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; +import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; @@ -52,24 +56,42 @@ import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.After; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; -import org.mockito.Captor; +import java.util.Arrays; import java.util.Collections; import java.util.Map; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; public class BigQuerySinkTaskTest { private static SinkTaskPropertiesFactory propertiesFactory; + private static AtomicLong spoofedRecordOffset = new AtomicLong(); + @BeforeClass public static void initializePropertiesFactory() { propertiesFactory = new SinkTaskPropertiesFactory(); } + @Before + public void setUp() { + MergeBatches.setStreamingBufferAvailabilityWait(0); + spoofedRecordOffset.set(0); + } + + @After + public void cleanUp() { + MergeBatches.resetStreamingBufferAvailabilityWait(); + } + @Test public void testSimplePut() { final String topic = "test-topic"; @@ -172,8 +194,6 @@ public void testEmptyRecordPut() { testTask.put(Collections.singletonList(emptyRecord)); } - @Captor ArgumentCaptor captor; - @Test public void testPutWhenPartitioningOnMessageTime() { final String topic = "test-topic"; @@ -314,6 +334,76 @@ public void testPutWhenPartitioningOnMessageTimeWhenNoTimestampType() { TimestampType.NO_TIMESTAMP_TYPE, null))); } + @Test + public void testPutWithUpsertDelete() throws Exception { + final String topic = "test-topic"; + final String key = "kafkaKey"; + final String value = "recordValue"; + + Map properties = propertiesFactory.getProperties(); + properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); + properties.put(BigQuerySinkConfig.UPSERT_ENABLED_CONFIG, "true"); + properties.put(BigQuerySinkConfig.DELETE_ENABLED_CONFIG, "true"); + properties.put(BigQuerySinkConfig.MERGE_INTERVAL_MS_CONFIG, "-1"); + properties.put(BigQuerySinkConfig.MERGE_RECORDS_THRESHOLD_CONFIG, "2"); + properties.put(BigQuerySinkConfig.KAFKA_KEY_FIELD_NAME_CONFIG, key); + + BigQuery bigQuery = mock(BigQuery.class); + Storage storage = mock(Storage.class); + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + + InsertAllResponse insertAllResponse = mock(InsertAllResponse.class); + when(bigQuery.insertAll(anyObject())).thenReturn(insertAllResponse); + when(insertAllResponse.hasErrors()).thenReturn(false); + + SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); + SchemaManager schemaManager = mock(SchemaManager.class); + Field keyField = Field.of(key, LegacySQLTypeName.STRING); + Field valueField = Field.of(value, LegacySQLTypeName.STRING); + com.google.cloud.bigquery.Schema intermediateSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder(MergeQueries.INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD, LegacySQLTypeName.INTEGER) + .setMode(Field.Mode.REQUIRED) + .build(), + Field.newBuilder(MergeQueries.INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME, LegacySQLTypeName.TIMESTAMP) + .setMode(Field.Mode.NULLABLE) + .build(), + Field.newBuilder(MergeQueries.INTERMEDIATE_TABLE_KEY_FIELD_NAME, LegacySQLTypeName.RECORD, keyField) + .setMode(Field.Mode.REQUIRED) + .build(), + Field.newBuilder(MergeQueries.INTERMEDIATE_TABLE_VALUE_FIELD_NAME, LegacySQLTypeName.RECORD, valueField) + .build() + ); + when(schemaManager.cachedSchema(any())).thenReturn(intermediateSchema); + + CountDownLatch executedMerges = new CountDownLatch(2); + CountDownLatch executedBatchClears = new CountDownLatch(2); + + when(bigQuery.query(any(QueryJobConfiguration.class))).then(invocation -> { + String query = invocation.getArgument(0, QueryJobConfiguration.class).getQuery(); + if (query.startsWith("MERGE")) { + executedMerges.countDown(); + } else if (query.startsWith("DELETE")) { + executedBatchClears.countDown(); + } + return null; + }); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + testTask.initialize(sinkTaskContext); + testTask.start(properties); + + // Insert a few regular records and one tombstone record + testTask.put(Arrays.asList( + spoofSinkRecord(topic, key, "4761", "value", "message text", TimestampType.NO_TIMESTAMP_TYPE, null), + spoofSinkRecord(topic, key, "489", "value", "other message text", TimestampType.NO_TIMESTAMP_TYPE, null), + spoofSinkRecord(topic, key, "28980", "value", "more message text", TimestampType.NO_TIMESTAMP_TYPE, null), + spoofSinkRecord(topic, key, "4761", null, null, TimestampType.NO_TIMESTAMP_TYPE, null) + )); + + assertTrue("Merge queries should be executed", executedMerges.await(5, TimeUnit.SECONDS)); + assertTrue("Batch clears should be executed", executedBatchClears.await(1, TimeUnit.SECONDS)); + } + // It's important that the buffer be completely wiped after a call to flush, since any exception // thrown during flush causes Kafka Connect to not commit the offsets for any records sent to the // task since the last flush @@ -582,38 +672,57 @@ public void testStop() { } /** - * Utility method for spoofing InsertAllRequests that should be sent to a BigQuery object. - * @param table The table to write to. - * @param rows The rows to write. - * @return The spoofed InsertAllRequest. + * Utility method for spoofing SinkRecords that should be passed to SinkTask.put() + * @param topic The topic of the record. + * @param keyField The field name for the record key; may be null. + * @param key The content of the record key; may be null. + * @param valueField The field name for the record value; may be null + * @param value The content of the record value; may be null + * @param timestampType The type of timestamp embedded in the message + * @param timestamp The timestamp in milliseconds + * @return The spoofed SinkRecord. */ - public static InsertAllRequest buildExpectedInsertAllRequest( - TableId table, - InsertAllRequest.RowToInsert... rows) { - return InsertAllRequest.newBuilder(table, rows) - .setIgnoreUnknownValues(false) - .setSkipInvalidRows(false) - .build(); + public static SinkRecord spoofSinkRecord(String topic, String keyField, String key, + String valueField, String value, + TimestampType timestampType, Long timestamp) { + Schema basicKeySchema = null; + Struct basicKey = null; + if (keyField != null) { + basicKeySchema = SchemaBuilder + .struct() + .field(keyField, Schema.STRING_SCHEMA) + .build(); + basicKey = new Struct(basicKeySchema); + basicKey.put(keyField, key); + } + + Schema basicValueSchema = null; + Struct basicValue = null; + if (valueField != null) { + basicValueSchema = SchemaBuilder + .struct() + .field(valueField, Schema.STRING_SCHEMA) + .build(); + basicValue = new Struct(basicValueSchema); + basicValue.put(valueField, value); + } + + return new SinkRecord(topic, 0, basicKeySchema, basicKey, + basicValueSchema, basicValue, spoofedRecordOffset.getAndIncrement(), timestamp, timestampType); } /** * Utility method for spoofing SinkRecords that should be passed to SinkTask.put() * @param topic The topic of the record. - * @param value The content of the record. + * @param field The field name for the record value. + * @param value The content of the record value. * @param timestampType The type of timestamp embedded in the message * @param timestamp The timestamp in milliseconds * @return The spoofed SinkRecord. */ public static SinkRecord spoofSinkRecord(String topic, String field, String value, TimestampType timestampType, Long timestamp) { - Schema basicRowSchema = SchemaBuilder - .struct() - .field(field, Schema.STRING_SCHEMA) - .build(); - Struct basicRowValue = new Struct(basicRowSchema); - basicRowValue.put(field, value); - return new SinkRecord(topic, 0, null, null, - basicRowSchema, basicRowValue, 0, timestamp, timestampType); + return spoofSinkRecord(topic, null, null, field, value, timestampType, timestamp); } /** diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java new file mode 100644 index 000000000..9675df8c2 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java @@ -0,0 +1,272 @@ +package com.wepay.kafka.connect.bigquery; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.LegacySQLTypeName; +import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.TableId; +import com.wepay.kafka.connect.bigquery.write.batch.KCBQThreadPoolExecutor; +import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class MergeQueriesTest { + + private static final String KEY = "kafkaKey"; + + private static final int BATCH_NUMBER = 42; + private static final TableId DESTINATION_TABLE = TableId.of("ds1", "t"); + private static final TableId INTERMEDIATE_TABLE = TableId.of("ds1", "t_tmp_6_uuid_epoch"); + private static final Schema INTERMEDIATE_TABLE_SCHEMA = constructIntermediateTable(); + + @Mock private MergeBatches mergeBatches; + @Mock private KCBQThreadPoolExecutor executor; + @Mock private BigQuery bigQuery; + @Mock private SchemaManager schemaManager; + @Mock private SinkTaskContext context; + + @Before + public void setUp() { + when(schemaManager.cachedSchema(INTERMEDIATE_TABLE)).thenReturn(INTERMEDIATE_TABLE_SCHEMA); + } + + private MergeQueries mergeQueries(boolean insertPartitionTime, boolean upsert, boolean delete) { + return new MergeQueries( + KEY, insertPartitionTime, upsert, delete, mergeBatches, executor, bigQuery, schemaManager, context + ); + } + + private static Schema constructIntermediateTable() { + List fields = new ArrayList<>(); + + List valueFields = Arrays.asList( + Field.of("f1", LegacySQLTypeName.STRING), + Field.of("f2", LegacySQLTypeName.RECORD, + Field.of("nested_f1", LegacySQLTypeName.INTEGER) + ), + Field.of("f3", LegacySQLTypeName.BOOLEAN), + Field.of("f4", LegacySQLTypeName.BYTES) + ); + Field wrappedValueField = Field + .newBuilder(MergeQueries.INTERMEDIATE_TABLE_VALUE_FIELD_NAME, LegacySQLTypeName.RECORD, valueFields.toArray(new Field[0])) + .setMode(Field.Mode.NULLABLE) + .build(); + fields.add(wrappedValueField); + + List keyFields = Arrays.asList( + Field.of("k1", LegacySQLTypeName.STRING), + Field.of("k2", LegacySQLTypeName.RECORD, + Field.of("nested_k1", LegacySQLTypeName.RECORD, + Field.of("doubly_nested_k", LegacySQLTypeName.BOOLEAN) + ), + Field.of("nested_k2", LegacySQLTypeName.INTEGER) + ) + ); + Field kafkaKeyField = Field.newBuilder(MergeQueries.INTERMEDIATE_TABLE_KEY_FIELD_NAME, LegacySQLTypeName.RECORD, keyFields.toArray(new Field[0])) + .setMode(Field.Mode.REQUIRED) + .build(); + fields.add(kafkaKeyField); + + Field partitionTimeField = Field + .newBuilder(MergeQueries.INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME, LegacySQLTypeName.TIMESTAMP) + .setMode(Field.Mode.NULLABLE) + .build(); + fields.add(partitionTimeField); + + Field batchNumberField = Field + .newBuilder(MergeQueries.INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD, LegacySQLTypeName.INTEGER) + .setMode(Field.Mode.REQUIRED) + .build(); + fields.add(batchNumberField); + + return Schema.of(fields); + } + + @Test + public void testUpsertQueryWithPartitionTime() { + String expectedQuery = + "MERGE " + table(DESTINATION_TABLE) + " " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "FROM " + table(INTERMEDIATE_TABLE) + " x " + + "WHERE batchNumber=" + BATCH_NUMBER + " " + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key " + + "WHEN MATCHED " + + "THEN UPDATE SET f1=`src`.value.f1, f2=`src`.value.f2, f3=`src`.value.f3, f4=`src`.value.f4 " + + "WHEN NOT MATCHED " + + "THEN INSERT (" + + KEY + ", " + + "_PARTITIONTIME, " + + "f1, f2, f3, f4) " + + "VALUES (" + + "`src`.key, " + + "CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), " + + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + ");"; + String actualQuery = mergeQueries(true, true, false) + .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); + System.out.println(actualQuery); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testUpsertQueryWithoutPartitionTime() { + String expectedQuery = + "MERGE " + table(DESTINATION_TABLE) + " " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "FROM " + table(INTERMEDIATE_TABLE) + " x " + + "WHERE batchNumber=" + BATCH_NUMBER + " " + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key " + + "WHEN MATCHED " + + "THEN UPDATE SET f1=`src`.value.f1, f2=`src`.value.f2, f3=`src`.value.f3, f4=`src`.value.f4 " + + "WHEN NOT MATCHED " + + "THEN INSERT (" + + KEY + ", " + + "f1, f2, f3, f4) " + + "VALUES (" + + "`src`.key, " + + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + ");"; + String actualQuery = mergeQueries(false, true, false) + .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); + System.out.println(actualQuery); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testDeleteQueryWithPartitionTime() { + String expectedQuery = + "MERGE " + table(DESTINATION_TABLE) + " " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "FROM " + table(INTERMEDIATE_TABLE) + " x " + + "WHERE batchNumber=" + BATCH_NUMBER + " " + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key AND `src`.value IS NULL " + + "WHEN MATCHED " + + "THEN DELETE " + + "WHEN NOT MATCHED " + + "THEN INSERT (" + + KEY + ", " + + "_PARTITIONTIME, " + + "f1, f2, f3, f4) " + + "VALUES (" + + "`src`.key, " + + "CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), " + + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + ");"; + String actualQuery = mergeQueries(true, false, true) + .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); + System.out.println(actualQuery); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testDeleteQueryWithoutPartitionTime() { + String expectedQuery = + "MERGE " + table(DESTINATION_TABLE) + " " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "FROM " + table(INTERMEDIATE_TABLE) + " x " + + "WHERE batchNumber=" + BATCH_NUMBER + " " + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key AND `src`.value IS NULL " + + "WHEN MATCHED " + + "THEN DELETE " + + "WHEN NOT MATCHED " + + "THEN INSERT (" + + KEY + ", " + + "f1, f2, f3, f4) " + + "VALUES (" + + "`src`.key, " + + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + ");"; + String actualQuery = mergeQueries(false, false, true) + .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); + System.out.println(actualQuery); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testUpsertDeleteQueryWithPartitionTime() { + String expectedQuery = + "MERGE " + table(DESTINATION_TABLE) + " " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "FROM " + table(INTERMEDIATE_TABLE) + " x " + + "WHERE batchNumber=" + BATCH_NUMBER + " " + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key " + + "WHEN MATCHED AND `src`.value IS NOT NULL " + + "THEN UPDATE SET f1=`src`.value.f1, f2=`src`.value.f2, f3=`src`.value.f3, f4=`src`.value.f4 " + + "WHEN MATCHED AND `src`.value IS NULL " + + "THEN DELETE " + + "WHEN NOT MATCHED AND `src`.value IS NOT NULL " + + "THEN INSERT (" + + KEY + ", " + + "_PARTITIONTIME, " + + "f1, f2, f3, f4) " + + "VALUES (" + + "`src`.key, " + + "CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), " + + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + ");"; + String actualQuery = mergeQueries(true, true, true) + .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); + System.out.println(actualQuery); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testUpsertDeleteQueryWithoutPartitionTime() { + String expectedQuery = + "MERGE " + table(DESTINATION_TABLE) + " " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "FROM " + table(INTERMEDIATE_TABLE) + " x " + + "WHERE batchNumber=" + BATCH_NUMBER + " " + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key " + + "WHEN MATCHED AND `src`.value IS NOT NULL " + + "THEN UPDATE SET f1=`src`.value.f1, f2=`src`.value.f2, f3=`src`.value.f3, f4=`src`.value.f4 " + + "WHEN MATCHED AND `src`.value IS NULL " + + "THEN DELETE " + + "WHEN NOT MATCHED AND `src`.value IS NOT NULL " + + "THEN INSERT (" + + KEY + ", " + + "f1, f2, f3, f4) " + + "VALUES (" + + "`src`.key, " + + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + ");"; String actualQuery = mergeQueries(false, true, true) + .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); + System.out.println(actualQuery); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testBatchClearQuery() { + String expectedQuery = + "DELETE FROM " + table(INTERMEDIATE_TABLE) + + " WHERE batchNumber <= " + BATCH_NUMBER + + " AND _PARTITIONTIME IS NOT NULL;"; + // No difference in batch clearing between upsert, delete, and both, or with or without partition time + String actualQuery = mergeQueries(false, false, false) + .batchClearQuery(INTERMEDIATE_TABLE, BATCH_NUMBER); + System.out.println(actualQuery); + assertEquals(expectedQuery, actualQuery); + } + + private String table(TableId table) { + return String.format("`%s`.`%s`", table.getDataset(), table.getTable()); + } +} From fdbde319c54f61709885b6a81a91a0711ba65250 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 17 Jun 2020 15:02:33 -0700 Subject: [PATCH 010/190] GH-264: Minor refactoring --- .../kafka/connect/bigquery/SchemaManager.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index fb49450aa..15df988e1 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -197,8 +197,7 @@ public void createTable(TableId table, Set records) { } catch (BigQueryException e) { if (e.getCode() == 409) { logger.debug("Failed to create {} as it already exists (possibly created by another task)", table(table)); - com.google.cloud.bigquery.Schema schema = bigQuery.getTable(table).getDefinition().getSchema(); - schemaCache.put(table, schema); + schemaCache.put(table, readTableSchema(table)); } } } @@ -210,12 +209,10 @@ public void createTable(TableId table, Set records) { * @param records The sink records used to update the schema. */ public void updateSchema(TableId table, Set records) { - synchronized (tableUpdateLocks.computeIfAbsent(table, t -> new Object())) { + synchronized (lock(tableUpdateLocks, table)) { TableInfo tableInfo = getTableInfo(table, records); - if (!schemaCache.containsKey(table)) { - logger.debug("Reading schema for {}", table(table)); - schemaCache.put(table, bigQuery.getTable(table).getDefinition().getSchema()); + schemaCache.put(table, readTableSchema(table)); } if (!schemaCache.get(table).equals(tableInfo.getDefinition().getSchema())) { @@ -453,6 +450,11 @@ private String table(TableId table) { + table; } + private com.google.cloud.bigquery.Schema readTableSchema(TableId table) { + logger.trace("Reading schema for {}", table(table)); + return bigQuery.getTable(table).getDefinition().getSchema(); + } + private Object lock(ConcurrentMap locks, TableId table) { return locks.computeIfAbsent(table, t -> new Object()); } From 7171d0676bf17a45251cca0e49db3a196cb4c792 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 18 Jun 2020 09:37:50 -0700 Subject: [PATCH 011/190] GH-264: Fix whitespace --- .../wepay/kafka/connect/bigquery/write/batch/MergeBatches.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java index 28eabc3a3..8a95ab317 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java @@ -124,7 +124,8 @@ private TableId newIntermediateTable(TableId destinationTable) { batchNumbers.put(result, new AtomicInteger()); batches.put(result, new ConcurrentHashMap<>()); - return result; } + return result; + } public TableId destinationTableFor(TableId intermediateTable) { return intermediateToDestinationTables.get(intermediateTable); From c37e8c7c4c9222fb224fdd26e5fd8cbc94dcdd38 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Sun, 21 Jun 2020 22:36:06 -0700 Subject: [PATCH 012/190] GH-264: Address code review --- .../kafka/connect/bigquery/SchemaManager.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 15df988e1..78e47f37d 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -165,11 +165,12 @@ public void createOrUpdateTable(TableId table, Set records) { synchronized (lock(tableCreateLocks, table)) { if (bigQuery.getTable(table) == null) { logger.debug("{} doesn't exist; creating instead of updating", table(table)); - createTable(table, records); - return; + if (createTable(table, records)) { + return; + } } } - + // Table already existed; attempt to update instead logger.debug("{} already exists; updating instead of creating", table(table)); updateSchema(table, records); @@ -179,13 +180,14 @@ public void createOrUpdateTable(TableId table, Set records) { * Create a new table in BigQuery. * @param table The BigQuery table to create. * @param records The sink records used to determine the schema. + * @return whether the table had to be created; if the table already existed, will return false */ - public void createTable(TableId table, Set records) { + public boolean createTable(TableId table, Set records) { synchronized (lock(tableCreateLocks, table)) { if (schemaCache.containsKey(table)) { // Table already exists; noop logger.debug("Skipping create of {} as it should already exist or appear very soon", table(table)); - return; + return false; } TableInfo tableInfo = getTableInfo(table, records); logger.info("Attempting to create {} with schema {}", @@ -194,11 +196,14 @@ public void createTable(TableId table, Set records) { bigQuery.create(tableInfo); logger.debug("Successfully created {}", table(table)); schemaCache.put(table, tableInfo.getDefinition().getSchema()); + return true; } catch (BigQueryException e) { if (e.getCode() == 409) { logger.debug("Failed to create {} as it already exists (possibly created by another task)", table(table)); schemaCache.put(table, readTableSchema(table)); + return false; } + throw e; } } } From 69c46fd04506354256c40a956e926cfc3def3577 Mon Sep 17 00:00:00 2001 From: Xin Xiang Date: Mon, 22 Jun 2020 15:18:03 -0700 Subject: [PATCH 013/190] Add pull request template to all conenct repos --- docs/pull_request_template.md | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 docs/pull_request_template.md diff --git a/docs/pull_request_template.md b/docs/pull_request_template.md new file mode 100644 index 000000000..ed95397ba --- /dev/null +++ b/docs/pull_request_template.md @@ -0,0 +1,28 @@ +## Problem + + +## Solution + + + +##### Does this solution apply anywhere else? +- [ ] yes +- [ ] no + +##### If yes, where? + + +## Test Strategy + + + +##### Testing done: +- [ ] Unit tests +- [ ] Integration tests +- [ ] System tests +- [ ] Manual tests + +## Release Plan + + + From d8aae10d2afbbb73cfc69ed420143ee2e50c8a2f Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 23 Jun 2020 12:35:13 -0700 Subject: [PATCH 014/190] GH-264: Apply suggestions from code review Co-authored-by: Nigel Liang --- .../kafka/connect/bigquery/config/BigQuerySinkConfig.java | 4 ++-- .../wepay/kafka/connect/bigquery/SinkPropertiesFactory.java | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 0a2dc8775..53b33e561 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -261,7 +261,7 @@ public class BigQuerySinkConfig extends AbstractConfig { private static final ConfigDef.Type INTERMEDIATE_TABLE_SUFFIX_TYPE = ConfigDef.Type.STRING; public static final String INTERMEDIATE_TABLE_SUFFIX_DEFAULT = "tmp"; private static final ConfigDef.Validator INTERMEDIATE_TABLE_SUFFIX_VALIDATOR = new ConfigDef.NonEmptyString(); - private static final ConfigDef.Importance INTERMEDIATE_TTABLE_SUFFIX_IMPORTANCE = ConfigDef.Importance.LOW; + private static final ConfigDef.Importance INTERMEDIATE_TABLE_SUFFIX_IMPORTANCE = ConfigDef.Importance.LOW; private static final String INTERMEDIATE_TABLE_SUFFIX_DOC = "A suffix that will be appended to the names of destination tables to create the names for " + "the corresponding intermediate tables. Multiple intermediate tables may be created for a " @@ -452,7 +452,7 @@ public static ConfigDef getConfig() { INTERMEDIATE_TABLE_SUFFIX_TYPE, INTERMEDIATE_TABLE_SUFFIX_DEFAULT, INTERMEDIATE_TABLE_SUFFIX_VALIDATOR, - INTERMEDIATE_TTABLE_SUFFIX_IMPORTANCE, + INTERMEDIATE_TABLE_SUFFIX_IMPORTANCE, INTERMEDIATE_TABLE_SUFFIX_DOC ).define( MERGE_INTERVAL_MS_CONFIG, diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java index 9198bb8c2..9e20cc73a 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java @@ -19,7 +19,6 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import java.util.HashMap; import java.util.Map; From 57fb6cf621c1a5666922ad4dfa4180e6c69fa7b8 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 29 Jun 2020 11:39:27 -0700 Subject: [PATCH 015/190] CC-8804: Integration tests for upsert/delete (#16) * GH-264: Add embedded integration test for upsert/delete Also fixes a bug in the schema retriever logic where key schemas were not being reported to schema retrievers, and improves shutdown logic so that tasks can stop gracefully when requested by the framework. * GH-264: Clean up shutdown logic, make logs easier to read * GC-264: Retain prior shutdown behavior when upsert/delete is not enabled * GC-264: Refactor merge query construction logic * GC-264: Fix infinite recursion bug in SchemaRetriever interface --- README.md | 37 +- build.gradle | 28 +- .../connect/bigquery/api/SchemaRetriever.java | 1 - .../connect/bigquery/BigQuerySinkTask.java | 57 ++- .../kafka/connect/bigquery/MergeQueries.java | 440 +++++++++++------- .../kafka/connect/bigquery/SchemaManager.java | 41 +- .../exception/ExpectedInterruptException.java | 28 ++ .../bigquery/utils/SinkRecordConverter.java | 1 + .../bigquery/utils/TableNameUtils.java | 36 ++ .../write/batch/KCBQThreadPoolExecutor.java | 16 +- .../bigquery/write/batch/MergeBatches.java | 65 +-- .../bigquery/write/batch/TableWriter.java | 3 +- .../write/row/AdaptiveBigQueryWriter.java | 5 +- .../bigquery/BigQuerySinkTaskTest.java | 6 +- .../connect/bigquery/MergeQueriesTest.java | 182 ++++---- .../bigquery/integration/BaseConnectorIT.java | 352 ++++++++++++++ .../UpsertDeleteBigQuerySinkConnectorIT.java | 408 ++++++++++++++++ .../src/test/resources/log4j.properties | 20 + 18 files changed, 1395 insertions(+), 331 deletions(-) create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/ExpectedInterruptException.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/TableNameUtils.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java create mode 100644 kcbq-connector/src/test/resources/log4j.properties diff --git a/README.md b/README.md index 6bbfc4a9a..657cfe19f 100644 --- a/README.md +++ b/README.md @@ -131,12 +131,41 @@ adjusting flags given to the Avro Console Producer and tweaking the config setti ## Integration Testing the Connector +There is a legacy Docker-based integration test for the connector, and newer integration tests that +programmatically instantiate an embedded Connect cluster. + +### Embedded integration tests + +Currently these tests only verify the connector's upsert/delete feature. They should eventually +replace all of the existing Docker-based tests. + +#### Configuring the tests + +You must supply the following environment variables in order to run the tests: + +- `$KCBQ_TEST_PROJECT`: The name of the BigQuery project to use for the test +- `$KCBQ_TEST_DATASET`: The name of the BigQuery dataset to use for the test +- `$KCBQ_TEST_KEYFILE`: The key (either file or raw contents) used to authenticate with BigQuery +during the test + +Additionally, the `$KCBQ_TEST_KEYSOURCE` variable can be supplied to specify whether the value of +`$KCBQ_TEST_KEYFILE` are a path to a key file (if set to `FILE`) or the raw contents of a key file +(if set to `JSON`). The default is `FILE`. + +#### Running the Integration Tests + +```bash +./gradlew embeddedIntegrationTest +``` + +### Docker-based tests + > **NOTE**: You must have [Docker] installed and running on your machine in order to run integration tests for the connector. This all takes place in the `kcbq-connector` directory. -### How Integration Testing Works +#### How Integration Testing Works Integration tests run by creating [Docker] instances for [Zookeeper], [Kafka], [Schema Registry], and the BigQuery Connector itself, then verifying the results using a [JUnit] test. @@ -148,7 +177,7 @@ The project and dataset they write to, as well as the specific JSON key file the specified by command-line flag, environment variable, or configuration file — the exact details of each can be found by running the integration test script with the `-?` flag. -### Data Corruption Concerns +#### Data Corruption Concerns In order to ensure the validity of each test, any table that will be written to in the course of integration testing is preemptively deleted before the connector is run. This will only be an issue @@ -161,7 +190,7 @@ tests will corrupt any existing data that is already on your machine, and there free up any of your ports that might currently be in use by real instances of the programs that are faked in the process of testing. -### Running the Integration Tests +#### Running the Integration Tests Running the series of integration tests is easy: @@ -176,7 +205,7 @@ the `--help` flag. > **NOTE:** You must have a recent version of [boot2docker], [Docker Machine], [Docker], etc. installed. Older versions will hang when cleaning containers, and linking doesn't work properly. -### Adding New Integration Tests +#### Adding New Integration Tests Adding an integration test is a little more involved, and consists of two major steps: specifying Avro data to be sent to Kafka, and specifying via JUnit test how to verify that such data made diff --git a/build.gradle b/build.gradle index 1652f5af4..13bfe3e49 100644 --- a/build.gradle +++ b/build.gradle @@ -28,6 +28,7 @@ project.ext { ioConfluentVersion = '5.5.0' junitVersion = '4.12' kafkaVersion = '2.5.0' + kafkaScalaVersion = '2.12' // For integration testing only mockitoVersion = '3.2.4' slf4jVersion = '1.6.1' } @@ -153,6 +154,26 @@ project(':kcbq-connector') { } } + test { + useJUnit { + // Exclude embedded integration tests from normal testing since they require BigQuery + // credentials and can take a while + excludeCategories 'org.apache.kafka.test.IntegrationTest' + } + } + + task embeddedIntegrationTest(type: Test) { + useJUnit { + includeCategories 'org.apache.kafka.test.IntegrationTest' + } + + // Enable logging for integration tests + testLogging { + outputs.upToDateWhen {false} + showStandardStreams = true + } + } + task integrationTestPrep() { dependsOn 'integrationTestTablePrep' dependsOn 'integrationTestBucketPrep' @@ -226,7 +247,12 @@ project(':kcbq-connector') { "junit:junit:$junitVersion", "org.mockito:mockito-core:$mockitoVersion", "org.mockito:mockito-inline:$mockitoVersion", - "org.apache.kafka:connect-api:$kafkaVersion" + "org.apache.kafka:kafka_$kafkaScalaVersion:$kafkaVersion", + "org.apache.kafka:kafka_$kafkaScalaVersion:$kafkaVersion:test", + "org.apache.kafka:kafka-clients:$kafkaVersion:test", + "org.apache.kafka:connect-api:$kafkaVersion", + "org.apache.kafka:connect-runtime:$kafkaVersion", + "org.apache.kafka:connect-runtime:$kafkaVersion:test", ) } diff --git a/kcbq-api/src/main/java/com/wepay/kafka/connect/bigquery/api/SchemaRetriever.java b/kcbq-api/src/main/java/com/wepay/kafka/connect/bigquery/api/SchemaRetriever.java index 704ec5828..b21a36932 100644 --- a/kcbq-api/src/main/java/com/wepay/kafka/connect/bigquery/api/SchemaRetriever.java +++ b/kcbq-api/src/main/java/com/wepay/kafka/connect/bigquery/api/SchemaRetriever.java @@ -30,5 +30,4 @@ public interface SchemaRetriever { * @return The value Schema for the given record. */ Schema retrieveValueSchema(SinkRecord record); - } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index ca30f382c..f29ec470a 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -62,6 +62,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; @@ -69,6 +70,8 @@ import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; +import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.intTable; + /** * A {@link SinkTask} used to translate Kafka Connect {@link SinkRecord SinkRecords} into BigQuery * {@link RowToInsert RowToInserts} and subsequently write them to BigQuery. @@ -131,6 +134,11 @@ public BigQuerySinkTask(BigQuery testBigQuery, SchemaRetriever schemaRetriever, @Override public void flush(Map offsets) { + if (upsertDelete) { + throw new ConnectException("This connector cannot perform upsert/delete on older versions of " + + "the Connect framework; please upgrade to version 0.10.2.0 or later"); + } + try { executor.awaitCurrentTasks(); } catch (InterruptedException err) { @@ -459,33 +467,36 @@ private void maybeStartMergeFlushTask() { @Override public void stop() { + maybeStopExecutor(loadExecutor, "load executor"); + maybeStopExecutor(executor, "table write executor"); + if (upsertDelete) { + mergeBatches.intermediateTables().forEach(table -> { + logger.debug("Deleting {}", intTable(table)); + getBigQuery().delete(table); + }); + } + + logger.trace("task.stop()"); + } + + private void maybeStopExecutor(ExecutorService executor, String executorName) { + if (executor == null) { + return; + } + try { if (upsertDelete) { - mergeBatches.intermediateTables().forEach(table -> { - logger.debug("Deleting intermediate table {}", table); - getBigQuery().delete(table); - }); - } - } finally { - try { + logger.trace("Forcibly shutting down {}", executorName); + executor.shutdownNow(); + } else { + logger.trace("Requesting shutdown for {}", executorName); executor.shutdown(); - executor.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT_SEC, TimeUnit.SECONDS); - if (loadExecutor != null) { - try { - logger.info("Attempting to shut down load executor."); - loadExecutor.shutdown(); - loadExecutor.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT_SEC, TimeUnit.SECONDS); - } catch (InterruptedException ex) { - logger.warn("Could not shut down load executor within {}s.", - EXECUTOR_SHUTDOWN_TIMEOUT_SEC); - } - } - } catch (InterruptedException ex) { - logger.warn("{} active threads are still executing tasks {}s after shutdown is signaled.", - executor.getActiveCount(), EXECUTOR_SHUTDOWN_TIMEOUT_SEC); - } finally { - logger.trace("task.stop()"); } + logger.trace("Awaiting termination of {}", executorName); + executor.awaitTermination(EXECUTOR_SHUTDOWN_TIMEOUT_SEC, TimeUnit.SECONDS); + logger.trace("Shut down {} successfully", executorName); + } catch (Exception e) { + logger.warn("Failed to shut down {}", executorName, e); } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java index 168e3e1b9..6612c1409 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java @@ -26,6 +26,7 @@ import com.google.cloud.bigquery.TableId; import com.google.common.annotations.VisibleForTesting; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; +import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException; import com.wepay.kafka.connect.bigquery.write.batch.KCBQThreadPoolExecutor; import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; import org.apache.kafka.connect.errors.ConnectException; @@ -37,9 +38,13 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.destTable; +import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.intTable; + public class MergeQueries { public static final String INTERMEDIATE_TABLE_KEY_FIELD_NAME = "key"; public static final String INTERMEDIATE_TABLE_VALUE_FIELD_NAME = "value"; + public static final String INTERMEDIATE_TABLE_ITERATION_FIELD_NAME = "i"; public static final String INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME = "partitionTime"; public static final String INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD = "batchNumber"; @@ -105,16 +110,16 @@ public void mergeFlushAll() { public void mergeFlush(TableId intermediateTable) { final TableId destinationTable = mergeBatches.destinationTableFor(intermediateTable); final int batchNumber = mergeBatches.incrementBatch(intermediateTable); - logger.trace("Triggering merge flush from intermediate table {} to destination table {} for batch {}", - intermediateTable, destinationTable, batchNumber); + logger.trace("Triggering merge flush from {} to {} for batch {}", + intTable(intermediateTable), destTable(destinationTable), batchNumber); executor.execute(() -> { try { mergeFlush(intermediateTable, destinationTable, batchNumber); } catch (InterruptedException e) { - throw new ConnectException(String.format( + throw new ExpectedInterruptException(String.format( "Interrupted while performing merge flush of batch %d from %s to %s", - batchNumber, intermediateTable, destinationTable)); + batchNumber, intTable(intermediateTable), destTable(destinationTable))); } }); } @@ -124,211 +129,312 @@ private void mergeFlush( ) throws InterruptedException{ // If there are rows to flush in this batch, flush them if (mergeBatches.prepareToFlush(intermediateTable, batchNumber)) { - logger.debug("Running merge query on batch {} from intermediate table {}", - batchNumber, intermediateTable); + logger.debug("Running merge query on batch {} from {}", + batchNumber, intTable(intermediateTable)); String mergeFlushQuery = mergeFlushQuery(intermediateTable, destinationTable, batchNumber); logger.trace(mergeFlushQuery); bigQuery.query(QueryJobConfiguration.of(mergeFlushQuery)); - logger.trace("Merge from intermediate table {} to destination table {} completed", - intermediateTable, destinationTable); + logger.trace("Merge from {} to {} completed", + intTable(intermediateTable), destTable(destinationTable)); logger.debug("Recording flush success for batch {} from {}", - batchNumber, intermediateTable); + batchNumber, intTable(intermediateTable)); mergeBatches.recordSuccessfulFlush(intermediateTable, batchNumber); // Commit those offsets ASAP context.requestCommit(); logger.info("Completed merge flush of batch {} from {} to {}", - batchNumber, intermediateTable, destinationTable); + batchNumber, intTable(intermediateTable), destTable(destinationTable)); } // After, regardless of whether we flushed or not, clean up old batches from the intermediate // table. Some rows may be several batches old but still in the table if they were in the // streaming buffer during the last purge. - logger.trace("Clearing batches from {} on back from intermediate table {}", batchNumber, intermediateTable); + logger.trace("Clearing batches from {} on back from {}", batchNumber, intTable(intermediateTable)); String batchClearQuery = batchClearQuery(intermediateTable, batchNumber); logger.trace(batchClearQuery); bigQuery.query(QueryJobConfiguration.of(batchClearQuery)); } - /* + @VisibleForTesting + String mergeFlushQuery(TableId intermediateTable, TableId destinationTable, int batchNumber) { + Schema intermediateSchema = schemaManager.cachedSchema(intermediateTable); - upsert+delete: - - MERGE ``.`` - USING ( - SELECT * FROM ( - SELECT ARRAY_AGG( - x ORDER BY partitionTime DESC LIMIT 1 - )[OFFSET(0)] src - FROM ``.`` x - WHERE batchNumber= - GROUP BY key.[, key....] - ) - ) - ON ``.=`src`.key - WHEN MATCHED AND `src`.value IS NOT NULL - THEN UPDATE SET =`src`.value.[, =`src`.value....] - WHEN MATCHED AND `src`.value IS NULL - THEN DELETE - WHEN NOT MATCHED AND `src`.value IS NOT NULL - THEN INSERT (, _PARTITIONTIME, [, ]) - VALUES ( - `src`.key, - CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), - `src`.value.[, `src`.value....] - ); + if (upsertEnabled && deleteEnabled) { + return upsertDeleteMergeFlushQuery(intermediateTable, destinationTable, batchNumber, intermediateSchema); + } else if (upsertEnabled) { + return upsertMergeFlushQuery(intermediateTable, destinationTable, batchNumber, intermediateSchema); + } else if (deleteEnabled) { + return deleteMergeFlushQuery(intermediateTable, destinationTable, batchNumber, intermediateSchema); + } else { + throw new IllegalStateException("At least one of upsert or delete must be enabled for merge flushing to occur."); + } + } + /* + MERGE ``.`` + USING ( + SELECT * FROM ( + SELECT ARRAY_AGG( + x ORDER BY i DESC LIMIT 1 + )[OFFSET(0)] src + FROM ``.`` x + WHERE batchNumber= + GROUP BY key.[, key....] + ) + ) + ON ``.=src.key + WHEN MATCHED AND src.value IS NOT NULL + THEN UPDATE SET =src.value.[, =src.value....] + WHEN MATCHED AND src.value IS NULL + THEN DELETE + WHEN NOT MATCHED AND src.value IS NOT NULL + THEN INSERT (, [_PARTITIONTIME, ][, ]) + VALUES ( + src.key, + [CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP),] + src.value.[, src.value....] + ); + */ + private String upsertDeleteMergeFlushQuery( + TableId intermediateTable, TableId destinationTable, int batchNumber, Schema intermediateSchema + ) { + List keyFields = listFields( + intermediateSchema.getFields().get(INTERMEDIATE_TABLE_KEY_FIELD_NAME).getSubFields(), + INTERMEDIATE_TABLE_KEY_FIELD_NAME + "." + ); - delete only: + List valueColumns = valueColumns(intermediateSchema); + + final String key = INTERMEDIATE_TABLE_KEY_FIELD_NAME; + final String i = INTERMEDIATE_TABLE_ITERATION_FIELD_NAME; + final String value = INTERMEDIATE_TABLE_VALUE_FIELD_NAME; + final String batch = INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD; + + return "MERGE " + table(destinationTable) + " " + + "USING (" + + "SELECT * FROM (" + + "SELECT ARRAY_AGG(" + + "x ORDER BY " + i + " DESC LIMIT 1" + + ")[OFFSET(0)] src " + + "FROM " + table(intermediateTable) + " x " + + "WHERE " + batch + "=" + batchNumber + " " + + "GROUP BY " + String.join(", ", keyFields) + + ")" + + ") " + + "ON `" + destinationTable.getTable() + "`." + keyFieldName + "=src." + key + " " + + "WHEN MATCHED AND src." + value + " IS NOT NULL " + + "THEN UPDATE SET " + valueColumns.stream().map(col -> col + "=src." + value + "." + col).collect(Collectors.joining(", ")) + " " + + "WHEN MATCHED AND src." + value + " IS NULL " + + "THEN DELETE " + + "WHEN NOT MATCHED AND src." + value + " IS NOT NULL " + + "THEN INSERT (" + + keyFieldName + ", " + + partitionTimePseudoColumn() + + String.join(", ", valueColumns) + ") " + + "VALUES (" + + "src." + key + ", " + + partitionTimeValue() + + valueColumns.stream().map(col -> "src." + value + "." + col).collect(Collectors.joining(", ")) + + ");"; + } - MERGE ``.`` - USING ( - SELECT * FROM ( - SELECT ARRAY_AGG( - x ORDER BY partitionTime DESC LIMIT 1 - )[OFFSET(0)] src - FROM ``.`` x - WHERE batchNumber= - GROUP BY key.[, key....] + /* + MERGE ``.`` + USING ( + SELECT * FROM ( + SELECT ARRAY_AGG( + x ORDER BY i DESC LIMIT 1 + )[OFFSET(0)] src + FROM ``.`` x + WHERE batchNumber= + GROUP BY key.[, key....] + ) ) - ) - ON ``.=`src`.key AND `src`.value IS NULL - WHEN MATCHED - THEN DELETE - WHEN NOT MATCHED - THEN INSERT (, _PARTITIONTIME, [, ]) - VALUES ( - `src`.key, - CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), - `src`.value.[, `src`.value....] - ); - + ON ``.=src.key + WHEN MATCHED + THEN UPDATE SET =src.value.[, ...] + WHEN NOT MATCHED + THEN INSERT ([, src.value....] + ); + */ + private String upsertMergeFlushQuery( + TableId intermediateTable, TableId destinationTable, int batchNumber, Schema intermediateSchema + ) { + List keyFields = listFields( + intermediateSchema.getFields().get(INTERMEDIATE_TABLE_KEY_FIELD_NAME).getSubFields(), + INTERMEDIATE_TABLE_KEY_FIELD_NAME + "." + ); - upsert only: + List valueColumns = valueColumns(intermediateSchema); + + final String key = INTERMEDIATE_TABLE_KEY_FIELD_NAME; + final String i = INTERMEDIATE_TABLE_ITERATION_FIELD_NAME; + final String value = INTERMEDIATE_TABLE_VALUE_FIELD_NAME; + final String batch = INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD; + + return "MERGE " + table(destinationTable) + " " + + "USING (" + + "SELECT * FROM (" + + "SELECT ARRAY_AGG(" + + "x ORDER BY " + i + " DESC LIMIT 1" + + ")[OFFSET(0)] src " + + "FROM " + table(intermediateTable) + " x " + + "WHERE " + batch + "=" + batchNumber + " " + + "GROUP BY " + String.join(", ", keyFields) + + ")" + + ") " + + "ON `" + destinationTable.getTable() + "`." + keyFieldName + "=src." + key + " " + + "WHEN MATCHED " + + "THEN UPDATE SET " + valueColumns.stream().map(col -> col + "=src." + value + "." + col).collect(Collectors.joining(", ")) + " " + + "WHEN NOT MATCHED " + + "THEN INSERT (" + + keyFieldName + ", " + + partitionTimePseudoColumn() + + String.join(", ", valueColumns) + ") " + + "VALUES (" + + "src." + key + ", " + + partitionTimeValue() + + valueColumns.stream().map(col -> "src." + value + "." + col).collect(Collectors.joining(", ")) + + ");"; + } - MERGE ``.`` - USING ( - SELECT * FROM ( - SELECT ARRAY_AGG( - x ORDER BY partitionTime DESC LIMIT 1 - )[OFFSET(0)] src - FROM ``.`` x - WHERE batchNumber= - GROUP BY key.[, key....] - ) - ) - ON ``.=`src`.key - WHEN MATCHED - THEN UPDATE SET =`src`.value.[, ...] - WHEN NOT MATCHED - THEN INSERT (`.`` + USING ( + SELECT batch.key AS key, [partitionTime, ]value + FROM ( + SELECT src.i, src.key FROM ( + SELECT ARRAY_AGG( + x ORDER BY i DESC LIMIT 1 + )[OFFSET(0)] src + FROM ( + SELECT * FROM ``.`` + WHERE batchNumber= + ) x + WHERE x.value IS NULL + GROUP BY key.[, key....])) AS deletes + RIGHT JOIN ( + SELECT * FROM ``.` + ) AS batch + USING (key) + WHERE deletes.i IS NULL OR batch.i >= deletes.i + ORDER BY batch.i ASC) AS src + ON ``.=src.key AND src.value IS NULL + WHEN MATCHED + THEN DELETE + WHEN NOT MATCHED AND src.value IS NOT NULL + THEN INSERT (, [_PARTITIONTIME, ][, ]) VALUES ( - `src`.key, - CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), - `src`.value.[, `src`.value....] + src.key, + [CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP),] + src.value.[, src.value....] ); - */ - @VisibleForTesting - String mergeFlushQuery(TableId intermediateTable, TableId destinationTable, int batchNumber) { - Schema intermediateSchema = schemaManager.cachedSchema(intermediateTable); - - String srcKey = INTERMEDIATE_TABLE_KEY_FIELD_NAME; - + private String deleteMergeFlushQuery( + TableId intermediateTable, TableId destinationTable, int batchNumber, Schema intermediateSchema + ) { List keyFields = listFields( - intermediateSchema.getFields().get(srcKey).getSubFields(), - srcKey + "." + intermediateSchema.getFields().get(INTERMEDIATE_TABLE_KEY_FIELD_NAME).getSubFields(), + INTERMEDIATE_TABLE_KEY_FIELD_NAME + "." ); - List dstValueFields = intermediateSchema.getFields().get(INTERMEDIATE_TABLE_VALUE_FIELD_NAME).getSubFields() + + List valueColumns = valueColumns(intermediateSchema); + + final String key = INTERMEDIATE_TABLE_KEY_FIELD_NAME; + final String i = INTERMEDIATE_TABLE_ITERATION_FIELD_NAME; + final String value = INTERMEDIATE_TABLE_VALUE_FIELD_NAME; + final String batch = INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD; + + return "MERGE " + table(destinationTable) + " " + + "USING (" + + "SELECT batch." + key + " AS " + key + ", " + partitionTimeColumn() + value + " " + + "FROM (" + + "SELECT src." + i + ", src." + key + " FROM (" + + "SELECT ARRAY_AGG(" + + "x ORDER BY " + i + " DESC LIMIT 1" + + ")[OFFSET(0)] src " + + "FROM (" + + "SELECT * FROM " + table(intermediateTable) + " " + + "WHERE " + batch + "=" + batchNumber + + ") x " + + "WHERE x." + value + " IS NULL " + + "GROUP BY " + String.join(", ", keyFields) + ")) AS deletes " + + "RIGHT JOIN (" + + "SELECT * FROM " + table(intermediateTable) + " " + + "WHERE " + batch + "=" + batchNumber + + ") AS batch " + + "USING (" + key + ") " + + "WHERE deletes." + i + " IS NULL OR batch." + i + " >= deletes." + i + " " + + "ORDER BY batch." + i + " ASC) AS src " + + "ON `" + destinationTable.getTable() + "`." + keyFieldName + "=src." + key + " AND src." + value + " IS NULL " + + "WHEN MATCHED " + + "THEN DELETE " + + "WHEN NOT MATCHED AND src." + value + " IS NOT NULL " + + "THEN INSERT (" + + keyFieldName + ", " + + partitionTimePseudoColumn() + + String.join(", ", valueColumns) + ") " + + "VALUES (" + + "src." + key + ", " + + partitionTimeValue() + + valueColumns.stream().map(col -> "src." + value + "." + col).collect(Collectors.joining(", ")) + + ");"; + } + + private String table(TableId tableId) { + return String.format("`%s`.`%s`", tableId.getDataset(), tableId.getTable()); + } + + private List valueColumns(Schema intermediateTableSchema) { + return intermediateTableSchema.getFields().get(INTERMEDIATE_TABLE_VALUE_FIELD_NAME).getSubFields() .stream() .map(Field::getName) .collect(Collectors.toList()); + } - List srcValueFields = dstValueFields.stream() - .map(field -> "`src`." + INTERMEDIATE_TABLE_VALUE_FIELD_NAME + "." + field) - .collect(Collectors.toList()); - List updateValues = dstValueFields.stream() - .map(field -> field + "=`src`." + INTERMEDIATE_TABLE_VALUE_FIELD_NAME + "." + field) - .collect(Collectors.toList()); + private String partitionTimePseudoColumn() { + return insertPartitionTime ? "_PARTITIONTIME, " : ""; + } - String partitionTimeField = insertPartitionTime ? "_PARTITIONTIME, " : ""; - String partitionTimeValue = insertPartitionTime - ? "CAST(CAST(DATE(`src`." + INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME + ") AS DATE) AS TIMESTAMP), " + private String partitionTimeValue() { + return insertPartitionTime + ? "CAST(CAST(DATE(src." + INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME + ") AS DATE) AS TIMESTAMP), " : ""; + } - String dst = destinationTable.getTable(); - - StringBuilder keysMatch = new StringBuilder("`").append(dst).append("`.").append(keyFieldName).append("=`src`.").append(srcKey); - - StringBuilder mergeOpening = new StringBuilder("MERGE `").append(destinationTable.getDataset()).append("`.`").append(destinationTable.getTable()).append("` ") - .append("USING (") - .append("SELECT * FROM (") - .append("SELECT ARRAY_AGG(") - .append("x ORDER BY ").append(INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME).append(" DESC LIMIT 1") - .append(")[OFFSET(0)] src ") - .append("FROM `").append(intermediateTable.getDataset()).append("`.`").append(intermediateTable.getTable()).append("` x ") - .append("WHERE ").append(INTERMEDIATE_TABLE_BATCH_NUMBER_FIELD).append("=").append(batchNumber).append(" ") - .append("GROUP BY ").append(String.join(", ", keyFields)) - .append(")") - .append(") "); - - StringBuilder insertClause = new StringBuilder("THEN INSERT (") - .append(keyFieldName).append(", ") - .append(partitionTimeField) - .append(String.join(", ", dstValueFields)) - .append(") ") - .append("VALUES (") - .append("`src`.").append(srcKey).append(", ") - .append(partitionTimeValue) - .append(String.join(", ", srcValueFields)) - .append(")"); - - StringBuilder updateClause = new StringBuilder("THEN UPDATE SET ") - .append(String.join(", ", updateValues)); - - StringBuilder valueIs = new StringBuilder("`src`.").append(INTERMEDIATE_TABLE_VALUE_FIELD_NAME).append(" IS "); - - if (upsertEnabled && deleteEnabled) { - // Delete rows with null values, and upsert all others - return mergeOpening - .append("ON ").append(keysMatch).append(" ") - .append("WHEN MATCHED AND ").append(valueIs).append("NOT NULL ") - .append(updateClause).append(" ") - .append("WHEN MATCHED AND ").append(valueIs).append("NULL ") - .append("THEN DELETE ") - .append("WHEN NOT MATCHED AND ").append(valueIs).append("NOT NULL ") - .append(insertClause) - .append(";") - .toString(); - } else if (deleteEnabled) { - // Delete rows with null values, and insert all others - return mergeOpening - .append("ON ").append(keysMatch).append(" ") - .append("AND ").append(valueIs).append("NULL ") - .append("WHEN MATCHED ") - .append("THEN DELETE ") - .append("WHEN NOT MATCHED ") - .append(insertClause) - .append(";") - .toString(); - } else if (upsertEnabled) { - // Assume all rows have non-null values and upsert them all - return mergeOpening - .append("ON ").append(keysMatch).append(" ") - .append("WHEN MATCHED ") - .append(updateClause).append(" ") - .append("WHEN NOT MATCHED ") - .append(insertClause) - .append(";") - .toString(); - } else { - throw new IllegalStateException("At least one of upsert or delete must be enabled for merge flushing to occur."); - } + private String partitionTimeColumn() { + return insertPartitionTime + ? INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME + ", " + : ""; } - // DELETE FROM `` WHERE batchNumber <= AND _PARTITIONTIME IS NOT NULL; + // DELETE FROM ``.`` WHERE batchNumber <= AND _PARTITIONTIME IS NOT NULL; @VisibleForTesting static String batchClearQuery(TableId intermediateTable, int batchNumber) { return new StringBuilder("DELETE FROM `").append(intermediateTable.getDataset()).append("`.`").append(intermediateTable.getTable()).append("` ") diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 78e47f37d..129de9a09 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -17,17 +17,19 @@ import com.wepay.kafka.connect.bigquery.convert.KafkaDataBuilder; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; - +import com.wepay.kafka.connect.bigquery.utils.TableNameUtils; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.SortedMap; import java.util.function.Function; import java.util.stream.Collectors; import java.util.concurrent.ConcurrentHashMap; @@ -249,8 +251,7 @@ private TableInfo getTableInfo(TableId table, Set records) { } catch (BigQueryConnectException exception) { throw new BigQueryConnectException("Failed to unionize schemas of records for the table " + table, exception); } - TableInfo tableInfo = constructTableInfo(table, schema, tableDescription); - return tableInfo; + return constructTableInfo(table, schema, tableDescription); } /** @@ -294,14 +295,8 @@ private com.google.cloud.bigquery.Schema getUnionizedSchema(List firstSchemaFields = firstSchema - .getFields() - .stream() - .collect(Collectors.toMap(Field::getName, Function.identity())); - Map secondSchemaFields = secondSchema - .getFields() - .stream() - .collect(Collectors.toMap(Field::getName, Function.identity())); + Map firstSchemaFields = schemaFields(firstSchema); + Map secondSchemaFields = schemaFields(secondSchema); for (Map.Entry entry : secondSchemaFields.entrySet()) { if (!firstSchemaFields.containsKey(entry.getKey())) { if (allowNewBQFields && (entry.getValue().getMode().equals(Field.Mode.NULLABLE) @@ -340,6 +335,18 @@ private String getUnionizedTableDescription(Set records) { return tableDescription; } + /** + * Returns a dictionary providing lookup of each field in the schema by name. The ordering of the + * fields in the schema is preserved in the returned map. + * @param schema The BigQuery schema + * @return A map allowing lookup of schema fields by name + */ + private Map schemaFields(com.google.cloud.bigquery.Schema schema) { + Map result = new LinkedHashMap<>(); + schema.getFields().forEach(field -> result.put(field.getName(), field)); + return result; + } + // package private for testing. TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema bigQuerySchema, String tableDescription) { StandardTableDefinition.Builder builder = StandardTableDefinition.newBuilder() @@ -416,6 +423,12 @@ private List getIntermediateSchemaFields(com.google.cloud.bigquery.Schema .build(); result.add(kafkaKeyField); + Field iterationField = Field + .newBuilder(MergeQueries.INTERMEDIATE_TABLE_ITERATION_FIELD_NAME, LegacySQLTypeName.INTEGER) + .setMode(Field.Mode.REQUIRED) + .build(); + result.add(iterationField); + Field partitionTimeField = Field .newBuilder(MergeQueries.INTERMEDIATE_TABLE_PARTITION_TIME_FIELD_NAME, LegacySQLTypeName.TIMESTAMP) .setMode(Field.Mode.NULLABLE) @@ -450,9 +463,9 @@ private List getRegularSchemaFields(com.google.cloud.bigquery.Schema valu } private String table(TableId table) { - return (intermediateTables ? "intermediate " : "") - + "table " - + table; + return intermediateTables + ? TableNameUtils.intTable(table) + : TableNameUtils.table(table); } private com.google.cloud.bigquery.Schema readTableSchema(TableId table) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/ExpectedInterruptException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/ExpectedInterruptException.java new file mode 100644 index 000000000..093e704d0 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/ExpectedInterruptException.java @@ -0,0 +1,28 @@ +package com.wepay.kafka.connect.bigquery.exception; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + + +import org.apache.kafka.connect.errors.ConnectException; + +public class ExpectedInterruptException extends ConnectException { + + public ExpectedInterruptException(String message) { + super(message); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java index 0cd9ac389..7e45dee40 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/SinkRecordConverter.java @@ -106,6 +106,7 @@ private Map getUpsertDeleteRow(SinkRecord record, TableId table) result.put(MergeQueries.INTERMEDIATE_TABLE_KEY_FIELD_NAME, convertedKey); result.put(MergeQueries.INTERMEDIATE_TABLE_VALUE_FIELD_NAME, convertedValue); + result.put(MergeQueries.INTERMEDIATE_TABLE_ITERATION_FIELD_NAME, totalBatchSize); if (usePartitionDecorator && useMessageTimeDatePartitioning) { if (record.timestampType() == TimestampType.NO_TIMESTAMP_TYPE) { throw new ConnectException( diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/TableNameUtils.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/TableNameUtils.java new file mode 100644 index 000000000..ed52c9b83 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/TableNameUtils.java @@ -0,0 +1,36 @@ +package com.wepay.kafka.connect.bigquery.utils; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + + +import com.google.cloud.bigquery.TableId; + +public class TableNameUtils { + + public static String table(TableId table) { + return String.format("table `%s`.`%s`", table.getDataset(), table.getTable()); + } + + public static String intTable(TableId table) { + return "intermediate " + table(table); + } + + public static String destTable(TableId table) { + return "destination " + table(table); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java index 9e128cdaa..9494e036b 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java @@ -20,15 +20,16 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.Collection; -import java.util.List; +import java.util.Objects; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -64,7 +65,8 @@ public KCBQThreadPoolExecutor(BigQuerySinkTaskConfig config, protected void afterExecute(Runnable runnable, Throwable throwable) { super.afterExecute(runnable, throwable); - if (throwable != null) { + // Skip interrupted exceptions, as they are thrown by design on task shutdown + if (throwable != null && !(throwable instanceof ExpectedInterruptException)) { logger.error("Task failed with {} error: {}", throwable.getClass().getName(), throwable.getMessage()); @@ -110,10 +112,8 @@ public void maybeThrowEncounteredErrors() { } private static String createErrorString(Collection errors) { - List exceptionTypeStrings = new ArrayList<>(errors.size()); - exceptionTypeStrings.addAll(errors.stream() - .map(throwable -> throwable.getClass().getName()) - .collect(Collectors.toList())); - return String.join(", ", exceptionTypeStrings); + return errors.stream() + .map(Objects::toString) + .collect(Collectors.joining(", ")); } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java index 8a95ab317..bae1effd0 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/MergeBatches.java @@ -25,6 +25,7 @@ import com.google.common.collect.HashBiMap; import com.google.common.collect.Maps; import com.wepay.kafka.connect.bigquery.MergeQueries; +import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException; import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; @@ -42,6 +43,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.intTable; + public class MergeBatches { private static final Logger logger = LoggerFactory.getLogger(MergeBatches.class); private static final long STREAMING_BUFFER_AVAILABILITY_WAIT_MS = 10_000L; @@ -156,8 +159,8 @@ public long addToBatch(SinkRecord record, TableId intermediateTable, Map allBatchesForTable = batches.get(intermediateTable); Batch batch = allBatchesForTable.remove(batchNumber); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index e8747f851..26a65e6ba 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -23,6 +23,7 @@ import com.google.cloud.bigquery.TableId; import com.wepay.kafka.connect.bigquery.utils.SinkRecordConverter; +import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; @@ -102,7 +103,7 @@ public void run() { } } } catch (InterruptedException err) { - throw new ConnectException("Thread interrupted while writing to BigQuery.", err); + throw new ExpectedInterruptException("Thread interrupted while writing to BigQuery."); } // Common case is 1 successful call and 0 failed calls: diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java index e3b970459..0fce65174 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java @@ -28,6 +28,7 @@ import com.wepay.kafka.connect.bigquery.SchemaManager; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import org.apache.kafka.connect.sink.SinkRecord; @@ -130,6 +131,7 @@ && onlyContainsInvalidSchemaErrors(writeResponse.getInsertErrors())) { writeResponse = bigQuery.insertAll(request); } catch (BigQueryException exception) { // no-op, we want to keep retrying the insert + logger.trace("insertion failed", exception); } } else { return writeResponse.getInsertErrors(); @@ -143,7 +145,7 @@ && onlyContainsInvalidSchemaErrors(writeResponse.getInsertErrors())) { try { Thread.sleep(RETRY_WAIT_TIME); } catch (InterruptedException e) { - // no-op, we want to keep retrying the insert + throw new ExpectedInterruptException("Interrupted while waiting to retry write"); } } logger.debug("table insertion completed successfully"); @@ -178,6 +180,7 @@ protected void attemptTableCreate(TableId tableId, Set records) { * This is why we can't have nice things, Google. */ private boolean onlyContainsInvalidSchemaErrors(Map> errors) { + logger.trace("write response contained errors: \n{}", errors); boolean invalidSchemaError = false; for (List errorList : errors.values()) { for (BigQueryError error : errorList) { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 51646dcc3..4192089f9 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyObject; import static org.mockito.Mockito.mock; @@ -40,6 +41,7 @@ import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.storage.Storage; +import com.wepay.kafka.connect.bigquery.api.KafkaSchemaRecordType; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; @@ -150,7 +152,9 @@ public void testSimplePutWhenSchemaRetrieverIsNotNull() { testTask.initialize(sinkTaskContext); testTask.start(properties); - testTask.put(Collections.singletonList(spoofSinkRecord(topic))); + SinkRecord spoofedRecord = + spoofSinkRecord(topic, "k", "key", "v", "value", TimestampType.NO_TIMESTAMP_TYPE, null); + testTask.put(Collections.singletonList(spoofedRecord)); testTask.flush(Collections.emptyMap()); verify(bigQuery, times(1)).insertAll(any(InsertAllRequest.class)); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java index 9675df8c2..bba3add2f 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java @@ -98,26 +98,25 @@ private static Schema constructIntermediateTable() { public void testUpsertQueryWithPartitionTime() { String expectedQuery = "MERGE " + table(DESTINATION_TABLE) + " " - + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY i DESC LIMIT 1)[OFFSET(0)] src " + "FROM " + table(INTERMEDIATE_TABLE) + " x " + "WHERE batchNumber=" + BATCH_NUMBER + " " + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " - + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key " + + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key " + "WHEN MATCHED " - + "THEN UPDATE SET f1=`src`.value.f1, f2=`src`.value.f2, f3=`src`.value.f3, f4=`src`.value.f4 " + + "THEN UPDATE SET f1=src.value.f1, f2=src.value.f2, f3=src.value.f3, f4=src.value.f4 " + "WHEN NOT MATCHED " - + "THEN INSERT (" + + "THEN INSERT (" + KEY + ", " - + "_PARTITIONTIME, " + + "_PARTITIONTIME, " + "f1, f2, f3, f4) " - + "VALUES (" - + "`src`.key, " - + "CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), " - + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + "VALUES (" + + "src.key, " + + "CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP), " + + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" + ");"; String actualQuery = mergeQueries(true, true, false) .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); - System.out.println(actualQuery); assertEquals(expectedQuery, actualQuery); } @@ -125,24 +124,23 @@ public void testUpsertQueryWithPartitionTime() { public void testUpsertQueryWithoutPartitionTime() { String expectedQuery = "MERGE " + table(DESTINATION_TABLE) + " " - + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY i DESC LIMIT 1)[OFFSET(0)] src " + "FROM " + table(INTERMEDIATE_TABLE) + " x " + "WHERE batchNumber=" + BATCH_NUMBER + " " + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " - + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key " + + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key " + "WHEN MATCHED " - + "THEN UPDATE SET f1=`src`.value.f1, f2=`src`.value.f2, f3=`src`.value.f3, f4=`src`.value.f4 " + + "THEN UPDATE SET f1=src.value.f1, f2=src.value.f2, f3=src.value.f3, f4=src.value.f4 " + "WHEN NOT MATCHED " + "THEN INSERT (" + KEY + ", " + "f1, f2, f3, f4) " + "VALUES (" - + "`src`.key, " - + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + "src.key, " + + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" + ");"; String actualQuery = mergeQueries(false, true, false) .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); - System.out.println(actualQuery); assertEquals(expectedQuery, actualQuery); } @@ -150,26 +148,41 @@ public void testUpsertQueryWithoutPartitionTime() { public void testDeleteQueryWithPartitionTime() { String expectedQuery = "MERGE " + table(DESTINATION_TABLE) + " " - + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " - + "FROM " + table(INTERMEDIATE_TABLE) + " x " - + "WHERE batchNumber=" + BATCH_NUMBER + " " - + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " - + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key AND `src`.value IS NULL " - + "WHEN MATCHED " - + "THEN DELETE " - + "WHEN NOT MATCHED " - + "THEN INSERT (" - + KEY + ", " - + "_PARTITIONTIME, " - + "f1, f2, f3, f4) " - + "VALUES (" - + "`src`.key, " - + "CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), " - + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + "USING (" + + "SELECT batch.key AS key, partitionTime, value " + + "FROM (" + + "SELECT src.i, src.key FROM (" + + "SELECT ARRAY_AGG(" + + "x ORDER BY i DESC LIMIT 1" + + ")[OFFSET(0)] src " + + "FROM (" + + "SELECT * FROM " + table(INTERMEDIATE_TABLE) + " " + + "WHERE batchNumber=" + BATCH_NUMBER + + ") x " + + "WHERE x.value IS NULL " + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) AS deletes " + + "RIGHT JOIN (" + + "SELECT * FROM " + table(INTERMEDIATE_TABLE) + " " + + "WHERE batchNumber=" + BATCH_NUMBER + + ") AS batch " + + "USING (key) " + + "WHERE deletes.i IS NULL OR batch.i >= deletes.i " + + "ORDER BY batch.i ASC) AS src " + + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key AND src.value IS NULL " + + "WHEN MATCHED " + + "THEN DELETE " + + "WHEN NOT MATCHED AND src.value IS NOT NULL " + + "THEN INSERT (" + + KEY + ", " + + "_PARTITIONTIME, " + + "f1, f2, f3, f4) " + + "VALUES (" + + "src.key, " + + "CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP), " + + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" + ");"; String actualQuery = mergeQueries(true, false, true) .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); - System.out.println(actualQuery); assertEquals(expectedQuery, actualQuery); } @@ -177,24 +190,39 @@ public void testDeleteQueryWithPartitionTime() { public void testDeleteQueryWithoutPartitionTime() { String expectedQuery = "MERGE " + table(DESTINATION_TABLE) + " " - + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " - + "FROM " + table(INTERMEDIATE_TABLE) + " x " - + "WHERE batchNumber=" + BATCH_NUMBER + " " - + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " - + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key AND `src`.value IS NULL " - + "WHEN MATCHED " - + "THEN DELETE " - + "WHEN NOT MATCHED " - + "THEN INSERT (" - + KEY + ", " - + "f1, f2, f3, f4) " - + "VALUES (" - + "`src`.key, " - + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + "USING (" + + "SELECT batch.key AS key, value " + + "FROM (" + + "SELECT src.i, src.key FROM (" + + "SELECT ARRAY_AGG(" + + "x ORDER BY i DESC LIMIT 1" + + ")[OFFSET(0)] src " + + "FROM (" + + "SELECT * FROM " + table(INTERMEDIATE_TABLE) + " " + + "WHERE batchNumber=" + BATCH_NUMBER + + ") x " + + "WHERE x.value IS NULL " + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) AS deletes " + + "RIGHT JOIN (" + + "SELECT * FROM " + table(INTERMEDIATE_TABLE) + " " + + "WHERE batchNumber=" + BATCH_NUMBER + + ") AS batch " + + "USING (key) " + + "WHERE deletes.i IS NULL OR batch.i >= deletes.i " + + "ORDER BY batch.i ASC) AS src " + + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key AND src.value IS NULL " + + "WHEN MATCHED " + + "THEN DELETE " + + "WHEN NOT MATCHED AND src.value IS NOT NULL " + + "THEN INSERT (" + + KEY + ", " + + "f1, f2, f3, f4) " + + "VALUES (" + + "src.key, " + + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" + ");"; String actualQuery = mergeQueries(false, false, true) .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); - System.out.println(actualQuery); assertEquals(expectedQuery, actualQuery); } @@ -202,28 +230,27 @@ public void testDeleteQueryWithoutPartitionTime() { public void testUpsertDeleteQueryWithPartitionTime() { String expectedQuery = "MERGE " + table(DESTINATION_TABLE) + " " - + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " - + "FROM " + table(INTERMEDIATE_TABLE) + " x " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY i DESC LIMIT 1)[OFFSET(0)] src " + + "FROM " + table(INTERMEDIATE_TABLE) + " x " + "WHERE batchNumber=" + BATCH_NUMBER + " " - + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " - + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key " - + "WHEN MATCHED AND `src`.value IS NOT NULL " - + "THEN UPDATE SET f1=`src`.value.f1, f2=`src`.value.f2, f3=`src`.value.f3, f4=`src`.value.f4 " - + "WHEN MATCHED AND `src`.value IS NULL " - + "THEN DELETE " - + "WHEN NOT MATCHED AND `src`.value IS NOT NULL " - + "THEN INSERT (" - + KEY + ", " - + "_PARTITIONTIME, " - + "f1, f2, f3, f4) " - + "VALUES (" - + "`src`.key, " - + "CAST(CAST(DATE(`src`.partitionTime) AS DATE) AS TIMESTAMP), " - + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key " + + "WHEN MATCHED AND src.value IS NOT NULL " + + "THEN UPDATE SET f1=src.value.f1, f2=src.value.f2, f3=src.value.f3, f4=src.value.f4 " + + "WHEN MATCHED AND src.value IS NULL " + + "THEN DELETE " + + "WHEN NOT MATCHED AND src.value IS NOT NULL " + + "THEN INSERT (" + + KEY + ", " + + "_PARTITIONTIME, " + + "f1, f2, f3, f4) " + + "VALUES (" + + "src.key, " + + "CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP), " + + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" + ");"; String actualQuery = mergeQueries(true, true, true) .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); - System.out.println(actualQuery); assertEquals(expectedQuery, actualQuery); } @@ -231,38 +258,35 @@ public void testUpsertDeleteQueryWithPartitionTime() { public void testUpsertDeleteQueryWithoutPartitionTime() { String expectedQuery = "MERGE " + table(DESTINATION_TABLE) + " " - + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY partitionTime DESC LIMIT 1)[OFFSET(0)] src " + + "USING (SELECT * FROM (SELECT ARRAY_AGG(x ORDER BY i DESC LIMIT 1)[OFFSET(0)] src " + "FROM " + table(INTERMEDIATE_TABLE) + " x " + "WHERE batchNumber=" + BATCH_NUMBER + " " + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " - + "ON `" + DESTINATION_TABLE.getTable() + "`.kafkaKey=`src`.key " - + "WHEN MATCHED AND `src`.value IS NOT NULL " - + "THEN UPDATE SET f1=`src`.value.f1, f2=`src`.value.f2, f3=`src`.value.f3, f4=`src`.value.f4 " - + "WHEN MATCHED AND `src`.value IS NULL " + + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key " + + "WHEN MATCHED AND src.value IS NOT NULL " + + "THEN UPDATE SET f1=src.value.f1, f2=src.value.f2, f3=src.value.f3, f4=src.value.f4 " + + "WHEN MATCHED AND src.value IS NULL " + "THEN DELETE " - + "WHEN NOT MATCHED AND `src`.value IS NOT NULL " + + "WHEN NOT MATCHED AND src.value IS NOT NULL " + "THEN INSERT (" + KEY + ", " + "f1, f2, f3, f4) " + "VALUES (" - + "`src`.key, " - + "`src`.value.f1, `src`.value.f2, `src`.value.f3, `src`.value.f4" + + "src.key, " + + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" + ");"; String actualQuery = mergeQueries(false, true, true) .mergeFlushQuery(INTERMEDIATE_TABLE, DESTINATION_TABLE, BATCH_NUMBER); - System.out.println(actualQuery); assertEquals(expectedQuery, actualQuery); } @Test public void testBatchClearQuery() { - String expectedQuery = + String expectedQuery = "DELETE FROM " + table(INTERMEDIATE_TABLE) + " WHERE batchNumber <= " + BATCH_NUMBER + " AND _PARTITIONTIME IS NOT NULL;"; // No difference in batch clearing between upsert, delete, and both, or with or without partition time - String actualQuery = mergeQueries(false, false, false) - .batchClearQuery(INTERMEDIATE_TABLE, BATCH_NUMBER); - System.out.println(actualQuery); + String actualQuery = MergeQueries.batchClearQuery(INTERMEDIATE_TABLE, BATCH_NUMBER); assertEquals(expectedQuery, actualQuery); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java new file mode 100644 index 000000000..980ec295b --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -0,0 +1,352 @@ +package com.wepay.kafka.connect.bigquery.integration; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.FieldValue; +import com.google.cloud.bigquery.QueryJobConfiguration; +import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.TableResult; +import com.wepay.kafka.connect.bigquery.BigQueryHelper; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; +import org.apache.kafka.connect.runtime.AbstractStatus; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.google.cloud.bigquery.LegacySQLTypeName.BOOLEAN; +import static com.google.cloud.bigquery.LegacySQLTypeName.BYTES; +import static com.google.cloud.bigquery.LegacySQLTypeName.DATE; +import static com.google.cloud.bigquery.LegacySQLTypeName.FLOAT; +import static com.google.cloud.bigquery.LegacySQLTypeName.INTEGER; +import static com.google.cloud.bigquery.LegacySQLTypeName.STRING; +import static com.google.cloud.bigquery.LegacySQLTypeName.TIMESTAMP; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.Assert.assertTrue; + +@Category(IntegrationTest.class) +public abstract class BaseConnectorIT { + private static final Logger logger = LoggerFactory.getLogger(BaseConnectorIT.class); + + private static final String KEYFILE_ENV_VAR = "KCBQ_TEST_KEYFILE"; + private static final String PROJECT_ENV_VAR = "KCBQ_TEST_PROJECT"; + private static final String DATASET_ENV_VAR = "KCBQ_TEST_DATASET"; + private static final String KEYSOURCE_ENV_VAR = "KCBQ_TEST_KEYSOURCE"; + + protected static final long OFFSET_COMMIT_INTERVAL_MS = TimeUnit.SECONDS.toMillis(10); + protected static final long COMMIT_MAX_DURATION_MS = TimeUnit.MINUTES.toMillis(5); + protected static final long OFFSETS_READ_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(10); + protected static final long CONNECTOR_STARTUP_DURATION_MS = TimeUnit.SECONDS.toMillis(60); + + protected EmbeddedConnectCluster connect; + private Admin kafkaAdminClient; + + protected void startConnect() { + Map workerProps = new HashMap<>(); + workerProps.put( + WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, Long.toString(OFFSET_COMMIT_INTERVAL_MS)); + // Allow per-connector consumer configuration for throughput testing + workerProps.put( + WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG, "All"); + + connect = new EmbeddedConnectCluster.Builder() + .name("kcbq-connect-cluster") + .workerProps(workerProps) + .build(); + + // start the clusters + connect.start(); + + kafkaAdminClient = connect.kafka().createAdminClient(); + + // the exception handler installed by the embedded zookeeper instance is noisy and unnecessary + Thread.setDefaultUncaughtExceptionHandler((t, e) -> { }); + } + + protected void stopConnect() { + if (kafkaAdminClient != null) { + kafkaAdminClient.close(); + kafkaAdminClient = null; + } + + // stop all Connect, Kafka and Zk threads. + if (connect != null) { + connect.stop(); + connect = null; + } + } + + protected Map baseConnectorProps(int tasksMax) { + Map result = new HashMap<>(); + + result.put(CONNECTOR_CLASS_CONFIG, "BigQuerySinkConnector"); + result.put(TASKS_MAX_CONFIG, Integer.toString(tasksMax)); + + result.put(BigQuerySinkConfig.PROJECT_CONFIG, project()); + result.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, dataset()); + result.put(BigQuerySinkConfig.KEYFILE_CONFIG, keyFile()); + result.put(BigQuerySinkConfig.KEY_SOURCE_CONFIG, keySource()); + + return result; + } + + protected BigQuery newBigQuery() { + return new BigQueryHelper() + .setKeySource(keySource()) + .connect(project(), keyFile()); + } + + protected void clearPriorTable(BigQuery bigQuery, String table) { + boolean deleted = bigQuery.delete(TableId.of(dataset(), table)); + if (deleted) { + logger.info("Deleted existing test table `{}`.`{}`", dataset(), table); + } + } + + protected void waitForCommittedRecords( + String connector, String topic, long numRecords, int numTasks + ) throws InterruptedException { + waitForCommittedRecords(connector, topic, numRecords, numTasks, COMMIT_MAX_DURATION_MS); + } + + protected void waitForCommittedRecords( + String connector, String topic, long numRecords, int numTasks, long timeoutMs) throws InterruptedException { + waitForCondition( + () -> { + long totalCommittedRecords = totalCommittedRecords(connector, topic); + if (totalCommittedRecords >= numRecords) { + return true; + } else { + // Check to make sure the connector is still running. If not, fail fast + assertTrue( + "Connector or one of its tasks failed during testing", + assertConnectorAndTasksRunning(connector, numTasks).orElse(false)); + logger.debug("Connector has only committed {} records for topic {} so far; {} expected", + totalCommittedRecords, topic, numRecords); + // Sleep here so as not to spam Kafka with list-offsets requests + Thread.sleep(OFFSET_COMMIT_INTERVAL_MS / 2); + return false; + } + }, + timeoutMs, + "Either the connector failed, or the message commit duration expired without all expected messages committed"); + } + + protected synchronized long totalCommittedRecords(String connector, String topic) throws TimeoutException, ExecutionException, InterruptedException { + // See https://github.com/apache/kafka/blob/f7c38d83c727310f4b0678886ba410ae2fae9379/connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java + // for how the consumer group ID is constructed for sink connectors + Map offsets = kafkaAdminClient + .listConsumerGroupOffsets("connect-" + connector) + .partitionsToOffsetAndMetadata() + .get(OFFSETS_READ_TIMEOUT_MS, TimeUnit.MILLISECONDS); + + logger.trace("Connector {} has so far committed offsets {}", connector, offsets); + + return offsets.entrySet().stream() + .filter(entry -> topic.equals(entry.getKey().topic())) + .mapToLong(entry -> entry.getValue().offset()) + .sum(); + } + + /** + * Read all rows from the given table. + * @param bigQuery used to connect to BigQuery + * @param tableName the table to read + * @param sortColumn a column to sort rows by (can use dot notation to refer to nested fields) + * @return a list of all rows from the table, in random order. + */ + protected List> readAllRows( + BigQuery bigQuery, String tableName, String sortColumn) throws InterruptedException { + + Table table = bigQuery.getTable(dataset(), tableName); + Schema schema = table.getDefinition().getSchema(); + + TableResult tableResult = bigQuery.query(QueryJobConfiguration.of(String.format( + "SELECT * FROM `%s`.`%s` ORDER BY %s ASC", + dataset(), + tableName, + sortColumn + ))); + + return StreamSupport.stream(tableResult.iterateAll().spliterator(), false) + .map(fieldValues -> convertRow(schema.getFields(), fieldValues)) + .collect(Collectors.toList()); + } + + private static List boxByteArray(byte[] bytes) { + Byte[] result = new Byte[bytes.length]; + for (int i = 0; i < bytes.length; i++) { + result[i] = bytes[i]; + } + return Arrays.asList(result); + } + + private Object convertField(Field fieldSchema, FieldValue field) { + if (field.isNull()) { + return null; + } + switch (field.getAttribute()) { + case PRIMITIVE: + if (fieldSchema.getType().equals(BOOLEAN)) { + return field.getBooleanValue(); + } else if (fieldSchema.getType().equals(BYTES)) { + // Do this in order for assertEquals() to work when this is an element of two compared + // lists + return boxByteArray(field.getBytesValue()); + } else if (fieldSchema.getType().equals(DATE)) { + DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + return LocalDate.parse(field.getStringValue(), dateFormatter) + .atStartOfDay(ZoneOffset.UTC) + .toInstant() + .toEpochMilli(); + } else if (fieldSchema.getType().equals(FLOAT)) { + return field.getDoubleValue(); + } else if (fieldSchema.getType().equals(INTEGER)) { + return field.getLongValue(); + } else if (fieldSchema.getType().equals(STRING)) { + return field.getStringValue(); + } else if (fieldSchema.getType().equals(TIMESTAMP)) { + return field.getTimestampValue(); + } else { + throw new RuntimeException("Cannot convert primitive field type " + + fieldSchema.getType()); + } + case REPEATED: + List result = new ArrayList<>(); + for (FieldValue arrayField : field.getRepeatedValue()) { + result.add(convertField(fieldSchema, arrayField)); + } + return result; + case RECORD: + List recordSchemas = fieldSchema.getSubFields(); + List recordFields = field.getRecordValue(); + return convertRow(recordSchemas, recordFields); + default: + throw new RuntimeException("Unknown field attribute: " + field.getAttribute()); + } + } + + private List convertRow(List rowSchema, List row) { + List result = new ArrayList<>(); + assert (rowSchema.size() == row.size()); + + for (int i = 0; i < rowSchema.size(); i++) { + result.add(convertField(rowSchema.get(i), row.get(i))); + } + + return result; + } + + /** + * Wait up to {@link #CONNECTOR_STARTUP_DURATION_MS maximum time limit} for the connector with the given + * name to start the specified number of tasks. + * + * @param name the name of the connector + * @param numTasks the minimum number of tasks that are expected + * @return the time this method discovered the connector has started, in milliseconds past epoch + * @throws InterruptedException if this was interrupted + */ + protected long waitForConnectorToStart(String name, int numTasks) throws InterruptedException { + TestUtils.waitForCondition( + () -> assertConnectorAndTasksRunning(name, numTasks).orElse(false), + CONNECTOR_STARTUP_DURATION_MS, + "Connector tasks did not start in time." + ); + return System.currentTimeMillis(); + } + + /** + * Confirm that a connector with an exact number of tasks is running. + * + * @param connectorName the connector + * @param numTasks the minimum number of tasks + * @return true if the connector and tasks are in RUNNING state; false otherwise + */ + protected Optional assertConnectorAndTasksRunning(String connectorName, int numTasks) { + try { + ConnectorStateInfo info = connect.connectorStatus(connectorName); + boolean result = info != null + && info.tasks().size() >= numTasks + && info.connector().state().equals(AbstractStatus.State.RUNNING.toString()) + && info.tasks().stream().allMatch(s -> s.state().equals(AbstractStatus.State.RUNNING.toString())); + return Optional.of(result); + } catch (Exception e) { + logger.error("Could not check connector state info.", e); + return Optional.empty(); + } + } + + private String readEnvVar(String var) { + String result = System.getenv(var); + if (result == null) { + throw new IllegalStateException(String.format( + "Environment variable '%s' must be supplied to run integration tests", + var)); + } + return result; + } + + private String readEnvVar(String var, String defaultVal) { + return System.getenv().getOrDefault(var, defaultVal); + } + + protected String keyFile() { + return readEnvVar(KEYFILE_ENV_VAR); + } + + protected String project() { + return readEnvVar(PROJECT_ENV_VAR); + } + + protected String dataset() { + return readEnvVar(DATASET_ENV_VAR); + } + + protected String keySource() { + return readEnvVar(KEYSOURCE_ENV_VAR, BigQuerySinkConfig.KEY_SOURCE_DEFAULT); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java new file mode 100644 index 000000000..67c30a3f2 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java @@ -0,0 +1,408 @@ +package com.wepay.kafka.connect.bigquery.integration; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + + +import com.google.cloud.bigquery.BigQuery; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.junit.Assert.assertEquals; + +@Category(IntegrationTest.class) +public class UpsertDeleteBigQuerySinkConnectorIT extends BaseConnectorIT { + + private static final Logger logger = LoggerFactory.getLogger(UpsertDeleteBigQuerySinkConnectorIT.class); + + private static final String CONNECTOR_NAME = "kcbq-sink-connector"; + private static final long NUM_RECORDS_PRODUCED = 20; + private static final int TASKS_MAX = 3; + private static final String KAFKA_FIELD_NAME = "kafkaKey"; + + private BigQuery bigQuery; + + @Before + public void setup() { + bigQuery = newBigQuery(); + startConnect(); + } + + @After + public void close() { + bigQuery = null; + stopConnect(); + } + + private Map upsertDeleteProps( + boolean upsert, + boolean delete, + long mergeRecordsThreshold) { + if (!upsert && !delete) { + throw new IllegalArgumentException("At least one of upsert or delete must be enabled"); + } + + Map result = new HashMap<>(); + + // use the JSON converter with schemas enabled + result.put(KEY_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + result.put(VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + + if (upsert) { + result.put(BigQuerySinkConfig.UPSERT_ENABLED_CONFIG, "true"); + } + if (delete) { + result.put(BigQuerySinkConfig.DELETE_ENABLED_CONFIG, "true"); + } + + // Hardcode merge flushes to just use number of records for now, as it's more deterministic and + // faster to test + result.put(BigQuerySinkConfig.MERGE_INTERVAL_MS_CONFIG, "-1"); + result.put(BigQuerySinkConfig.MERGE_RECORDS_THRESHOLD_CONFIG, Long.toString(mergeRecordsThreshold)); + + result.put(BigQuerySinkConfig.KAFKA_KEY_FIELD_NAME_CONFIG, KAFKA_FIELD_NAME); + + return result; + } + + @Test + public void testUpsert() throws Throwable { + // create topic in Kafka + final String topic = "test-upsert"; + // Make sure each task gets to read from at least one partition + connect.kafka().createTopic(topic, TASKS_MAX); + + final String table = "test_upsert"; + clearPriorTable(bigQuery, table); + + // setup props for the sink connector + Map props = baseConnectorProps(TASKS_MAX); + props.put(SinkConnectorConfig.TOPICS_CONFIG, topic); + + props.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); + props.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, IdentitySchemaRetriever.class.getName()); + props.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); + + // Enable only upsert and not delete, and merge flush every other record + props.putAll(upsertDeleteProps(true, false, 2)); + + // start a sink connector + connect.configureConnector(CONNECTOR_NAME, props); + + // wait for tasks to spin up + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); + + // Instantiate the converters we'll use to send records to the connector + Converter keyConverter = converter(true); + Converter valueConverter = converter(false); + + // Send records to Kafka + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + // Each pair of records will share a key. Only the second record of each pair should be + // present in the table at the end of the test + String kafkaKey = key(keyConverter, topic, i / 2); + String kafkaValue = value(valueConverter, topic, i, false); + logger.debug("Sending message with key '{}' and value '{}' to topic '{}'", kafkaKey, kafkaValue, topic); + connect.kafka().produce(topic, kafkaKey, kafkaValue); + } + + // wait for tasks to write to BigQuery and commit offsets for their records + waitForCommittedRecords(CONNECTOR_NAME, topic, NUM_RECORDS_PRODUCED, TASKS_MAX); + + List> allRows = readAllRows(bigQuery, table, KAFKA_FIELD_NAME + ".k1"); + List> expectedRows = LongStream.range(0, NUM_RECORDS_PRODUCED / 2) + .mapToObj(i -> Arrays.asList( + "another string", + (i - 1) % 3 == 0, + (i * 2 + 1) / 0.69, + Collections.singletonList(i))) + .collect(Collectors.toList()); + assertEquals(expectedRows, allRows); + } + + @Test + public void testDelete() throws Throwable { + // create topic in Kafka + final String topic = "test-delete"; + // Make sure each task gets to read from at least one partition + connect.kafka().createTopic(topic, TASKS_MAX); + + final String table = "test_delete"; + clearPriorTable(bigQuery, table); + + // setup props for the sink connector + Map props = baseConnectorProps(TASKS_MAX); + props.put(SinkConnectorConfig.TOPICS_CONFIG, topic); + + props.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); + props.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, IdentitySchemaRetriever.class.getName()); + props.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); + + // Enable only delete and not upsert, and merge flush every other record + props.putAll(upsertDeleteProps(false, true, 2)); + + // start a sink connector + connect.configureConnector(CONNECTOR_NAME, props); + + // wait for tasks to spin up + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); + + // Instantiate the converters we'll use to send records to the connector + Converter keyConverter = converter(true); + Converter valueConverter = converter(false); + + // Send records to Kafka + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + // Each pair of records will share a key. Because upsert is not enabled, no deduplication will take place + // and, unless a tombstone is written for that key, both will be inserted + String kafkaKey = key(keyConverter, topic, i / 2); + // Every fourth record will be a tombstone, so every record pair with an odd-numbered key will be dropped + String kafkaValue = value(valueConverter, topic, i, i % 4 == 3); + logger.debug("Sending message with key '{}' and value '{}' to topic '{}'", kafkaKey, kafkaValue, topic); + connect.kafka().produce(topic, kafkaKey, kafkaValue); + } + + // wait for tasks to write to BigQuery and commit offsets for their records + waitForCommittedRecords(CONNECTOR_NAME, topic, NUM_RECORDS_PRODUCED, TASKS_MAX); + + // Since we have multiple rows per key, order by key and the f3 field (which should be + // monotonically increasing in insertion order) + List> allRows = readAllRows(bigQuery, table, KAFKA_FIELD_NAME + ".k1, f3"); + List> expectedRows = LongStream.range(0, NUM_RECORDS_PRODUCED) + .filter(i -> i % 4 < 2) + .mapToObj(i -> Arrays.asList( + i % 4 == 0 ? "a string" : "another string", + i % 3 == 0, + i / 0.69, + Collections.singletonList(i * 2 / 4))) + .collect(Collectors.toList()); + assertEquals(expectedRows, allRows); + } + + @Test + public void testUpsertDelete() throws Throwable { + // create topic in Kafka + final String topic = "test-upsert-delete"; + // Make sure each task gets to read from at least one partition + connect.kafka().createTopic(topic, TASKS_MAX); + + final String table = "test_upsert_delete"; + clearPriorTable(bigQuery, table); + + // setup props for the sink connector + Map props = baseConnectorProps(TASKS_MAX); + props.put(SinkConnectorConfig.TOPICS_CONFIG, topic); + + props.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); + props.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, IdentitySchemaRetriever.class.getName()); + props.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); + + // Enable upsert and delete, and merge flush every other record + props.putAll(upsertDeleteProps(true, true, 2)); + + // start a sink connector + connect.configureConnector(CONNECTOR_NAME, props); + + // wait for tasks to spin up + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); + + // Instantiate the converters we'll use to send records to the connector + Converter keyConverter = converter(true); + Converter valueConverter = converter(false); + + // Send records to Kafka + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + // Each pair of records will share a key. Only the second record of each pair should be + // present in the table at the end of the test + String kafkaKey = key(keyConverter, topic, i / 2); + // Every fourth record will be a tombstone, so every record pair with an odd-numbered key will be dropped + String kafkaValue = value(valueConverter, topic, i, i % 4 == 3); + logger.debug("Sending message with key '{}' and value '{}' to topic '{}'", kafkaKey, kafkaValue, topic); + connect.kafka().produce(topic, kafkaKey, kafkaValue); + } + + // wait for tasks to write to BigQuery and commit offsets for their records + waitForCommittedRecords(CONNECTOR_NAME, topic, NUM_RECORDS_PRODUCED, TASKS_MAX); + + // Since we have multiple rows per key, order by key and the f3 field (which should be + // monotonically increasing in insertion order) + List> allRows = readAllRows(bigQuery, table, KAFKA_FIELD_NAME + ".k1, f3"); + List> expectedRows = LongStream.range(0, NUM_RECORDS_PRODUCED) + .filter(i -> i % 4 == 1) + .mapToObj(i -> Arrays.asList( + "another string", + i % 3 == 0, + i / 0.69, + Collections.singletonList(i * 2 / 4))) + .collect(Collectors.toList()); + assertEquals(expectedRows, allRows); + } + + @Test + @Ignore("Skipped during regular testing; comment-out annotation to run") + public void testUpsertDeleteHighThroughput() throws Throwable { + final long numRecords = 1_000_000L; + final int numPartitions = 10; + final int tasksMax = 1; + + // create topic in Kafka + final String topic = "test-upsert-delete-throughput"; + connect.kafka().createTopic(topic, numPartitions); + + final String table = "test_upsert_delete_throughput"; + clearPriorTable(bigQuery, table); + + // Instantiate the converters we'll use to send records to the connector + Converter keyConverter = converter(true); + Converter valueConverter = converter(false); + + // Send records to Kafka. Pre-populate Kafka before starting the connector as we want to measure + // the connector's throughput cleanly + logger.info("Pre-populating Kafka with test data"); + for (int i = 0; i < numRecords; i++) { + if (i % 10000 == 0) { + logger.info("{} records produced so far", i); + } + // Each pair of records will share a key. Only the second record of each pair should be + // present in the table at the end of the test + String kafkaKey = key(keyConverter, topic, i / 2); + // Every fourth record will be a tombstone, so every record pair with an odd-numbered key will + // be dropped + String kafkaValue = value(valueConverter, topic, i, i % 4 == 3); + connect.kafka().produce(topic, kafkaKey, kafkaValue); + } + + // setup props for the sink connector + // use a single task + Map props = baseConnectorProps(tasksMax); + props.put(SinkConnectorConfig.TOPICS_CONFIG, topic); + // Allow for at most 10,000 records per call to poll + props.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + + ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + "10000"); + // Try to get at least 1 MB per partition with each request + props.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + + ConsumerConfig.FETCH_MIN_BYTES_CONFIG, + Integer.toString(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES * numPartitions)); + // Wait up to one second for each batch to reach the requested size + props.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + + ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, + "1000" + ); + + props.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); + props.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, IdentitySchemaRetriever.class.getName()); + props.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); + + // Enable upsert and delete, and schedule ten total flushes + props.putAll(upsertDeleteProps(true, true, numRecords / 10)); + + logger.info("Pre-population complete; creating connector"); + long start = System.currentTimeMillis(); + // start a sink connector + connect.configureConnector(CONNECTOR_NAME, props); + + // wait for tasks to spin up + waitForConnectorToStart(CONNECTOR_NAME, tasksMax); + + // wait for tasks to write to BigQuery and commit offsets for their records + waitForCommittedRecords(CONNECTOR_NAME, topic, numRecords, tasksMax, TimeUnit.MINUTES.toMillis(10)); + long time = System.currentTimeMillis() - start; + logger.info("All records have been read and committed by the connector; " + + "total time from start to finish: {} seconds", time / 1000.0); + + // Since we have multiple rows per key, order by key and the f3 field (which should be + // monotonically increasing in insertion order) + List> allRows = readAllRows(bigQuery, table, KAFKA_FIELD_NAME + ".k1, f3"); + List> expectedRows = LongStream.range(0, numRecords) + .filter(i -> i % 4 == 1) + .mapToObj(i -> Arrays.asList( + "another string", + i % 3 == 0, + i / 0.69, + Collections.singletonList(i * 2 / 4))) + .collect(Collectors.toList()); + assertEquals(expectedRows, allRows); + } + + private Converter converter(boolean isKey) { + Map props = new HashMap<>(); + props.put(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, true); + Converter result = new JsonConverter(); + result.configure(props, isKey); + return result; + } + + private String key(Converter converter, String topic, long iteration) { + final Schema schema = SchemaBuilder.struct() + .field("k1", Schema.INT64_SCHEMA) + .build(); + + final Struct struct = new Struct(schema) + .put("k1", iteration); + + return new String(converter.fromConnectData(topic, schema, struct)); + } + + private String value(Converter converter, String topic, long iteration, boolean tombstone) { + final Schema schema = SchemaBuilder.struct() + .optional() + .field("f1", Schema.STRING_SCHEMA) + .field("f2", Schema.BOOLEAN_SCHEMA) + .field("f3", Schema.FLOAT64_SCHEMA) + .build(); + + if (tombstone) { + return new String(converter.fromConnectData(topic, schema, null)); + } + + final Struct struct = new Struct(schema) + .put("f1", iteration % 2 == 0 ? "a string" : "another string") + .put("f2", iteration % 3 == 0) + .put("f3", iteration / 0.69); + + return new String(converter.fromConnectData(topic, schema, struct)); + } +} diff --git a/kcbq-connector/src/test/resources/log4j.properties b/kcbq-connector/src/test/resources/log4j.properties new file mode 100644 index 000000000..60a108fb4 --- /dev/null +++ b/kcbq-connector/src/test/resources/log4j.properties @@ -0,0 +1,20 @@ +log4j.rootLogger=INFO, stdout + +# Send the logs to the console. +# +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout + +connect.log.pattern=[%d] %p %X{connector.context}%m (%c:%L)%n +log4j.appender.stdout.layout.ConversionPattern=${connect.log.pattern} +log4j.appender.connectAppender.layout.ConversionPattern=${connect.log.pattern} + +# These are used in the log4j properties file that ships by default with Connect +log4j.logger.org.apache.zookeeper=ERROR +log4j.logger.org.reflections=ERROR + +# We see a lot of WARN-level messages from this class when a table is created by the connector and +# then written to shortly after. No need for that much noise during routine tests +log4j.logger.com.wepay.kafka.connect.bigquery.write.batch.TableWriter=ERROR +# Logs a message at INFO on every http request +log4j.logger.org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster=WARN \ No newline at end of file From 854d26a02917d95c3f9153d0af180c4e96247c18 Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Mon, 13 Jul 2020 15:04:19 -0700 Subject: [PATCH 016/190] CC-9208: Include original trace in worker trace when task fails. (#18) * Surface underlying error trace. * Include throwable cause message. * Handle write failures such as 403 errors. * Remove unused import. * Test exception thrown in put if errors are encountered. * Update test to check error message. * CC-9208: Prevent exceptions from being discarded if flush is called before put * Remove clear error buffer test. * Test BigQueryConnectException without toString. * Put back BigQueryConnectException toString override. * Remove unused import in BigQuerySinkTaskTest. * Add override dec. Co-authored-by: Chris Egerton --- .../connect/bigquery/BigQuerySinkTask.java | 6 +-- .../exception/BigQueryConnectException.java | 6 +++ .../write/batch/KCBQThreadPoolExecutor.java | 2 - .../bigquery/write/batch/TableWriter.java | 11 +++-- .../bigquery/BigQuerySinkTaskTest.java | 41 ++++++++++--------- .../write/row/BigQueryWriterTest.java | 5 +-- 6 files changed, 37 insertions(+), 34 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index f29ec470a..2f1e1fc2c 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -214,10 +214,8 @@ private PartitionedTableId getRecordTable(SinkRecord record) { @Override public void put(Collection records) { - if (upsertDelete) { - // Periodically poll for errors here instead of doing a stop-the-world check in flush() - executor.maybeThrowEncounteredErrors(); - } + // Periodically poll for errors here instead of doing a stop-the-world check in flush() + executor.maybeThrowEncounteredErrors(); logger.debug("Putting {} records in the sink.", records.size()); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java index 40fefd7da..e129480bf 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java @@ -61,4 +61,10 @@ private static String formatInsertAllErrors(Map> error } return messageBuilder.toString(); } + + @Override + public String toString() { + return getCause() != null ? + super.toString() + "\nCaused by: " + getCause().getLocalizedMessage() : super.toString(); + } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java index 9494e036b..91a3c9876 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java @@ -29,7 +29,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -105,7 +104,6 @@ public void awaitCurrentTasks() throws InterruptedException, BigQueryConnectExce public void maybeThrowEncounteredErrors() { if (encounteredErrors.size() > 0) { String errorString = createErrorString(encounteredErrors); - encounteredErrors.clear(); throw new BigQueryConnectException("Some write threads encountered unrecoverable errors: " + errorString + "; See logs for more detail"); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index 26a65e6ba..11d544784 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -23,11 +23,11 @@ import com.google.cloud.bigquery.TableId; import com.wepay.kafka.connect.bigquery.utils.SinkRecordConverter; +import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; -import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -98,7 +98,10 @@ public void run() { logger.warn("Could not write batch of size {} to BigQuery.", currentBatchList.size(), err); if (isBatchSizeError(err)) { failureCount++; - currentBatchSize = getNewBatchSize(currentBatchSize); + currentBatchSize = getNewBatchSize(currentBatchSize, err); + } else { + // Throw exception on write errors such as 403. + throw new BigQueryConnectException("Failed to write to table", err); } } } @@ -119,10 +122,10 @@ public void run() { onFinish.accept(rows.values()); } - private static int getNewBatchSize(int currentBatchSize) { + private static int getNewBatchSize(int currentBatchSize, Throwable err) { if (currentBatchSize == 1) { // todo correct exception type? - throw new ConnectException("Attempted to reduce batch size below 1."); + throw new BigQueryConnectException("Attempted to reduce batch size below 1.", err); } // round batch size up so we don't end up with a dangling 1 row at the end. return (int) Math.ceil(currentBatchSize / 2.0); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 4192089f9..28b222aec 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyObject; @@ -408,41 +407,43 @@ public void testPutWithUpsertDelete() throws Exception { assertTrue("Batch clears should be executed", executedBatchClears.await(1, TimeUnit.SECONDS)); } - // It's important that the buffer be completely wiped after a call to flush, since any exception - // thrown during flush causes Kafka Connect to not commit the offsets for any records sent to the - // task since the last flush - @Test - public void testBufferClearOnFlushError() { - final String dataset = "scratch"; - final String topic = "test_topic"; - + // Throw an exception on the first put, and assert the Exception will be exposed in subsequent + // put call. + @Test(expected = BigQueryConnectException.class, timeout = 30000L) + public void testSimplePutException() throws InterruptedException { + final String topic = "test-topic"; Map properties = propertiesFactory.getProperties(); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); - properties.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, dataset); + properties.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, "scratch"); BigQuery bigQuery = mock(BigQuery.class); Table mockTable = mock(Table.class); when(bigQuery.getTable(any())).thenReturn(mockTable); Storage storage = mock(Storage.class); - when(bigQuery.insertAll(any(InsertAllRequest.class))) - .thenThrow(new RuntimeException("This is a test")); + String error = "Cannot add required fields to an existing schema."; + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + when(bigQuery.insertAll(any())) + .thenThrow( + new BigQueryException(400, error, new BigQueryError("invalid", "global", error))); SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = + new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); testTask.initialize(sinkTaskContext); testTask.start(properties); + testTask.put(Collections.singletonList(spoofSinkRecord(topic))); try { - testTask.put(Collections.singletonList(spoofSinkRecord(topic))); - testTask.flush(Collections.emptyMap()); - fail("An exception should have been thrown by now"); - } catch (BigQueryConnectException err) { - testTask.flush(Collections.emptyMap()); - verify(bigQuery, times(1)).insertAll(any(InsertAllRequest.class)); + while (true) { + Thread.sleep(100); + testTask.put(Collections.emptyList()); + } + } catch (Exception e) { + assertTrue(e.getMessage().contains(error)); + throw e; } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index d1ab8b77b..3ba448227 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -142,7 +142,7 @@ public void testAutoCreateTables() { verify(bigQuery, times(2)).insertAll(anyObject()); } - @Test + @Test(expected = BigQueryConnectException.class) public void testNonAutoCreateTables() { final String topic = "test_topic"; final String dataset = "scratch"; @@ -174,9 +174,6 @@ public void testNonAutoCreateTables() { testTask.put( Collections.singletonList(spoofSinkRecord(topic, 0, 0, "some_field", "some_value"))); testTask.flush(Collections.emptyMap()); - - verify(schemaManager, times(0)).createTable(anyObject(), anyObject()); - verify(bigQuery, times(2)).insertAll(anyObject()); } @Test From 31befe9c9869e6fb9475552d74f15a813d148ab5 Mon Sep 17 00:00:00 2001 From: Alex Diachenko Date: Thu, 13 Aug 2020 10:05:35 -0700 Subject: [PATCH 017/190] RCCA-1483: Bump up ioConfluentVersion to 5.5.1 to include the latest kafka-connect-avro-converter. (#23) --- build.gradle | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/build.gradle b/build.gradle index 13bfe3e49..e37be875a 100644 --- a/build.gradle +++ b/build.gradle @@ -25,7 +25,7 @@ project.ext { googleCloudVersion = '1.79.0' googleAuthVersion = '0.9.0' googleCloudGsonVersion = '2.8.5' - ioConfluentVersion = '5.5.0' + ioConfluentVersion = '5.5.1' junitVersion = '4.12' kafkaVersion = '2.5.0' kafkaScalaVersion = '2.12' // For integration testing only @@ -213,7 +213,7 @@ project(':kcbq-connector') { javadoc { options.links 'http://docs.oracle.com/javase/8/docs/api/' - options.links 'http://docs.confluent.io/3.2.0/connect/javadocs/' + options.links 'http://docs.confluent.io/5.5.1/connect/javadocs/' options.links 'https://googleapis.dev/java/google-cloud-clients/0.97.0-alpha/' options.links 'https://kafka.apache.org/0100/javadoc/' options.links 'https://avro.apache.org/docs/1.8.1/api/java/' @@ -288,7 +288,7 @@ project('kcbq-api') { javadoc { options.links 'http://docs.oracle.com/javase/8/docs/api/' - options.links 'http://docs.confluent.io/3.2.0/connect/javadocs/' + options.links 'http://docs.confluent.io/5.5.1/connect/javadocs/' } dependencies { @@ -344,7 +344,7 @@ project('kcbq-confluent') { javadoc { options.links 'http://docs.oracle.com/javase/8/docs/api/' - options.links 'http://docs.confluent.io/3.2.0/connect/javadocs/' + options.links 'http://docs.confluent.io/5.5.1/connect/javadocs/' } dependencies { From ce31e807454b787a03e6db1b4c7deb8f6f624527 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 14:47:31 -0400 Subject: [PATCH 018/190] Bump version to 2.0.0 --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 7e259101b..5b861fdf6 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,2 @@ group=com.wepay.kcbq -version=1.6.5 +version=2.0.0 From 31ffaa83cd32375dd236f0b122ca891a1d696659 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 16:15:53 -0400 Subject: [PATCH 019/190] Bump to next snapshot version for 1.1.x --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 301b5832f..1e4ec7128 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,3 +1,3 @@ group=com.wepay.kcbq sourceCompatibility=1.8 -version=1.1.2 +version=1.1.3-SNAPSHOT From 9cedaeb6c4e74d08be9dd04c6b53883a32b0dfd2 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 16:16:28 -0400 Subject: [PATCH 020/190] Bump to next snapshot version for 1.2.x --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 711f84905..43ce10728 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,3 +1,3 @@ group=com.wepay.kcbq sourceCompatibility=1.8 -version=1.2.0 +version=1.2.1-SNAPSHOT From 4931d3f62ce898612a4206043caca81613488172 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 16:21:37 -0400 Subject: [PATCH 021/190] Bump to next snapshot version for 1.3.x --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 0a3633cb8..a3370a016 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,2 @@ group=com.wepay.kcbq -version=1.3.0-SNAPSHOT +version=1.3.1-SNAPSHOT From 3f3a3292f6c3de4834bdee436901cd0d5e7af0a0 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 16:22:22 -0400 Subject: [PATCH 022/190] Bump to next snapshot version for 1.4.x --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 7540460ac..cdf539f0e 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,2 @@ group=com.wepay.kcbq -version=1.4.1 +version=1.4.2-SNAPSHOT From dbbe599e1c48a91b61b94a406ea6ad2ac5883345 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 16:22:57 -0400 Subject: [PATCH 023/190] Bump to next snapshot version for 1.5.x --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 23cee74fe..a18c42fd8 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,2 @@ group=com.wepay.kcbq -version=1.5.2 +version=1.5.3-SNAPSHOT From 603e3a3f60ac0084c6fcb5e64e4d7d9a355b524a Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 16:23:45 -0400 Subject: [PATCH 024/190] Bump to next snapshot version for 1.6.x --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 7e259101b..5e43a8f51 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,2 @@ group=com.wepay.kcbq -version=1.6.5 +version=1.6.6-SNAPSHOT From b8d0ac4191b6d3acebb511f0f92fcd7ba6fdfaa9 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 16:24:30 -0400 Subject: [PATCH 025/190] Revert to snapshot version since 2.0.0 has not been released yet --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 5b861fdf6..3546a4f69 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,2 @@ group=com.wepay.kcbq -version=2.0.0 +version=2.0.0-SNAPSHOT From 160a198ba68fb4b887a5655fad67a25e0430d302 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 10 Sep 2020 16:25:19 -0400 Subject: [PATCH 026/190] Bump to next snapshot version for 2.1.x --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 3546a4f69..c75bdf6e6 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,2 @@ group=com.wepay.kcbq -version=2.0.0-SNAPSHOT +version=2.1.0-SNAPSHOT From c4787b10c8dedefbc784df889007eef0d5d93e2b Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 11 Sep 2020 11:11:39 -0400 Subject: [PATCH 027/190] Add a brief summary of the WePay->Confluent migration to the README (#24) --- README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/README.md b/README.md index 657cfe19f..9ee9f62a0 100644 --- a/README.md +++ b/README.md @@ -6,6 +6,14 @@ This is an implementation of a sink connector from [Apache Kafka] to [Google BigQuery], built on top of [Apache Kafka Connect]. For a comprehensive list of configuration options, see the [Connector Configuration Wiki]. +## History + +This connector was [originally developed by WePay](https://github.com/wepay/kafka-connect-bigquery). +In late 2020 the project moved to [Confluent](https://github.com/confluentinc/kafka-connect-bigquery), +with both companies taking on maintenance duties. All new activity such as filing issues and opening +pull requests should now target the [Confluent](https://github.com/confluentinc/kafka-connect-bigquery) +fork of the project. + ## Download The latest releases are available in the GitHub release tab, or via [tarballs in Maven central](http://search.maven.org/#search%7Cga%7C1%7Ca%3A%22kcbq-connector%22). From 66fb689c5e80d5be7324c6bb3dbf1e7a6f53c6dd Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 11 Sep 2020 13:21:09 -0400 Subject: [PATCH 028/190] MINOR: Fix integration test to use new datasets config property (#25) --- kcbq-connector/test/integrationtest.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kcbq-connector/test/integrationtest.sh b/kcbq-connector/test/integrationtest.sh index a88c5518b..08b842675 100755 --- a/kcbq-connector/test/integrationtest.sh +++ b/kcbq-connector/test/integrationtest.sh @@ -245,7 +245,7 @@ CONNECTOR_PROPS="$DOCKER_DIR/connect/properties/connector.properties" cp "$RESOURCES_DIR/connector-template.properties" "$CONNECTOR_PROPS" cat << EOF >> $CONNECTOR_PROPS project=$KCBQ_TEST_PROJECT -datasets=.*=$KCBQ_TEST_DATASET +defaultDataset=$KCBQ_TEST_DATASET gcsBucketName=$KCBQ_TEST_BUCKET gcsFolderName=$KCBQ_TEST_FOLDER topics=$test_topics From 2efd29a4a9b2073f202ac348528428759224fe24 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 15 Sep 2020 16:36:49 -0400 Subject: [PATCH 029/190] MINOR: Remove PR template (#29) --- docs/pull_request_template.md | 28 ---------------------------- 1 file changed, 28 deletions(-) delete mode 100644 docs/pull_request_template.md diff --git a/docs/pull_request_template.md b/docs/pull_request_template.md deleted file mode 100644 index ed95397ba..000000000 --- a/docs/pull_request_template.md +++ /dev/null @@ -1,28 +0,0 @@ -## Problem - - -## Solution - - - -##### Does this solution apply anywhere else? -- [ ] yes -- [ ] no - -##### If yes, where? - - -## Test Strategy - - - -##### Testing done: -- [ ] Unit tests -- [ ] Integration tests -- [ ] System tests -- [ ] Manual tests - -## Release Plan - - - From d9f602dbdf45d2e058e3852c52e1fd2d3cb33eb0 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 23 Sep 2020 08:37:00 -0400 Subject: [PATCH 030/190] GH-32: Switch from Gradle to Maven for build tool (#34) --- .gitignore | 4 +- .travis.yml | 31 -- README.md | 10 +- build.gradle | 315 ---------------- codecov.yml | 43 --- gradle.properties | 3 - gradle/wrapper/gradle-wrapper.jar | Bin 53556 -> 0 bytes gradle/wrapper/gradle-wrapper.properties | 6 - gradlew | 164 --------- kcbq-api/pom.xml | 54 +++ kcbq-confluent/pom.xml | 96 +++++ .../src/test/resources/log4j.properties | 14 + kcbq-connector/logos/BigQuery.png | Bin 0 -> 6210 bytes kcbq-connector/logos/confluent.png | Bin 0 -> 3156 bytes kcbq-connector/pom.xml | 182 +++++++++ .../it/BigQueryConnectorIntegrationTest.java | 0 .../bigquery/it/utils/TableClearer.java | 0 .../src/test/resources/log4j.properties | 14 + .../test/docker/connect/connect-docker.sh | 3 +- kcbq-connector/test/integrationtest.sh | 29 +- pom.xml | 346 ++++++++++++++++++ settings.gradle | 1 - 22 files changed, 727 insertions(+), 588 deletions(-) delete mode 100644 .travis.yml delete mode 100644 build.gradle delete mode 100644 codecov.yml delete mode 100644 gradle.properties delete mode 100644 gradle/wrapper/gradle-wrapper.jar delete mode 100644 gradle/wrapper/gradle-wrapper.properties delete mode 100755 gradlew create mode 100644 kcbq-api/pom.xml create mode 100644 kcbq-confluent/pom.xml create mode 100644 kcbq-confluent/src/test/resources/log4j.properties create mode 100644 kcbq-connector/logos/BigQuery.png create mode 100644 kcbq-connector/logos/confluent.png create mode 100644 kcbq-connector/pom.xml rename kcbq-connector/src/{integration-test => test}/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java (100%) rename kcbq-connector/src/{integration-test => test}/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java (100%) create mode 100644 kcbq-connector/src/test/resources/log4j.properties create mode 100644 pom.xml delete mode 100644 settings.gradle diff --git a/.gitignore b/.gitignore index 0fd805b49..6b35bdd71 100644 --- a/.gitignore +++ b/.gitignore @@ -1,8 +1,10 @@ bin/ build/ +target/ *.class *.jar *.tar +*.zip .gradle @@ -21,5 +23,5 @@ build/ key.json test.conf -kcbq-connector/src/integration-test/resources/test.properties +kcbq-connector/src/test/resources/test.properties kcbq-connector/test/docker/connect/properties/ diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index 2ce0f9e86..000000000 --- a/.travis.yml +++ /dev/null @@ -1,31 +0,0 @@ -language: java -sudo: true -dist: trusty -group: edge - -jdk: - - oraclejdk8 - - openjdk8 - -matrix: - fast_finish: true - -script: - - gradle test - -after_success: - - if [ -e ./gradlew ]; then ./gradlew jacocoTestReport; else gradle jacocoTestReport; fi - - bash <(curl -s https://codecov.io/bash) - -before_cache: - - rm -f $HOME/.gradle/caches/modules-2/modules-2.lock - - rm -fr $HOME/.gradle/caches/*/plugin-resolution/ - -cache: - directories: - - $HOME/.gradle/caches/ - - $HOME/.gradle/wrapper/ - -notifications: - email: - - open-source@wepay.com diff --git a/README.md b/README.md index 91aad2b20..2fad00b76 100644 --- a/README.md +++ b/README.md @@ -38,7 +38,7 @@ save the properties file. Once you get more familiar with the connector, you might want to revisit the `connector.properties` file and experiment with tweaking its settings. -### Building and Extracting a Tarball +### Building and Extracting a Confluent Hub archive If you haven't already, move into the repository's top-level directory: @@ -46,16 +46,16 @@ If you haven't already, move into the repository's top-level directory: $ cd /path/to/kafka-connect-bigquery/ ``` -Begin by creating a tarball of the connector with the Confluent Schema Retriever included: +Begin by creating Confluent Hub archive of the connector with the Confluent Schema Retriever included: ```bash -$ ./gradlew clean confluentTarBall +$ mvn clean package -DskipTests ``` And then extract its contents: ```bash -$ mkdir bin/jar/ && tar -C bin/jar/ -xf bin/tar/kcbq-connector-*-confluent-dist.tar +$ mkdir -p bin/jar/ && cp kcbq-connector/target/components/packages/wepay-kafka-connect-bigquery-*/wepay-kafka-connect-bigquery-*/lib/*.jar bin/jar/ ``` ### Setting-Up Background Processes @@ -193,7 +193,7 @@ cannot occupy more than one line** (this inconvenience is due to limitations in Console Producer, and may be addressed in future commits). To specify data verification, add a new JUnit test to the file -`src/integration-test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java`. +`src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java`. Rows that are retrieved from BigQuery in the test are only returned as _Lists_ of _Objects_. The names of their columns are not tracked. Construct a _List_ of the _Objects_ that you expect to be stored in the test's BigQuery table, retrieve the actual _List_ of _Objects_ stored via a call to diff --git a/build.gradle b/build.gradle deleted file mode 100644 index 901987f9d..000000000 --- a/build.gradle +++ /dev/null @@ -1,315 +0,0 @@ -// BEGIN ALL PROJECTS // -allprojects { - apply plugin: 'java' -} -// END ALL PROJECTS - -// BEGIN SUBPROJECTS // -subprojects { subproject -> - - apply plugin: 'maven' - apply plugin: 'signing' - apply plugin: 'checkstyle' - apply plugin: 'findbugs' - apply plugin: 'idea' - apply plugin: 'eclipse' - - jar.baseName = subproject.name - - [compileJava, compileTestJava].each { - it.options.compilerArgs << '-Xlint:unchecked' - } - - checkstyle { - configFile = file("${rootDir}/config/checkstyle/google_checks.xml") - toolVersion = '6.18' - } - - task javadocJar(type: Jar) { - classifier = 'javadoc' - from javadoc - } - - task sourcesJar(type: Jar) { - classifier = 'sources' - from sourceSets.main.allSource - } - - task tarBall(type: Tar) { - classifier = 'dist' - baseName = subproject.name - from subproject.configurations.runtime - from jar - } - - signing { - sign configurations.archives - required { - gradle.taskGraph.hasTask('uploadArchives') - } - } - - uploadArchives { - repositories { - mavenDeployer { - beforeDeployment { - MavenDeployment deployment -> signing.signPom(deployment) - } - - repository(url: 'https://oss.sonatype.org/service/local/staging/deploy/maven2') { - authentication(userName: findProperty('ossrhUsername') ?: '', password: findProperty('ossrhPassword') ?: '') - } - - snapshotRepository(url: 'https://oss.sonatype.org/content/repositories/snapshots') { - authentication(userName: findProperty('ossrhUsername') ?: '', password: findProperty('ossrhPassword') ?: '') - } - - pom.project { - - licenses { - license { - name 'The Apache License, Version 2.0' - url 'http://www.apache.org/licenses/LICENSE-2.0.txt' - } - } - - scm { - connection 'scm:git:git://github.com/wepay/kafka-connect-bigquery.git' - developerConnection 'scm:git:ssh://github.com:wepay/kafka-connect-bigquery.git' - url 'https://github.com/wepay/kafka-connect-bigquery' - } - - developers { - developer { - id 'C0urante' - name 'Chris Egerton' - email 'fearthecellos@gmail.comw' - } - - developer { - id 'moirat' - name 'Moira Tagle' - email 'moirat@wepay.com' - } - } - } - } - } - } -} -// END SUBPROJECTS - -// BEGIN ROOT PROJECT -task confluentTarBall(type: Tar) { - destinationDir = file("${rootDir}/bin/tar/") - baseName = 'kcbq-connector' - classifier = 'confluent-dist' - with project(':kcbq-connector').tarBall - with project(':kcbq-confluent').tarBall - exclude 'jackson-core-2.1.3.jar' // Runtime conflicts occur if this is left in; thankfully, 2.5.4 is compatible -} - -clean { - delete "${rootDir}/bin/" -} -// END ROOT PROJECT - -// BEGIN INDIVIDUAL PROJECTS -project(':kcbq-connector') { - apply plugin: 'jacoco' - - jar { - manifest { - attributes 'Implementation-Title': 'Kafka Connect BigQuery Connector', - 'Implementation-Version': version - - } - } - - repositories { - mavenCentral() - } - - sourceSets { - integrationTest { - java { - compileClasspath += main.output - runtimeClasspath += main.output - srcDir file('src/integration-test/java') - } - resources.srcDir file('src/integration-test/resources') - } - } - - task integrationTestPrep(type: JavaExec) { - main = 'com.wepay.kafka.connect.bigquery.it.utils.TableClearer' - classpath = sourceSets.integrationTest.runtimeClasspath - args findProperty('kcbq_test_keyfile') ?: '' - args findProperty('kcbq_test_project') ?: '' - args findProperty('kcbq_test_dataset') ?: '' - if (findProperty('kcbq_test_tables') != null) - args findProperty('kcbq_test_tables').split(' ') - } - - task integrationTest(type: Test) { - testClassesDir = sourceSets.integrationTest.output.classesDir - classpath = sourceSets.integrationTest.runtimeClasspath - } - - compileIntegrationTestJava.options.compilerArgs << '-Xlint:unchecked' - - configurations { - integrationTestCompile.extendsFrom testCompile - integrationTestRuntime.extendsFrom testRuntime - } - - javadoc { - options.links 'http://docs.oracle.com/javase/8/docs/api/' - options.links 'http://docs.confluent.io/3.2.0/connect/javadocs/' - options.links 'http://googlecloudplatform.github.io/google-cloud-java/0.2.7/apidocs/' - options.links 'https://kafka.apache.org/0100/javadoc/' - options.links 'https://avro.apache.org/docs/1.8.1/api/java/' - } - - jacocoTestReport { - reports { - html.destination "${buildDir}/reports/jacoco/" - xml.enabled true - } - } - - dependencies { - compile project(':kcbq-api') - - compile group: 'com.google.cloud', name: 'google-cloud', version: '0.25.0-alpha' - compile group: 'com.google.auth', name: 'google-auth-library-oauth2-http', version: '0.9.0' - - compile group: 'io.debezium', name: 'debezium-core', version:'0.4.0' - - compile group: 'org.apache.kafka', name: 'connect-api', version: '1.0.0' - compile group: 'org.apache.kafka', name: 'kafka-clients', version: '1.0.0' - - compile group: 'org.slf4j', name: 'slf4j-api', version: '1.6.1' - compile group: 'org.slf4j', name: 'slf4j-simple', version: '1.6.1' - - testCompile group: 'junit', name: 'junit', version: '4.12' - testCompile group: 'org.mockito', name: 'mockito-core', version: '1.10.19' - } - - artifacts { - archives javadocJar, sourcesJar, tarBall, rootProject.confluentTarBall - } - - uploadArchives { - repositories { - mavenDeployer { - pom.project { - name 'Kafka Connect BigQuery Connector' - packaging 'jar' - description 'A Kafka Connector used to load data into BigQuery' - url 'https://github.com/wepay/kafka-connect-bigquery' - } - } - } - } -} - -project('kcbq-api') { - jar { - manifest { - attributes 'Implementation-Title': 'Kafka Connect BigQuery API', - 'Implementation-Version': version - } - } - - repositories { - mavenCentral() - } - - javadoc { - options.links 'http://docs.oracle.com/javase/8/docs/api/' - options.links 'http://docs.confluent.io/3.2.0/connect/javadocs/' - } - - dependencies { - compile group: 'com.google.cloud', name: 'google-cloud', version: '0.25.0-alpha' - - compile group: 'org.apache.kafka', name: 'connect-api', version: '1.0.0' - } - - artifacts { - archives javadocJar, sourcesJar - } - - uploadArchives { - repositories { - mavenDeployer { - pom.project { - name 'Kafka Connect BigQuery Connector API' - packaging 'jar' - description 'A small API for the Kafka Connector used to load data into BigQuery' - url 'https://github.com/wepay/kafka-connect-bigquery' - } - } - } - } -} - -project('kcbq-confluent') { - jar { - manifest { - attributes 'Implementation-Title': 'Kafka Connect BigQuery Schema Registry Schema Retriever', - 'Implementation-Version': version - } - } - - repositories { - mavenCentral() - maven { - url 'http://packages.confluent.io/maven' - } - jcenter() - } - - javadoc { - options.links 'http://docs.oracle.com/javase/8/docs/api/' - options.links 'http://docs.confluent.io/3.2.0/connect/javadocs/' - } - - dependencies { - compile project(':kcbq-api') - - compile group: 'com.google.cloud', name: 'google-cloud', version: '0.25.0-alpha' - - compile group: 'io.confluent', name: 'kafka-connect-avro-converter', version: '3.2.0' - compile group: 'io.confluent', name: 'kafka-schema-registry-client', version: '3.2.0' - - compile group: 'org.apache.avro', name: 'avro', version: '1.8.1' - - compile group: 'org.apache.kafka', name: 'connect-api', version: '1.0.0' - compile group: 'org.apache.kafka', name: 'kafka-clients', version: '1.0.0' - - compile group: 'org.slf4j', name: 'slf4j-api', version: '1.6.1' - - testCompile group: 'junit', name: 'junit', version: '4.12' - testCompile group: 'org.mockito', name: 'mockito-core', version: '1.10.19' - } - - artifacts { - archives javadocJar, sourcesJar - } - - uploadArchives { - repositories { - mavenDeployer { - pom.project { - name 'Kafka Connect BigQuery Connector Schema Registry Schema Retriever' - packaging 'jar' - description 'A Schema Registry-based schema retriever for the Kafka Connector used to load data into BigQuery' - url 'https://github.com/wepay/kafka-connect-bigquery' - } - } - } - } -} -// END INDIVIDUAL PROJECTS diff --git a/codecov.yml b/codecov.yml deleted file mode 100644 index c644d5794..000000000 --- a/codecov.yml +++ /dev/null @@ -1,43 +0,0 @@ -codecov: - branch: master - bot: skyzyx - -coverage: - precision: 2 - round: down - range: "70...100" - - status: - project: - default: - target: auto - threshold: 1.25 - branches: - - master - - feature/* - - patch: - default: - target: auto - branches: - - master - - feature/* - - changes: - default: - branches: - - master - - feature/* - - ignore: - - config/.* - - gradle/.* - - test/.* - - .*/vendor/.* - -comment: - layout: "header, diff, changes, sunburst, uncovered, tree" - behavior: default - branches: - - master - - feature/* diff --git a/gradle.properties b/gradle.properties deleted file mode 100644 index 1e4ec7128..000000000 --- a/gradle.properties +++ /dev/null @@ -1,3 +0,0 @@ -group=com.wepay.kcbq -sourceCompatibility=1.8 -version=1.1.3-SNAPSHOT diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar deleted file mode 100644 index ca78035ef0501d802d4fc55381ef2d5c3ce0ec6e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 53556 zcmafaW3XsJ(%7|a+qP}nwr$(CZQFj=wr$(@UA(+xH(#=wO)^z|&iv@9neOWDX^nz3 zFbEU?00abpJ7cBo`loO)|22l7HMDRNfRDr(;s(%6He@B!R zl#>(_RaT*s6?>AMo|2KKrCWfNrlp#lo@-WOSZ3Zod7P#lmzMGa(ZwA{NHx8{)|HLtOGBmL<{ePk& z|0}Aylc9rysnh?l#3IPVtoSeL%3mP<&r3w?-R*4b4NXWG>5Od*ot=GSWT6Hb5JLAX zShc9#=!2lw!t#FMI}pFJc zw6Uj8`Bst|cD2?nsG(d*ZG#%NF?Y80v0PGQSJPsUg@n3BQIkW_dR~d>N{{*bSH}Pd zIWdTJ#iH#>%S&)$tqoH6b*V7fLp<>(xL_ji`jq2`%oD)~iD7`@hsO@Vy3*qM{u`G^ zc0*TD{z`zuUlxn}e`r+pbapYdRdBNZ%Pbd5Q|G@k4^Kf?7YkE67fWM97kj6FFrif0 z)*eX^!4Hihd~D&c(x5hVbJa`bB+7ol01GlU5|UB2N>+y7))3gd&fUa5@v;6n+Lq-3 z{Jl7)Ss;}F5czIs_L}Eunuojl?dWXn4q(#5iYPV+5*ifPnsS@1F)kK`O<80078hB& z!Uu$#cM=e$$6FUI2Uys(|$Fxqmy zG@_F97OGMH;TUgxma36@BQi`!B{e(ZeayiDo z;os4R9{50YQVC-ThdC9S{Ee)4ikHa8|X*ach%>dfECip|EPi!8S zDh{J&bjYD?EYtrlYx3Xq_Uu~2x$3X9ZT$tJ|15Qq|5LU8AycBUzy2x~OxU04i>D z9w@yRqlcbqC}2T_XT5eNHYx5)7rtz8{DE*J?o>>OiS)0JC!ZaB0JL-Ob1w)8zanZ< zR(Xiz3$ioy*%XQmL-bJnNfvE$rI2P~LX90G#gt4nb9mku*6S{mqFw`_kt{LAkj!x21fSFo(-^4px?_hH9-@XW8zqNrs(RYSX5R zn7kQuX>YGYLyM(G>^wtn&><_Q!~W27r537fQwZIqYL965<@&T|=xUF6c$g=5 z9B|kBeu>}r8R@-o3b!=}4_HG6sot1tgjjbmglPS~q)5GX6CU&gxsD0v9llaw7Bh7W zG`o>aya0{@c}L+Gw`1PRqcl6e6}@o3Bcd#mP)9H<2a|Wi{ZWqCzX%93IfRpvQ5Gba z7lEPC4fM4WC?*W3IpV-cRPh5Sc}Q>vS@2qu<+V(nS%!Sm&*^W!gSj)# z5h9&o{KIKp2kov&g`CP%-CqAqA#o0Mw?;q#0Dk{<4VeG4n2LHB+qgPgx|xbu+L#I& z8=E>i%Np7lnw$R9>ZhtnJ0P3l{ISg3VawG!KBZ_pvN2DYtK&W!-f06 z`*U{p=QkVw&*us(0Q^xhL0e%n5Ms&j;)%FBf*#J>kq82xOVpI4<0WK)`n9DXCuv$A zfn4!kd?3Iqh$3+WD+l&4vj>}m@*Jom+}vj&2m=KQGoVRm7M2KY7**ns0|M5px)Deh zez6~hUk1`@NgO%XoGXd)&6$_Hs|(2|X^7HUDkEtbwHV#1wRTpbb)rHlLu^njhFg9S zx+)}U8(USDXm>S%pp;a_Y<5>3i_Hp_vWwtzt5uj8ewqTFEE)E15)Wjvv?x}}8HMiX z;^3-OH85AzcV_0O-Exhrj`RpUZ;j$qjmZ|L#+*_US5`JV%8wqakxhD&XCpyuWo{N- z+bNS}p+afKlpHI>3VBBeq|G8boGeUaC)(Ru3u`YLW30>~)5=GL=sUjLgu65%VcPGs}PA z2_OLv=2)9Xm11f*FTt*o*yc8FG>4G~q{mOUX#}$!=u>KSGyX(=*}&rI;2K(U?Koxp z7F-pc*}}pO@m;7sff=FGTE4TA9ZNTRx%XWeaa|lx9o$qjHByj0HxuO5TvpM}CwTW> z#R=1vZp)76kO?#z;(>6Mu&gCwrlvRCVG_g8sMl;^DrH)&-*)v5ZHl3IWWpPi!|ZNQ z4&vdL!lWNaYH)lo!KJkFQfoCqF_@w-in(c2pNkpCKo6my8_yVs_Uj=zGVLKUT#^z^ z-)|f>)fuk#(@A>3(o0VqQ1$4+z_E9HCQ7R^ z30tu-(OIxDiiOEkGpXw&zReM}VP+C}bFAvU5%L?0cQ@?`fBSwH7!4o)d`OImPc+X< zrwk1#`^<8L8#>HOQb0pxt)HxXg%o|3x3nsPjSioaPqZ^lnSNOaJHg}1zqdDur0PoP zRVh{xV61JsNFuq`Xd6MtK*HtXN?NH20{)o}s_-I*YU7#=qn8b)kV`MS%A%ewrx<5I zY9{WpWlK^G^SP=5nvS-WEy+2%2}G?;#q01CSQ@%UJgw>}sHVEQip4`tToFyKHmwTV z-vWa!(`#8lj^drh)TLYVZLU!F!ak3OPw(qUajt(mO&u~ANUN%r3KUzV%k%|1=7Iat z5Pt`rL>P6u2G|qX<$)j~A0r2ZdE%y2n!@s>8}^KzEQEj6Kc?A%>r0ye>xB@wj|1Ob47`2EH4(rA(O{ zU}u2kj}N3&2?^3EQ{aT{?2g=~RLM;{)T7k%gI$^7qr`&%?-K{7Z|xhUKgd+!`-Yie zuE4Z_s?8kT>|npn6{66?E4$Pc2K(`?YTz3q(aigbu-ShRhKK|(f0cCh1&Q1?!Rr=v&a!K}wA-|$Gr{J~k~ z7@gS_x|i#V?>C5h_S4>+&Y9UC;Z@h2@kZgiJ|M%c)C38h@es^Y`p#a9|M_8mi3pR( z6*QJ0&b&7q+!3NCbBMs(x}XlEUyQp~0K9id;Wx1KycVf%ae(I8KJgjc!$0vE-NSwS zEu2^31P|2W6P)+j90blNtRJ5=DmAN?R}TD4!&z=N=@IeHhDTl-!_-e0hc?;+-;cCJ zm~zCBdd&GjPVt9?QcvkJQtf#Mv5mGLq7;pHYUils+`Yo8=kJB06UOcuYC;cMU2)oG zMH>rDE_p-R8=u3n)w%~+lE$>My@gq^RU(c_#Yk|`!Sjm$ug=Rfte#lnU+3im?EmV# zsQ)8&61KN9vov>gGIX)DxBI8_l58uFEQm1nXX|V=m@g=xsEFu>FsERj84_NVQ56PN z!biByA&vMXZd;f2LD`as@gWp{0NymGSG%BQYnYw6nfWRI`$p&Ub8b!_;Pjp%TsmXI zfGrv)2Ikh0e{6<_{jJk;U`7Zl+LFg){?(TM{#uQ_K{wp6!O_Bx33d!Brgr9~942)4 zchrS8Old{AF_&$zBx^bCTQ74ka9H84%F{rOzJ`rkJjSB_^^pZqe9`VQ^HyUpX_!ZA z+f0In>sw`>{d(L>oA+{4&zo5_^6t%TX0Gj0^M@u0@~^-f=4Gt9HMY&X&b`K%xjauF z8_!X>V|CrL;+a6gp zKd)6{;@wH+A{&U6?dAu>etSxBD)@5z;S~6%oQqH(uVW(Ajr>Dy{pPKUlD+ zFbjJ6c69Zum)+VkzfW(gW7%C{gU6X+a{LH?s2^BS64n$B%cf()0AWRUIbQPhQ|q|& z55=zLH=!8-f5HKjA|4`9M&54<=^^w{`bc~@pMec>@~;_k-6-b93So0uesmwYOL zmrx9lp%heN8h0j@P=!rO5=@h9UIZ^85wMay-2UO?xo>XOHLK<6Q|uyT6%*f4V!dYTC-$swh8fk{pCMlf5hw+9jV|?GlEBEAx zj#np5nqD`peZ6m5`&-xKetv((^8@xo*!!N3lmt=YUou<_xyn#yJp3Y#wf`tEP?IB4 z>Mq>31$Blx^|cr*L09CYlW3$Ek;PY`k@ToRobo6~q}E71Oxr##L$~JJ9_?1@As_if z`YlL&yDtoy733P&wytI4>Gd;vxHw2O@+@KgbPa)>3z8mMkyAS%Fna#8Sg!uWhMEubF;n{i3Ae4j{$p>dYj-^9?1ysjK~i0Q(4XUQE? zq8WLEcE@FsQ%hrS`3O$YbyPGkF6o;%&dxfHG?_n@Z&K4vR@ieBC{}cst~pIc4R0u& zj`QUL>5UQF@PgvVoBbRAtoQ_wyeeA9wsSN9mXX-dN^aFG=EB_B_b{U`BenI&D=;Fj zT!n`sy{aPu9YibsEpvrQ^0t(q&Inj%Pca%Yu&!K1ORT4wD6j-dc+{?5(JAouXgIy8 z%-H6Fbhd6%S=KCeIm`}PC!@`F>UKx&(#(Exk?s77w@&*`_tZ&sgzQ!_QK=DBnare8 z;)ocuEeZw)R1@{BuzGzIj$Z6EqM#s17Zv{q88!cq88!bXFpB=ZG^k$1C)OSWOnz4h zh&DA{Lx8q4*47TCo_gzx?MlHD(Bx{$87ha%T$XB*_{8uv@LhK>VV`UY=tPjwOandObAG0 z65^99S$7U)%^i%0Rnv*|IFjxg{!=`YHMJK^XV#j)p>*^S8FcuGV-BAwAU)a(e+)Wj z<=0$&0zB{usg@89sQBDI-|(HM1iz{8?zwn?5-k8jfM6Uf#vp^D4ozQhw#0tB@N(_V z5G#8|@Ta&(7#{whu<-X6VG66*t5~?Wlg0j8JGkpMEo%Sg1fExMxWXFTg2;1a+bNC~ zMiFaxTcU3ZKjv)V5kM}`LLzVunn%c$N*BoJj-NZ6`Q{g=3;*E#!f_{#*C?+ad~5zZ z=keRIuK5M;04KWI+Ycv(7YzExxp+b(xFaY3Z^kf3mPKNCd{OQbO%F%7nd8P(nBNon z_?lN|<`FF*oN)KZYNm_512Er;<8GEqpFWsK<1M&j{|B zo5C*08{%HJJyGfROq44Q!PMdxq^&J+j?ahYI=`%GLh<*U*BGQ36lvssxuhS-weUq^_|F7sRH2KqhQ2}MFKYfgn|}o{=of1QHP+(v0l0HYK}G+OiNO_D__5DAvd@{ul69am-m8ERsfZLSCNp9cTU% zmH*GrZ`geV`DBTGGoW+_>cFiEGR0sT5#0!Gq3u)$0>Q+2gNXQYFn7##$e~T?O6@UKnaPmHYrr;IL66 zpHCH6FCU(hv{CKW&}j6$b_zL?RWjo+BMls3=9G<#5Tzqzb=To%u9RQYw&j~}FJ@T0 zwqYi7d0bfhOvCF+KQ?e8GFX^6Wr;#sLd>z=9rOo+Sn!Gx#S!8{JZOiICy=>JL!*Db z?0=i<6a%%-Qb$_VMK#jDzwycH@RdM&ODTf(BM+(VE<)*OfvATsOZ?;*Z|+KHl#LYV zwB(~69*ivMM^es;_qv2a`F=yr7hG(h9F_QsJdxq1W);`Gg)XvElwdAOhjO9z zZr>li{sH_~k(_n9ib4ek0I-7t03iF%BB@~LVj<}4Y-(%tUl(nv+J`Z=I^xgjDynBP zN0jq=Yp@Y{EX@X*q%wsh^8JcPZT)X5xy=r1Yhrts;iZ@>npp;KAbS=u^ z7C^t_c%Z%wUF|lirC0D?_B+enX?Etl?DjuDbKmTMIivlD98rUKIU`CqV0Ocly#&IF zVJ8$a8*L_yNF&jX!-@&G+9c#)>ZeLLirXnS+DtWKjc8+nJ|uDRlm6xpN-+4*hewV+ zK>0BT%8ou*`H3UuqFuNnXC^;BIAixsF!~XP(TYBlVf14Qq4mS}s)|2ZF#71(dk7cV zj6Tw*_G9cDz}0~ zXB=I`eTPx>~gi%8(4o7@g1GNnp$hJ_%Mg1`VLZDvLJeHGr+zT1&yk_ z)dbBKq?T{~APy~$Nlig_@z&C!xIWPDo3m~uxHe!qrNb26;xt|ht-7c7np#s+cje~J zZ~taj5)DfMbEaGGQw!+3dN0G2S=fRaa3rl z7Osx|l1jjjIOhCoaPxPQt1`ZxtLxIkA`VmUHN|vTlJRWNz<2C9m^>k4usuSUG})b%|D<wP^rU?JNVjdb*1yWsZBE8HZC}Q5va#I zsBwfZp;FX)RpB3EoWZyd4Bs{TNmbQ{0Kzz-0SgBPl2=f6IWi{9_QZu%rTT_|l31Q_ zycR4qyR5Il(L|CofDAL(ez5(KmRFo@U&>^{qK1eq^QMA`FZE_d6`2iXL�H$uJM z5b&uBBCA_wdL?^xw19P_F!l$XIUCIG0(Uznb36A^l7CS!0R}%?tUXwj0HwXsK4>8v zWE@fGYQ(q1F-!wr2v#*y7wWza-i5khqjQYc`6WHxhz85!iY%{Wb*z~zziBKpL+~P= z5yWtFJwj0m!TPZcI??gVUnnQOG_s*FMi>bxB)n3@mOYG~$F8 zl_Xm}#nH#t1z6WP61iq!0zB{Jh{o+KuI9xVM*x|TC7COi#tnUn_I;MA4`P!sk}}W2 z$gGS}m_|3n{2>Nib`R}0pU=AR9)Uh6;G*?1T2ZSB5`4PjrO>Bt2=i6u=qr=bN)Jho zMV?Wtn1yFbC*Io^`FFE6o|ePN6GG{zD$mtIc0OSsefFkNdF;nI-VNeuPS?6%IPVoN zZsFOKggP&tnTdglp;!r1nb~ME!H<>dW?N62A>Q1QI7WDZr;ehh?{L3L=pIMlpL9<- zCZ-fg1i?An;l=twL*C@`7quCoH<3MF6KapUt`yRJpF@_5T*SKkjpGkuc&h|H=`ud? z`ZbMU&m4ld%TU}+A+8V~1;8C{f84t#jj{05Rv(nfKmS(5<=Ac8!Twv+zNQ2KAo$N0 ztE8Q?i=mCpKTj(+=3sG#PuZ69xtt)EQ_E$H(y>G9(Tc1>K{$_6M z*(L~w^!?vvr`|bde{$}8^!2_!m&7A22>lTX_-4~b$zzFP^|OM2SO6_YC(5x3nDFZF zLEs;<=Rhe2kWFopSdxKt#+6GlvG$4b&}%<@1KN1(I;X?0JG+# zOZ+SI(Rz6pJnLxoojp_o=1!h~JgSvFTm#aA(MK;!EfdNVDQXa* z&OSYBpIIn<0tfRSotyL5B*mozW{+MLZ6NMLdlU~=0cuYk{B}v^W)@XIJ)rGX--$xE zOcvV!YR_%}tq!75cM%KJ4z>o<-#?T-I%Kk_LSFz{9lHk$0c_9Q_`|<#-aCblZ)o=E z*hH(RzI&AO5E03$9B2e^8%VO=Ic`s>OC%|BVCLoQQbv;^DMQ^Uw~-6%GO^F}H0Q~q z^f33U->p7+w08Mu`8u@@tTTdOW34aQ*zLPo3M*ZgM$1;R*;#AtJ6(i#%35VYXVR~_ zpR*$Hu4*h>k<4nGL6_ctd(c>3Fj`0BNeVt%XZj?1n3pFSWG&#xyR5p9Jv$6nTu7ep z?1&YWZQu<{`E%?dM-RU+EZMY2%EDea9xT>s>$*;qAlk-5oOIejvmMX=Dq4!!RUk=a zamTctj!;C0!kjqf;w{^1TIo=<;5h(Fc&cSFE^CdtNLq|vxH@9x>|8h1&ggl0X!ym_ zxDkU%TWQgqxL#tcz=HsPkx1(`m~!V*zIMr!EW@nJ8EsF5D1i?_3bVt6HC-~|(pC+o zolB0hY3Npl)MYwqOg)KHp8bH;7}-IT!ab|vHd#`jh;fZ<<}KC7PEI6)jPuAiRJGC5 z2&o+9RNmrt5uHY7Ei0NyCNA<4mLnKiFYNv_Zb#Nii3WTZ0arZ8AT4M0>{%QkfFKHD z$$+eh87@<>*<{1qeS%#EY7=9pnWpm2e2)YsTnSN=OZ;bh@jzvAJ7{9b^qHwKQXd&- z%P@H^nn=iub17MjB9)=GFUvK6%wfa84NFp5%?$!9s);AdXonKo1(r8TF-+CxrZNsr z&~Nv31)}ejFF>%}r3{F{mBb*6PpWF=m1;g?!&1Yw@g9xX(CztT)5@3!PJ$MraL?jJ zjIfepZ3R}0DTSdM7v5{g4CqqENzH&qX~|~OOAZ?k(03=3VqR=omosOJO0#<^kry}S zMOVziT*;@o#igZ%dH=|V33S4P3X#diBc9o-J2t^IYq9m{K7GEtHmM_yBtV6$dz7+GSDI~g-K~b{o`Ud#% za0>r2$Osa6KCfwq^?pc*f*-YeG33x$$Cz>r@k4A{>e&zlHn~AYPNFAkSGe@|SF%2qflcY{3Q}TP1xU;;lixI`{PI_{1MwPU# zb8@!|+^PX>d@Px~2o3tYZS<^mg8`s&^A%j$#_ecM)T0-=M6*JcsBjG$6!qH-)6k^r z=hP|(rciXq{A45YWNjc*3tE28s-&}Y*eX(?Dl3}SRu~$6>Iiz?;9=wGO3&_yuud9e zI;ydoyIqTk1TB7ZTT{o1+!@^A%5#rZX4&G?bC6Vjp}Q)V%s16{j$h#-0dMi5>oaC* zU7@wAR|uZ!g;*b6%$SP9WYJtzOSYZDh1c(z!EV*QKzo%BvfbkQv*RPPRQm&M)gPX{ zsGE;rsTtrJ$#Y-96Z*&W0@1o8i1XD}SJet-l%J+a?+-Q*x7&~$2T(*W!GkT;zTp0% zNA(Z6)VBxSak^X6;6eB5FV>%~$+vsI)VmXV3FrLDw`e5ziZ6n180=s3hq09zred)+ zgJxaVKHB88?P~L<=_F^?2OWvaMvl_Lf>sx1GE2t38EFH4*y%WGwX9|A`ZH11xDv-% z3(>w@i{-S_vscw(nT*5!zMm)OY9HA?0x+)$lY58XGTd?$B3bT8G>2Nx$&v++LtnP3 zw}ctz1peYD;s&U(-^Myl#2TRgMq>XF?%dT=NcS~K*x?!t!7>qNE z#XC*r*1Tmas=7$c($69)&0Q|gv4u14v;$|>JCPh{TE18`JLEk$4XUNT)N=8{H?x*& zvob>*k&1|Mkkd%B@&YU_Lcn6yuNS9U<3xC>F0xW3NJsSKU{z_OEIUWa!kVhos3p^e znKBiVqZGn&Zfiz_FCObw-B89YT-{>XtOQQPL1W`9eIoGH-yu`;QO593{jOJqGn?rW z=RZk&t9S(Xl|LZ(OCOgW*&y;4vV)EVx-q4}3kS|HZRW|V9K(LmDf^v;cNIA<6Xu;r zr&oQ^+#ynltMZM`QGV&B_LCdX;Ne^G^-p>$C`a&0*)GRI%e-E{tr+g{@f;iM4wUfPv7pnd_ccS(@ z4{d>u?2E(%@tJmuYw(j8bKAF*cbJo=l*&?B*~c9JD0L7D9LGrhr;Cdt zncS<5VKKJXK?NvGezTQjVUEao!!?}QQz%e#pJ`pN*=dEnReH3bA86g#Q&aLzn9ReZ zzJ$1Y2xzkQdOGVMvC7*9JIRk=IPkJQ2Q3hL%S@dl8N9sAYwsaPHJ_V#Ur9yFWa?cX zjz$+PT{j#E`o?A)2J@8F_`LjHqe`B}I=iKBH6G%zkONe{6sF|Z1v_YQ5&iJov>WGX zipwqW?lIMTBKC>nGA2tsNMx`5CdJY5t}Sz&K$ILDLDC^Pxs_SN&B&jwR}-G3CYZ?b zgKQIgD&Y5pU|OO#CgM zDGuh11j==SAiOZK7m6XE5XW7K(-=sL% zH&+Fz#zLnR(xemV8{F6vc-V`jR7;uVCP}E6Ih=qbmD+TbZ0%-$&Jvj$24?|h9`H!y zP_Tq~oX$EP6%+(9dat$vf8(7vrhU`tFbifgmbiJH(c??;^VknrH z0hsB`p0zIK60yzL%uq8HIxikY-MQKue-X0Bb=6c(wEk*{u0TF8t-_|Q3?O!7wDN;z z>J}_l#!p35Wa#!8&${i&4N1dhNxC7AoA!|VwT*p2*5ZBdic8_~ zkfY8g0D2OPVnL0=o~egN@WK#FU(X>U<#}TGn5vFj1{rPxmoMy%^)Wv?A{ASoTusuuqHD7a5BYf}yH8T5&ox(ckKBEO7Rd?Y?Lp&5oNE!c_F zq_zlC1$F{`-KoyC!}LT)RKJ8?u*ioiyHCbjkW@hWoNawAxb?(^dk1pHOkmE}1>J0> zG}DEB*XNnF=GEwAtr6@@RUF?=NFRWh9Yu~`=$C7-iLKM&68Z7$lSa2Q*@8# zr=^)HLw~**-4mMU9p_K_q(NUfgw!mT!&mU6UzRR3?O6+Kf?Bml+DG)4;NHTg#V->s zyl2!8bbaR#xq4a%wC5$AyIvN$3K^|=d2<_Bszp}&D?5ICjvp_Di}EDG=9VygTzAmMB#^O zss~=SJf03Zqu>_Z_sevE`Gw-k0H0vQK&)s_8m#@KSCn1IhS-8236Qy3u!>h&Myz`1Kd8B~HlYtAU=gA11kqTr1`MN9eyqp7elU7>IHRBL9eHY4UWJ;U)t{yN*Rm)~+ss$M3* zIi`3)<{@3Z1heF9@JR!C+xWC##A~Hh6;Jo%oqCK$fPG6;Q%&iwSVez+S&H&4Q3Lap zUzp_C?Bd3k@N0J(XK%I*Y8R~CI>_d(Na+h|_@M&n3!V+t$ONDV-MniLcA-)o=n`-A z<8ttu7TbY&f9C8tiFVKgy;}5p4$ktRr@!JYKa+g+S!26-yZ6r1b6BM82c`o(|AP?0 zWsdI&53A&;EqYJ|$mNdP4zuWK+h<-`H>2EvRYzSDeze~owhCzF^0Iu^xV^Sv!nqE-4@O&@C z!xw^61W&#Ioa2BSBx>;v{M8g!r2;OpS_^Wo%k?M z1ce90s~<)S-q0se_|)Ik!#!_j=fCxaOQcL`BqD`8@WsGWMqEx#v)r zTb_n1GZNvTYT}r9Ag$(i!8X6 zNU$YbD2sh6*}S%!#>qseXVzSBf>J|g&tP1*6;F(7o@z5yBV>-A-B7jDD$%}mKu=Sk zf%YTL_D!P3ujNo-A&!SXL@>`t8oeE<)7Iexa;)be(pOWnJo`y_%5?g?Bb{Z}ptE2I}2DbF^CCr)96 zZd?xW*TqH)B}#ln^QHMl0vFi9DB#20TVb)V^Qgcn0)Pn5QtC|S*aXu1d0YZVxclWn zla0V*_UL8ZB}?}GpxUEvE}5UU{g&yp2-u3POD?+vzbH_ZIN zRg;d~&1^c-`zGviyarVb*dbjO!waqeW4;Cq;S+k3wYM35$?xwUuWHYeBT!~ui^?u2 zDTZnl*=D}kWhrQysw44&$Nj-HI2T1J7ejOO7yPtWc&(=}{Xst2-Xpm5Hw^?R(nORl zSOwG`MxuD_>usNDbhm*wP?Gs$a<)_xk^J>MS8yA#9>Iynllll{WARg{G;EHXW5~Rm zL-|Z^83y%jy-5Zok}|{6-5&6+f3dejs1#g2J()gyET`p4#!=Gv&R=kKKGLVG{l$(k zuBnqP2gKL?<)D89(n(*PI=2Aj@{|2D7901rk8$xu|E<3{jctG{$?BJZ`OP_jqll%=o>SRg|iFp>7h4N6Qe#g*&gbN`CDKxlneuB#GKMN82a|&*-r|8(MUx|XCNs?v_@JrwJ}g0 z1b>lmV2^)q7zrPHc~=+}f7ci!e^K~w(iTHcLQ(?qQO+vdSOVfHybl9#9F<`NjAfiL zpzfSzYhGQp%_aHC$W(cOU0HnZBS5*)rKKjoVXk#yv8|-c70uVW{NZaZa+h72-E7fR zVcaym*Yi3l2bwmQgK^|i|uC9JmO6AKTOo5vSaE7!I z7ZHBuWomktl`=e+6bx-^L31&#i>t|oUVeMQkI}O>)vi3Otn+MRh-9msb!l8`zjS>e zMnz@@b3)gQ)5J>%)w9Zk?$$!iRb}du99&z~D;Ki_0S#o?vL)fjY*wm?^GxM${*Gun zIEbK*(gVC5#6>583s9<3>=)c3k{hbUdh)$UU|bAPFuY&}(krSDl(Zn43%S=hmgshs z=rhpKIIsC!BgObZ!2HuPa&6Q#rAL%7pzPV<=a#n$B&0YL-_V(;Nhr&F=vu37+#xim z{vkE!+&$}q(@;FxP`p?e9ZC z4vpX_#JUbq>_JIgbvIfvrRMIGnav%=hkdOyHPk2j&C_|64`1BE^$=?XOI`Or;6f`i z%+&w0(j-K^MUP-Qc|Xl$J1UgL%$O@>;R1MDR;90qh}(>`OjQIL#PO^Ud7^a} zKEP||e^%jto&@%3V@I!Aq8DlAuW`A;?t{==&x;q%Ah_q{ix0630P2@y;*klP4#WSD zaYvrc6eb!k*X9f+Blw4B+{c_A%nYIP2d0RBGh&eqBaZ_z#;*Yt=}#OjhOqCy=#yQI zhLnTKKJa9b`vB$(Ao&k6%Y3HIpu=gwm5)Ip7dYg$+zm3+8Nuv4&&&(s1N6d8d!kDL zlIe#s9t-S|d?E&24++OCMt$N4hjc`}+dEZx>O6oyo_|611-z}D z72Qwu`{x!>AM|UH_ypY=KYux@1-d~&Lm`*!P$2dQUO7(kmUGD(27|Z}pD-<%rw|?YSLpf58810bgRZon-0n3jtyb004^rTxa-a zKd7jOsj=&SJqSxx_cXv!#rz}NG-1cK6k?auMoCFSYP&ciI<=EVEUAn&zGAbORkS*B z%c8k{9kQ{32LVMvK~;o9gd!qZ+b(zk77BjX0nkOz|t%ZyQwv6Ar9!-%hi0EWRDop&s8J{t(y0 z909e1K0*rT`AAn#<;Vb(bB}h&+k}H;$ou5^)5N2{!G|CKe)3JY>CrILmm~o5W0!tN z9QZxM2S4Fvh-nIpfqDROrU(*+G56EtRg<3&eRzWdV<7qQ+Xp}&Vm}(thcbX3{5}<+k7`Q(^&cHM; zpl;S8UR>zsRN-u#ZSFLxXXd&w^ZzvKkH|Sx|QW;}y zwwjPUwZ>^iUL(>(T;Vp?Oug3rW|qX_4^=p`p$h~p-0jjdiZAZ8#u6qq`J`B(vzM0q zNULLZBad0hD+w7&%@y->WE`Y&H2F)MZLeV;-OxonwCUHW9SFHb;wf~iO&b;(Y@u? z4%$Tw*5v5}98V zAZ>y~BgD&16*=U&=dz6A*+(*dzh4#d=V|EhLBCRaXjJAGzl4-l>$eh+yQQ<~dAmqa zl9#Dzi85)r)=V+bZkEbESsx^rK}j9w%QKNhO3EVOuo4|as4O`0gg{%5M33={#iFwY zV;t7oFqNM>lkPhc4SLqt@NKudj9#nk@;Mm_B2%2BatkFH9*8KcQl|t{KtSjgY z*dyH1Y4R-;uFe>yuk6y09p9}tk*IiQ^&8^Sb@1RwZbDM_s%t=P>0%2-4+(#p&v01E za#7~6OOU}-)7YC^v^1Zg8OOp&zdawbSLKP_iyYi*wnEqBrE)tmr5bIJ9x3%`j7r}x zrGnd+LZ!r@`U&7y(%e?A*VWQee<0^6K6LGn9LX2e#T!d7ldXD>cKA|dyXwhakc>^Y zU|}vjw2zC)R^_3#xlE0`peQcn#`>Y_{xiPi0P;tf?S~YbRn&_m@tTckq9Zo#x#_-- zXdr7e1=gl};Kd#_?fo}C;+H;8`Jv}5%78(8)LH9o3C7p&40<_JO;wcAkjx!LfDGk8DQwau;V^g~l&8@j40GToR?g^-kw zg`U~VD4<;(?gO>o8QOw*o2eOY%b-hogBy+^-P~}9oIk8=OqN)mPV%ErQIVr$u9Zim zPWVp?=}kFPByX$Q9>3O3){Eu(Mmz!xX_{dUCp)ZOqg4dAitL=*7skIWF`qgcKR`=| z73~K%jpmF&%RNio5*}ZrrMQ@dS9P9qEzVREVS!Mjv5?wQ z$NUT#V;GsVUyHZuVn+B#;-QoqrCZjcW86wvJ2!mql*$(h9N|>;flzX+%cPISgz!D)|S2qu8H6sywRqb zH0|YusE-pxerVLq91EJ(4y$S#*5sVlS{7Q1Vm^3dsVzb!C&%owKGo#j+`M5C)`bgSG;KJ7N}V}!HM{-L%%=~hF|}OP z4B=oEPu$ARBWjggMLMW@qnJ2F=a@E5j$x(taAwVba*-i(rC~K~U~CT&AZ^_$pKLC_ zcrJm`yAp)aa#0pU5qG|83u#T|UXiQLGw56RvP9?Plv-;wZG0inQw`1tRbIDlZMG=$ zS|gNO>O<1ZoG2U9Lc!4dAc0qg5MG))j%e(Yjl)iQ)Ae*@?MLAFvMW%2jj zZ2vR`>O-0iRM!3s%B4PpaPN0j&1YI~KjGefFmdX8yi?5`G;JSPJLX19CW%R>L$-2l zg0ubJ)Vj=k4Sqv6*<&4k)JnT|?F343%AoH?&=Y+|^>*VWRx+B?3toG)Nif@!Q1Iad zAo=-XKjdoIpdAq?5jDKyD4h?#;w42Jw}jb;b*m9wl&veNO;Nd&u%acq5R)&6OCxD! zcTzK&>e)#3gsx=jR&3DNKxMOeUipkG=-Fjo@&fs9jJ;EIW!=8+orlHDoo3JJSd@`y+1I$tN#2dj6pE~%ELv|P#LU> zoiF2g3Sa$N)aTgCV{So-dAT@qt|W;9pT34JdcC5%fP$a_bA0s+=%|1Bqa8i?P%GQFXn@ny5sv z$hoFJZ8|eCPH#@tHZK+Tk_}5%!xkj!5;*zf_RumpDb~VeFVHCD+&r(RPP=$s%-meK zfpkJYx{;+d6gVYZPvz&>>KD{MD&A_eUz; z-J>?U)P~OOTL_uhm5ERMn+V;@p2SyC3*99lwtX+3|X>OZn3?WV`e1N zXMW#8K>SF|`4Jx?KQ_Q1E%qsv(Z^0Ie7$A+R*LA{#tw0PH|hO)PDff)ym7Y`Z*&E^ zDZ+Yc_Mo2gbbJf_&bLba=M&AU<83pI@xe zAfIp-=gbZ;@$sWxHKEQuk7E3cXJ^T7d}w9M9Z>>&r;O?BDyV5{s3_nYDCrkn+umNA zOZiEk0Wn2Ny@?YgUS$IccYX#1?rn3#Sd`=nY;)0h7|LD6 z4JU?z?sUhmpzmdYC~N~f`AmT&Mf)%bA!>^fQlb9wjItGcQk(q_d~vMLb==xB60|tB zEF;4Y&$XPOOxnP^N)nQpni)u`BLp{Cu{|h{TG373ctzG70Szai zdfAf((wJP2MV02XykIG=+?}sw7xYe%t{B6UaVTXMqI!xa^+=NHM?&0k*l~#_s6E4Q ze)jCi&R!#Bp-eV%!Th|L=U_jRTp9|PyePmbxDD~5)DLo3j)xuNDrB1@@7j4;1@$KI z^*3w#-=Vm@(fLKcGAtIFAS|eawsoXFid<^@6CwsQmC@&vsL}E_w*8+L5W71w3t^A!F zl?Lt|G9LC=8i4Gwb@DA@+6j_Ik?3s1w|^#r>AzP&-KkbuNJijd=jchdM4=1O>X)08 zKux(&W|)oV8+Rz6@XMlw3dvGNmfk3{DF$t5h*cZ3eq{q4TKgu1J`^u!)RrnAr7jXi zE+v{qGR{^f0gk4a7baDwfg;VSNLGH@$aO{Y&X>RdrQ|@vZEB2Igd-?QyEG`O^kZ8w zy)4Ycu&uY5osWQ{YPMF;Es_aEC@wWyCVHVEufUY#pd8om7#d$T)hG`-V-tnXBFJ*( zn^lHck;P1$k=Wq;AZ(qI6ugCD5*jA_21gs!uFjz*zZM<6srgenF)rCbeo%1*xT?fZ z2vyO1MWI!`SmoTHmLg4U81JUm*YJ%Y@;xzaF~{IC_pSR0M6DLd?BB4>FuvCtXo10OHYn7xB7?}dW9r^o3f0noO8z zF>xgry-GF@6OL`HwL930GNbNg_h<-BW7jz&8XTs|i)sx%VBH-Q#88$Icy+pX!RTK9 zcxw^A8AC{E;u3X*UM@Xm%5Zh}4W*!o2PTvgPls}qtCt*d^J&#!4AO+hLPy4-JZ;0} z)T!r7-3@^#<{=_gkS+&>QH>fC5Rq5jOx0K0-*8oJmN=xdepoqZA&PgVvptyZc<;W0 zX95C&fYzzwnx0%i22m7!auQA+@Zw=&)|kCx@Jg1AVo43 zIOTE=Td=~Y&Lg0d{(~LNCgF0hE^b-V8o3hgviLq-lg|e#AySvbG7Ir|PvIiGjR{X+ zv?YZl{&p>S#N{aQt$fC97*TabZKq+3|BUl zBFl@DF+;NCYxCAoK=CVxf{-T@@t@oJ~7q;_6QAcfWv6uFimU(pZO(^ zF-0ufSPgBLiQYW+*)U8s`<-|_N|@r9^hVDn@C2FKoQ+7sxSc7#yoFr0U# z{|=&N0M`8FhB)*yhb_{b-T^_m=Syi-sgDEWO zE3~Y^lESRO&!w-e?yzhJP2^EcEXmhm{^vN{o^&=(9mlO_jB{NS8<_S?B+k`|W5b8tCkk`ik! zP~h89#WaF*P$$MsOLBLn(4~TKt}W=VgxtUi9R(u{^I_s56?k)T2=0@3{ANXIJhj$1 zsop=_rnp7pnDsO_%p48jW7TsnZtN62+zodXtB-J_dq?mQYM3?SYMfCnZ&t9ZQ2iD< z%s+p%U9>l>s+z3c{<^B~NU2WnysqvAu(B6BSm2}-)mhB=P@bmuALR|h=r}|(Yk_Ld zuX-YtlQG&CU87jzYOT)lgk64hU*=LzTZYkbSx#1!+t#_VtPf!J*XxIbz7!^VP2&!f z$*=J6Lo)4DABzQsAIElQO5W@6#@P3G({;4-Pa$L6xcRq3uFsoqFWi7jS^IF~k-0Lu zxVf?^CFn-|oMv@(tH~H%C1qN^JXBO)Si|rLX%Faj^15i~>OA2)9`zw>p6#0-vw38w z%^KUDx&}Vh7|lSweto0PKO&?3qAF9EBr}9l>_qB=Tbxp(zu3ZPNJ$)AB=eC5uVL^5cMRB{MgKHK|1?ka5N82HCX*|`5o0^Kr*!6s(rJl$ zUi9}JvbAXx_uNlBK;!3`uKyRw>7UW_|3ai?sav_>E};Wga5TetCGoy|Q49fRB%)cB zf`|DgC-jxaUyzAdZf{stdw8BGh9z53oRlIDDYvtqbQZKI)r}C@TpCxalCuyY##ms z9Br^GU+*Occnm#%zBrDsIt_h!DmCg5lM{?WO}oZmK1#GmU=Uf>J0>3pfW??`@d;jn zQ+MxF&^~MjP;FocZ4pzt5>BK;j9D=SU_v)HS4;U`<7O~6pjxceCb_})9L$|h4?(&( zeC{8N-OG%~Kd~r-7HX~cdB>EC*?_3#-Eqh7hzH)|UkJf;3=op9PI;r0b!x>)zA z;p5gSir0i{+gC)(u2$}|Z&nu|G0ds^P~tNfwe%-N1+A&pUu2%1K6B~K-NJQ_d;V$_ zcb1uGMXEV<$G1CiS02>P_rkrV4Dx~n9G^cImHGw$V9}~FbZ(d9eJ2labLk9G=H42C zLU~ggxxVqjC)`8g{u8=@;$65e|Lg=#c%F(PU~+M6z^K1o%pfO$OTPFkdI5+%DQ2%W zLcxjI_rv)O{Wz@+Y+6_?kEr=uFZXuQZppLE$nmq#$oAl&KW)1a6+wb*6q|}hgE0z> zqwhGL1zL5tJzl_+XYpE6b!@0lDs7aK-ddFRex=`|#E@Oi?NT-ES?$rLr>qLlj234~2cbg)dCFsEaUxhCoE zww0TaG%V5#wg_G`j+??MojaIy<4@DgatbDG@`VVOOyd4xC4jX{iP@I_$JlVdg=)*2 z(wel+EVi;yhs+uJ)R}`lfn&}0E!WdnC@b9hYfv8jKcP`aN9|S#2ut9dNuaAKa=6ZAS4Z`GuXW zT8W2UBIBT)zI;ivj1_UmSc%Dey)IGhVLhSUhYTD3Sk_cC$;-$9Ev5Te;LeN%zbX0{nOfuo7z*QMb^k3f#%fd`zl&1JA5gzOCnxado&-u%_+4DYBck!@s#A< zk+9k$Z`H@otY;3_U7CjqPDmA~Z6qs)ly>|;OVFp%{n65d)dIb~SkElpuf-SpHMw6e zfRe=kPA9%ALxxC(v9t~*XxUb!Lq#RoT>@WK&Pvx^JwpqFPCo-A0CN7ZYHQ37Hcvz> zEbopS-zUWaMV8I(1m7npodZ2Z^lX5#$)>j_3`s}@$kC<(LFp>tphVF-2BKU@1qTUrnmoVYOjUiM)UZ^ozdL6Q8~hHW%PC5LhQ zBs_;iO|!EG^~HCyoJRKM&WNq_0+}5r?P?I8Zapm0&tmRc8s87)<#tP-$ZJZ(a@d1V zrTi`?sO#+ER&s94`aX7NxxV=uEvpK(0D_lnSq}^(YQNYr>R8_F_`!a@RU|5gP0jRU zlO>{4Qc=(jk!(>lSwNA8v0Hi5I3235_G;YA2U$n9lFR+kRXFd6HXAm@kA^(kvGZ@4 z$ZPDaAfmj`$ohP}c&48ls=w+4-QE0RE{3%vMb^UvI6CT+zQU?DjNh@cSKjCB-U=vx zH|Mqg4CH<{#JV(T!4M|g+Tr^ok zq9qm#qcJfxqQ!U#jEYP)A}z3OBrq_kM8B8yo)I~w%=|<8WUZ*(zvHPdBjN5%vDyX0 z-v)NE6UL{$M)!O^9^(HI0JZrqBhC!68-dhYu_v9*z0&A$uGwbqSy6J*~BQg z7L03dlL1HDWS`Pr^}s=9I3E^bL^ZP)jG8|PDdLFKa3+wNpkLg?TV{Afm399sb^47Y zI?}$f;mZOnf#RpzrpB71eCy#YID~miHph#Te>sBYtvRHA(;8Vr{hS^?_3R0#EYnRFnTZ;&44bWTgAcK-dcy~?t$qUrAwTw<7ryWu7g=J$OS(UT zN+cMOR%{Ss>N3KF2ZMk6HQI{yqNOU+paXkg_vATjx0A;%)t0=hBbhGG;bZXtU-|dm zEop(9oct!8V7R0PpJiHfMaI=9X%ZKKL<*)ttaxPjQ5HXJ1o5)KT)QDie_5&oL2HfE zcJ1_MV^vB0aBqIq@ri@}rZ!&u?4XAl=cL9_P`ADWbPVBA%qf^APzGsGm&d5MjZUY@ zX1EsL)!D&nc(T>&Tck+M{=Syeid4Jlw`cJxG$2QmnT!!h52Mv8)WcdOW^B@8150}r z%6)i0m)C>n4n;%AyjiCj`lf%!$JL<~ruSEf}2q{)TvJDv4E8I!H5|tKJ8d zN;J!19IOdr1O^#R`6BCqyzAlhDiLB6PTOJHHQUOiq}(f>Y*t6ZxwzY}FjEt@M#WaE z#n~pj9y}fWH=Jy^_t6GOB~hp+lW*3(wsQXGJiPs}lW+Zr#Qk>TYie2|9F~W{ib_ZH zT1|J=LCuc52_76NZfTyvKXP3JoCe)jR@})ZWJsw34iSF<&Z|t`Q#Gpy$T`Qn)!d>^ z4=Kqiqg!)iu;|QqpuuMX(#RB@(l-hbnL(mj}F2LsgwwtRm$e z;>p;v3>W6B5e^6~`+PV6rhEexRyU)}uq-#Aj-Q-@FgU}0363wojO?NfvC8((hnsq< zx7;u`!puGdHiIQ+L;!#+bAd4m2AjcxGY0P9*ilZL_j{BI8~b2ky3mqzf1l`FC+$8u zLduO30@ck)Ij49|NI>Kd^Jg;OqTLmD)nOBao<2L1H@N}yH@yKu5k|sZ!nEI!JKY!0ajCD+xk}j#bA0onRWj}^<*xn%QMxQG_tvgu+zmapC zKg6h4eVcxj;O%PZNxjz8a+uVpYmTq7NX|(GICWQj-E|AtC(i2yS<|sk8>(yv2o(zU zj*pb5wEJ`jcKg)mHDHVeWeqqLw07+TJk1Ox)A!m*?d9g-@P^#;0PVdw7#QsW7iyy} zt3}0@Ej5xGSXJ#8?waSy(&*hQwxb8{WK0($)xL_g8qK6xsn^ainS4zuEmZbOdqw5h z^|PAVR3;AP;dc*=J6QUSvmK=m+~rYlRaJ4A^KxbtZT6K#lm?6qJ$xh)q!{NROG+pG z?$$=`v=#`^iTiaa?Zo-Fv&gR%I@4!oT{&~hFa=UFA6!fYYJ6g_`hSj(v*D4I6X@;A z)CjUxE?Xrk(^xGf_%1Fn2wlV)nh7@H&E}?C4>Bej2MtO5A-ioUoJ`P4BWCv@d$osVx0k5HbVIb`K9FSZDdmXbO+FU(VmfcVWw?4a^wERqZ z0%yOzT&+d;SdVZzwXMwf`aGc)US&7jxIATx3cGD4=>XEr+~F-M(abJK7bklpZV6oF(x}wL*Q}q_dWDYFXW0)b1?@Z43nRbxCV<&Fg$- z5FIy<)2tZE6Om?vBrl$HSa-Wp^G!321jwK`v-Mob-y^7Wr;;k>gIKXnsB#?`-M`3& z!I{g=T1}w#e~r`sVg)HGwt_g0;@8SXf;o$Ei&<;SI9p%!lFwWk5I~RBMY(V zJ^K}>W3fAQeiny1_x`~z`%$e0qm~Y}6`l;0l4#ux8|VY!oHZ;PsP*omSt;HqZRWlR zB6k-I@<;dK)sTdc2zSs=hM$?m-^~Es)sWOR?&~$VR7V^0=p1sJJ#O6gK+sk+xJO>X z*QYoH#I|RmwP$GM7fJ(8NmE`?TV7$-95N6Fg?(O=8YS1@`V~sA!1@*#00^CUOvMeB zseSBQWczm@0~;qT8Z4+l{ASD_tp%RZi>wTSCY*M*IB}=uewB=4DI^v-<=(w zlT8mztmRo1Du}aho(8}ElpxB677Mry!i(F7DdNaBM|`X!w%I$ri9Q}LyS~Ajp1tjo z5d@{<-SQ-GfkSFb8oAgf76~s7|Cxk{w{wQ4+$YcHvamH|Z2)@I6+u;P2Ot%wirk_6 z0BvLwDHTiI;>XCYOwl96=;V|UqLYe|Of!o32>N0{&3^)D!Zb*I$(R zfAZ_;-2Mqxr27X}-u@GdLvR0o!0XD>Q}R?(lByDtvJ;aNv}2Pq`$~^fGs^a~luC@u zs*H>c%&d*f%xdV2kOq9Uy`STz8JE7=t04 z|CF{%DAr@Y5X%>2lqK!%QIWi(XNl1l)$|!TXi7M zo){E*mvAjx*_@2YqN)4TM3_l9j?ANMA$G{LD--m-NEYvxLk$dEQixD|c;r$l0cO%; z9CuTj9JPCdIdx4+F9Nw98zH#$m$r`0Ns%XF@;3?>C;t|8{OdpXeC_{J7~xa!{iFK8 zzbXqDSzG)^ser$3j~#tT=KZ8?DSy(onEw0if`)%Z#EqPV?QCp5A%Zd%wkDs%OxI70 z{(ptVlT>s+nfYjZU~myM&7n3`+p|cA1RV%v+kV3dxNR2FF`mUe|3-M_WJvKfgba_MxO;Fc&AQY{-4lU+`y=o`gKO z@ICM$@I?XcL%(!1O+t_EO5nAC*YmZo@Kxguz<<)stuPilVX0HqWt;qoV0*>*TMdkDTiha*-sp3LP?b zAOR`-NZW9li*1_jgwtdTTE4~v%WB6Xc8duYAwVL63~#=^IW(YJa^8x5iH~+P>WPkN zC&0i;uXnO<8;S|7>m)G=yOJvSoa<*ZrG+u0o==^}kM?ek*}4(?ic{`vvXFr43w;ar z{BbB}Lh7ph+Hgy(b|INkII#sn*o+=mRl)}KUp7CMB>Q`90Fy2&Ng^=6B~v*i_6QKM z!#Prs0gIjFfJ-uw;E73*r686I2YI;+A%r}Xw*ziLVOOV>8UNRL!@fzzP94t17ms+N z1{Psaw?E`6)Obyc4_2D5G~d1poou5JOHbvoNp|39im|J;g8UYgLvu5ag3`yKX(S){ zq9Gc70hE?Vr!APSQq0c(Ev81=@d6hYgBhBQCPiu{7i9R6~sH#@ZA%TU6(SX zrr+}Kl&!y-BJ&TEnBvbSc=CDuEu{Nb%l)?|s9@mu37!8hUp6>W@UPMpq95i>T5zt1 z?V(n}GYV+nqJ3WnT}$aKKqY_K)ARa=pepOM+wK+8oTKrHPve9nb;I_HcJoOKKO`j2xWK&4P9U~HBfTN9ymDTn-VlD#rFs8tq*4-s z!7u&nc2A!UH1B`!cK`idWi6bXENso>?f+Vt3p$#89@ua;`BxGnNmqVBA8q7ghP}P& z+&Gu0n;A2)i^wR{-=92yfk}?FPd`8%sWOcXs63Cc&Cq!}jQdWcCy`Hj+mEyp!kk?~ z=Y%UgoJ@YnB|r0$wbJ+x5MFK&Iy%#V>Y!q10xQ{41vP4FvY9B=ln4{<5F6ysx(kA| z2-67T!)ii~{l?rSLP`gB;Ny2_pdL%x{t4oM&RTuNQ27*1vEC+A)Ly!3g@Ym$uF%sv zdGz;Ws_}4Q_$Q13p=QGGwh6@brmB=Vf)=ga>Kn_KCEgo_3A^=815>iLxJpQfq*ri( z^Y|XdoYBPP{CCZ|2<2KA*`ng|)MTprb}cUR)+>JEiuH#nZ|Dr^Iw}#k)v~q|ZFB&} zmI~$`QU>h!WOG4lm+#L0k1Ov%WXp68Sk!aO+e>n7Zb%C_L?&V62_5-DO=eCRiaKT> z1NYs4Envw3o!H4#WM>iOVxRZlNI;_zi-XivwN0x$0sSQ|yZsml1zA!d@)#x~fxjIj%rIH1V`Q_i0LLMg z-S_<{yoFY@Tnt{m?~2hge_G^|t}fsVFDgP7yoCutdwQ`3(*|- zIq~rQZ+gH#o4)d=J!Nb5*+1+JKAFw`Rk$TfW#$vvjP}R0-Ne8q@2)_C81Y=Jr*~mw+j+EYB}u`1(rqd(w0R#&WWp|B z$PHMNN(19wbh-BdOX1-@n7Ijh#3*mVD{#;wTkl(yI#!M9eD#)sWjy&fw@(x5ULssc z#6>Gu$jRrwUxwn_gEl`vumO)I11N&ZVfDWl%BQ}s9}$wZv-HMhp3E1>l$S+1 zt-a=Sm`z;W)Gg#SL65?K?3ue{;hpnGxL2HMawPU}KlSkI=)EM`3!0h-`M1VpTO1Un zt#8Fb@jR`<1Qd=HqdW9-6C@#C2Nq@cB-v4+J%uun){c2M_^%}I^o*-#FTYr9^h-43 zDdj?@;uAB}7}?kqcV+8&;}d=*vj8ETVTa4~qwkn_5pNq(;cN(uj9JhKg}xLV@DW8U z5&`wU$j81w{9gy|ubJ(H6yZ+%Q{g;6I!tRD@#FBvz86bS^rg|D%46+KxhDCYi-eQXPn}=G!bT&Gpjc0)|)ThluVM+ z=yU;^n+MsOzky%x{@lJo?!Zr>!mctKY={Cy1ADoS14{S;Ui19q3Cl1QQ9R#O98g?i z0N}yWT&CcvIdHBSL!`x!&S(}zM-%>H!sV@F$A-jNH$gjtDbx=_q9Z8x0ij+g%+Y07 zxTC?a4XI%dXI%P7R4Mt=JHxb+=H_KRI>?PF?!SxS$))(yUY6~day9cMe-)vF7j;jn z^j5dsZoE#cmVHT73^Ec5&b^OON4fBw>X{H3H)?Jbf%ABWGd=u1368Iu^~*VXp=04n zMo{nKJv^GMg5Bj1QSDb5Q^ovidJ!k3kuD2-1+y9O1lyyl<8t~Itu3dP57=mD0M$?r zF_|?mSr(39<*?wo!vAj$`Cnf}0Mq3Bn;HB zaz{Hv_w6xG&?E-~1cUrkD@l(vc0&3RG22L-UkLb)D-+qcZr~;Z$-%Obwg!GNB&B@` z)SG2j^Qwbh_xve^D%82CSDXK9IbZ(c(c_iZ=XE=$iqFi{wIKso8z%7kIO9I+db8W< z_w?1!N4DRW?>t*cbr5dVxn#rzUyV>@u!%JyCGYM$^sM#p^mK~lC9#l5cAf*HFtelqM%$T+vi?Dh0-czyF$9rpC*i}W(F9`IrQ>+&vj!$LyHN{Jw{M1AUTy zCadsJ>96^;%M~g=`PfJPR=7u@K?y-?DZzO*H5O;C@d^ z^UJ#7VOEwcv(#7LDOcwX@(jO_?`<`LJ7=F%0$vealnikU{acm62CT56Ne4Fd6#MX2 zpRbTu#Is79%e0>CE;`bM&&f$XAx#cdY=<~u%lrclr`ALMOoo=W~gYcNZIV{~UEg$aF0*BD6^F2>CeNnTX}J9!KzadQ4kmp+W!BaJXAWmzmGO z;VImJY7~a)7kRBrO~zWZ4t)B;Jh+9b;g(<_o7%1VX$i6#*{`V}eE?ij+b(}oiLiM`GF^xIaP zh$cxnT+WBNek$mL4O0u>nzmnw0Mw~{Trdr=(?)WAPVQp;_po}s5wN}^eJAS~Qmv3n zmSXJ%awpB*#xD%JPpE%#cVaFA1$Kp^uix(!ZEYwRjai(QJT!ww zGyG{hjDm>Z>s9HFcECK{>|}*xjy7b+ifoK~1-#|C8j+Wt@+YBh)}llrKbRjfnnhv6 zdDEHg)eKZ@uedah3aW?HM3l+fg4Mf*#WlWQNK8^6ip9gv!9b*nA&ND&G*YXpSogV5Yzx zd}qFZR%m{Y)<1VPi>4-00Yj5>`)y0)JSo0OZVd>!t1RCe5?&9l)aPwKC-6#KD(u)v^$P!LaC`wg9Zg-Sdx>5z~nU0o?HDF zb$7RZ`MtuBQ#SVyCR*tyU<6W%o3|*}{8=h{a+J!f)14|pAal2e%%;%YA5T&a!{lOA za?wQd#H*@3cSY^y4<7rg7RRp_Yr_0F7aYPz|CwO9LOWj*Zcugf=w4djSFa4yTNE{I z(cYy1(;BN++>8=Mr?Ypz7eh;i+`!y;r&Zn%ZmE%1i2>GpS{t0GIC4T$p@3q+PP#wc zE*LhNu*^rzB)-#wUJ*?K=ZX-nN#G( zvQxf+5P`?FGw~;aN69qAz+_A#zBR(0qCM4`cOA^xMcR${(JNv2d=W#Ey}|BOE43@^ zHN$tzHPiOg+2~j8`wpql8y(4dWc+Zaj`SI^8%3_8G=iBx)sxbQi`)B+rYEVff8zop z3WJNP$Kq^*mAq@i{LS&j2eQtX@C@DuePG@#BMJ=oQi-2hh+VqMHnq8e7kDjPbmGIN z1DM>ZGh0;~v&FNDK3YQzRBEOLQl+Jzp9N`@ugd9G@vP^SRj@56z--J`3KJY99JRKy zcq9~z5-q*qL%haz1QXrR4wK%Q>^1td^)jMd&jv8e>*7K_;gsT8P^4R0s_9mFMjI?e z{EQ+}Ze!oy>WkC656{B!h5h7=x|Gij(?P(fAU-?SY0{v1ERkP>8lP0-xJcip^A;q1 z;5VIO7r)lPnQNMxIMs3DcyIw^VOy0<#!L`|W zQ%2pQrrgDMIh+z=vK|7^T2$*b>i``QW;o|~jADj}&?0yE2HbU)Ic*d3?62EeUF&ik z;e{283NT{q;HY(Vp8|+jOW)hPwQ*Hkw&Ghh$@C4dY-8-wos0eH1p@^wW>oVp<`C2; z#iNFr=3tMjl@l0@es*NFs$(Q^@(ekjU)*qQBnf+im!rY8bc@lR;=N#9&%u~M6vtXLu@~Fw7~zShp5_G z{r{-wF4YO8&viT>-`F<;=I_wRx51&5W603Ec_g7EMMbJ;TEX@DE8mp&PmBTSGKoKK ze&|S`$53PX`hV;Uuk=UZacJAScuW;bUlFZ&9W;8e19j&sh)*|LUed_I|VT!LOhX3N<96LN9k=NMEKN%O^5{6`td^m+$qtxeOq z$`^t9t6rAz5@7Nd$IbWizO9F8(eEjlbcyz;soC2mCtE&xdX7<2k}Z5n99e6*wMNRH z`{8FBTk)}8%vlyK^5I5=^II0Vwi}U5di$h~<6HI4Ookj-y*Fn9thFAlTXyx0d{i=e zsZ<8V*kW2=7ABT6!?kCx)AHZTjJUq;MNxasQA~D*+kR7dASx3QObIuD7pu$NBgZIc z9b$Z%S?FV2LfZgYTp&ue5jTF_WycIRU^W5Hk=zGJ4}bQaV&GG>S5z`DPCEt=!Uj z#*(`$O2o?LO6V2vwl7at z@QRC!_!E(eb?t8&=QxNCW0SJDE^1Dw=y*q5K%%iKKe$%Y9*?T3b|%3<52b@!NOT&J z%ASlb0J6cQv;;*cpgdKkiawC^{TNFOEXzpZH+O{U@O5MmQx08(+}!|Lm=T7h#+%Xf z9;>QH7%!@!wW$MN<=fv@pd_ASTJfL$R~iDy-|I^J&GG){s`FodubQ^gf*SIlM68KA zQB?TBT>>J1qpzD7poxVF&@JC3{0k+8b4BY^#Z}^TG>_(gcfG@PK2#kRAvG%Z7fw3A z4hoySQoIVU`--a>uhmNzCxlIBFJ%Mm+m`@as5+nZSZ&)$&9$8*=1bxdA3e^ z;Z1`dirpv4?7{9~HV5f$-KB>&U^W5NMuKAe(bH#T0kN#aU8IHi?zF?XBlhBy+fjYU zeWCZKTwK!~xj%nl>I4-2v4$O+P;~v^>eG(D?pt9zy zRCBU=@K~i~#-dc{xoLO(_pDV34(N7s?WFn2D_SYeP3ZOdh_?JH40yT}j)%?CrpChb zU`0oWPW@S*$G)Ibi z0o-p_#Y^7jWw=dEjzjvU+Cp|SD$WJDFp$pkZdnZlr?oX~c`~TW76Y|c5OvKZP@DwX z@9OH%5)9Z{z2CaI4YUONO*vX_2B{W*luoTGv<_IM*BiJ0qz#Z4U-%eEkshR~Fg$L$ zZ_o9TA3ck`Dc>Qoo^Qn1&DYX1MuXs~lNQtb8Q2B;7%DDiP7QmtmmT>VmOx*o@Ava} zAvYs=WAD-(QtwH`Wu2IFlV+Z!{0-PggPs8So3a2fp;!2vh)c`|rXN;9+xmnIP1>;Y zSo*uiR&Mw%KMYm+)StEbI7nQ#BdAqFyd8I=lihTbCM)+`e@tp{dl9B(cX&qg!Tx|i zHEegYsGD`^LeeoEt4+?qx$_e0m?=eB&^-$&f(;8`M*0Je~WfkLFTSB_qLr#Un;^imfV0Hb73uErgp`POj|0alOCq z2;6?9j1Mr;FKD$Y=$1vE+J3sv$+SNN+ZwNSl7*#zb=CA8CPVdzy(6~t73U$*VKB)S z8s`<>*i>#55d3z}vdkygSRB_t6Dry2Xb*vpN??c^+&Xw47B>M`c#MUZSFvOcxp)j|3z&$SR; z+F4&$!&qzrgX|iVBh5d$!(2KP9!K_ZJwgl+<24>IL-rA_$2y>yBM=Nt%6)pSA>}N6 zdUDMtMXA)g7bGuQF0TDFt{hI0j&j{0cpgC#zhe+YGGG@wHfo-Vj(k^J2(_NmY|f4y z?+@bh4vx|`r!dCwZ{nqY%i!F7A4?nkS|~JayO4&{OZwY=*oOe3gkg=-M=RkJteO>H zx9zre%h8!))600?Dc=KK5{9C)wfW8x)zB1TgL1jLRIa)gm4Pr}sSZ?C>Sa}FYe*Z{ zEN|>}-#clZO}+gO!+*NHnbtZpC7*6@@qbU={%utM*FNU|!%|FA()}xW%h#aU;3_NI zn7-#0NhL;Qi}vFiiTQW50N6O*XLd=z<*2EeDFxX_K~JH4F#j{yYeBdh`xg{A3s-{a ztd8UC2|l+!Z}0E$JIFu0jcZQ_hKfVtLu>#SWh(QTOvdG2HjphSPvFAcR7tJa4?IHK z_i`d>L#CUDiWycG*ZYN5-D5!pyN_d|8bF6EXdv_EY|Unqk`M<;_O}4aktvN3!BP(f zR6&mT&mw(KZD(uz1?}TJaohvmm6VG|V(?RKhW z>)r?39>@;pkaPt_u;Zn z=`T`(jm${Y`Pw0ZjG0Uy{rX-ce+I548vA_wL_#|j1Al&oZf#_zEo=>yr=mCD8p@x- zq;)c(^%Xja99ruciXiQm;EhtNOHQsTc|)*78aFwyHkkeuM?s71ODWI!%= z2v|m57c?QM(^v2Q8GhBo&XLYV7X#h6)j`eqjB(6R+=6x^k3=wcr|#4-kj+M?7<+U5 zw8e7p7VZ2Iy^ntDt7_g!F6YY@R8m~sXJ{j!(IBsTbj3DT;DqZUEjEOP}W!cw(XdQd{t4{@N0BwKhO zeeYB zVc&2TNFZWt5nZ~pRv(mNw3&)Drj=d8&|xNdkWhjw46#p5 z&?EOXo>8;KZHAKTvolyyERY%)Iq)!jvF1)L!DGm9k^}-I_dXjpje2|}0(^63ov+oY zR&?O}?)PwY71kIDZek>DCOW*=tV#3yX#GP0HBnl1VR<;JzpxB0KQMvNnOW^N)yRsP+0ZKbhI5@cghs85i$Ah~><{GmaoK>F$l<7@@m zkNf-6)!~Os~H2L#;zXe3dEjx@Z#c8XS=1y?F zKFIG3e)}7mPCFz@&LA+z7;#~M`-;CYqK`|S+3bCN262^o!+br+PIQlx3pFEMSs6pr*6=;25LB?-~(_9{L z;s!oQ1Z|C!UI^bwd9sS>Oi4MZvcJ0TAxFFGp2w(1t!OVzh;*ZFN#Q3V9*cpG1QVze zd_!ElcJk+yXeETb@~Vg$vS*N~^w-${i}`B$ibQI6wnDm7F*P?T=998nMq{|rK@F@Zm<3U5fGY`% zXmfVDmWWt{&b<}QH4l+yWm!L#gP*m-_Gr7(NsD9Js2@Y;?lTHE2c|9DFQu#eg|WON zj*MHb48iyGp_&zy*mN5nEq*XsWa2q5ty7=Pi>+&i5e5{Dhl+k;c<4(c-C&PEu#CAu zc8YVr>+DM_C**$?v4OEB7Ktd_2{{P0dNP_TyCE)-isKd|;O3*`C*#>fd_`_I>Teq+ z+2)^CZHq`qhRZ8W97J|DcipI)7)TM`>y52gDKDQecIrjAPxt~ zo^U*Bf?+AH-dGojd#b%dDvFGaVKNKZOEeI}O7KYekg5q097f_!`HbPoT$L!y-GNCd zfuOyJ|V<~p1&NNY+KF+1* zZOG=s*BI+0srNv0PV`44+OjL4SK=?Xw-2P-K%cvVEXvOkF4w{tXAD#_;kASq>DdDs zp{v*fic>86eSyX6%0QB%yzR-Vdk6%P zX#Go#)u;|e$@|xuz^JSIpu&Cp^gzpk%q<`%7Hj$JArr@J{h-k@-wqs#|!ZC8>KY#S1c$RQFW1-Cu({B=)HVxRsi2fV}0A7ruZiglW8%MvYmV={vSa>gxq*v zb!8uQfM6lpZxYLeQD>82Tnlo=Gnfa$JcoRgP$qlv<=F$pCQ1>*oX{rC$$l!w>V-qT zT$qeZBlGYE0z=h;?o3 zrBp6&42|3-X9WWM!c9sqJ4A-BRQKj_ONI85_C_Q3NN1&PmPq4}XTTzm&LaFHaHs;` z1i#;I<-ME<;-nx7eCfU5r{gIx9exFgj$2kb7h?C>;82T7^15Lf7izUOA67+i~zUjk) zP@wYF$hNr9`Dg{tazc^aAcq(`4G8rwb1S@0kE6CkazSzQ1)O zFT8x>g2ZU1TqglAUV;EjFe1OV=}%4geW5O>ZL1H^Bh$CAHMTQ$(Eqb9Ql9)@4zWyb zG;2E1bvLR#A@Ow0d3QPl;SxFmBqjor*U!LG4d%@q5&-(0o@+e`$v1D^u0%0UX|ScB z!H@+LU3W(tcSpG$uXf8VSD!I|dinghETh;ysW*3P9IS#}gGr{vTA{alfSx1=6}wK* zJ8E*6vpTLg7;Me$e#c4iH!gkImhvR4_TZg7i0Kpe6d3S4R2l31>Ni!JHxp-ynWOr2 zpW>J-nq!&PgF7w(k%>3O%FUry6XHHK9lGe69tCI7mU@@cbjtWKO)2t1d`!?XhSiV# zfZ@m0)T`C#N;T@Q4{c~R5yF-UhtiJA6ME+y;1sz|2ooqNRqEszXX}hL97RBNn@f*{|d*bZD zi={%gD9boJ3+=+CHW|j~4=l*wMv3eolu6AJ`Z~z!VCf7kUsf63=wz^USJV~}2P|Kj zFqnx%?#vyB;m*c3@pN5zAJ7tv zIPu7!u_;{rbp-Oyt3fwJ0s`s<#OWgY7rphnu}~G-NnyHHi~5{BHugD5G?4F0BKQH_ z7$5%0fA0pGBMr*Qi(}Ga__UJs4nG-v){Ta7nUjsiwDV-l%DFC7rQU> zn4KP9uBb1%TDmT}n5yr$UnM0COTm#{ZEhZMyOy`kEF7Ml);g|yxoJceVh)wvnSi_V zy!|4~gFmoaj`fu`;Xwxfa4Som^Z4yVVX*2ZPMV#uCMV|6%zT$t(hT#JacW8*=kC5j zM}W-jOM%U3PSmsaFGqKMUcT63+G0}MBuaz(gn=J9ZTvEFa;|)m1n+c{Y5N-FRthCV zoKv$a)?I^!*l@rwBuwh^jM->l(%r4Dm&p!_K6DEyT++Ts=gK;%X8SW_e+bmA0+cV+ zI+r|8wUBJBg#%tjm+h8(=9xwsnr&_Gxt-eJIg3`Nb-2usQpRCEb=N+GkDN3T2cbHtjVCS}!+3ye@#T-t26W&Ci0RsX6Cdu--aVtL)mO z)qg_eOlg_!8_9sF-&4mShPd60FPI zJ~~2%$)uN9F1(&Wx{OJ8Cd6tOs?X9pV3dXlJ9yfi$+d## zhb7OWZCPh1hg+BiM)E7M2Jm`Lb1h|PWM?goiy0<1ZZf8# zCa&0MK(xoe+?Y634zmSqXWP$wV8Gr;(I~~R@LQWTG5levz*@>-N`$TIf!M<`W=jUl zP>xN4N*L1owyb7uHg}|%q^LB&SiUOVjN_%_A-W$pl88eC0^hh4ydBMBsD_ofC~(cM zt42n&FhoUK4bmgH*b}Si2_cK^$3v|JvMe1$9f zu{x7OR(ixG`Pj-h>MH#XR0e9rey4he+PVT7*4cZ1&+q@c&(W~TB*&_8A zeqBU^!PCXx<8O($cPt=a8D=M(BG&~O5sBHI{Tc(q4t?2tjK66zlWxo$Y?wrQAk&Q{JeJP7`w$7e8W&?R|_(}%PXF1AOvt$rz}j3OFQwmJarzxTrTbVm@#oP}AEc=bMYx%IEnO>%?rc1D`G zb+45})SH3B4YK;;ZgZ1!fPhTAU`izo8fX|ELSyz` z%y1SDxxIF8BGOWk=L>a7gec9Lxa=kJ{_G}nu7^EL`F#c`;JQ5q5D;S%noB-J1ZK4g zA!u~LN$tj;>PfIo4u-ARk?2^})k27kO{Gg<$wiaRlU0_&dP5ySH;;Rms0x*oYgOwb+g}-6DftAw}7|73aWwqB*#0Fk%#g=akp-mZ*fc1z)Y>^KLBh`Q##f>rQ z-}MC*tYTl5?6lfgzD@HszA9)Jg#{0hJr`kcbh6^y8_;REP5o;10p*4{A#Z)neJ4ls zc7GrDHQm>i{fM5@2!43TE9(}k%#x3s?-f;fUB+lVeVcX+v(N^)%Q2CUVxWvR*P1Hq ztde+%o;P*yp?+CoF3Y{J%gcFW_AlOJp1JLfOgiqO@C#^@fOAJr&&x%Hn*qL5ptsfs zuQ4#AJEnTW?u62?WYLRNvTS{s>Dx4ptHdjk5XXtSdW&mtt<=~mx;e0@Cl@TJ+RVQ~ z?qHXcrGmykp-G^^&~NhCBF&sSK61RVw4^dSqe7G&Dxt(4zd=m0H(6KlK^yvU_;~Rw z%|K5e5ks|gb{MDEmT#sy5DlhYrFmPkBb>Gr0l(a8CAo}1f|Poak$l!oZQePUiQ1uZ zDY-Sj=>k|2$2lWkE!Kw@Pkeb<5=Rk#-k?YB66SsRBC32p67zXLiIsYbravW26gniE zP^UQf4)x#`Yka6j8EfJ2s6z;ML5Iw9XvK*}t90VTh3x3E(M$el^+Y(>&s&7nY`S~H zvO-2^RU{uJSa$s@7GCWkuYvDp>k1YI`uc?7)Z@PuF(Aq`A3HBmv1LwlJ3fpf54(k9 z#ms-#vRG=NpC0`@_A+0kkN6p6`^}VTNcI{37tZ_ep3pK}o-68s4rqQC2$*Mw`*f7Z zsf?}!b1zG?$}noMj`gH*a=XHoyYD-EWb;f7UU6j;Ym^lqFd76Zshwq(OcL)-*D<*r>u&zKlR5PU!Ub$Q6^?!y|+2b^6VOSt-_^ z%Zj-Kwug+V*7zm|^-FH%If>ATTAX%Y2v4`;K3YdBfAuY*jdSIZdth&*-na%thggU> zP55NW&^X>@q{{1@91&BWP^0ykyA)$7v^*l-h%!9acAw`0CMETx06Yk#7#z8THCA+7 zhUPF&qhd0}h4K`maf~H-aJiLv1LF*6Q$UPNE#MTmqBsZAE**)!*B}OgptX6AFlbH` zelmf<&@?UQz0J^Ih~f)wfk>SPh`Xxe^0mjV3yem;!b5_K zkI%6kdAHdv<@x33tG5nv1oE{wa}q>mujS?BRlQt|r39Vv!+WOtjvcSZ+4BY6Ub}eY zTaMje$@;HO3L4^Vkbg<B<2*zN2goBm-=O4XuI)X% zz8YgjIC}QMPWaXS^%mVpR&{YJt3D!y0YvG}?3bJEHi1&w582Qa?-gh{CC8h%AzxQq zy0%a@4Tu&V(W81d;YXNj=U5SLFRQZy zcfd)~HK@`fUIVR$Ge@wFD|9>2YRaIGqp3+MM+JK>8dKZLGigfG+99ioRVoRoVslF# zUm$_*H`j!FfE8U+2;sj5Ps^r{%!G){lSvojYDmo1kg!e{)m#$eawb0BFrOMpvm-st zE4~3bUKcf{$4dbq;}I=4i_+P_;=@A72OQtmpG1$@Z+u^ck449?ZOtgqVY1@ zZ{+Z~!Beiu8ARl`GonjbyIZ{;AYB-|Ic*t;Fw5UH66Tu$L71&IVN2jhJbyt8ssWy+ zx&@ttD$isCH5DnDR49BffwHnzO;I)ANC) zqJa+%=sRO~U-7z6>44p9f(o-b!H}`kqdQ`HeCWOL)NHn# z3#r4>m3ZUNbbZ8LV;grw{=x!j{nk}jl*AJdC!ymr(jA)7k^G;sgLduwG1(3$&BUS6@z zUh0GLzCvxTO~N_kT6+R&_HD=U$IC-^yI{#ZLn4B$OrtpNPzNnYu)JlGebSoAke5EP z(|yL~wczW7k}q&ua+zxN(p0h{XNtEaZj!t^hnDDG$;Sd4O*Msc*C1l6A&8wABG$!s-l)&{$j{CzLL{$%t%8a?!@hpW!{iWjf>Yoo7&hK0?1+v^3&y z&upm#Spa!u@s;{3_SKFk@3T90D$j8HT$j_XI$-pnJ>Cvt@Fo9`Y5SSwd!D{C0eA2~ zRigX#kWuD=`g*hEgNM(_;~R>Wg-?Rv$IJMlT^+(j35&_)LT~O1YYQuAqk+Xx4 z`4!k>wiaW~7pr$8UyIR9jtj1LK_-i_j(D&E-S>K^Es^9I(%H{|quk_fUgw4=P&L2P zI^jclwgL@I zdvSq#qc{xFX@(SE7zCq_{GR1L4(La2c|HzoaDIqXWy|ca1$miYg`gH>Nix5p-6-1- zk*@|y-JSw;V*CLbw`dN$>57KR1!tJ&%&@jw(lkFDBB^A3w<1jD8|{#Q!?3 z%>XaRcyw7XRr+3S1RH@dXwNIbnm{#eR2H&ej`zEwwdyEV}2i}E` z*{yiz!bZG-S70@4O}2YL3m<(S$ZFVpEpW#!a4k=GpPX)f1J5&&12C*o0ye^#{)MTE zgx>%VPv9>%2;0BxR;BO$&u6;tu^#(y4-A_k=p(cbA9P$+b`XP{8^nMRvR!ZsgQF?# zbQz1I@EP%qrW;|fM0PNK2fY5v`r@3bXdeb?myaCRORF5aE4GUn?QLIyUiF56p-y5| zCGL}pD>D=mhC9QOp((^E(lBlvcvKH?7jHPRb~*K+!&VbEY%drr+Ygg#)R>vtuNwLj z+76wiuCaD)*;U<3y(4TrPzRwC>$-EOHV7?f*@@9_*qCip-|mcd(USsKmkA~G+|_>@ z+Gh#ecb(g`<6Ng=?_8`OYl0Vs6N*VjNVaiEd8iZHUOtcg44r?mpPo_Exo6d8a$Bow z3BqraMah5_^R))Eo{eTK%=0#M!S@ZF^i%PRa>k6ASgfv5uH6zZvO{UFS0g`vyj^KJ z{aQ$NtqkVqIvtNghbP{n2u5FmyPg<3uw8)~mj-%E#UzEJ59wRCZW-G2wIjNeVPTtz zE_9eUu*FStC}J&xdLh$f+&i`TF5xk_NRNS8tw;@|`chYF(@0;&-=5lb`oDBMKv8nZk_Bn;-R z_kk)ffhEmn;VKZG<=I7$_-~yzU}T+&u$ab}xCx7_7MR!sK7M4L{Za ziY3XMotWpD>CIu({=}D4bll)52GHkI0hvWyX=|=123Z2G~+6Oe6;8X%oW2>KhkL(BxYwr)y4F zz3F-$z5Umd9m@;Fqw`gITq}^c}ShpKft<&t#Fi5X{#66orY0f}mq9sVL zH*2O`a$4`;_ZWZ5F5vL_U}=7%jdqhF3BvK%i+}YMESElo+jdiDImb%~kYhE|^wpYV z9!vJlBCa~cb2Zu%R=rTRC3wF#?BV3klJX(m%<(U-XUsZ>-i4t_e)Y>2DBm=7>IVv# zMW1ly$tX$|KAQAlRy0P#ghKzo0CVP|3BsS%RKxd4?JVZt9!lEM<=#WHrDl7q&y{Le zGAKeDgVP2hdM7%921ZA#(8vj(3`GrtyquSDx+o)f!?p&}&WFmd8jT$T;x z0ZcEz>y^tj8;@}~m6yq7NSMPSCk1yOPT(Z)0~gnlKE|PKW8U?}pmQ_r64>~$V>$IXD3UmIY)&R|H#^@?lB$Ry3=4u+4VVCNa7WV4s5o?}>7y9N1iI6^pNX6i!4 zXI^voflM;=zo!^_oBH_{4hFdaj6$|fdoVU!XKT`2$eiarh6+PFakM0!_8N4)hrl9_ zh(v&IoM8YSxMWCy4`S1Yso$-X~g7AWAwNqd|hG5-WL{GUJcQm=1cq9A{$Lf#)gT~ z#S;v}RO;QiO)(hDC)^ssSZv1r(Ra|l?m#$^Z7942h>BuC0|9aUKCJ&8E9T#9f&u~q zI$|lJJix(7F(&Q!WU-Kyio>7+!&9&^sgB7QC(xj!p)f3($Joh2ahs8(8BOYx zBFZVJg|@m=8I@TmAZet2pK@x6WM{*>>9n7BZ6xRl?$h&B62@ zAckY(`YMX?u|O&r*<8jtvAk;Cfjw{Nyay{zjNU?Cqg-c)n_YyXV>FUb-#&y zK3}ldPx+zj3buc~F?v-Q+JR^TO>XcY!Pz#CE9ZE7!&9?UOPS8O$O`AGT4aRgy(3F{ zr;#VRyZ2%YK-&gGM0Vlb*^7Mr;kRntx|pYeh|vjhd~&@sZ{#Yev%8hAgp3%k&V+4M0v^eO$__iD zj{53M-z;|ZJTMnlj1_Mv$ZrrLoRk1zj%+AfG^lsdXVw-`ylX9k#hqqZi+?>p`Y6Tg<9Ydgr!N1wjyeIZzZj%xfsGG%lhUg7GP(PJ=HbS5Z$_mP|f zjKg_m5N1o<7Or8!>b4L}gUbg(kK zlLv;*vYe;dW%@M|3t9(sBJS-UsyEXtJ5rVr-y>JS-puI0-puMSqhe#sJwC8CW7Y9zxoj)blmO&LRZU-w})h;h5yZSZ%D#DWIVP{N~Zg# z=#_?B9}Y9y_~Lx#AP|wEyE_BB1w%d^BUFj{g^E@P1)(A2S%!`ITcIWxy?6_AO#zya zc4KpVV{>77{ygv!N3~hvOw)ANTM|v&Cao7(++vM5ustP*^7Fe)#ND^=Xlzm@+?cPB zHeo?BE{DxyRSS<*1**1HJ81=$_xmP4Uoh}k-%b6ba`f$#QfyiaY71a)CIHOMG`|mA zzd2?8eA*&hUj6?1CwG`x14fr-G(;|98 zeI#qU$qbf=5^@J@>3=+Wk%uDgmXyYEpLXiD%E8qB==S*REh06g-m6z~QiMJN@OShX z+1mjjDdIG_QC{i2v@~Sa>K>=>8>ri_x2keC+CspgkX(n&td;rmtA?%;S3dg{D*GMM zQtuT)b?ImgtwR|!c_jE$56}pfyF^rkZ8PSPNOU4;sq!2tujc-ge2U+~_SGYRS`w)Dhz*RzvdialDZ+5wRt(0}qn2 zHi3;aB><1wVEp=)HvtpRfDCf&cFD$@E>oXkXuo|IhE2jpxvd&DiCVLZB(&t>I z2Gc0APSg4QuLer3n>+nUzY@Ifcfe$f)Vhm5G;7%*dPRM|RM66P%$`42)3}@Drw(__ zxR??AVA?dWswDl{&of9HBZ=zxOu6N)ZGjxceWwjpabp3D+zYI#^>mW(ZhHrf-5>(z zlKK0ud!1Z7EBQ(e>e&Vss-K-0x%X5HGl~6cBC1u!7=oBMEp!!nvLi@oidDudLs$a* zUu}mQwo%s6tlw@cv4}CjTtiFNa=|c>Z@zqqkCnJ`ECIJr+ao_3MfgZ(Sh#`r9D}S& znTu;xYq?y9?bKdy3unJFiVQHS+U=)CB$8k?mpb*u zJfbEN@xULK<)?ig|Ct6pe1xFKfI*-VX8V1>k#Oc$5*DIvXULpq=TNsus7(3oe79rk zq5Nfvm7(M_>%r@cWv|lLsd|CaxnXMLgg2S8g;@CF-35QuoU2b;wRd)}53xJAM{(_NQ;||h zB=7)5}m37tuE{8(oj2!aw#7Zh`^kwqF7SBo?U?E?c zhJ=?;(W_A)!T__zak@fEch%1Kr(;gZU6Osh-_F3j8!N|}!oUKVx6oL9h?~pWR+iQq zh$6hGjH(m-+GwxCmHYzCy4~buN!shUZO(OB#@ah{(#CNYNR8Dp6~Ce5(Ufw(6Hn;Q z5r++5wA(Q1>Uo6}KBKqx$+QB&9w;=j@Tt9>V zTEBwhXgdc0k4QJb7s0;@V<(_*U}>W-Vr*k;CvUIwz5f6D`t4CNmq%6xoRY7yvaU7~ zgMC*wC+5qi1;Jm;hX9Qjg%oTa$2wOptui^SH#=`u^bl0ng%Tr4_pj_)Wy{f}$*#=r77`8Z=m`G^)G;3-= zk`1G0!HG1sB@lD4n2bssGhh{?*7ChzJntBSq$5(p5bD@JmOztt;HBkT!7MoNOk$~4!>lz} z8xvtfy`RCruS!rkSIcni@3=A&C)XGmU}m=-=|({tbWzDC2jSqHbVxxrqNa8Q`DnKc zSqBn26Jhr3G(**$f%YXph0JLOIf=ht!)wz?ybiOQbuvnf41Y1;bn>1Q6rG+-#eE2Y zm$Rcv(RhlvOUwQBOmfD9z@&a|650UOI+4YwFj?;*@+8a$-!H=nct-jun_Qq&5=1&l z>qWcKtdZ_O+Y~4l9E^{0rfr8 z!Z@;uO7|8#c$kxZSO3ao!PKri8SIUr0BY*%>iig*b4{leF0DePS~$mf>W#1GVES{L zvuj`BZ`!-1Q@g2&E;6Aexxzqwvs)(n;WOS}U0l0F8n79k6lewac>2?!$sT=pWEydI z%2=4x3D*?FR~PWo>;u=s&S&Y=jdSb5l&dAh?hC^e@A2?H z#k@oQ_`&_=`E%%rpbPSevfC+HfUwhxUSq5vL@np0$PYSuH5Xi?C|?IUnLw`TFKqC$ zvge|4qO}NDofooQ@ly8;f)8NBsuaU2SxDwM8O?lGLOB8-^b=G<+X5h^kjxp9v!mgk z9T5b8;JU|ciR)m!Mj%mba&CB8DmG;+O6!oR)Na*4Y!Em3$EuBX0ppW!SLyIp}tB3Lc5y#8vg&`qc7j%Pg1N~)&IFFn3 zSGJfh_`i-Ju|Ql&-#n|o0LEyJ-^XZqXIndc^M7MgNQ)Vg=;A{O_&8T=URyU~GA+Es zB7iK^?T;RXhW?uF)xJkE-efchGTEfSiiENcG=4`Q61g!#A%C}OD%1JL$C1>=7SEQp zXC2SX5(wbKiOf*4RQ*PP%}_Ii2|Nd1l6{2KTeyqjs~hSQ%Um$TTaj8u3~}YOiFb#}Vb@Tvt`+q2fwGX=^3*mQDXf1&E{)4eX7Aiqk-L z$Ypz+fe@%dCXg_2u4pDs_p3f-6z|Pv66R$_9#y5i_{<#q$0kmtwc{1ArIWT@Mu4z0 zhEqw|76|NL`dA7VH8Wp`c%w|kwA)sIb6l>;4FLy_W^YtsB~c;2v%RO|1ME0JN>J_S zR>J9{Qrr3tQZuwcO@o|}Smn1})OfMBXC=|u(SnZ9WOEf70iG|i)u4)aOpnwaL4Ivg zT2vz+a6of51B^wCzc=Ym)9!c2>fe@^@8nl4CtjgE$WWp{+jcA|Fe9_!(6b)6F=0rP zBqv6hLmI%lHuH5g#i`pa(%$jjZiJHY+<@NzzPQZi^?X5$C(`k+Q%~J?Qx{h~JsyCq zfciwR7FikRMzc*eF&${8Xqh3Bl+!P=XZ;jftp(`0K8%r;IB@UdX@%XF-BH}}xJoR) zCHR7z_0n86)xd7Y-*2h%RaUV}bkJPVBSBs*z4Van!)G)%LdDCjM1g7W^hwAqgnwoqFN{ahS1VOpL#z5IdLpx4sY^qT^T8S4q}i zcEch!1ldo-p-?1KI_Wnvs$Ctf-3%S8n>pGa-0tBB0)!Dqf|w_eP{)0O#H#q|0<0uE zD!djon5YCg61}*9dxf2>W&MKgf$<>3=%-RFrvwNF$I>RkHAoEmi=9bhMv9|z+bRi7 zizyZ5(e!dMF|4cblv$=*`sk+*%^u4ANwsJzLjf_Tonr2aI>$Oe&(*Q1L(UYm24cH2 zCaP^b#90;E=%BclGz03oP30NL6m#Ah)G38T!AykZQ;IOsp+iBbhO^&cu)_szTo}O9 zMv6;2lfXzf#WU!4Nm(Wrl|hOz)-1HRqf$zDy3D7j#jXxUx0GxXVNSlP)o9U}*gbN_ zWW8OB566+!z{GRsSgs;3kPwhW*Pm`{HAhDO6!i?|(D3tmT34uQ&$m{r^J(fd17VBmlO53H<*I809%Yxf}ul$Pr-T0}%fw z>^)$3_+X4=ji5Q#d^XuyB+uBNNTWA~pEw%78 z@58WKBHu!2-vSJJzvdkeAZq%Dyet1D%>l4=7#JJc1L9``V#)tG?|Lr7t1*Bo;Rd`* z^nYg@@T~E^L--@~)Akets709lw~XgG(>EyrG7bc&oo_?N-&c+I0_q>pr7R8qYb}i0 z9EP9*98D|$W&U<9>hG(@+Z><)@`qaZMfUE`#b;lsTgC>wVn={cfZ%UHz_Z4?7m(jS zU;<7B+G(4a{TXe!Ln^o%P?_%lmHBHs;RE``AJ7CWE$zPPZdgfc8(RR3u0PZ^o^}DT znR=2*K>s2J6!n{C!rxbo_X~jN-yfjAcL8B1eO>$igin8p>W7tETm?WC0H9L+4GDPG zc#8`D5%sT^;yd=YO#iteo@(y?4PE2SFY`y-@74O>hM%Vzhd=NL0R#FUO8-mK|2M_M zr?v4^Kko+%welZX{&~cCDx32I&iBoKX3y^f@E>Q;pY!)^ck8L@%@07-xBp!O=PAm! zRNr37Z`U{7n7^)X^BAV~FQxnz!{%w?rz$dkC$I4q`#tgBegZ$O*PmElpTa*?2KfO$ zsry^reuDk}b;?Z^FOFcP5z1MzXYCt3jZ`_`VV+PvwwpB-V*;5LH#M!)8MN=sPygr1=U}b_P?s@ zY5d9`B!Q0qg5;m0Sw1b%({O)3$a-Ap#72PxsJ&ATyQ!hWvYH`V0EcJL*ph@pSL< z2NhY>KT-XUx%BCl-4ED+>VJa$K4ARA2Hw*GJT>h9U>dCdjp^z4!%ubhKMM5J*!+Vg zt?@USpJ2Zi==jD1h7jz91(n*Rm \(.*\)$'` - if expr "$link" : '/.*' > /dev/null; then - PRG="$link" - else - PRG=`dirname "$PRG"`"/$link" - fi -done -SAVED="`pwd`" -cd "`dirname \"$PRG\"`/" >/dev/null -APP_HOME="`pwd -P`" -cd "$SAVED" >/dev/null - -APP_NAME="Gradle" -APP_BASE_NAME=`basename "$0"` - -# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. -DEFAULT_JVM_OPTS="" - -# Use the maximum available, or set MAX_FD != -1 to use that value. -MAX_FD="maximum" - -warn ( ) { - echo "$*" -} - -die ( ) { - echo - echo "$*" - echo - exit 1 -} - -# OS specific support (must be 'true' or 'false'). -cygwin=false -msys=false -darwin=false -nonstop=false -case "`uname`" in - CYGWIN* ) - cygwin=true - ;; - Darwin* ) - darwin=true - ;; - MINGW* ) - msys=true - ;; - NONSTOP* ) - nonstop=true - ;; -esac - -CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar - -# Determine the Java command to use to start the JVM. -if [ -n "$JAVA_HOME" ] ; then - if [ -x "$JAVA_HOME/jre/sh/java" ] ; then - # IBM's JDK on AIX uses strange locations for the executables - JAVACMD="$JAVA_HOME/jre/sh/java" - else - JAVACMD="$JAVA_HOME/bin/java" - fi - if [ ! -x "$JAVACMD" ] ; then - die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME - -Please set the JAVA_HOME variable in your environment to match the -location of your Java installation." - fi -else - JAVACMD="java" - which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. - -Please set the JAVA_HOME variable in your environment to match the -location of your Java installation." -fi - -# Increase the maximum file descriptors if we can. -if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then - MAX_FD_LIMIT=`ulimit -H -n` - if [ $? -eq 0 ] ; then - if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then - MAX_FD="$MAX_FD_LIMIT" - fi - ulimit -n $MAX_FD - if [ $? -ne 0 ] ; then - warn "Could not set maximum file descriptor limit: $MAX_FD" - fi - else - warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" - fi -fi - -# For Darwin, add options to specify how the application appears in the dock -if $darwin; then - GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" -fi - -# For Cygwin, switch paths to Windows format before running java -if $cygwin ; then - APP_HOME=`cygpath --path --mixed "$APP_HOME"` - CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` - JAVACMD=`cygpath --unix "$JAVACMD"` - - # We build the pattern for arguments to be converted via cygpath - ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` - SEP="" - for dir in $ROOTDIRSRAW ; do - ROOTDIRS="$ROOTDIRS$SEP$dir" - SEP="|" - done - OURCYGPATTERN="(^($ROOTDIRS))" - # Add a user-defined pattern to the cygpath arguments - if [ "$GRADLE_CYGPATTERN" != "" ] ; then - OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" - fi - # Now convert the arguments - kludge to limit ourselves to /bin/sh - i=0 - for arg in "$@" ; do - CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` - CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option - - if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition - eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` - else - eval `echo args$i`="\"$arg\"" - fi - i=$((i+1)) - done - case $i in - (0) set -- ;; - (1) set -- "$args0" ;; - (2) set -- "$args0" "$args1" ;; - (3) set -- "$args0" "$args1" "$args2" ;; - (4) set -- "$args0" "$args1" "$args2" "$args3" ;; - (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; - (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; - (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; - (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; - (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; - esac -fi - -# Split up the JVM_OPTS And GRADLE_OPTS values into an array, following the shell quoting and substitution rules -function splitJvmOpts() { - JVM_OPTS=("$@") -} -eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS -JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME" - -exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@" diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml new file mode 100644 index 000000000..7e4d3c45e --- /dev/null +++ b/kcbq-api/pom.xml @@ -0,0 +1,54 @@ + + + + 4.0.0 + + + com.wepay.kcbq + kcbq-parent + 1.1.3-SNAPSHOT + .. + + + kcbq-api + kafka-connect-bigquery-api + + + + org.apache.kafka + connect-api + + + + com.google.cloud + google-cloud + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml new file mode 100644 index 000000000..5545e657d --- /dev/null +++ b/kcbq-confluent/pom.xml @@ -0,0 +1,96 @@ + + + + 4.0.0 + + + com.wepay.kcbq + kcbq-parent + 1.1.3-SNAPSHOT + .. + + + kcbq-confluent + kafka-connect-bigquery-confluent + + + + com.wepay.kcbq + kcbq-api + + + + org.apache.kafka + connect-api + + + + com.google.cloud + google-cloud + + + org.slf4j + slf4j-api + + + org.apache.avro + avro + + + io.confluent + kafka-connect-avro-converter + + + io.confluent + kafka-schema-registry-client + + + + junit + junit + + + org.mockito + mockito-core + + + org.slf4j + slf4j-log4j12 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + diff --git a/kcbq-confluent/src/test/resources/log4j.properties b/kcbq-confluent/src/test/resources/log4j.properties new file mode 100644 index 000000000..2a5a0233d --- /dev/null +++ b/kcbq-confluent/src/test/resources/log4j.properties @@ -0,0 +1,14 @@ +log4j.rootLogger=INFO, stdout + +# Send the logs to the console. +# +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout + +connect.log.pattern=[%d] %p %X{connector.context}%m (%c:%L)%n +log4j.appender.stdout.layout.ConversionPattern=${connect.log.pattern} +log4j.appender.connectAppender.layout.ConversionPattern=${connect.log.pattern} + +# These are used in the log4j properties file that ships by default with Connect +log4j.logger.org.apache.zookeeper=ERROR +log4j.logger.org.reflections=ERROR diff --git a/kcbq-connector/logos/BigQuery.png b/kcbq-connector/logos/BigQuery.png new file mode 100644 index 0000000000000000000000000000000000000000..a7e0a715624aa53317e0d5f3f9bd0aab9d5f695c GIT binary patch literal 6210 zcmai3c|4Te+rMYbFvv1Bw#Sq$SsF_yvNS_Tda{-!+k~W$J!@kO6*EX#vkqC3!bJ9E z2&KeCMP{;RknSX( z?sm?TUJeCbO`=6leHL|RO}X{8>~0vOp2A#MD=v5ST#N6P5?2tac2L8)eM)=u(r`s; zMD&TGu6MI?`5fm!5S{a2k(0uy8^??A!V$}Kt)qAy_3osle1%$eA7 zc%Inv;;eccV&t_28~|qEjO;igh;N3AOUWIr)Of&KWH{cX+SCmmlf+io| z){5A|X~%>z?n+ic@mF?~*EXW3hky~1wg#dI$ZXkF04{s)!RVqyAN?SX0zf!QoEV59 zJUS;t4D?`9fsN*k2h}YpMp{QSkO=vk8ySbm-Z>*N&XgiHFs{e7U};z7 z8RJ8D4G6b^2>b2Mk^;K3Hqa8j#|aF5y&a#ZvE^^TsN^3zs#lC};}216(W!08oj5IIPlNT}5-6#Qt2QNwYKJ1>|cnC=W1 zh+~=h!Gl=@XxVLlp55Vg!ucHr#@1vMt$mFLN2ni;O0KWVWl=94SITN<5Lgih3|rw>F61&-|GlRSyBt%eM_I zg1P9!4`%+=Lt!tQ*mKc+TI0+uwle+G4g=Y_Z|E?WlPzrFv9`w^4JJajM zC3t-)1+a!caMSNc1Lo^e?n%a#uKFs6fg$#xNq?nKX+Zbe{p2uRFXM@5e*kpWV{=%F)8P6=?Drx$wgn?jYTDT(Wu${l6}*r zCYC{)HxZ!25J=Z*2O9qE-N{Oi1@pKpWjY@hy>17Ae3NF!_;?-{m|eN=NI^05196+$zh4FUo z^mwe}3;adw)b)V7-UCa&;R-0~Ub4-hwbxQMTBXDSXd8BjSK6-ApDQ$vzwYYU?3fni{ zdH06nk-Uks0ZAP3c1J6o1B*ct)5G;=67_Z%_9B{>}U+3PB4Q=GSNDR!+bq~5%9Sg$R=JDzti0X5fEYscuj zi*t%pUi-R}5}{b5LI8$$bFFe%*3pT7UPo5(X!|&zi47i`E}M?etEc1Hk(L{jhG;j5 z$Pf3u1)2`hOfsgS^x^&^H_H#8BP-BeZ9;!ngPqyBF*^%pYNu-YR6QPu)oDn1p7Wwq zZ@l<;=}?D5zwJir1_VD>=TZ5p>{ut4Z({hc-?L;{Fev-A&CfWJ^vqduEsi0lIq#J) zqVDkYFx5?xM!PwHTo zq=t4N`^Yo8o4{pi?MPlvv&rKVA#*6mjezX9+J<>RlwEdt`}&rgUn5q@ai!2u{C(BQ zf7-1-3f1m>$lv+NKe{K;YKxD4bBJbA$%LxH=2yOlE#?2p$4!6rKYH%loRL@gg{p}6 zrp#EPG(Nu75Hm2D1^BhJ>4M?ey!1NYy&IvDXG}rL#N1qMsy5G({ zis`riv)QClT*V%a3+NI6c7{s9DWXxOxMn3b*|4zY}lFRj_|2eqA_|naZAL*H#+beOQ8sTYmnw%%DJ;(w5cIhSt$c zGfiZ3r3`vTu}h|_w{5_PFk&+`iwz~Mj&xKgt-E>rK29daC{yjYK$N4y7An_w22;d3 zM>hjL$~I{~%<3KY>sKn9peLPE{KyKGFv1o7dMv(tMFa?J*>fJXZWwjB-CG&h@8YQ? zsUuhA_1@I|VKKs_9vEzJ0E>(2TFUR8Yqd+-TcerjZo96A<1-O%9c@D_i+i4KVn(=} zXRg}w47HXhUSe)@#^h|+2y zrnEA_J=F)JWNa+^>(qN!IO~!F`9Bf^qWo>8-J>TVcb)dr zfwk<5iY|;7A;ZrWrf|CZXWcaDI2Qt^t!K2Xjr>;q@;S{4_T#Cd#&Z>W2D=UjOti8KKQD4*!zb^yR%b z%#OIex0AD}6D+{P$1wwool(@IC0Ho*IDDUP~8`jbk$KGKv1n(zi+!~2H?DBw{0`u(>O9I;$R6A zzIMG2Qng-Za;-tC=8r(cUW6Db6s~P`#Gsgd?+Gfpqvm!|-Cdw&(Wc&OFQx9QponxM zS4X2ZO?C(sO{$3nG#1a)Ul7dLedcE_4_#-Vp)Q-j!P_c>On9_7sp4r_MXEGD1rRI` z1V~N0oD(fkrH66X8X^aa5xtg|J4#B=fR%1La-HMCZZDmI8X5c9lT|1!xDvuO1DK2( zNxn5BLUHm?h%kf)C_upMVDdmL_-z1eZEU=FnDvflr#Vf`yY1zt)a zl?!%rqe<(ruOV{qaa4Bx$qH_5;>cxIbw%>=gYCaQr$Ug?)&(G$;u#Y*OwD~c15j8frrB0PRqD!rHpWOjZyyN5PhTjX5jvmL7ur}r7BIvwr9^}MO(hdyvOH1h$`|ORVO2Br`4>8R{z6%y zPdIHIcgap33K#1p4Yu*TIA6@0{A0(w6EEJtlyAMRwe(uAj-qVwc+{oqYO=AkpvuYv zIGkR?jpi{~EFX(sw~*GGSgQUG7KH-$CCYH%$59D@uxV-9bG;%Ok)t}jPCf)!!y4&! z7Lf;Z)4QgyCWMQD21nx{; z`>THVnBa4MUgl&7)^wt)FFUV@Bc|s@O~@!JFU>cq@rwL<|LShzY<{wla7bzQ!-n$C z^}`@tS}G<`GnMZv<47O{QQFqgIZ}q|>ta`$Pc#grR2{r z{A))*g3bpq5?H%Zp$5& zZr0M~D$UqG8QBXHSz6EpxYJ|axYYZ8LF*hH(p0rSzfJwYX`IRI-KTB~m2n8v6pIor zodBP5!u2mq8RCMhEG`G65Y)QH?kG%Ao_x|1j6MX?O=_lu#>=mhwiT~#7e%!58Nzw# z^d3E-I7FTVqSvhE@O;CyKfmi7S;ILZIv_a_)$nP_&3^@|1$xrh`bB;Ad5|Bl{+xHo zeIlyt6m1dzosCO?c-r0kIYtI*0IfGKh26eysq)XuXKa02ap&A`%Q}|Bx@?0Okt1ILb-KTfbbY4!>2Xl5h@1L?9Y(BODn~>RQs!s;CgdG-q znP}XoBkm*-0<2=ZPJoRj^!xoOs}#~^&VPwi|InKun+*JVdRjVf8)G$ih4uGNM3ui8 z^+Z&u#5p(}6@3bLV?QvdyXnX699Gg>Yp`U1$We?W6H!?6PHC$XEj2)fft*~?lw445)?Fq_3-Mueb zoGF8?ZC_McvuH=535e-r@{YF%kRmhwm6X1rdzmcJvMK~pz{rTiZy8p@yVc_9wdzk0 z!$WMQ8dtKW2!peml-|S2?Raw0z7zAj#5xg-AzF${Uv_GOkBg7OdBjEz9r97OEs2+? z0BylzJSm7vv2rw!*v^I)S+SklmS;oT>$c>fR?>KdwLp;c(d`(g_VvYq1iPvB`KMdb zzIacbH~mQ};zm}mSR=e`0mqqC!T%N@h7A&+CJLe)Hc$`+xVcv{gh^}v+CM(=8$VArZ1?6hL@ z?4>1+H6^$*LK7Km2J0w39vPMUESEG35$Tt>vdnvq`G{8p$)$-bMp(c)dpr(fANBh~ z#+f7(M8$A?bJ0hiO!SiWtDml2q)m}e@XOSUrLBGMzMrY0 z+-YBdf|MH3xw-^6y0gvPcfkvuOcm7?nxg{LES@MqmY`o(HK=S_y(+8h;)qUw9~OcD z9e-d!sOV4b9?qXaw!U*?QCI*Sq_)=d#z(6`@Tp*=10&2`m=`HqAG*uPx>#}hpCgX$ z;H#(6lIxQOv(pWLs&Ek^8EV5S9yW(_{dX1FeKpp3u_!TgslWqMpV6Nyjr-}^#>|VX zWu_1}K@--;JE<=tZ-WwjBfjW|MlJ-VVYBo)v~&Ug?M<`@7?+dy{onZWx(KE`xKRtD z1&qXCM+mdOEju+QKtww17~-PHMsg`F{~bS3nP^7ebFXen3tv;b0}=xiBtP&YWp!8k zG%<&FU(n*%@EGFw`ajJ9P`X{R^!WS9`~JNzd|HD-fCRV#l;DOop2<&*{K7M)VTQ1R zZFfjrc5REui8nnlZXn7VE+Av@US5d(>jy9coq?^KqYaxMhwRH`pr>51-_W=|_`dHv z?~`fyxBr7cy#`uyv1{1;9M?V|Rs8#{y!r1DkhkJIf@C6p^YfdtArGuAPjfmV-1J}J z4egTm^rAdJxZITanf1-F5zvOb>hW+H_$~H(<(e4iz|`77RYH0AY-tk*$m#bjgbPW2 zs`yqtILE6FLd%b^y^!q9cF4XSPw$>45O558xdfLvmodo@=L!Av_UY+tVVoO(h?5M^f@>{ZG{nxqlk2aJI=uf52iw*LS> z2;b8TPT1L=0YpK?SU=UB+^{bHyGN{XY6Wdf&&+sQ)K4k_zKUCpv-mUI-_H*KBX-U~ z(4@k!*=a9`NvHH#fY5jF2|~*?W7@zV987!3 zT;j<{AvU}R!hkcl?G~gG7b&f+V}w>Mbzxol%R+>S(BAAR7YX*JGdmoStcel@s-la# zCWye5xdn5sDh;La5kNb7QcN>pu{yf`jefmJpbua~l(z6*!p`5fy`umu8{vu4IHdf` z4J?0e3!l~5cZVSJhjBGyPj8@peDresR`U`_XfVkne|IBP#g8S)2;S+=1}bkCkW4v5QzrMuQ~72ux&z3;zH5PT{C literal 0 HcmV?d00001 diff --git a/kcbq-connector/logos/confluent.png b/kcbq-connector/logos/confluent.png new file mode 100644 index 0000000000000000000000000000000000000000..14cd8c506e46fc70ccd274c4c9f4f655ffb14800 GIT binary patch literal 3156 zcmV-a46E~rP) zd30699mlUwYHdZ^g2uun>JqI2H9W8*6n3Z6Hpv8-!2c@~#0Z6;^@O;5l%OlI9Omnl%sR z6s>q5r+6T6Y>8JTRI}{EzF0$W+7=N-LD%IqyXxH z?Co7)!?Hge;Jn{+( z3E2rrc7yQe=*@XCtL^BE;g}>}*_Qw&Qve`)Dwn-8{a#QM9aANZd%Sl-_6W}3DYEvq&?+&uhcAFqnOpz!eV*Eg27Tv}A>fKdLnlc(P-`IHqDmYRe=3p)D9 zo&v^#zk?lMCnyBJ2M>V;BGy9oWRMBo2HSXE#b7=d46e5<>{g&Rm-0{idMK~`5PdIr z7eBA$*WErqB`$F*BKZyQBv{S!xyvV=jqG1#?#xb*2k9z)4?_Z^`4;#Mzji8@&EOp{ z3EUGPpKF;r%ajFP1?vEn>s}V_=7p%dH-g#VLFOU5M)-rgtQH}U6_9-wWM2x&8youU zW%uIWL14h5q9X(HHyyeGvO8RMXT>{3&6cj+JpjTFq!c&A>bsJ=e@hl7HVP zD#IWE5XLb+L5>3AAeA3y--cQQ!%a+3RlG0pjaSgjH z%J`oVIn4~W$V+B7Wbb1k`++KHGukkg-aG^xA`&dg9{CxvJCNN`WOpFD1KF(xx5@>5 z4U|b2bQ~-IG=O^^H<62{vQW9`i5l4(gCZq%F?gA|Ezk_`H|6*reJbJ~Jnkb<0CKN#nPAR`#{Mu)N zS-dQED>0AxoI8uhl{5FvKyUJC{9F^ji~O^aNS^5}7-a7u%kCk(Ec=3dy^8$S#Xc1HYRzg`1|x_PU`WR`Hlg zrV6{5byH&uf_^Er^e_2T@M0c!6x?VbV=S^$*?!3~-@cM?O*Ju@QkiYxX>|F0m$@@d zB0j|sE7lGyW~P!JVi~PJaN|Q~;Ky|qvgb#AZjwC=Wdule_cNb?88SLKrn5t%U_&2$ zwNw~)d*vg^E_f4c!c8}&0f+SYIA-6H2;}b008NaLeX=Pp$nNkF{}dj727FPDxsY|U z%(Cz+H8VNa@>+w)E@G&w_65s){bX+?mDLK%JgGax2Z_e+nUMWI{&^-jXGgVfRgo1s zA1=R)){g-vLUcTTD8dhaaM>$AvzJ|<#B`Dh?@?Vo9h(q zOtMcgF13W^&%d$XulNM>7uAhF^TgFh~7+*{rmjbL=Pq*qw$cvg+cad%;IS_%O3gt z5uVpcX5W$tx=NMt4UgkNWhY8`nqh`C%X!RYhTKbQ84a=O8f<}8BR|wHSvK$4~P}83hyZs&ySlM2T=Xt#v zV|p==8sTxwD7%N~#5OMb6NYovr0j!O<(tlL8c{3z`Rv;S*Sk)H@2QuRW{gMF~%2E+&x<(Hkl4QgRVyvyP%#V3q zJ7Y*oGKl{EAdll#-57;pp5`e-Oijw(o5xfzcMK7e?9!$+oG0`Nm;DD)8j_b5e`r4a zJp(&9T{P#8MfMRq%?E}UE7_|;qGttM8mD+k_6ynVWaO@pZH4H#L&~drz_b&1y7SCk zd8`iE`4p-4wO!M)cN0^f=SHSB0}kPnuVk*XWm(sEyG{hX@ zG5vHgQNQRGhs1pQob?|$1*Yx2lI#VAWl_Sj%2@-(E8nzoN2E*fcPPhhlaA4A^>&}U zz5`D4_;XDE<>tC5JHfkBNf0P0WRWl5Umt$>b&E>!qTR0zahF4Oo0eUy6Z+Ch&5;6SM=j@bf>w+^qnUA$txa?^KPli-zxgLrk`Gt@J%#TK9-rckq(773cs` zL5fE9ROxsG$es$>Q)JnN@X|@gvS|54L4NG{ zw*T&d(ezlJ%*|7M(Zfs|ok}7zgbjh#)#eqrqx*;)QzKQ$-%sAsa(Fv+{WQj{!#&@rLIqVem?{a{%H)xCksMO z^we`y`lkGzJim1A?mMc&)fjxE`;>H-GNo<0&JMiXXg4M8TJrE!05iPLZ9hFVZD396rZB>UqFcROh>ZSwsiNcYa(5ym9@ z(CL2;6aNZIP<_;K*-3Dj>XtnreSM-1%dU!Eld>n|uMgBw*}X&u%^~|Ukah-`uJ8)~ zcQ{0EAj#emJPT&hamYRcvQLB1cl!ySkiV``2W9V{w^hgdo-?=j{?7<{^5)nF;Em?EeRy<<9*ujhS5l0000 + + + 4.0.0 + + + com.wepay.kcbq + kcbq-parent + 1.1.3-SNAPSHOT + .. + + + kcbq-connector + kafka-connect-bigquery + + + + org.apache.kafka + connect-api + + + + com.fasterxml.jackson.core + jackson-core + + + com.google.guava + guava + + + com.google.code.findbugs + jsr305 + + + com.google.http-client + google-http-client + + + com.google.http-client + google-http-client-jackson2 + + + com.google.cloud + google-cloud + + + com.google.auth + google-auth-library-oauth2-http + + + org.xerial.snappy + snappy-java + + + org.slf4j + slf4j-api + + + io.debezium + debezium-core + + + + com.wepay.kcbq + kcbq-api + + + com.wepay.kcbq + kcbq-confluent + + + + junit + junit + + + org.mockito + mockito-core + + + org.slf4j + slf4j-log4j12 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + org.apache.maven.plugins + maven-failsafe-plugin + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + io.confluent + kafka-connect-maven-plugin + + + + kafka-connect + + + BigQuery Sink Connector + kafka-connect-bigquery + + A sink connector for writing to Google BigQuery, with support for automatic table creation and schema evolution. + + logos/BigQuery.png + https://docs.confluent.io/current/connect/kafka-connect-bigquery/ + https://github.com/confluentinc/kafka-connect-bigquery + + Confluent, Inc. + supports WePay's BigQuery connector version 1.1.2 and later, as part of a Confluent Platform subscription. + ]]> + https://docs.confluent.io/current/connect/kafka-connect-bigquery/ + logos/confluent.png + + wepay + organization + WePay + https://go.wepay.com/ + + true + + + sink + + + + cloud + analytics + data + gcp + google + bigquery + warehouse + platform + nosql + + + + Apache Kafka 0.11 or higher / Confluent Platform 3.3 or higher + Java 1.8 or higher + Active Google Cloud Platform (GCP) account with authorization to create resources + Kafka Connect 0.11 or higher / Confluent Platform 3.3 or higher + + + + + + + + diff --git a/kcbq-connector/src/integration-test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java similarity index 100% rename from kcbq-connector/src/integration-test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java rename to kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java diff --git a/kcbq-connector/src/integration-test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java similarity index 100% rename from kcbq-connector/src/integration-test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java rename to kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java diff --git a/kcbq-connector/src/test/resources/log4j.properties b/kcbq-connector/src/test/resources/log4j.properties new file mode 100644 index 000000000..2a5a0233d --- /dev/null +++ b/kcbq-connector/src/test/resources/log4j.properties @@ -0,0 +1,14 @@ +log4j.rootLogger=INFO, stdout + +# Send the logs to the console. +# +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout + +connect.log.pattern=[%d] %p %X{connector.context}%m (%c:%L)%n +log4j.appender.stdout.layout.ConversionPattern=${connect.log.pattern} +log4j.appender.connectAppender.layout.ConversionPattern=${connect.log.pattern} + +# These are used in the log4j properties file that ships by default with Connect +log4j.logger.org.apache.zookeeper=ERROR +log4j.logger.org.reflections=ERROR diff --git a/kcbq-connector/test/docker/connect/connect-docker.sh b/kcbq-connector/test/docker/connect/connect-docker.sh index f5d103863..d0f78fafd 100755 --- a/kcbq-connector/test/docker/connect/connect-docker.sh +++ b/kcbq-connector/test/docker/connect/connect-docker.sh @@ -14,8 +14,7 @@ # specific language governing permissions and limitations # under the License. -tar -C /usr/share/java/kafka-connect-bigquery/ -xf /usr/share/java/kafka-connect-bigquery/kcbq.tar -tar -C /usr/share/java/kafka-connect-bigquery/ -xf /usr/share/java/kafka-connect-bigquery/retriever.tar +unzip -j -d /usr/share/java/kafka-connect-bigquery /usr/share/java/kafka-connect-bigquery/kcbq.zip 'wepay-kafka-connect-bigquery-*/lib/*.jar' connect-standalone \ /etc/kafka-connect-bigquery/standalone.properties \ diff --git a/kcbq-connector/test/integrationtest.sh b/kcbq-connector/test/integrationtest.sh index a65ae7038..df6f17de2 100755 --- a/kcbq-connector/test/integrationtest.sh +++ b/kcbq-connector/test/integrationtest.sh @@ -84,7 +84,6 @@ log() { } BASE_DIR=$(dirname "$0") -GRADLEW="$BASE_DIR/../../gradlew" #################################################################################################### # Configuration processing @@ -194,19 +193,17 @@ docker start -a "$POPULATE_DOCKER_NAME" # Deleting existing BigQuery tables warn 'Deleting existing BigQuery test tables' -"$GRADLEW" -p "$BASE_DIR/.." \ - -Pkcbq_test_keyfile="$KCBQ_TEST_KEYFILE" \ - -Pkcbq_test_project="$KCBQ_TEST_PROJECT" \ - -Pkcbq_test_dataset="$KCBQ_TEST_DATASET" \ - -Pkcbq_test_tables="$(basename "$BASE_DIR"/resources/test_schemas/* | sed -E -e 's/[^a-zA-Z0-9_]/_/g' -e 's/^(.*)$/kcbq_test_\1/' | xargs echo -n)" \ - integrationTestPrep +TEST_TABLES="$(basename "$BASE_DIR"/resources/test_schemas/* | sed -E -e 's/[^a-zA-Z0-9_]/_/g' -e 's/^(.*)$/kcbq_test_\1/' | xargs echo -n)" +mvn -f "$BASE_DIR/.." clean test-compile exec:java \ + -Dexec.mainClass=com.wepay.kafka.connect.bigquery.it.utils.TableClearer \ + -Dexec.classpathScope=test \ + -Dexec.args="${KCBQ_TEST_KEYFILE} ${KCBQ_TEST_PROJECT} ${KCBQ_TEST_DATASET} ${TEST_TABLES}" #################################################################################################### # Executing connector in standalone mode (this is the execution portion of the actual test) statusupdate 'Executing Kafka Connect in Docker' -# Run clean task to ensure there's only one connector tarball in the build directory -"$GRADLEW" -q -p "$BASE_DIR/../.." clean confluentTarBall +mvn -f "$BASE_DIR/.." install -Dskip.unit.tests=true [[ ! -e "$DOCKER_DIR/connect/properties" ]] && mkdir "$DOCKER_DIR/connect/properties" RESOURCES_DIR="$BASE_DIR/resources" @@ -232,8 +229,7 @@ echo >> "$CONNECTOR_PROPS" CONNECT_DOCKER_IMAGE='kcbq/connect' CONNECT_DOCKER_NAME='kcbq_test_connect' -cp "$BASE_DIR"/../../bin/tar/kcbq-connector-*-confluent-dist.tar "$DOCKER_DIR/connect/kcbq.tar" -cp "$BASE_DIR"/../../bin/tar/kcbq-connector-*-confluent-dist.tar "$DOCKER_DIR/connect/retriever.tar" +cp "$BASE_DIR"/../target/components/packages/wepay-kafka-connect-bigquery-*.zip "$DOCKER_DIR/connect/kcbq.zip" cp "$KCBQ_TEST_KEYFILE" "$DOCKER_DIR/connect/key.json" if ! dockerimageexists "$CONNECT_DOCKER_IMAGE"; then @@ -242,8 +238,7 @@ fi docker create --name "$CONNECT_DOCKER_NAME" \ --link "$KAFKA_DOCKER_NAME:kafka" --link "$SCHEMA_REGISTRY_DOCKER_NAME:schema-registry" \ -t "$CONNECT_DOCKER_IMAGE" /bin/bash -docker cp "$DOCKER_DIR/connect/kcbq.tar" "$CONNECT_DOCKER_NAME:/usr/share/java/kafka-connect-bigquery/kcbq.tar" -docker cp "$DOCKER_DIR/connect/retriever.tar" "$CONNECT_DOCKER_NAME:/usr/share/java/kafka-connect-bigquery/retriever.tar" +docker cp "$DOCKER_DIR/connect/kcbq.zip" "$CONNECT_DOCKER_NAME:/usr/share/java/kafka-connect-bigquery/kcbq.zip" docker cp "$DOCKER_DIR/connect/properties/" "$CONNECT_DOCKER_NAME:/etc/kafka-connect-bigquery/" docker cp "$DOCKER_DIR/connect/key.json" "$CONNECT_DOCKER_NAME:/tmp/key.json" docker start -a "$CONNECT_DOCKER_NAME" @@ -252,12 +247,12 @@ docker start -a "$CONNECT_DOCKER_NAME" # Checking on BigQuery data via Java test (this is the verification portion of the actual test) statusupdate 'Verifying that test data made it successfully to BigQuery' -INTEGRATION_TEST_RESOURCE_DIR="$BASE_DIR/../src/integration-test/resources" -[[ ! -d "$INTEGRATION_TEST_RESOURCE_DIR" ]] && mkdir -p "$INTEGRATION_TEST_RESOURCE_DIR" -INTEGRATION_TEST_PROPERTIES_FILE="$INTEGRATION_TEST_RESOURCE_DIR/test.properties" +TEST_RESOURCE_DIR="$BASE_DIR/../src/test/resources" +[[ ! -d "$TEST_RESOURCE_DIR" ]] && mkdir -p "$TEST_RESOURCE_DIR" +INTEGRATION_TEST_PROPERTIES_FILE="$TEST_RESOURCE_DIR/test.properties" echo "keyfile=$KCBQ_TEST_KEYFILE" > "$INTEGRATION_TEST_PROPERTIES_FILE" echo "project=$KCBQ_TEST_PROJECT" >> "$INTEGRATION_TEST_PROPERTIES_FILE" echo "dataset=$KCBQ_TEST_DATASET" >> "$INTEGRATION_TEST_PROPERTIES_FILE" -"$GRADLEW" -p "$BASE_DIR/.." cleanIntegrationTest integrationTest +mvn -f "$BASE_DIR/.." -Dskip.unit.tests=true integration-test \ No newline at end of file diff --git a/pom.xml b/pom.xml new file mode 100644 index 000000000..b05ee38e7 --- /dev/null +++ b/pom.xml @@ -0,0 +1,346 @@ + + + + 4.0.0 + + com.wepay.kcbq + kcbq-parent + 1.1.3-SNAPSHOT + pom + + + kcbq-api + kcbq-confluent + kcbq-connector + + + + 8 + + 1.8.1 + 3.2.0 + 0.4.0 + 3.0.0 + 0.9.0 + 0.25.0-alpha + 1.22.0 + 20.0 + 2.6.3 + 1.0.0 + 1.7.25 + 1.1.4 + + 4.12 + 1.10.19 + + 2.15 + 6.18 + 3.8.1 + 0.8.5 + 0.11.1 + 2.5.3 + 3.0.0-M4 + + ${maven.test.skip} + + + kafka-connect-bigquery-parent + + https://github.com/confluentinc/kafka-connect-bigquery + + 2016 + + + + Apache License 2.0 + https://www.apache.org/licenses/LICENSE-2.0 + repo + + + + + scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git + scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git + https://github.com/confluentinc/kafka-connect-bigquery + HEAD + + + + + C0urante + Chris Egerton + fearthecellos@gmail.com + America/New_York + + + moirat + Moira Tagle + moirat@wepay.com + America/Los_Angeles + + + + + + confluent + http://packages.confluent.io/maven/ + + + jcenter + https://jcenter.bintray.com + + + + + + confluent + http://packages.confluent.io/maven/ + + + jcenter + https://jcenter.bintray.com + + + + + + + + com.wepay.kcbq + kcbq-api + ${project.version} + + + com.wepay.kcbq + kcbq-confluent + ${project.version} + + + + + org.apache.kafka + connect-api + ${kafka.version} + provided + + + org.apache.kafka + kafka-clients + ${kafka.version} + provided + + + + com.google.cloud + google-cloud + ${google.cloud.version} + + + com.google.auth + google-auth-library-oauth2-http + ${google.auth.version} + + + org.slf4j + slf4j-api + ${slf4j.version} + + + io.debezium + debezium-core + ${debezium.version} + + + org.apache.avro + avro + ${avro.version} + + + io.confluent + kafka-connect-avro-converter + ${confluent.version} + + + io.confluent + kafka-schema-registry-client + ${confluent.version} + + + + + org.xerial.snappy + snappy-java + ${snappy.version} + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + com.google.guava + guava + ${guava.version} + + + com.google.code.findbugs + jsr305 + ${findbugs.jsr305.version} + + + com.google.http-client + google-http-client + ${google.http.version} + + + com.google.http-client + google-http-client-jackson2 + ${google.http.version} + + + + junit + junit + ${junit.version} + test + + + org.mockito + mockito-core + ${mockito.version} + test + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + test + + + + + + + + org.apache.maven.plugins + maven-release-plugin + ${release.plugin.version} + + true + false + v@{project.version} + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + ${compiler.plugin.version} + + ${java.version} + ${java.version} + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire.plugin.version} + + + **/*IntegrationTest.java + + ${skip.unit.tests} + + + + org.apache.maven.plugins + maven-failsafe-plugin + ${surefire.plugin.version} + + + **/*IntegrationTest.java + + + + + integration-test + + integration-test + + + + + + org.jacoco + jacoco-maven-plugin + ${jacoco.plugin.version} + + + pre-unit-test + + prepare-agent + + + + report + verify + + report + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + ${checkstyle.plugin.version} + + + validate + validate + + ${project.parent.basedir}/config/checkstyle/google_checks.xml + + + check + + + + + + com.puppycrawl.tools + checkstyle + ${checkstyle.version} + + + + + io.confluent + kafka-connect-maven-plugin + ${kafka.connect.plugin.version} + + + + + diff --git a/settings.gradle b/settings.gradle deleted file mode 100644 index a24368b32..000000000 --- a/settings.gradle +++ /dev/null @@ -1 +0,0 @@ -include 'kcbq-connector', 'kcbq-api', 'kcbq-confluent' From 51ae613656b9b9bb432a8ef333456195a836b801 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 23 Sep 2020 11:42:46 -0400 Subject: [PATCH 031/190] GH-27: Add allowSchemaUnionization config property (#28) * GH-27: Add allowSchemaUnionization config property Still needed: unit and possibly integration tests for the logic in the SchemaManager class * GH-27: Tweak schema change validation logic * GH-27: Fix schema update bugs, add unit tests --- .../connect/bigquery/BigQuerySinkTask.java | 3 +- .../kafka/connect/bigquery/SchemaManager.java | 162 ++++++++--- .../bigquery/config/BigQuerySinkConfig.java | 32 ++- .../write/row/AdaptiveBigQueryWriter.java | 11 +- .../bigquery/write/row/GCSToBQWriter.java | 7 +- .../write/row/UpsertDeleteBigQueryWriter.java | 7 +- .../connect/bigquery/SchemaManagerTest.java | 258 +++++++++++++++++- 7 files changed, 411 insertions(+), 69 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 2f1e1fc2c..85ef9a67b 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -316,8 +316,9 @@ private SchemaManager newSchemaManager() { Optional> clusteringFieldName = config.getClusteringPartitionFieldName(); boolean allowNewBQFields = config.getBoolean(config.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); boolean allowReqFieldRelaxation = config.getBoolean(config.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); + boolean allowSchemaUnionization = config.getBoolean(config.ALLOW_SCHEMA_UNIONIZATION_CONFIG); return new SchemaManager(schemaRetriever, schemaConverter, getBigQuery(), - allowNewBQFields, allowReqFieldRelaxation, + allowNewBQFields, allowReqFieldRelaxation, allowSchemaUnionization, kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, clusteringFieldName); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 129de9a09..b08c9238e 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -7,11 +7,11 @@ import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.LegacySQLTypeName; import com.google.cloud.bigquery.StandardTableDefinition; -import com.google.cloud.bigquery.Table; import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TableInfo; import com.google.cloud.bigquery.TimePartitioning; import com.google.cloud.bigquery.TimePartitioning.Type; +import com.google.common.annotations.VisibleForTesting; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.convert.KafkaDataBuilder; @@ -28,10 +28,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; -import java.util.SortedMap; -import java.util.function.Function; -import java.util.stream.Collectors; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -47,6 +43,7 @@ public class SchemaManager { private final BigQuery bigQuery; private final boolean allowNewBQFields; private final boolean allowBQRequiredFieldRelaxation; + private final boolean allowSchemaUnionization; private final Optional kafkaKeyFieldName; private final Optional kafkaDataFieldName; private final Optional timestampPartitionFieldName; @@ -63,6 +60,7 @@ public class SchemaManager { * @param bigQuery Used to communicate create/update requests to BigQuery. * @param allowNewBQFields If set to true, allows new fields to be added to BigQuery Schema. * @param allowBQRequiredFieldRelaxation If set to true, allows changing field mode from REQUIRED to NULLABLE + * @param allowSchemaUnionization If set to true, allows existing and new schemas to be unionized * @param kafkaKeyFieldName The name of kafka key field to be used in BigQuery. * If set to null, Kafka Key Field will not be included in BigQuery. * @param kafkaDataFieldName The name of kafka data field to be used in BigQuery. @@ -79,6 +77,7 @@ public SchemaManager( BigQuery bigQuery, boolean allowNewBQFields, boolean allowBQRequiredFieldRelaxation, + boolean allowSchemaUnionization, Optional kafkaKeyFieldName, Optional kafkaDataFieldName, Optional timestampPartitionFieldName, @@ -89,6 +88,7 @@ public SchemaManager( bigQuery, allowNewBQFields, allowBQRequiredFieldRelaxation, + allowSchemaUnionization, kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, @@ -105,6 +105,7 @@ private SchemaManager( BigQuery bigQuery, boolean allowNewBQFields, boolean allowBQRequiredFieldRelaxation, + boolean allowSchemaUnionization, Optional kafkaKeyFieldName, Optional kafkaDataFieldName, Optional timestampPartitionFieldName, @@ -118,6 +119,7 @@ private SchemaManager( this.bigQuery = bigQuery; this.allowNewBQFields = allowNewBQFields; this.allowBQRequiredFieldRelaxation = allowBQRequiredFieldRelaxation; + this.allowSchemaUnionization = allowSchemaUnionization; this.kafkaKeyFieldName = kafkaKeyFieldName; this.kafkaDataFieldName = kafkaDataFieldName; this.timestampPartitionFieldName = timestampPartitionFieldName; @@ -135,6 +137,7 @@ public SchemaManager forIntermediateTables() { bigQuery, allowNewBQFields, allowBQRequiredFieldRelaxation, + allowSchemaUnionization, kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, @@ -163,7 +166,7 @@ public com.google.cloud.bigquery.Schema cachedSchema(TableId table) { * @param table The BigQuery table to create, * @param records The sink records used to determine the schema. */ - public void createOrUpdateTable(TableId table, Set records) { + public void createOrUpdateTable(TableId table, List records) { synchronized (lock(tableCreateLocks, table)) { if (bigQuery.getTable(table) == null) { logger.debug("{} doesn't exist; creating instead of updating", table(table)); @@ -184,7 +187,7 @@ public void createOrUpdateTable(TableId table, Set records) { * @param records The sink records used to determine the schema. * @return whether the table had to be created; if the table already existed, will return false */ - public boolean createTable(TableId table, Set records) { + public boolean createTable(TableId table, List records) { synchronized (lock(tableCreateLocks, table)) { if (schemaCache.containsKey(table)) { // Table already exists; noop @@ -215,7 +218,7 @@ public boolean createTable(TableId table, Set records) { * @param table The BigQuery table to update. * @param records The sink records used to update the schema. */ - public void updateSchema(TableId table, Set records) { + public void updateSchema(TableId table, List records) { synchronized (lock(tableUpdateLocks, table)) { TableInfo tableInfo = getTableInfo(table, records); if (!schemaCache.containsKey(table)) { @@ -232,7 +235,6 @@ public void updateSchema(TableId table, Set records) { logger.debug("Skipping update of {} since current schema should be compatible", table(table)); } } - } /** @@ -241,17 +243,36 @@ public void updateSchema(TableId table, Set records) { * @param records The sink records used to determine the schema for constructing the table info * @return The resulting BigQuery table information */ - private TableInfo getTableInfo(TableId table, Set records) { - List bigQuerySchemas = getSchemasList(table, records); - com.google.cloud.bigquery.Schema schema; + private TableInfo getTableInfo(TableId table, List records) { + com.google.cloud.bigquery.Schema proposedSchema; String tableDescription; try { - schema = getUnionizedSchema(bigQuerySchemas); + proposedSchema = getAndValidateProposedSchema(table, records); tableDescription = getUnionizedTableDescription(records); } catch (BigQueryConnectException exception) { throw new BigQueryConnectException("Failed to unionize schemas of records for the table " + table, exception); } - return constructTableInfo(table, schema, tableDescription); + return constructTableInfo(table, proposedSchema, tableDescription); + } + + @VisibleForTesting + com.google.cloud.bigquery.Schema getAndValidateProposedSchema( + TableId table, List records) { + com.google.cloud.bigquery.Schema result; + if (allowSchemaUnionization) { + List bigQuerySchemas = getSchemasList(table, records); + result = getUnionizedSchema(bigQuerySchemas); + } else { + com.google.cloud.bigquery.Schema existingSchema = readTableSchema(table); + result = convertRecordSchema(records.get(records.size() - 1)); + if (existingSchema != null) { + validateSchemaChange(existingSchema, result); + if (allowBQRequiredFieldRelaxation) { + result = relaxFieldsWhereNecessary(existingSchema, result); + } + } + } + return result; } /** @@ -260,21 +281,22 @@ private TableInfo getTableInfo(TableId table, Set records) { * @param records The sink records' schemas to add to the list of schemas * @return List of BigQuery schemas */ - private List getSchemasList(TableId table, Set records) { + private List getSchemasList(TableId table, List records) { List bigQuerySchemas = new ArrayList<>(); - if (bigQuery.getTable(table) != null) { - Table bigQueryTable = bigQuery.getTable(table.getDataset(), table.getTable()); - bigQuerySchemas.add(bigQueryTable.getDefinition().getSchema()); - } + Optional.ofNullable(readTableSchema(table)).ifPresent(bigQuerySchemas::add); for (SinkRecord record : records) { - Schema kafkaValueSchema = schemaRetriever.retrieveValueSchema(record); - Schema kafkaKeySchema = kafkaKeyFieldName.isPresent() ? schemaRetriever.retrieveKeySchema(record) : null; - com.google.cloud.bigquery.Schema schema = getBigQuerySchema(kafkaKeySchema, kafkaValueSchema); - bigQuerySchemas.add(schema); + bigQuerySchemas.add(convertRecordSchema(record)); } return bigQuerySchemas; } + private com.google.cloud.bigquery.Schema convertRecordSchema(SinkRecord record) { + Schema kafkaValueSchema = schemaRetriever.retrieveValueSchema(record); + Schema kafkaKeySchema = kafkaKeyFieldName.isPresent() ? schemaRetriever.retrieveKeySchema(record) : null; + com.google.cloud.bigquery.Schema result = getBigQuerySchema(kafkaKeySchema, kafkaValueSchema); + return result; + } + /** * Returns a unionized schema from a list of BigQuery schemas * @param schemas The list of BigQuery schemas to unionize @@ -282,8 +304,11 @@ private List getSchemasList(TableId table, Set */ private com.google.cloud.bigquery.Schema getUnionizedSchema(List schemas) { com.google.cloud.bigquery.Schema currentSchema = schemas.get(0); + com.google.cloud.bigquery.Schema proposedSchema; for (int i = 1; i < schemas.size(); i++) { - currentSchema = unionizeSchemas(currentSchema, schemas.get(i)); + proposedSchema = unionizeSchemas(currentSchema, schemas.get(i)); + validateSchemaChange(currentSchema, proposedSchema); + currentSchema = proposedSchema; } return currentSchema; } @@ -294,31 +319,78 @@ private com.google.cloud.bigquery.Schema getUnionizedSchema(List firstSchemaFields = schemaFields(firstSchema); Map secondSchemaFields = schemaFields(secondSchema); - for (Map.Entry entry : secondSchemaFields.entrySet()) { - if (!firstSchemaFields.containsKey(entry.getKey())) { - if (allowNewBQFields && (entry.getValue().getMode().equals(Field.Mode.NULLABLE) - || (entry.getValue().getMode().equals(Field.Mode.REQUIRED) && allowBQRequiredFieldRelaxation))) { - firstSchemaFields.put(entry.getKey(), entry.getValue().toBuilder().setMode(Field.Mode.NULLABLE).build()); - } else { + Map unionizedSchemaFields = new LinkedHashMap<>(); + + firstSchemaFields.forEach((name, firstField) -> { + Field secondField = secondSchemaFields.get(name); + if (secondField == null) { + unionizedSchemaFields.put(name, firstField.toBuilder().setMode(Field.Mode.NULLABLE).build()); + } else if (isFieldRelaxation(firstField, secondField)) { + unionizedSchemaFields.put(name, secondField); + } else { + unionizedSchemaFields.put(name, firstField); + } + }); + + secondSchemaFields.forEach((name, secondField) -> { + if (!unionizedSchemaFields.containsKey(name)) { + unionizedSchemaFields.put(name, secondField.toBuilder().setMode(Field.Mode.NULLABLE).build()); + } + }); + return com.google.cloud.bigquery.Schema.of(unionizedSchemaFields.values()); + } + + private void validateSchemaChange( + com.google.cloud.bigquery.Schema existingSchema, com.google.cloud.bigquery.Schema proposedSchema) { + Map earliestSchemaFields = schemaFields(existingSchema); + Map proposedSchemaFields = schemaFields(proposedSchema); + for (Map.Entry entry : proposedSchemaFields.entrySet()) { + if (!earliestSchemaFields.containsKey(entry.getKey())) { + if (!isValidFieldAddition(entry.getValue())) { throw new BigQueryConnectException("New Field found with the name " + entry.getKey() - + " Ensure that " + BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG + " is true and " + BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG + - " is true if " + entry.getKey() + " has mode REQUIRED in order to update the Schema"); + + " Ensure that " + BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG + " is true and " + + BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG + + " is true if " + entry.getKey() + " has mode REQUIRED in order to update the Schema"); } - } else { - if (firstSchemaFields.get(entry.getKey()).getMode().equals(Field.Mode.REQUIRED) && secondSchemaFields.get(entry.getKey()).getMode().equals(Field.Mode.NULLABLE)) { - if (allowBQRequiredFieldRelaxation) { - firstSchemaFields.put(entry.getKey(), entry.getValue().toBuilder().setMode(Field.Mode.NULLABLE).build()); - } else { - throw new BigQueryConnectException( entry.getKey() + " has mode REQUIRED. Set " + BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG - + " to true, to change the mode to NULLABLE"); - } + } else if (isFieldRelaxation(earliestSchemaFields.get(entry.getKey()), entry.getValue())) { + if (!allowBQRequiredFieldRelaxation) { + throw new BigQueryConnectException( entry.getKey() + " has mode REQUIRED. Set " + + BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG + + " to true, to change the mode to NULLABLE"); } } } - return com.google.cloud.bigquery.Schema.of(firstSchemaFields.values()); + } + + private boolean isFieldRelaxation(Field currentField, Field proposedField) { + return currentField.getMode().equals(Field.Mode.REQUIRED) + && proposedField.getMode().equals(Field.Mode.NULLABLE); + } + + private boolean isValidFieldAddition(Field newField) { + return allowNewBQFields && ( + newField.getMode().equals(Field.Mode.NULLABLE) || + (newField.getMode().equals(Field.Mode.REQUIRED) && allowBQRequiredFieldRelaxation)); + } + + private com.google.cloud.bigquery.Schema relaxFieldsWhereNecessary( + com.google.cloud.bigquery.Schema existingSchema, + com.google.cloud.bigquery.Schema proposedSchema) { + Map existingSchemaFields = schemaFields(existingSchema); + Map proposedSchemaFields = schemaFields(proposedSchema); + List newSchemaFields = new ArrayList<>(); + for (Map.Entry entry : proposedSchemaFields.entrySet()) { + if (!existingSchemaFields.containsKey(entry.getKey())) { + newSchemaFields.add(entry.getValue().toBuilder().setMode(Field.Mode.NULLABLE).build()); + } else { + newSchemaFields.add(entry.getValue()); + } + } + return com.google.cloud.bigquery.Schema.of(newSchemaFields); } /** @@ -326,7 +398,7 @@ private com.google.cloud.bigquery.Schema unionizeSchemas(com.google.cloud.bigque * @param records The records used to get the unionized table description * @return The resulting table description */ - private String getUnionizedTableDescription(Set records) { + private String getUnionizedTableDescription(List records) { String tableDescription = null; for (SinkRecord record : records) { Schema kafkaValueSchema = schemaRetriever.retrieveValueSchema(record); @@ -470,7 +542,9 @@ private String table(TableId table) { private com.google.cloud.bigquery.Schema readTableSchema(TableId table) { logger.trace("Reading schema for {}", table(table)); - return bigQuery.getTable(table).getDefinition().getSchema(); + return Optional.ofNullable(bigQuery.getTable(table)) + .map(t -> t.getDefinition().getSchema()) + .orElse(null); } private Object lock(ConcurrentMap locks, TableId table) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 53b33e561..583f87031 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -217,28 +217,36 @@ public class BigQuerySinkConfig extends AbstractConfig { private static final String TABLE_CREATE_DOC = "Automatically create BigQuery tables if they don't already exist"; - public static final String AUTO_CREATE_BUCKET_CONFIG = "autoCreateBucket"; - private static final ConfigDef.Type AUTO_CREATE_BUCKET_TYPE = ConfigDef.Type.BOOLEAN; - public static final Boolean AUTO_CREATE_BUCKET_DEFAULT = true; + public static final String AUTO_CREATE_BUCKET_CONFIG = "autoCreateBucket"; + private static final ConfigDef.Type AUTO_CREATE_BUCKET_TYPE = ConfigDef.Type.BOOLEAN; + public static final Boolean AUTO_CREATE_BUCKET_DEFAULT = true; private static final ConfigDef.Importance AUTO_CREATE_BUCKET_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String AUTO_CREATE_BUCKET_DOC = "Whether to automatically create the given bucket, if it does not exist. " + "Only relevant if enableBatchLoad is configured."; - public static final String ALLOW_NEW_BIGQUERY_FIELDS_CONFIG = "allowNewBigQueryFields"; - private static final ConfigDef.Type ALLOW_NEW_BIGQUERY_FIELDS_TYPE = ConfigDef.Type.BOOLEAN; - public static final Boolean ALLOW_NEW_BIGQUERY_FIELDS_DEFAULT = false; + public static final String ALLOW_NEW_BIGQUERY_FIELDS_CONFIG = "allowNewBigQueryFields"; + private static final ConfigDef.Type ALLOW_NEW_BIGQUERY_FIELDS_TYPE = ConfigDef.Type.BOOLEAN; + public static final Boolean ALLOW_NEW_BIGQUERY_FIELDS_DEFAULT = false; private static final ConfigDef.Importance ALLOW_NEW_BIGQUERY_FIELDS_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String ALLOW_NEW_BIGQUERY_FIELDS_DOC = "If true, new fields can be added to BigQuery tables during subsequent schema updates"; - public static final String ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG = "allowBigQueryRequiredFieldRelaxation"; - private static final ConfigDef.Type ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_TYPE = ConfigDef.Type.BOOLEAN; - public static final Boolean ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_DEFAULT = false; + public static final String ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG = "allowBigQueryRequiredFieldRelaxation"; + private static final ConfigDef.Type ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_TYPE = ConfigDef.Type.BOOLEAN; + public static final Boolean ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_DEFAULT = false; private static final ConfigDef.Importance ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_DOC = "If true, fields in BigQuery Schema can be changed from REQUIRED to NULLABLE"; + public static final String ALLOW_SCHEMA_UNIONIZATION_CONFIG = "allowSchemaUnionization"; + private static final ConfigDef.Type ALLOW_SCHEMA_UNIONIZATION_TYPE = ConfigDef.Type.BOOLEAN; + public static final Boolean ALLOW_SCHEMA_UNIONIZATION_DEFAULT = false; + private static final ConfigDef.Importance ALLOW_SCHEMA_UNIONIZATION_IMPORTANCE = ConfigDef.Importance.MEDIUM; + private static final String ALLOW_SCHEMA_UNIONIZATION_DOC = + "If true, the existing table schema (if one is present) will be unionized with new " + + "record schemas during schema updates"; + public static final String UPSERT_ENABLED_CONFIG = "upsertEnabled"; private static final ConfigDef.Type UPSERT_ENABLED_TYPE = ConfigDef.Type.BOOLEAN; public static final boolean UPSERT_ENABLED_DEFAULT = false; @@ -435,6 +443,12 @@ public static ConfigDef getConfig() { ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_DEFAULT, ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_IMPORTANCE, ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_DOC + ).define( + ALLOW_SCHEMA_UNIONIZATION_CONFIG, + ALLOW_SCHEMA_UNIONIZATION_TYPE, + ALLOW_SCHEMA_UNIONIZATION_DEFAULT, + ALLOW_SCHEMA_UNIONIZATION_IMPORTANCE, + ALLOW_SCHEMA_UNIONIZATION_DOC ).define( UPSERT_ENABLED_CONFIG, UPSERT_ENABLED_TYPE, diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java index 5af298908..9a6e61aa3 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java @@ -35,6 +35,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -105,14 +106,14 @@ public Map> performWriteRequest( // Should only perform one schema update attempt. if (writeResponse.hasErrors() && onlyContainsInvalidSchemaErrors(writeResponse.getInsertErrors())) { - attemptSchemaUpdate(tableId, rows.keySet()); + attemptSchemaUpdate(tableId, new ArrayList<>(rows.keySet())); } } catch (BigQueryException exception) { // Should only perform one table creation attempt. if (isTableNotExistedException(exception) && autoCreateTables) { - attemptTableCreate(tableId.getBaseTableId(), rows.keySet()); + attemptTableCreate(tableId.getBaseTableId(), new ArrayList<>(rows.keySet())); } else if (isTableMissingSchema(exception)) { - attemptSchemaUpdate(tableId, rows.keySet()); + attemptSchemaUpdate(tableId, new ArrayList<>(rows.keySet())); } else { throw exception; } @@ -152,7 +153,7 @@ && onlyContainsInvalidSchemaErrors(writeResponse.getInsertErrors())) { return new HashMap<>(); } - protected void attemptSchemaUpdate(PartitionedTableId tableId, Set records) { + protected void attemptSchemaUpdate(PartitionedTableId tableId, List records) { try { schemaManager.updateSchema(tableId.getBaseTableId(), records); } catch (BigQueryException exception) { @@ -161,7 +162,7 @@ protected void attemptSchemaUpdate(PartitionedTableId tableId, Set r } } - protected void attemptTableCreate(TableId tableId, Set records) { + protected void attemptTableCreate(TableId tableId, List records) { try { schemaManager.createTable(tableId, records); } catch (BigQueryException exception) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriter.java index 10a37b660..fb9068e5a 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriter.java @@ -40,11 +40,12 @@ import org.slf4j.LoggerFactory; import java.io.UnsupportedEncodingException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Random; -import java.util.Set; import java.util.SortedMap; /** @@ -118,7 +119,7 @@ public void writeRows(SortedMap rows, // Check if the table specified exists // This error shouldn't be thrown. All tables should be created by the connector at startup if (autoCreateTables && bigQuery.getTable(tableId) == null) { - attemptTableCreate(tableId, rows.keySet()); + attemptTableCreate(tableId, new ArrayList<>(rows.keySet())); } int attemptCount = 0; @@ -198,7 +199,7 @@ private void waitRandomTime() throws InterruptedException { Thread.sleep(retryWaitMs + random.nextInt(WAIT_MAX_JITTER)); } - private void attemptTableCreate(TableId tableId, Set records) { + private void attemptTableCreate(TableId tableId, List records) { try { logger.info("Table {} does not exist, auto-creating table ", tableId); schemaManager.createTable(tableId, records); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java index 20e30226d..0dd32af0e 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java @@ -26,9 +26,8 @@ import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import org.apache.kafka.connect.sink.SinkRecord; +import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.concurrent.Future; public class UpsertDeleteBigQueryWriter extends AdaptiveBigQueryWriter { @@ -64,7 +63,7 @@ public UpsertDeleteBigQueryWriter(BigQuery bigQuery, } @Override - protected void attemptSchemaUpdate(PartitionedTableId tableId, Set records) { + protected void attemptSchemaUpdate(PartitionedTableId tableId, List records) { // Update the intermediate table here... super.attemptSchemaUpdate(tableId, records); try { @@ -77,7 +76,7 @@ protected void attemptSchemaUpdate(PartitionedTableId tableId, Set r } @Override - protected void attemptTableCreate(TableId tableId, Set records) { + protected void attemptTableCreate(TableId tableId, List records) { // Create the intermediate table here... super.attemptTableCreate(tableId, records); if (autoCreateTables) { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index fe5fa73cb..5f6449d09 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -25,21 +25,29 @@ import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.LegacySQLTypeName; import com.google.cloud.bigquery.StandardTableDefinition; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableDefinition; import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TableInfo; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; +import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.sink.SinkRecord; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.stubbing.OngoingStubbing; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; public class SchemaManagerTest { @@ -70,7 +78,7 @@ public void testBQTableDescription() { Optional kafkaKeyFieldName = Optional.of("kafkaKey"); Optional kafkaDataFieldName = Optional.of("kafkaData"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, kafkaKeyFieldName, kafkaDataFieldName, Optional.empty(), Optional.empty()); + mockBigQuery, false, false, false, kafkaKeyFieldName, kafkaDataFieldName, Optional.empty(), Optional.empty()); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -88,7 +96,7 @@ public void testBQTableDescription() { public void testTimestampPartitionSet() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -108,7 +116,7 @@ public void testClusteringPartitionSet() { Optional timestampPartitionFieldName = Optional.of("testField"); Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -124,4 +132,248 @@ public void testClusteringPartitionSet() { testField.get(), definition.getClustering().getFields()); } + + @Test + public void testSuccessfulUpdateWithOnlyRelaxedFields() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema relaxedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(false, true, false); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, relaxedSchema, relaxedSchema); + } + + @Test(expected = BigQueryConnectException.class) + public void testDisallowedUpdateWithOnlyRelaxedFields() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema relaxedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, false, false); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, relaxedSchema, null); + } + + @Test + public void testSuccessfulUpdateWithOnlyNewFields() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, false, false); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expandedSchema); + } + + @Test(expected = BigQueryConnectException.class) + public void testDisallowedUpdateWithOnlyNewFields() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(false, true, false); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, null); + } + + @Test(expected = BigQueryConnectException.class) + public void testDisallowedUpdateWithOnlyNewRequiredFields() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.REQUIRED).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, false, false); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, null); + } + + @Test + public void testSuccessfulUpdateWithNewAndRelaxedFields() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expandedAndRelaxedSchema = com.google.cloud.bigquery.Schema.of( + // Relax an existing field from required to nullable + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build(), + // Add a new nullable field + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.NULLABLE).build(), + // Add a new required field (that should be relaxed to nullable automatically) + Field.newBuilder("f3", LegacySQLTypeName.NUMERIC).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.NULLABLE).build(), + Field.newBuilder("f3", LegacySQLTypeName.NUMERIC).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, false); + + testGetAndValidateProposedSchema + (schemaManager, existingSchema, expandedAndRelaxedSchema, expectedSchema); + } + + @Test + public void testSuccessfulUnionizedUpdateWithNewAndRelaxedFields() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema disjointSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, true); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, disjointSchema, expectedSchema); + } + + @Test(expected = BigQueryConnectException.class) + public void testDisallowedUnionizedUpdateWithNewField() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(false, true, true); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, null); + } + + @Test(expected = BigQueryConnectException.class) + public void testDisallowedUnionizedUpdateWithRelaxedField() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, false, true); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, null); + } + + @Test + public void testUnionizedUpdateWithMultipleSchemas() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema firstNewSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build() + ); + com.google.cloud.bigquery.Schema secondNewSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.REQUIRED).build() + ); + com.google.cloud.bigquery.Schema thirdNewSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + List newSchemas = + Arrays.asList(firstNewSchema, secondNewSchema, thirdNewSchema); + + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, true); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, newSchemas, expectedSchema); + } + + private SchemaManager createSchemaManager( + boolean allowNewFields, boolean allowFieldRelaxation, boolean allowUnionization) { + return new SchemaManager(new IdentitySchemaRetriever(), mockSchemaConverter, mockBigQuery, + allowNewFields, allowFieldRelaxation, allowUnionization, + Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); + } + + private void testGetAndValidateProposedSchema( + SchemaManager schemaManager, + com.google.cloud.bigquery.Schema existingSchema, + com.google.cloud.bigquery.Schema newSchema, + com.google.cloud.bigquery.Schema expectedSchema) { + testGetAndValidateProposedSchema( + schemaManager, existingSchema, Collections.singletonList(newSchema), expectedSchema); + } + + private void testGetAndValidateProposedSchema( + SchemaManager schemaManager, + com.google.cloud.bigquery.Schema existingSchema, + List newSchemas, + com.google.cloud.bigquery.Schema expectedSchema) { + Table existingTable = existingSchema != null ? tableWithSchema(existingSchema) : null; + + SinkRecord mockSinkRecord = recordWithValueSchema(mockKafkaSchema); + List incomingSinkRecords = Collections.nCopies(newSchemas.size(), mockSinkRecord); + + when(mockBigQuery.getTable(tableId)).thenReturn(existingTable); + OngoingStubbing converterStub = + when(mockSchemaConverter.convertSchema(mockKafkaSchema)); + for (com.google.cloud.bigquery.Schema newSchema : newSchemas) { + // The converter will return the schemas in the order that they are provided to it with the + // call to "thenReturn" + converterStub = converterStub.thenReturn(newSchema); + } + + com.google.cloud.bigquery.Schema proposedSchema = + schemaManager.getAndValidateProposedSchema(tableId, incomingSinkRecords); + + if (expectedSchema != null) { + Assert.assertEquals(expectedSchema, proposedSchema); + } + } + + private Table tableWithSchema(com.google.cloud.bigquery.Schema schema) { + TableDefinition definition = mock(TableDefinition.class); + when(definition.getSchema()).thenReturn(schema); + + Table result = mock(Table.class); + when(result.getDefinition()).thenReturn(definition); + + return result; + } + + private SinkRecord recordWithValueSchema(Schema valueSchema) { + SinkRecord result = mock(SinkRecord.class); + when(result.valueSchema()).thenReturn(valueSchema); + return result; + } } From b6e82986866e942dc87fb532e0d476f0d5aceed6 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 23 Sep 2020 14:59:10 -0400 Subject: [PATCH 032/190] MINOR: Add Jenkinsfile (#35) * MINOR: Add Jenkinsfile * Fix build error while running 'mvn site' * Prevent integration test classes from running during `mvn integration-test` * Disable Checkstyle on Jenkins * Fix integration test --- Jenkinsfile | 7 +++++ config/checkstyle/suppressions.xml | 8 +++++ kcbq-connector/pom.xml | 4 --- kcbq-connector/test/integrationtest.sh | 4 +-- pom.xml | 41 ++++++++++++++++++++++---- 5 files changed, 52 insertions(+), 12 deletions(-) create mode 100644 Jenkinsfile create mode 100644 config/checkstyle/suppressions.xml diff --git a/Jenkinsfile b/Jenkinsfile new file mode 100644 index 000000000..551c15919 --- /dev/null +++ b/Jenkinsfile @@ -0,0 +1,7 @@ +#!/usr/bin/env groovy +common { + slackChannel = '#connect-warn' + nodeLabel = 'docker-oraclejdk8' + publish = false + downStreamValidate = false +} diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml new file mode 100644 index 000000000..28a683b60 --- /dev/null +++ b/config/checkstyle/suppressions.xml @@ -0,0 +1,8 @@ + + + + + + \ No newline at end of file diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index d1a4a2a4b..6f364dab3 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -107,10 +107,6 @@ org.apache.maven.plugins maven-surefire-plugin - - org.apache.maven.plugins - maven-failsafe-plugin - org.jacoco jacoco-maven-plugin diff --git a/kcbq-connector/test/integrationtest.sh b/kcbq-connector/test/integrationtest.sh index df6f17de2..4ea70bef7 100755 --- a/kcbq-connector/test/integrationtest.sh +++ b/kcbq-connector/test/integrationtest.sh @@ -17,7 +17,7 @@ #################################################################################################### # Basic script setup -set -e +set -ex if [[ -t 1 ]]; then KCBQ_TEST_COLORS='true' @@ -255,4 +255,4 @@ echo "keyfile=$KCBQ_TEST_KEYFILE" > "$INTEGRATION_TEST_PROPERTIES_FILE" echo "project=$KCBQ_TEST_PROJECT" >> "$INTEGRATION_TEST_PROPERTIES_FILE" echo "dataset=$KCBQ_TEST_DATASET" >> "$INTEGRATION_TEST_PROPERTIES_FILE" -mvn -f "$BASE_DIR/.." -Dskip.unit.tests=true integration-test \ No newline at end of file +mvn -f "$BASE_DIR/.." clean test-compile -Dskip.unit.tests=true failsafe:integration-test@verify-docker-test \ No newline at end of file diff --git a/pom.xml b/pom.xml index b05ee38e7..adea1235d 100644 --- a/pom.xml +++ b/pom.xml @@ -53,6 +53,7 @@ 0.8.5 0.11.1 2.5.3 + 3.7.1 3.0.0-M4 ${maven.test.skip} @@ -277,17 +278,17 @@ org.apache.maven.plugins maven-failsafe-plugin ${surefire.plugin.version} - - - **/*IntegrationTest.java - - - integration-test + verify-docker-test integration-test + + + **/*IntegrationTest.java + + @@ -321,6 +322,7 @@ validate ${project.parent.basedir}/config/checkstyle/google_checks.xml + ${project.parent.basedir}/config/checkstyle/suppressions.xml check @@ -335,6 +337,15 @@ + + + org.apache.maven.plugins + maven-site-plugin + ${site.plugin.version} + io.confluent kafka-connect-maven-plugin @@ -343,4 +354,22 @@ + + + jenkins + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + true + + + + + + + From f0ec33958239d2e9783d54ba89a4a2ae796f27f7 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 23 Sep 2020 15:51:43 -0400 Subject: [PATCH 033/190] MINOR: Fix BigQueryWriterTest unit test (#36) --- .../connect/bigquery/write/row/BigQueryWriterTest.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index 7c0e871e5..ca5743f44 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -116,8 +116,7 @@ public void testAutoCreateTables() { when(insertAllResponse.hasErrors()).thenReturn(false); when(insertAllResponse.getInsertErrors()).thenReturn(emptyMap); - BigQueryException missTableException = mock(BigQueryException.class); - when(missTableException.getCode()).thenReturn(404); + BigQueryException missTableException = new BigQueryException(404, "Table is missing"); when(bigQuery.insertAll(anyObject())).thenThrow(missTableException).thenReturn(insertAllResponse); @@ -151,8 +150,7 @@ public void testNonAutoCreateTables() { when(insertAllResponse.hasErrors()).thenReturn(false); when(insertAllResponse.getInsertErrors()).thenReturn(emptyMap); - BigQueryException missTableException = mock(BigQueryException.class); - when(missTableException.getCode()).thenReturn(404); + BigQueryException missTableException = new BigQueryException(404, "Table is missing"); when(bigQuery.insertAll(anyObject())).thenThrow(missTableException).thenReturn(insertAllResponse); From 4eee99ed48f2207cf66e88cf219a3c7d80a9bc27 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 2 Oct 2020 14:44:02 -0400 Subject: [PATCH 034/190] GH-39: Fix NPE when null inner values are encountered with field sanitization enabled (#40) --- .../bigquery/utils/FieldNameSanitizer.java | 21 +++++++++-------- .../utils/FieldNameSanitizerTest.java | 23 +++++++++++++++++++ 2 files changed, 35 insertions(+), 9 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizer.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizer.java index 09aeb70c2..880d5fd22 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizer.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizer.java @@ -1,5 +1,6 @@ package com.wepay.kafka.connect.bigquery.utils; +import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; @@ -20,15 +21,17 @@ public static String sanitizeName(String name) { // letters, numbers, and underscores. // Note: a.b and a/b will have the same value after sanitization which will cause Duplicate key // Exception. + @SuppressWarnings("unchecked") public static Map replaceInvalidKeys(Map map) { - return map.entrySet().stream().collect(Collectors.toMap( - (entry) -> sanitizeName(entry.getKey()), - (entry) -> { - if (entry.getValue() instanceof Map) { - return replaceInvalidKeys((Map) entry.getValue()); - } - return entry.getValue(); - } - )); + Map result = new HashMap<>(); + map.forEach((key, value) -> { + String sanitizedKey = sanitizeName(key); + if (value instanceof Map) { + result.put(sanitizedKey, replaceInvalidKeys((Map) value)); + } else { + result.put(sanitizedKey, value); + } + }); + return result; } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizerTest.java index 3358c1386..7b02c009b 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/FieldNameSanitizerTest.java @@ -1,5 +1,6 @@ package com.wepay.kafka.connect.bigquery.utils; +import java.util.Collections; import java.util.Map; import org.junit.Before; import org.junit.Test; @@ -75,4 +76,26 @@ public void testInvalidSymbol() { // Validate map size. assertEquals(5, sanitizedMap.size()); } + + /** + * Verifies that null values are acceptable while sanitizing keys. + */ + @Test + public void testNullValue() { + assertEquals( + Collections.singletonMap("abc", null), + FieldNameSanitizer.replaceInvalidKeys(Collections.singletonMap("abc", null))); + } + + @Test + public void testDeeplyNestedNullValues() { + testMap = new HashMap<>(); + testMap.put("top", null); + testMap.put("middle", Collections.singletonMap("key", null)); + testMap.put("bottom", Collections.singletonMap("key", Collections.singletonMap("key", null))); + assertEquals( + testMap, + FieldNameSanitizer.replaceInvalidKeys(testMap) + ); + } } From 5e47b9574747791bd6c6d343f7b93ef3126d2978 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 5 Oct 2020 09:51:47 -0400 Subject: [PATCH 035/190] GH-31: Port Dockerized integration tests to embedded integration tests (#37) * GH-31: Port Dockerized integration tests to embedded integration tests * GH-31: Try to get integration tests working on Jenkins * GH-31: Address review feedback * GH-31: Clear buckets before deleting them in integration tests --- Jenkinsfile | 3 + README.md | 109 ++--- kcbq-connector/pom.xml | 22 + .../connect/bigquery/BigQuerySinkTask.java | 26 +- .../bigquery/config/BigQuerySinkConfig.java | 2 +- .../bigquery/integration/BaseConnectorIT.java | 31 +- .../integration/BigQuerySinkConnectorIT.java | 292 +++++++++++++ .../UpsertDeleteBigQuerySinkConnectorIT.java | 3 +- .../utils/BucketClearer.java | 34 +- .../integration/utils/TableClearer.java | 52 +++ .../it/BigQueryConnectorIntegrationTest.java | 393 ------------------ .../bigquery/it/utils/TableClearer.java | 60 --- .../gcs-load/data.json | 0 .../gcs-load/schema.json | 0 .../logical-types/data.json | 0 .../logical-types/schema.json | 0 .../matryoshka-dolls/data.json | 0 .../matryoshka-dolls/schema.json | 0 .../integration_test_cases}/nulls/data.json | 0 .../integration_test_cases}/nulls/schema.json | 0 .../primitives/data.json | 0 .../primitives/schema.json | 0 kcbq-connector/test/docker/connect/Dockerfile | 41 -- .../test/docker/connect/connect-docker.sh | 26 -- .../test/docker/populate/Dockerfile | 32 -- .../test/docker/populate/populate-docker.sh | 24 -- kcbq-connector/test/integrationtest.sh | 288 ------------- .../resources/connector-template.properties | 41 -- .../resources/standalone-template.properties | 27 -- pom.xml | 45 +- 30 files changed, 496 insertions(+), 1055 deletions(-) create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java rename kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/{it => integration}/utils/BucketClearer.java (59%) create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/TableClearer.java delete mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java delete mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/gcs-load/data.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/gcs-load/schema.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/logical-types/data.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/logical-types/schema.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/matryoshka-dolls/data.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/matryoshka-dolls/schema.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/nulls/data.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/nulls/schema.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/primitives/data.json (100%) rename kcbq-connector/{test/resources/test_schemas => src/test/resources/integration_test_cases}/primitives/schema.json (100%) delete mode 100644 kcbq-connector/test/docker/connect/Dockerfile delete mode 100755 kcbq-connector/test/docker/connect/connect-docker.sh delete mode 100644 kcbq-connector/test/docker/populate/Dockerfile delete mode 100755 kcbq-connector/test/docker/populate/populate-docker.sh delete mode 100755 kcbq-connector/test/integrationtest.sh delete mode 100644 kcbq-connector/test/resources/connector-template.properties delete mode 100644 kcbq-connector/test/resources/standalone-template.properties diff --git a/Jenkinsfile b/Jenkinsfile index 551c15919..07318f3bc 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -4,4 +4,7 @@ common { nodeLabel = 'docker-oraclejdk8' publish = false downStreamValidate = false + secret_file_list = [ + ['gcp/kcbq', 'creds', '/tmp/creds.json', 'KCBQ_TEST_KEYFILE'] + ] } diff --git a/README.md b/README.md index b647e1e8f..d9076fe05 100644 --- a/README.md +++ b/README.md @@ -131,15 +131,7 @@ adjusting flags given to the Avro Console Producer and tweaking the config setti ## Integration Testing the Connector -There is a legacy Docker-based integration test for the connector, and newer integration tests that -programmatically instantiate an embedded Connect cluster. - -### Embedded integration tests - -Currently these tests only verify the connector's upsert/delete feature. They should eventually -replace all of the existing Docker-based tests. - -#### Configuring the tests +### Configuring the tests You must supply the following environment variables in order to run the tests: @@ -147,37 +139,33 @@ You must supply the following environment variables in order to run the tests: - `$KCBQ_TEST_DATASET`: The name of the BigQuery dataset to use for the test - `$KCBQ_TEST_KEYFILE`: The key (either file or raw contents) used to authenticate with BigQuery during the test +- `$KCBQ_TEST_BUCKET`: The name of the GCS bucket to use (for testing the GCS batch loading feature) -Additionally, the `$KCBQ_TEST_KEYSOURCE` variable can be supplied to specify whether the value of -`$KCBQ_TEST_KEYFILE` are a path to a key file (if set to `FILE`) or the raw contents of a key file +Optionally, the `$KCBQ_TEST_KEYSOURCE` variable can be supplied to specify whether the value of +`$KCBQ_TEST_KEYFILE` is a path to a key file (if set to `FILE`) or the raw contents of a key file (if set to `JSON`). The default is `FILE`. -#### Running the Integration Tests +The `$KCBQ_TEST_FOLDER` variable can be supplied to specify which subfolder of the GCS bucket should +be used when testing the GCS batch loading feature; if not supplied, the top-level folder will be +used. + +### Running the Integration Tests ```bash -mvn failsafe:integration-test@embedded-integration-test +# (Re)builds the project and runs the integration tests, skipping unit tests to save a bit of time +mvn clean package failsafe:integration-test@embedded-integration-test -Dskip.unit.tests=true ``` -### Docker-based tests +### How Integration Testing Works -> **NOTE**: You must have [Docker] installed and running on your machine in order to run integration -tests for the connector. - -This all takes place in the `kcbq-connector` directory. - -#### How Integration Testing Works - -Integration tests run by creating [Docker] instances for [Zookeeper], [Kafka], [Schema Registry], +Integration tests run by creating embedded instances for [Zookeeper], [Kafka], [Schema Registry], and the BigQuery Connector itself, then verifying the results using a [JUnit] test. -They use schemas and data that can be found in the `test/docker/populate/test_schemas/` directory, -and rely on a user-provided JSON key file (like in the `quickstart` example) to access BigQuery. - -The project and dataset they write to, as well as the specific JSON key file they use, can be -specified by command-line flag, environment variable, or configuration file — the exact details of -each can be found by running the integration test script with the `-?` flag. +They use schemas and data that can be found in the +`kcbq-connector/src/test/resources/integration_test_cases/` directory, and rely on a user-provided +JSON key file (like in the `quickstart` example) to access BigQuery. -#### Data Corruption Concerns +### Data Corruption Concerns In order to ensure the validity of each test, any table that will be written to in the course of integration testing is preemptively deleted before the connector is run. This will only be an issue @@ -185,70 +173,49 @@ if you have any tables in your dataset whose names begin with `kcbq_test_` and m name of any of the `test_schema` subdirectories. If that is the case, you should probably consider writing to a different project/dataset. -Because Kafka and Schema Registry are run in Docker, there is no risk that running integration -tests will corrupt any existing data that is already on your machine, and there is also no need to -free up any of your ports that might currently be in use by real instances of the programs that are -faked in the process of testing. - -#### Running the Integration Tests - -Running the series of integration tests is easy: - -```bash -$ test/integrationtest.sh -``` - -This assumes that the project, dataset, and key file have been specified by variable or -configuration file. For more information on how to specify these, run the test script with -the `--help` flag. - -> **NOTE:** You must have a recent version of [boot2docker], [Docker Machine], [Docker], etc. -installed. Older versions will hang when cleaning containers, and linking doesn't work properly. +Kafka, Schema Registry, Zookeeper, and Kafka Connect are all run as temporary embedded instances, so +there is no risk that running integration tests will corrupt any existing data that is already on +your machine, and there is also no need to free up any of your ports that might currently be in use +by instances of the services that are brought up in the process of testing. -#### Adding New Integration Tests +### Adding New Integration Tests Adding an integration test is a little more involved, and consists of two major steps: specifying -Avro data to be sent to Kafka, and specifying via JUnit test how to verify that such data made +Avro data to be sent to Kafka, and specifying via JUnit test how to verify that such data made it to BigQuery as expected. -To specify input data, you must create a new directory in the `test/resources/test_schemas/` -directory with whatever name you want the Kafka topic of your test to be named, and whatever -string you want the name of your test's BigQuery table to be derived from. Then, create two files -in that directory: +To specify input data, you must create a new directory in the +`kcbq-connector/src/test/resources/integration_test_cases/` directory with whatever name you want +the Kafka topic of your test to be named, and whatever string you want the name of your test's +BigQuery table to be derived from. Then, create two files in that directory: * `schema.json` will contain the Avro schema of the type of data the new test will send through the connector. -* `data.json` will contain a series of JSON objects, each of which should represent an [Avro] record -that matches the specified schema. **Each JSON object must occupy its own line, and each object -cannot occupy more than one line** (this inconvenience is due to limitations in the Avro +* `data.json` will contain a series of JSON objects, each of which should represent an [Avro] record +that matches the specified schema. **Each JSON object must occupy its own line, and each object +cannot occupy more than one line** (this inconvenience is due to limitations in the Avro Console Producer, and may be addressed in future commits). -To specify data verification, add a new JUnit test to the file -`src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java`. -Rows that are retrieved from BigQuery in the test are only returned as _Lists_ of _Objects_. The -names of their columns are not tracked. Construct a _List_ of the _Objects_ that you expect to be -stored in the test's BigQuery table, retrieve the actual _List_ of _Objects_ stored via a call to -`readAllRows()`, and then compare the two via a call to `testRows()`. +To specify data verification, add to the test cases present in the +`kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java` > **NOTE**: Because the order of rows is not guaranteed when reading test results from BigQuery, -you must include a row number as the first field of any of your test schemas, and every row of test -data must have a unique value for its row number (row numbers are one-indexed). +you must include a numeric column named "row" number in all of your test schemas, and every row of +test data must have a unique value for its row number. When data is read back from BigQuery to +verify its accuracy, it will be returned in ascending order based on that "row" column. [Apache Avro]: https://avro.apache.org - [Apache Kafka Connect]: http://docs.confluent.io/3.0.0/connect/ + [Apache Kafka Connect]: http://docs.confluent.io/current/connect/ [Apache Kafka]: http://kafka.apache.org [Apache Maven]: https://maven.apache.org [Avro]: https://avro.apache.org [BigQuery]: https://cloud.google.com/bigquery/ - [boot2docker]: http://boot2docker.io - [Confluent Platform]: http://docs.confluent.io/3.0.0/installation.html + [Confluent Platform]: http://docs.confluent.io/current/installation.html [Connector Configuration Wiki]: https://github.com/wepay/kafka-connect-bigquery/wiki/Connector-Configuration - [Docker Machine]: https://docs.docker.com/machine/ - [Docker]: https://www.docker.com [Google BigQuery]: https://cloud.google.com/bigquery/ [JUnit]: http://junit.org - [Kafka Connect]: http://docs.confluent.io/3.0.0/connect/ + [Kafka Connect]: http://docs.confluent.io/current/connect/ [Kafka]: http://kafka.apache.org [Maven]: https://maven.apache.org [Schema Registry]: https://github.com/confluentinc/schema-registry diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 65eb56d7f..b73856ec2 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -102,6 +102,24 @@ test test-jar + + io.confluent + kafka-schema-registry + + + io.confluent + kafka-schema-registry + tests + test-jar + + + io.confluent + kafka-connect-avro-converter + + + io.confluent + kafka-avro-serializer + @@ -114,6 +132,10 @@ org.apache.maven.plugins maven-surefire-plugin + + org.apache.maven.plugins + maven-failsafe-plugin + org.jacoco jacoco-maven-plugin diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 85ef9a67b..6a070b3b5 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -93,6 +93,7 @@ public class BigQuerySinkTask extends SinkTask { private boolean upsertDelete; private MergeBatches mergeBatches; private MergeQueries mergeQueries; + private volatile boolean stopped; private TopicPartitionManager topicPartitionManager; @@ -139,6 +140,11 @@ public void flush(Map offsets) { + "the Connect framework; please upgrade to version 0.10.2.0 or later"); } + // Return immediately here since the executor will already be shutdown + if (stopped) { + return; + } + try { executor.awaitCurrentTasks(); } catch (InterruptedException err) { @@ -382,6 +388,8 @@ private SinkRecordConverter getConverter(BigQuerySinkTaskConfig config) { @Override public void start(Map properties) { logger.trace("task.start()"); + stopped = false; + final boolean hasGCSBQTask = properties.remove(BigQuerySinkConnector.GCS_BQ_TASK_CONFIG_KEY) != null; try { @@ -466,13 +474,17 @@ private void maybeStartMergeFlushTask() { @Override public void stop() { - maybeStopExecutor(loadExecutor, "load executor"); - maybeStopExecutor(executor, "table write executor"); - if (upsertDelete) { - mergeBatches.intermediateTables().forEach(table -> { - logger.debug("Deleting {}", intTable(table)); - getBigQuery().delete(table); - }); + try { + maybeStopExecutor(loadExecutor, "load executor"); + maybeStopExecutor(executor, "table write executor"); + if (upsertDelete) { + mergeBatches.intermediateTables().forEach(table -> { + logger.debug("Deleting {}", intTable(table)); + getBigQuery().delete(table); + }); + } + } finally { + stopped = true; } logger.trace("task.stop()"); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 583f87031..9ad323b77 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -104,7 +104,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String GCS_FOLDER_NAME_CONFIG = "gcsFolderName"; private static final ConfigDef.Type GCS_FOLDER_NAME_TYPE = ConfigDef.Type.STRING; - private static final Object GCS_FOLDER_NAME_DEFAULT = ""; + public static final String GCS_FOLDER_NAME_DEFAULT = ""; private static final ConfigDef.Importance GCS_FOLDER_NAME_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String GCS_FOLDER_NAME_DOC = "The name of the folder under the bucket in which gcs blobs used to batch load to BigQuery " diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java index 3c02bc0a7..ae7b1a4cf 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -22,6 +22,8 @@ import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -77,6 +79,8 @@ public abstract class BaseConnectorIT { private static final String PROJECT_ENV_VAR = "KCBQ_TEST_PROJECT"; private static final String DATASET_ENV_VAR = "KCBQ_TEST_DATASET"; private static final String KEYSOURCE_ENV_VAR = "KCBQ_TEST_KEYSOURCE"; + private static final String GCS_BUCKET_ENV_VAR = "KCBQ_TEST_BUCKET"; + private static final String GCS_FOLDER_ENV_VAR = "KCBQ_TEST_FOLDER"; protected static final long OFFSET_COMMIT_INTERVAL_MS = TimeUnit.SECONDS.toMillis(10); protected static final long COMMIT_MAX_DURATION_MS = TimeUnit.MINUTES.toMillis(5); @@ -151,14 +155,15 @@ protected void clearPriorTable(BigQuery bigQuery, String table) { protected void waitForCommittedRecords( String connector, String topic, long numRecords, int numTasks ) throws InterruptedException { - waitForCommittedRecords(connector, topic, numRecords, numTasks, COMMIT_MAX_DURATION_MS); + waitForCommittedRecords(connector, Collections.singleton(topic), numRecords, numTasks, COMMIT_MAX_DURATION_MS); } protected void waitForCommittedRecords( - String connector, String topic, long numRecords, int numTasks, long timeoutMs) throws InterruptedException { + String connector, Collection topics, long numRecords, int numTasks, long timeoutMs + ) throws InterruptedException { waitForCondition( () -> { - long totalCommittedRecords = totalCommittedRecords(connector, topic); + long totalCommittedRecords = totalCommittedRecords(connector, topics); if (totalCommittedRecords >= numRecords) { return true; } else { @@ -170,8 +175,8 @@ protected void waitForCommittedRecords( } catch (AssertionError e) { throw new NoRetryException(e); } - logger.debug("Connector has only committed {} records for topic {} so far; {} expected", - totalCommittedRecords, topic, numRecords); + logger.debug("Connector has only committed {} records for topics {} so far; {} expected", + totalCommittedRecords, topics, numRecords); // Sleep here so as not to spam Kafka with list-offsets requests Thread.sleep(OFFSET_COMMIT_INTERVAL_MS / 2); return false; @@ -181,7 +186,7 @@ protected void waitForCommittedRecords( "Either the connector failed, or the message commit duration expired without all expected messages committed"); } - protected synchronized long totalCommittedRecords(String connector, String topic) throws TimeoutException, ExecutionException, InterruptedException { + protected synchronized long totalCommittedRecords(String connector, Collection topics) throws TimeoutException, ExecutionException, InterruptedException { // See https://github.com/apache/kafka/blob/f7c38d83c727310f4b0678886ba410ae2fae9379/connect/runtime/src/main/java/org/apache/kafka/connect/util/SinkUtils.java // for how the consumer group ID is constructed for sink connectors Map offsets = kafkaAdminClient @@ -192,7 +197,7 @@ protected synchronized long totalCommittedRecords(String connector, String topic logger.trace("Connector {} has so far committed offsets {}", connector, offsets); return offsets.entrySet().stream() - .filter(entry -> topic.equals(entry.getKey().topic())) + .filter(entry -> topics.contains(entry.getKey().topic())) .mapToLong(entry -> entry.getValue().offset()) .sum(); } @@ -222,7 +227,7 @@ protected List> readAllRows( .collect(Collectors.toList()); } - private static List boxByteArray(byte[] bytes) { + protected static List boxByteArray(byte[] bytes) { Byte[] result = new Byte[bytes.length]; for (int i = 0; i < bytes.length; i++) { result[i] = bytes[i]; @@ -320,7 +325,7 @@ protected Optional assertConnectorAndTasksRunning(String connectorName, && info.tasks().stream().allMatch(s -> s.state().equals(AbstractStatus.State.RUNNING.toString())); return Optional.of(result); } catch (Exception e) { - logger.error("Could not check connector state info.", e); + logger.debug("Could not check connector state info.", e); return Optional.empty(); } } @@ -354,4 +359,12 @@ protected String dataset() { protected String keySource() { return readEnvVar(KEYSOURCE_ENV_VAR, BigQuerySinkConfig.KEY_SOURCE_DEFAULT); } + + protected String gcsBucket() { + return readEnvVar(GCS_BUCKET_ENV_VAR); + } + + protected String gcsFolder() { + return readEnvVar(GCS_FOLDER_ENV_VAR, BigQuerySinkConfig.GCS_FOLDER_NAME_DEFAULT); + } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java new file mode 100644 index 000000000..aec6c10d8 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java @@ -0,0 +1,292 @@ +package com.wepay.kafka.connect.bigquery.integration; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.integration.utils.BucketClearer; +import com.wepay.kafka.connect.bigquery.integration.utils.TableClearer; +import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; +import io.confluent.connect.avro.AvroConverter; +import io.confluent.kafka.formatter.AvroMessageReader; +import io.confluent.kafka.schemaregistry.ClusterTestHarness; +import io.confluent.kafka.schemaregistry.CompatibilityLevel; +import io.confluent.kafka.schemaregistry.RestApp; +import io.confluent.kafka.schemaregistry.rest.SchemaRegistryConfig; +import kafka.common.MessageReader; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Scanner; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +import static io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG; +import static org.junit.Assert.assertEquals; + +@Category(IntegrationTest.class) +public class BigQuerySinkConnectorIT extends BaseConnectorIT { + + private static final Map>> TEST_CASES; + static { + Map>> testCases = new HashMap<>(); + + List> expectedGcsLoadRows = new ArrayList<>(); + expectedGcsLoadRows.add(Arrays.asList( + 1L, + null, + false, + 4242L, + 42424242424242L, + 42.42, + 42424242.42424242, + "forty-two", + boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) + )); + expectedGcsLoadRows.add(Arrays.asList( + 2L, + 5L, + true, + 4354L, + 435443544354L, + 43.54, + 435443.544354, + "forty-three", + boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) + )); + expectedGcsLoadRows.add(Arrays.asList( + 3L, + 8L, + false, + 1993L, + 199319931993L, + 19.93, + 199319.931993, + "nineteen", + boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) + )); + testCases.put("gcs-load", expectedGcsLoadRows); + + List> expectedNullsRows = new ArrayList<>(); + expectedNullsRows.add(Arrays.asList(1L, "Required string", null, 42L, false)); + expectedNullsRows.add(Arrays.asList(2L, "Required string", "Optional string", 89L, null)); + expectedNullsRows.add(Arrays.asList(3L, "Required string", null, null, true)); + expectedNullsRows.add(Arrays.asList(4L, "Required string", "Optional string", null, null)); + testCases.put("nulls", expectedNullsRows); + + List> expectedMatryoshkaRows = new ArrayList<>(); + expectedMatryoshkaRows.add(Arrays.asList( + 1L, + Arrays.asList( + Arrays.asList(42.0, 42.42, 42.4242), + Arrays.asList( + 42L, + "42" + ) + ), + Arrays.asList( + -42L, + "-42" + ) + )); + testCases.put("matryoshka-dolls", expectedMatryoshkaRows); + + List> expectedPrimitivesRows = new ArrayList<>(); + expectedPrimitivesRows.add(Arrays.asList( + 1L, + null, + false, + 4242L, + 42424242424242L, + 42.42, + 42424242.42424242, + "forty-two", + boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) + )); + testCases.put("primitives", expectedPrimitivesRows); + + List> expectedLogicalTypesRows = new ArrayList<>(); + expectedLogicalTypesRows.add(Arrays.asList(1L, 0L, 0L)); + expectedLogicalTypesRows.add(Arrays.asList(2L, 42000000000L, 362880000000L)); + expectedLogicalTypesRows.add(Arrays.asList(3L, 1468275102000000L, 1468195200000L)); + testCases.put("logical-types", expectedLogicalTypesRows); + + TEST_CASES = Collections.unmodifiableMap(testCases); + } + + private static final String TEST_CASE_PREFIX = "kcbq_test_"; + + private static final Collection TEST_TOPICS = TEST_CASES.keySet().stream() + .map(tc -> TEST_CASE_PREFIX + tc) + .collect(Collectors.toList()); + + private static final Collection TEST_TABLES = TEST_TOPICS.stream() + .map(FieldNameSanitizer::sanitizeName) + .collect(Collectors.toList()); + + private RestApp restApp; + private String schemaRegistryUrl; + private Producer valueProducer; + private int numRecordsProduced; + + @Before + public void setup() throws Exception { + BucketClearer.clearBucket(keyFile(), project(), gcsBucket(), keySource()); + TableClearer.clearTables(newBigQuery(), dataset(), TEST_TABLES); + + startConnect(); + restApp = new RestApp( + ClusterTestHarness.choosePort(), + null, + connect.kafka().bootstrapServers(), + SchemaRegistryConfig.DEFAULT_KAFKASTORE_TOPIC, + CompatibilityLevel.BACKWARD.name, + true, + null); + + restApp.start(); + + schemaRegistryUrl = restApp.restClient.getBaseUrls().current(); + + Map producerProps = new HashMap<>(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, connect.kafka().bootstrapServers()); + valueProducer = new KafkaProducer<>( + producerProps, Serdes.ByteArray().serializer(), Serdes.ByteArray().serializer()); + + numRecordsProduced = 0; + } + + @After + public void close() throws Exception { + if (restApp != null) { + restApp.stop(); + } + stopConnect(); + } + + @Test + public void testAll() throws Exception { + final int tasksMax = 1; + final String connectorName = "bigquery-connector"; + + TEST_CASES.keySet().forEach(this::populate); + + connect.configureConnector(connectorName, connectorProps(tasksMax)); + + waitForConnectorToStart(connectorName, tasksMax); + + waitForCommittedRecords( + "bigquery-connector", TEST_TOPICS, numRecordsProduced, tasksMax, TimeUnit.MINUTES.toMillis(3)); + + TEST_CASES.forEach(this::verify); + } + + private void populate(String testCase) { + String topic = TEST_CASE_PREFIX + testCase; + connect.kafka().createTopic(topic); + + String testCaseDir = "integration_test_cases/" + testCase + "/"; + + InputStream schemaStream = BigQuerySinkConnectorIT.class.getClassLoader() + .getResourceAsStream(testCaseDir + "schema.json"); + Scanner schemaScanner = new Scanner(schemaStream).useDelimiter("\\A"); + String schemaString = schemaScanner.next(); + + Properties messageReaderProps = new Properties(); + messageReaderProps.put(SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl); + messageReaderProps.put("value.schema", schemaString); + messageReaderProps.put("topic", topic); + InputStream dataStream = BigQuerySinkConnectorIT.class.getClassLoader() + .getResourceAsStream(testCaseDir + "data.json"); + MessageReader messageReader = new AvroMessageReader(); + messageReader.init(dataStream, messageReaderProps); + + ProducerRecord message = messageReader.readMessage(); + while (message != null) { + try { + valueProducer.send(message).get(1, TimeUnit.SECONDS); + numRecordsProduced++; + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + message = messageReader.readMessage(); + } + } + + private Map connectorProps(int tasksMax) { + Map result = baseConnectorProps(tasksMax); + + result.put( + ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, + AvroConverter.class.getName()); + result.put( + ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG + "." + SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistryUrl); + result.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, + AvroConverter.class.getName()); + result.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + "." + SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistryUrl); + + result.put(SinkConnectorConfig.TOPICS_CONFIG, String.join(",", TEST_TOPICS)); + + result.put(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG, "true"); + result.put(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG, "true"); + result.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); + result.put(BigQuerySinkConfig.ENABLE_BATCH_CONFIG, "kcbq_test_gcs-load"); + result.put(BigQuerySinkConfig.BATCH_LOAD_INTERVAL_SEC_CONFIG, "10"); + result.put(BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG, gcsBucket()); + result.put(BigQuerySinkConfig.GCS_FOLDER_NAME_CONFIG, gcsFolder()); + result.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, IdentitySchemaRetriever.class.getName()); + + return result; + } + + private void verify(String testCase, List> expectedRows) { + List> testRows; + try { + testRows = readAllRows(newBigQuery(), TEST_CASE_PREFIX + FieldNameSanitizer.sanitizeName(testCase), "row"); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + + assertEquals(expectedRows, testRows); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java index 67c30a3f2..946f37146 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java @@ -348,7 +348,8 @@ public void testUpsertDeleteHighThroughput() throws Throwable { waitForConnectorToStart(CONNECTOR_NAME, tasksMax); // wait for tasks to write to BigQuery and commit offsets for their records - waitForCommittedRecords(CONNECTOR_NAME, topic, numRecords, tasksMax, TimeUnit.MINUTES.toMillis(10)); + waitForCommittedRecords( + CONNECTOR_NAME, Collections.singleton(topic), numRecords, tasksMax, TimeUnit.MINUTES.toMillis(10)); long time = System.currentTimeMillis() - start; logger.info("All records have been read and committed by the connector; " + "total time from start to finish: {} seconds", time / 1000.0); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/BucketClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java similarity index 59% rename from kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/BucketClearer.java rename to kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java index c7dc97948..0b734561d 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/BucketClearer.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java @@ -1,4 +1,4 @@ -package com.wepay.kafka.connect.bigquery.it.utils; +package com.wepay.kafka.connect.bigquery.integration.utils; /* * Copyright 2016 WePay, Inc. @@ -20,7 +20,6 @@ import com.google.cloud.storage.Blob; import com.google.cloud.storage.Bucket; - import com.google.cloud.storage.Storage; import com.wepay.kafka.connect.bigquery.GCSBuilder; import org.slf4j.Logger; @@ -29,37 +28,28 @@ public class BucketClearer { private static final Logger logger = LoggerFactory.getLogger(BucketClearer.class); - private static String keySource; /** - * Clears tables in the given project and dataset, using a provided JSON service account key. + * Clear out a GCS bucket. Useful in integration testing to provide a clean slate before creating + * a connector and writing to that bucket. + * @param key The GCP credentials to use (can be a filename or a raw JSON string). + * @param project The GCP project the bucket belongs to. + * @param bucketName The bucket to clear. + * @param keySource The key source. If "FILE", then the {@code key} parameter will be treated as a + * filename; if "JSON", then {@code key} will be treated as a raw JSON string. */ - public static void main(String[] args) { - if (args.length < 3 || args.length > 4) { - usage(); - } else if (args.length == 3) { - keySource = "FILE"; - } else { - keySource = args[3]; - } - Storage gcs = new GCSBuilder(args[1]).setKey(args[0]).setKeySource(keySource).build(); - - String bucketName = args[2]; + public static void clearBucket(String key, String project, String bucketName, String keySource) { + Storage gcs = new GCSBuilder(project).setKey(key).setKeySource(keySource).build(); Bucket bucket = gcs.get(bucketName); if (bucket != null) { logger.info("Deleting objects in the Bucket {}", bucketName); for (Blob blob : bucket.list().iterateAll()) { gcs.delete(blob.getBlobId()); } + bucket.delete(); + logger.info("Bucket {} deleted successfully", bucketName); } else { logger.info("Bucket {} does not exist", bucketName); } } - - private static void usage() { - System.err.println( - "usage: BucketClearer []" - ); - System.exit(1); - } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/TableClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/TableClearer.java new file mode 100644 index 000000000..ecac43627 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/TableClearer.java @@ -0,0 +1,52 @@ +package com.wepay.kafka.connect.bigquery.integration.utils; + +/* + * Copyright 2016 WePay, Inc. + * + * 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. + */ + + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.TableId; +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; + +import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.table; + +public class TableClearer { + private static final Logger logger = LoggerFactory.getLogger(TableClearer.class); + + // TODO: Might want to add support for table sanitization here + /** + * Clear out one or more BigQuery tables. Useful in integration testing to provide a clean slate + * before creating a connector and writing to those tables. + * @param bigQuery The BigQuery client to use when sending table deletion requests. + * @param dataset The dataset that the to-be-cleared tables belong to. + * @param tables The tables to clear. + */ + public static void clearTables(BigQuery bigQuery, String dataset, Collection tables) { + for (String tableName : tables) { + TableId table = TableId.of(dataset, FieldNameSanitizer.sanitizeName(tableName)); + if (bigQuery.delete(table)) { + logger.info("{} deleted successfully", table(table)); + } else { + logger.info("{} does not exist", table(table)); + } + } + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java deleted file mode 100644 index 46775b418..000000000 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java +++ /dev/null @@ -1,393 +0,0 @@ -package com.wepay.kafka.connect.bigquery.it; - -/* - * Copyright 2016 WePay, Inc. - * - * 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. - */ - - -import static com.google.cloud.bigquery.LegacySQLTypeName.BOOLEAN; -import static com.google.cloud.bigquery.LegacySQLTypeName.BYTES; -import static com.google.cloud.bigquery.LegacySQLTypeName.DATE; -import static com.google.cloud.bigquery.LegacySQLTypeName.FLOAT; -import static com.google.cloud.bigquery.LegacySQLTypeName.INTEGER; -import static com.google.cloud.bigquery.LegacySQLTypeName.STRING; -import static com.google.cloud.bigquery.LegacySQLTypeName.TIMESTAMP; - -import static org.junit.Assert.assertEquals; - -import com.google.cloud.bigquery.BigQuery; -import com.google.cloud.bigquery.Field; -import com.google.cloud.bigquery.FieldValue; -import com.google.cloud.bigquery.FieldValueList; -import com.google.cloud.bigquery.Schema; -import com.google.cloud.bigquery.Table; -import com.google.cloud.bigquery.TableResult; - -import com.wepay.kafka.connect.bigquery.BigQueryHelper; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; - -import org.junit.BeforeClass; -import org.junit.Test; - -import java.io.FileNotFoundException; -import java.io.InputStream; - -import java.time.LocalDate; -import java.time.ZoneOffset; -import java.time.format.DateTimeFormatter; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.Properties; - -public class BigQueryConnectorIntegrationTest { - public static final String TEST_PROPERTIES_FILENAME = "/test.properties"; - public static final String KEYFILE_PROPERTY = "keyfile"; - public static final String PROJECT_PROPERTY = "project"; - public static final String DATASET_PROPERTY = "dataset"; - public static final String KEY_SOURCE_PROPERTY = "keySource"; - - private static String keyfile; - private static String project; - private static String dataset; - private static String keySource; - - private static BigQuery bigQuery; - - @BeforeClass - public static void initialize() throws Exception { - initializeTestProperties(); - initializeBigQuery(); - } - - private static void initializeTestProperties() throws Exception { - try (InputStream propertiesFile = - BigQueryConnectorIntegrationTest.class.getResourceAsStream(TEST_PROPERTIES_FILENAME)) { - if (propertiesFile == null) { - throw new FileNotFoundException( - "Resource file '" + TEST_PROPERTIES_FILENAME - + "' must be provided in order to run integration tests" - ); - } - - Properties properties = new Properties(); - properties.load(propertiesFile); - - keyfile = properties.getProperty(KEYFILE_PROPERTY); - if (keyfile == null) { - throw new SinkConfigConnectException( - "'" + KEYFILE_PROPERTY - + "' property must be specified in test properties file" - ); - } - - project = properties.getProperty(PROJECT_PROPERTY); - if (project == null) { - throw new SinkConfigConnectException( - "'" + PROJECT_PROPERTY - + "' property must be specified in test properties file" - ); - } - - dataset = properties.getProperty(DATASET_PROPERTY); - if (dataset == null) { - throw new SinkConfigConnectException( - "'" + DATASET_PROPERTY - + "' property must be specified in test properties file" - ); - } - - keySource = properties.getProperty(KEY_SOURCE_PROPERTY); - } - } - - private static void initializeBigQuery() throws Exception { - bigQuery = new BigQueryHelper().setKeySource(keySource).connect(project, keyfile); - } - - private static List boxByteArray(byte[] bytes) { - Byte[] result = new Byte[bytes.length]; - for (int i = 0; i < bytes.length; i++) { - result[i] = bytes[i]; - } - return Arrays.asList(result); - } - - private Object convertField(Field fieldSchema, FieldValue field) { - if (field.isNull()) { - return null; - } - switch (field.getAttribute()) { - case PRIMITIVE: - if (fieldSchema.getType().equals(BOOLEAN)) { - return field.getBooleanValue(); - } else if (fieldSchema.getType().equals(BYTES)) { - // Do this in order for assertEquals() to work when this is an element of two compared - // lists - return boxByteArray(field.getBytesValue()); - } else if (fieldSchema.getType().equals(DATE)) { - DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); - long millisecondsSinceEpoch = LocalDate.parse(field.getStringValue(), dateFormatter) - .atStartOfDay(ZoneOffset.UTC) - .toInstant() - .toEpochMilli(); - return millisecondsSinceEpoch; - } else if (fieldSchema.getType().equals(FLOAT)) { - return field.getDoubleValue(); - } else if (fieldSchema.getType().equals(INTEGER)) { - return field.getLongValue(); - } else if (fieldSchema.getType().equals(STRING)) { - return field.getStringValue(); - } else if (fieldSchema.getType().equals(TIMESTAMP)) { - return field.getTimestampValue(); - } else { - throw new RuntimeException("Cannot convert primitive field type " - + fieldSchema.getType()); - } - case REPEATED: - List result = new ArrayList<>(); - for (FieldValue arrayField : field.getRepeatedValue()) { - result.add(convertField(fieldSchema, arrayField)); - } - return result; - case RECORD: - List recordSchemas = fieldSchema.getSubFields(); - List recordFields = field.getRecordValue(); - return convertRow(recordSchemas, recordFields); - default: - throw new RuntimeException("Unknown field attribute: " + field.getAttribute()); - } - } - - private List convertRow(List rowSchema, List row) { - List result = new ArrayList<>(); - assert (rowSchema.size() == row.size()); - - for (int i = 0; i < rowSchema.size(); i++) { - if (rowSchema.get(i).getName().equals("row")) { - result.add(convertField(rowSchema.get(i), row.get(i))); - } - } - for (int i = 0; i < rowSchema.size(); i++) { - if (!rowSchema.get(i).getName().equals("row")) { - result.add(convertField(rowSchema.get(i), row.get(i))); - } - } - - return result; - } - - private List> readAllRows(String tableName) { - Table table = bigQuery.getTable(dataset, tableName); - Schema schema = table.getDefinition().getSchema(); - - List> rows = new ArrayList<>(); - TableResult tableResult = table.list(); - - while (tableResult != null) { - Iterable fieldValueLists = tableResult.iterateAll(); - for (FieldValueList fieldValueList : fieldValueLists) { - rows.add(convertRow(schema.getFields(), fieldValueList)); - } - tableResult = tableResult.getNextPage(); - } - return rows; - } - - @Test - public void testNull() { - List> expectedRows = new ArrayList<>(); - - // {"row":1,"f1":"Required string","f2":null,"f3":{"int":42},"f4":{"boolean":false}} - expectedRows.add(Arrays.asList(1L, "Required string", null, 42L, false)); - // {"row":2,"f1":"Required string","f2":{"string":"Optional string"},"f3":{"int":89},"f4":null} - expectedRows.add(Arrays.asList(2L, "Required string", "Optional string", 89L, null)); - // {"row":3,"f1":"Required string","f2":null,"f3":null,"f4":{"boolean":true}} - expectedRows.add(Arrays.asList(3L, "Required string", null, null, true)); - // {"row":4,"f1":"Required string","f2":{"string":"Optional string"},"f3":null,"f4":null} - expectedRows.add(Arrays.asList(4L, "Required string", "Optional string", null, null)); - - testRows(expectedRows, readAllRows("test_nulls")); - } - - @Test - public void testMatryoshka() { - List> expectedRows = new ArrayList<>(); - - /* { "row": 1, - "middle": - { "middle_array": [42.0, 42.42, 42.4242], - "inner": - { "inner_int": 42, - "inner_string": "42" - } - }, - "inner": - { "inner_int": -42, - "inner_string": "-42" - } - } */ - expectedRows.add(Arrays.asList( - 1L, - Arrays.asList( - Arrays.asList(42.0, 42.42, 42.4242), - Arrays.asList( - 42L, - "42" - ) - ), - Arrays.asList( - -42L, - "-42" - ) - )); - - testRows(expectedRows, readAllRows("test_matryoshka_dolls")); - } - - @Test - public void testPrimitives() { - List> expectedRows = new ArrayList<>(); - - /* { "row": 1, - "null_prim": null, - "boolean_prim": false, - "int_prim": 4242, - "long_prim": 42424242424242, - "float_prim": 42.42, - "double_prim": 42424242.42424242, - "string_prim": "forty-two", - "bytes_prim": "\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078" - } */ - expectedRows.add(Arrays.asList( - 1L, - null, - false, - 4242L, - 42424242424242L, - 42.42, - 42424242.42424242, - "forty-two", - boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) - )); - - testRows(expectedRows, readAllRows("test_primitives")); - } - - @Test - public void testLogicalTypes() { - List> expectedRows = new ArrayList<>(); - - // {"row": 1, "timestamp-test": 0, "date-test": 0} - expectedRows.add(Arrays.asList(1L, 0L, 0L)); - // {"row": 2, "timestamp-test": 42000000, "date-test": 4200} - expectedRows.add(Arrays.asList(2L, 42000000000L, 362880000000L)); - // {"row": 3, "timestamp-test": 1468275102000, "date-test": 16993} - expectedRows.add(Arrays.asList(3L, 1468275102000000L, 1468195200000L)); - - testRows(expectedRows, readAllRows("test_logical_types")); - } - - @Test - public void testGCSLoad() { - List> expectedRows = new ArrayList<>(); - - /* {"row":1, - "null_prim":null, - "boolean_prim":false, - "int_prim":4242, - "long_prim":42424242424242, - "float_prim":42.42, - "double_prim":42424242.42424242, - "string_prim":"forty-two", - "bytes_prim":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"} - */ - expectedRows.add(Arrays.asList( - 1L, - null, - false, - 4242L, - 42424242424242L, - 42.42, - 42424242.42424242, - "forty-two", - boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) - )); - /* {"row":2, - "null_prim":{"int":5}, - "boolean_prim":true, - "int_prim":4354, - "long_prim":435443544354, - "float_prim":43.54, - "double_prim":435443.544354, - "string_prim":"forty-three", - "bytes_prim":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"} - */ - expectedRows.add(Arrays.asList( - 2L, - 5L, - true, - 4354L, - 435443544354L, - 43.54, - 435443.544354, - "forty-three", - boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) - )); - /* {"row":3, - "null_prim":{"int":8}, - "boolean_prim":false, - "int_prim":1993, - "long_prim":199319931993, - "float_prim":19.93, - "double_prim":199319.931993, - "string_prim":"nineteen", - "bytes_prim":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"} - */ - expectedRows.add(Arrays.asList( - 3L, - 8L, - false, - 1993L, - 199319931993L, - 19.93, - 199319.931993, - "nineteen", - boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) - )); - - testRows(expectedRows, readAllRows("test_gcs_load")); - } - - private void testRows( - List> expectedRows, - List> testRows) { - assertEquals("Number of expected rows should match", expectedRows.size(), testRows.size()); - - for (List testRow : testRows) { - int rowNumber = (int) (((Long) testRow.get(0)).longValue()); - List expectedRow = expectedRows.get(rowNumber - 1); - expectedRow.sort(Comparator.nullsLast(Comparator.comparing(Object::toString))); - testRow.sort(Comparator.nullsLast(Comparator.comparing(Object::toString))); - assertEquals( - "Row " + rowNumber + " (if these look identical, it's probably a type mismatch)", - expectedRow, - testRow - ); - } - } -} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java deleted file mode 100644 index cedf5f936..000000000 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java +++ /dev/null @@ -1,60 +0,0 @@ -package com.wepay.kafka.connect.bigquery.it.utils; - -/* - * Copyright 2016 WePay, Inc. - * - * 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. - */ - - -import com.google.cloud.bigquery.BigQuery; - -import com.wepay.kafka.connect.bigquery.BigQueryHelper; - -import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TableClearer { - private static final Logger logger = LoggerFactory.getLogger(TableClearer.class); - - /** - * Clears tables in the given project and dataset, using a provided JSON service account key. - */ - public static void main(String[] args) { - if (args.length < 4) { - usage(); - } - int tablesStart = 3; - BigQuery bigQuery = new BigQueryHelper().connect(args[1], args[0]); - for (int i = tablesStart; i < args.length; i++) { - // May be consider using sanitizeTopics property value in future to decide table name - // sanitization but as currently we always run test cases with sanitizeTopics value as true - // hence sanitize table name prior delete. This is required else it makes test cases flaky. - String table = FieldNameSanitizer.sanitizeName(args[i]); - if (bigQuery.delete(args[2], table)) { - logger.info("Table {} in dataset {} deleted successfully", table, args[2]); - } else { - logger.info("Table {} in dataset {} does not exist", table, args[2]); - } - } - } - - private static void usage() { - System.err.println( - "usage: TableClearer [
...]" - ); - System.exit(1); - } -} diff --git a/kcbq-connector/test/resources/test_schemas/gcs-load/data.json b/kcbq-connector/src/test/resources/integration_test_cases/gcs-load/data.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/gcs-load/data.json rename to kcbq-connector/src/test/resources/integration_test_cases/gcs-load/data.json diff --git a/kcbq-connector/test/resources/test_schemas/gcs-load/schema.json b/kcbq-connector/src/test/resources/integration_test_cases/gcs-load/schema.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/gcs-load/schema.json rename to kcbq-connector/src/test/resources/integration_test_cases/gcs-load/schema.json diff --git a/kcbq-connector/test/resources/test_schemas/logical-types/data.json b/kcbq-connector/src/test/resources/integration_test_cases/logical-types/data.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/logical-types/data.json rename to kcbq-connector/src/test/resources/integration_test_cases/logical-types/data.json diff --git a/kcbq-connector/test/resources/test_schemas/logical-types/schema.json b/kcbq-connector/src/test/resources/integration_test_cases/logical-types/schema.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/logical-types/schema.json rename to kcbq-connector/src/test/resources/integration_test_cases/logical-types/schema.json diff --git a/kcbq-connector/test/resources/test_schemas/matryoshka-dolls/data.json b/kcbq-connector/src/test/resources/integration_test_cases/matryoshka-dolls/data.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/matryoshka-dolls/data.json rename to kcbq-connector/src/test/resources/integration_test_cases/matryoshka-dolls/data.json diff --git a/kcbq-connector/test/resources/test_schemas/matryoshka-dolls/schema.json b/kcbq-connector/src/test/resources/integration_test_cases/matryoshka-dolls/schema.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/matryoshka-dolls/schema.json rename to kcbq-connector/src/test/resources/integration_test_cases/matryoshka-dolls/schema.json diff --git a/kcbq-connector/test/resources/test_schemas/nulls/data.json b/kcbq-connector/src/test/resources/integration_test_cases/nulls/data.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/nulls/data.json rename to kcbq-connector/src/test/resources/integration_test_cases/nulls/data.json diff --git a/kcbq-connector/test/resources/test_schemas/nulls/schema.json b/kcbq-connector/src/test/resources/integration_test_cases/nulls/schema.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/nulls/schema.json rename to kcbq-connector/src/test/resources/integration_test_cases/nulls/schema.json diff --git a/kcbq-connector/test/resources/test_schemas/primitives/data.json b/kcbq-connector/src/test/resources/integration_test_cases/primitives/data.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/primitives/data.json rename to kcbq-connector/src/test/resources/integration_test_cases/primitives/data.json diff --git a/kcbq-connector/test/resources/test_schemas/primitives/schema.json b/kcbq-connector/src/test/resources/integration_test_cases/primitives/schema.json similarity index 100% rename from kcbq-connector/test/resources/test_schemas/primitives/schema.json rename to kcbq-connector/src/test/resources/integration_test_cases/primitives/schema.json diff --git a/kcbq-connector/test/docker/connect/Dockerfile b/kcbq-connector/test/docker/connect/Dockerfile deleted file mode 100644 index 1c73c57f8..000000000 --- a/kcbq-connector/test/docker/connect/Dockerfile +++ /dev/null @@ -1,41 +0,0 @@ -# Copyright 2016 WePay, Inc. -# -# 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. -# -# Builds a docker image for the Kafka-BigQuery Connector. -# Expects links to "kafka" and "schema-registry" containers. -# -# Usage: -# docker build -t kcbq/connect connect -# docker run --name kcbq_test_connect \ -# --link kcbq_test_kafka:kafka --link kcbq_test_schema-registry:schema-registry \ -# kcbq/connect - -FROM confluentinc/cp-kafka-connect-base:4.1.2 - -RUN ["apt", "update"] -RUN ["apt", "install", "unzip"] - -COPY connect-docker.sh /usr/local/bin/ - -RUN ["chmod", "+x", "/usr/local/bin/connect-docker.sh"] - -RUN ["mkdir", "/usr/logs"] -RUN ["chmod", "a+rwx", "/usr/logs"] - -RUN ["mkdir", "-p", "/usr/local/share/kafka/plugins/kafka-connect-bigquery"] -RUN ["chmod", "a+rwx", "/usr/local/share/kafka/plugins/kafka-connect-bigquery"] - -USER root -ENTRYPOINT ["/usr/local/bin/connect-docker.sh"] diff --git a/kcbq-connector/test/docker/connect/connect-docker.sh b/kcbq-connector/test/docker/connect/connect-docker.sh deleted file mode 100755 index a8be46892..000000000 --- a/kcbq-connector/test/docker/connect/connect-docker.sh +++ /dev/null @@ -1,26 +0,0 @@ -#! /usr/bin/env bash -# Copyright 2016 WePay, Inc. -# -# 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. - -unzip -j -d /usr/local/share/kafka/plugins/kafka-connect-bigquery/ /usr/local/share/kafka/plugins/kafka-connect-bigquery/kcbq.zip 'wepay-kafka-connect-bigquery-*/lib/*.jar' - -connect-standalone \ - /etc/kafka-connect-bigquery/standalone.properties \ - /etc/kafka-connect-bigquery/connector.properties & - -# Time (seconds) to wait for the process for inserting rows into BigQuery to be done. -# This time can be adjusted if necessary. -sleep 180 -kill $! diff --git a/kcbq-connector/test/docker/populate/Dockerfile b/kcbq-connector/test/docker/populate/Dockerfile deleted file mode 100644 index c91dcecf6..000000000 --- a/kcbq-connector/test/docker/populate/Dockerfile +++ /dev/null @@ -1,32 +0,0 @@ -# Copyright 2016 WePay, Inc. -# -# 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. -# -# Populates Kafka and Schema Registry with test data -# Expects links to "kafka" and "schema-registry" containers. -# -# Usage: -# docker build -t kcbq/populate populate -# docker run --name kcbq_test_populate \ -# --link kcbq_test_kafka:kafka --link kcbq_test_schema-registry:schema-registry \ -# kcbq/populate - -FROM confluentinc/cp-schema-registry:4.1.2 - -COPY populate-docker.sh /usr/local/bin/ - -RUN ["chmod", "+x", "/usr/local/bin/populate-docker.sh"] - -USER root -ENTRYPOINT ["/usr/local/bin/populate-docker.sh"] diff --git a/kcbq-connector/test/docker/populate/populate-docker.sh b/kcbq-connector/test/docker/populate/populate-docker.sh deleted file mode 100755 index aed681140..000000000 --- a/kcbq-connector/test/docker/populate/populate-docker.sh +++ /dev/null @@ -1,24 +0,0 @@ -#! /usr/bin/env bash -# Copyright 2016 WePay, Inc. -# -# 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. - -for schema_dir in /tmp/schemas/*; do - kafka-avro-console-producer \ - --topic "kcbq_test_`basename $schema_dir`" \ - --broker-list 'kafka:29092' \ - --property value.schema="`cat \"$schema_dir/schema.json\"`" \ - --property schema.registry.url='http://schema-registry:8081' \ - < "$schema_dir/data.json" -done diff --git a/kcbq-connector/test/integrationtest.sh b/kcbq-connector/test/integrationtest.sh deleted file mode 100755 index 75b1698f7..000000000 --- a/kcbq-connector/test/integrationtest.sh +++ /dev/null @@ -1,288 +0,0 @@ -#! /usr/bin/env bash -# Copyright 2016 WePay, Inc. -# -# 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. - -#################################################################################################### -# Basic script setup - -set -ex - -if [[ -t 1 ]]; then - NORMAL="$(tput sgr0)" - BOLD="$(tput bold)" - RED="$(tput setaf 1)" - GREEN="$(tput setaf 2)" - YELLOW="$(tput setaf 3)" -else - unset NORMAL BOLD RED GREEN YELLOW -fi - -usage() { - echo -e "usage: $0\n" \ - "[-k|--key-file ]\n" \ - "[-k|--key-source ] (path must be absolute; relative paths will not work)\n" \ - "[-p|--project ]\n" \ - "[-d|--dataset ]\n" \ - "[-b|--bucket \n]" \ - "[-f|--folder \n]" \ - 1>&2 - echo 1>&2 - echo "Options can also be specified via environment variable:" \ - "KCBQ_TEST_KEYFILE, KCBQ_TEST_PROJECT, KCBQ_TEST_DATASET, KCBQ_TEST_BUCKET, and KCBQ_TEST_FOLDER" \ - "respectively control the keyfile, project, dataset, and bucket." \ - 1>&2 - echo 1>&2 - echo "Options can also be specified in a file named 'test.conf'" \ - "placed in the same directory as this script, with a series of = lines." \ - "The properties are 'keyfile', 'project', 'dataset', and 'bucket'." \ - 1>&2 - echo 1>&2 - echo "The descending order of priority for each of these forms of specification is:" \ - "command line option, environment variable, configuration file." \ - 1>&2 - # Accept an optional exit value parameter - exit ${1:-0} -} - -msg() { printf "$1%s: $2$NORMAL\n" "$(basename $0)"; } -error() { msg "$RED" "$*"; exit 1; } >&2 -warn() { msg "$YELLOW" "$*"; } >&2 -statusupdate() { msg "$GREEN" "$*"; } -log() { msg "$BOLD" "$*"; } - - -BASE_DIR=$(dirname "$0") - -#################################################################################################### -# Configuration processing - -# Read in properties file, if it exists and can be read -PROPERTIES_FILE="$BASE_DIR/test.conf" -[[ -f "$PROPERTIES_FILE" ]] && [[ -r "$PROPERTIES_FILE" ]] && source "$PROPERTIES_FILE" - -# Copy the file's properties into actual test variables, -# without overriding any that have already been specified -KCBQ_TEST_KEYFILE=${KCBQ_TEST_KEYFILE:-$keyfile} -KCBQ_TEST_PROJECT=${KCBQ_TEST_PROJECT:-$project} -KCBQ_TEST_DATASET=${KCBQ_TEST_DATASET:-$dataset} -KCBQ_TEST_BUCKET=${KCBQ_TEST_BUCKET:-$bucket} -KCBQ_TEST_FOLDER=${KCBQ_TEST_FOLDER:-$folder} -KCBQ_TEST_KEYSOURCE=${KCBQ_TEST_KEYSOURCE:-$keysource} - -# Capture any command line flags -while [[ $# -gt 0 ]]; do - case "$1" in - -k|--key-file) - [[ -z "$2" ]] && { error "key filename must follow $1 flag"; usage 1; } - shift - KCBQ_TEST_KEYFILE="$1" - ;; - -p|--project) - [[ -z "$2" ]] && { error "project name must follow $1 flag"; usage 1; } - shift - KCBQ_TEST_PROJECT="$1" - ;; - -d|--dataset) - [[ -z "$2" ]] && { error "dataset name must follow $1 flag"; usage 1; } - shift - KCBQ_TEST_DATASET="$1" - ;; - -b|--bucket) - [[ -z "$2" ]] && { error "bucket name must follow $1 flag"; usage 1; } - shift - KCBQ_TEST_BUCKET="$1" - ;; - -b|--folder) - [[ -z "$2" ]] && { error "folder name must follow $1 flag"; usage 1; } - shift - KCBQ_TEST_FOLDER="$1" - ;; - -h|--help|'-?') - usage 0 - ;; - -kf|--key-source) - [[ -z "$2" ]] && { error "key filename must follow $1 flag"; usage 1; } - shift - KCBQ_TEST_KEYSOURCE="$1" - ;; - *) - error "unrecognized option: '$1'"; usage 1 - ;; - esac - shift -done - -# Make sure required arguments have been provided one way or another -[[ -z "$KCBQ_TEST_KEYFILE" ]] && { error 'a key filename is required'; usage 1; } -[[ -z "$KCBQ_TEST_PROJECT" ]] && { error 'a project name is required'; usage 1; } -[[ -z "$KCBQ_TEST_DATASET" ]] && { error 'a dataset name is required'; usage 1; } -[[ -z "$KCBQ_TEST_BUCKET" ]] && { error 'a bucket name is required'; usage 1; } - -#################################################################################################### -# Schema Registry Docker initialization - -if echo | xargs --no-run-if-empty; then - xargs() { command xargs --no-run-if-empty "$@"; } -else - xargs() { command xargs "$@"; } -fi 2> /dev/null - - -dockercleanup() { - log 'Cleaning up leftover Docker containers' - docker ps -aq -f 'name=kcbq_test_(zookeeper|kafka|schema-registry|populate|connect)' \ - | xargs docker rm -f > /dev/null -} - -dockerimageexists() { - docker images --format '{{ .Repository }}' | grep -q "$1" -} - -# Cleanup these on exit in case something goes wrong -trap dockercleanup EXIT -# And remove any that are still around right now -dockercleanup - -DOCKER_DIR="$BASE_DIR/docker" - -ZOOKEEPER_DOCKER_NAME='kcbq_test_zookeeper' -KAFKA_DOCKER_NAME='kcbq_test_kafka' -SCHEMA_REGISTRY_DOCKER_NAME='kcbq_test_schema-registry' - -statusupdate 'Creating Zookeeper Docker instance' -docker run --name "$ZOOKEEPER_DOCKER_NAME" \ - -d \ - -e ZOOKEEPER_CLIENT_PORT=32181 \ - confluentinc/cp-zookeeper:4.1.2 - -statusupdate 'Creating Kafka Docker instance' -docker run --name "$KAFKA_DOCKER_NAME" \ - --link "$ZOOKEEPER_DOCKER_NAME":zookeeper \ - --add-host kafka:127.0.0.1 \ - -d \ - -e KAFKA_ZOOKEEPER_CONNECT=zookeeper:32181 \ - -e KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://kafka:29092 \ - -e KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR=1 \ - confluentinc/cp-kafka:4.1.2 - -statusupdate 'Creating Schema Registry Docker instance' -# Have to pause here to make sure Zookeeper/Kafka get on their feet first -sleep 5 -docker run --name "$SCHEMA_REGISTRY_DOCKER_NAME" \ - --link "$ZOOKEEPER_DOCKER_NAME":zookeeper --link "$KAFKA_DOCKER_NAME":kafka \ - --env SCHEMA_REGISTRY_AVRO_COMPATIBILITY_LEVEL=none \ - -d \ - -e SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL=zookeeper:32181 \ - -e SCHEMA_REGISTRY_HOST_NAME=schema-registry \ - confluentinc/cp-schema-registry:4.1.2 - -#################################################################################################### -# Writing data to Kafka Docker instance via Avro console producer -statusupdate 'Populating Kafka/Schema Registry Docker instances with test data' - -POPULATE_DOCKER_IMAGE='kcbq/populate' -POPULATE_DOCKER_NAME='kcbq_test_populate' - -if ! dockerimageexists "$POPULATE_DOCKER_IMAGE"; then - docker build -q -t "$POPULATE_DOCKER_IMAGE" "$DOCKER_DIR/populate" -fi -# Have to pause here to make sure the Schema Registry gets on its feet first -sleep 35 -docker create --name "$POPULATE_DOCKER_NAME" \ - --link "$KAFKA_DOCKER_NAME:kafka" --link "$SCHEMA_REGISTRY_DOCKER_NAME:schema-registry" \ - "$POPULATE_DOCKER_IMAGE" -docker cp "$BASE_DIR/resources/test_schemas/" "$POPULATE_DOCKER_NAME:/tmp/schemas/" -docker start -a "$POPULATE_DOCKER_NAME" - -#################################################################################################### -# Deleting existing BigQuery tables/bucket -warn 'Deleting existing BigQuery test tables and existing GCS bucket' - -unset TEST_TABLES -unset TEST_TOPICS -for file in "$BASE_DIR"/resources/test_schemas/*; do - TEST_TABLES+="${TEST_TABLES:+ }test_$(basename "${file/-/_}")" - TEST_TOPICS+="${TEST_TOPICS:+,}kcbq_test_$(basename "$file")" -done - -mvn -f "$BASE_DIR/.." clean test-compile -mvn -f "$BASE_DIR/.." exec:java -Dexec.mainClass=com.wepay.kafka.connect.bigquery.it.utils.TableClearer \ - -Dexec.classpathScope=test \ - -Dexec.args="${KCBQ_TEST_KEYFILE} ${KCBQ_TEST_PROJECT} ${KCBQ_TEST_DATASET} ${TEST_TABLES}" -mvn -f "$BASE_DIR/.." exec:java -Dexec.mainClass=com.wepay.kafka.connect.bigquery.it.utils.BucketClearer \ - -Dexec.classpathScope=test \ - -Dexec.args="${KCBQ_TEST_KEYFILE} ${KCBQ_TEST_PROJECT} ${KCBQ_TEST_BUCKET}" - -#################################################################################################### -# Executing connector in standalone mode (this is the execution portion of the actual test) -statusupdate 'Executing Kafka Connect in Docker' - -mvn -f "$BASE_DIR/.." install -Dskip.unit.tests=true - -[[ ! -e "$DOCKER_DIR/connect/properties" ]] && mkdir "$DOCKER_DIR/connect/properties" -RESOURCES_DIR="$BASE_DIR/resources" - -STANDALONE_PROPS="$DOCKER_DIR/connect/properties/standalone.properties" -cp "$RESOURCES_DIR/standalone-template.properties" "$STANDALONE_PROPS" - -CONNECTOR_PROPS="$DOCKER_DIR/connect/properties/connector.properties" -cp "$RESOURCES_DIR/connector-template.properties" "$CONNECTOR_PROPS" -cat << EOF >> $CONNECTOR_PROPS -project=$KCBQ_TEST_PROJECT -defaultDataset=$KCBQ_TEST_DATASET -gcsBucketName=$KCBQ_TEST_BUCKET -gcsFolderName=$KCBQ_TEST_FOLDER -topics=$TEST_TOPICS - -EOF - -CONNECT_DOCKER_IMAGE='kcbq/connect' -CONNECT_DOCKER_NAME='kcbq_test_connect' - -cp "$BASE_DIR"/../target/components/packages/wepay-kafka-connect-bigquery-*.zip "$DOCKER_DIR/connect/kcbq.zip" -if [[ "$KCBQ_TEST_KEYSOURCE" == "JSON" ]]; then - echo "$KCBQ_TEST_KEYFILE" > "$DOCKER_DIR/connect/key.json" -else - cp "$KCBQ_TEST_KEYFILE" "$DOCKER_DIR/connect/key.json" -fi - -if ! dockerimageexists "$CONNECT_DOCKER_IMAGE"; then - docker build -q -t "$CONNECT_DOCKER_IMAGE" "$DOCKER_DIR/connect" -fi -docker create --name "$CONNECT_DOCKER_NAME" \ - --link "$KAFKA_DOCKER_NAME:kafka" --link "$SCHEMA_REGISTRY_DOCKER_NAME:schema-registry" \ - -t "$CONNECT_DOCKER_IMAGE" /bin/bash -docker cp "$DOCKER_DIR/connect/kcbq.zip" "$CONNECT_DOCKER_NAME:/usr/local/share/kafka/plugins/kafka-connect-bigquery/kcbq.zip" -docker cp "$DOCKER_DIR/connect/properties/" "$CONNECT_DOCKER_NAME:/etc/kafka-connect-bigquery/" -docker cp "$DOCKER_DIR/connect/key.json" "$CONNECT_DOCKER_NAME:/tmp/key.json" -docker start -a "$CONNECT_DOCKER_NAME" - -#################################################################################################### -# Checking on BigQuery data via Java test (this is the verification portion of the actual test) -statusupdate 'Verifying that test data made it successfully to BigQuery' - -TEST_RESOURCE_DIR="$BASE_DIR/../src/test/resources" -[[ ! -d "$TEST_RESOURCE_DIR" ]] && mkdir -p "$TEST_RESOURCE_DIR" - -cat << EOF > "$TEST_RESOURCE_DIR/test.properties" -keyfile=$KCBQ_TEST_KEYFILE -project=$KCBQ_TEST_PROJECT -dataset=$KCBQ_TEST_DATASET -bucket=$KCBQ_TEST_BUCKET -folder=$KCBQ_TEST_FOLDER -keysource=$KCBQ_TEST_KEYSOURCE -EOF - -mvn -f "$BASE_DIR/.." clean test-compile -Dskip.unit.tests=true failsafe:integration-test@verify-docker-test \ No newline at end of file diff --git a/kcbq-connector/test/resources/connector-template.properties b/kcbq-connector/test/resources/connector-template.properties deleted file mode 100644 index a8aa8e766..000000000 --- a/kcbq-connector/test/resources/connector-template.properties +++ /dev/null @@ -1,41 +0,0 @@ -# Copyright 2016 WePay, Inc. -# -# 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. - -name=bigquery-connector -connector.class=com.wepay.kafka.connect.bigquery.BigQuerySinkConnector -tasks.max=1 -defaultDataset= -autoUpdateSchemas=true - -sanitizeTopics=true - -bufferSize=100000 -maxWriteSize=1000 -tableWriteWait=1000 - -keyfile=/tmp/key.json -schemaRetriever=com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever - -enableBatchLoad=kcbq_test_gcs-load -batchLoadIntervalSec=10 - -# An example regex router SMT that strips (kcbq_) from the topic name. -# Replace with relevant regex to replace the topic of each sink record with -# destination dataset and table name in the format : or only the destination -# table name in the format -transforms=RegexTransformation -transforms.RegexTransformation.type=org.apache.kafka.connect.transforms.RegexRouter -transforms.RegexTransformation.regex=(kcbq_)(.*) -transforms.RegexTransformation.replacement=$2 diff --git a/kcbq-connector/test/resources/standalone-template.properties b/kcbq-connector/test/resources/standalone-template.properties deleted file mode 100644 index cb9998503..000000000 --- a/kcbq-connector/test/resources/standalone-template.properties +++ /dev/null @@ -1,27 +0,0 @@ -# Copyright 2016 WePay, Inc. -# -# 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. - -bootstrap.servers=kafka:29092 -key.converter=io.confluent.connect.avro.AvroConverter -key.converter.schema.registry.url=http://schema-registry:8081 -value.converter=io.confluent.connect.avro.AvroConverter -value.converter.schema.registry.url=http://schema-registry:8081 -internal.key.converter=org.apache.kafka.connect.json.JsonConverter -internal.value.converter=org.apache.kafka.connect.json.JsonConverter -internal.key.converter.schemas.enable=false -internal.value.converter.schemas.enable=false -offset.storage.file.filename=/tmp/connect.offsets -offset.flush.interval.ms=10000 -plugin.path=/usr/local/share/kafka/plugins diff --git a/pom.xml b/pom.xml index b8139c4ce..3c3356217 100644 --- a/pom.xml +++ b/pom.xml @@ -225,6 +225,28 @@ test-jar test + + io.confluent + kafka-schema-registry + ${confluent.version} + + + io.confluent + kafka-schema-registry + ${confluent.version} + tests + test-jar + + + io.confluent + kafka-connect-avro-converter + ${confluent.version} + + + io.confluent + kafka-avro-serializer + ${confluent.version} + @@ -258,7 +280,6 @@ ${surefire.plugin.version} - **/*IntegrationTest.java **/*IT.java ${skip.unit.tests} @@ -269,17 +290,6 @@ maven-failsafe-plugin ${surefire.plugin.version} - - verify-docker-test - - integration-test - - - - **/*IntegrationTest.java - - - embedded-integration-test @@ -368,6 +378,17 @@ true + + org.apache.maven.plugins + maven-failsafe-plugin + + + connect-205118 + jenkinsKcbqIntegrationTesting + jenkins-kcbq-integration-testing + + + From 4ab4917209e812bb0edd8130c83795f3f54148bb Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 6 Oct 2020 13:48:02 -0400 Subject: [PATCH 036/190] MINOR: Modify license headers, add automated license header generation and enforcement (#38) * GH-32: Switch from Gradle to Maven for build tool * Add license plugin and reformat copyright notice on (almost) all files --- Jenkinsfile | 18 +++++ config/checkstyle/suppressions.xml | 20 +++++ config/copyright/custom-header-styles.xml | 44 +++++++++++ kcbq-api/pom.xml | 39 ++++++---- .../connect/bigquery/api/SchemaRetriever.java | 19 +++++ kcbq-confluent/pom.xml | 39 ++++++---- .../SchemaRegistrySchemaRetriever.java | 19 +++++ .../SchemaRegistrySchemaRetrieverConfig.java | 19 +++++ .../SchemaRegistrySchemaRetrieverTest.java | 19 +++++ .../src/test/resources/log4j.properties | 19 +++++ kcbq-connector/pom.xml | 39 ++++++---- .../quickstart/avro-console-producer.sh | 6 +- kcbq-connector/quickstart/connector.sh | 6 +- kcbq-connector/quickstart/kafka.sh | 6 +- .../properties/connector.properties | 6 +- .../properties/standalone.properties | 6 +- kcbq-connector/quickstart/schema-registry.sh | 6 +- kcbq-connector/quickstart/zookeeper.sh | 6 +- .../connect/bigquery/BigQueryHelper.java | 8 +- .../bigquery/BigQuerySinkConnector.java | 7 +- .../connect/bigquery/BigQuerySinkTask.java | 7 +- .../kafka/connect/bigquery/SchemaManager.java | 19 +++++ .../bigquery/config/BigQuerySinkConfig.java | 7 +- .../config/BigQuerySinkConnectorConfig.java | 7 +- .../config/BigQuerySinkTaskConfig.java | 7 +- .../convert/BigQueryRecordConverter.java | 7 +- .../convert/BigQuerySchemaConverter.java | 7 +- .../bigquery/convert/RecordConverter.java | 7 +- .../bigquery/convert/SchemaConverter.java | 7 +- .../kafkadata/KafkaDataBQRecordConverter.java | 7 +- .../kafkadata/KafkaDataBQSchemaConverter.java | 7 +- .../DebeziumLogicalConverters.java | 7 +- .../logicaltype/KafkaLogicalConverters.java | 7 +- .../logicaltype/LogicalConverterRegistry.java | 7 +- .../logicaltype/LogicalTypeConverter.java | 7 +- .../exception/BigQueryConnectException.java | 7 +- .../exception/ConversionConnectException.java | 7 +- .../exception/SinkConfigConnectException.java | 7 +- .../retrieve/MemorySchemaRetriever.java | 19 +++++ .../bigquery/utils/PartitionedTableId.java | 7 +- .../bigquery/utils/TopicToTableResolver.java | 7 +- .../kafka/connect/bigquery/utils/Version.java | 7 +- .../write/batch/CountDownRunnable.java | 7 +- .../write/batch/KCBQThreadPoolExecutor.java | 7 +- .../bigquery/write/batch/TableWriter.java | 7 +- .../write/row/AdaptiveBigQueryWriter.java | 7 +- .../bigquery/write/row/BigQueryWriter.java | 7 +- .../write/row/SimpleBigQueryWriter.java | 7 +- .../bigquery/BigQuerySinkConnectorTest.java | 7 +- .../bigquery/BigQuerySinkTaskTest.java | 7 +- .../connect/bigquery/SchemaManagerTest.java | 7 +- .../SinkConnectorPropertiesFactory.java | 7 +- .../bigquery/SinkPropertiesFactory.java | 7 +- .../bigquery/SinkTaskPropertiesFactory.java | 7 +- .../config/BigQuerySinkConfigTest.java | 7 +- .../BigQuerySinkConnectorConfigTest.java | 7 +- .../config/BigQuerySinkTaskConfigTest.java | 7 +- .../convert/BigQueryRecordConverterTest.java | 7 +- .../convert/BigQuerySchemaConverterTest.java | 7 +- .../KafkaDataBQRecordConvertTest.java | 7 +- .../KafkaDataBQSchemaConverterTest.java | 7 +- .../DebeziumLogicalConvertersTest.java | 7 +- .../KafkaLogicalConvertersTest.java | 7 +- .../it/BigQueryConnectorIntegrationTest.java | 7 +- .../bigquery/it/utils/TableClearer.java | 7 +- .../retrieve/MemorySchemaRetrieverTest.java | 21 ++++- .../utils/PartitionedTableIdTest.java | 7 +- .../utils/TopicToTableResolverTest.java | 8 +- .../write/row/BigQueryWriterTest.java | 7 +- .../src/test/resources/log4j.properties | 19 +++++ .../src/test/resources/test.properties | 22 ++++++ kcbq-connector/test/docker/connect/Dockerfile | 13 +--- .../test/docker/connect/connect-docker.sh | 6 +- .../test/docker/populate/Dockerfile | 13 +--- .../test/docker/populate/populate-docker.sh | 6 +- kcbq-connector/test/integrationtest.sh | 6 +- .../resources/connector-template.properties | 6 +- .../resources/standalone-template.properties | 6 +- pom.xml | 76 +++++++++++++++---- 79 files changed, 670 insertions(+), 236 deletions(-) create mode 100644 config/copyright/custom-header-styles.xml create mode 100644 kcbq-connector/src/test/resources/test.properties diff --git a/Jenkinsfile b/Jenkinsfile index 551c15919..a080d23d6 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -1,4 +1,22 @@ #!/usr/bin/env groovy +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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. + */ common { slackChannel = '#connect-warn' nodeLabel = 'docker-oraclejdk8' diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml index 28a683b60..4bf4c40b5 100644 --- a/config/checkstyle/suppressions.xml +++ b/config/checkstyle/suppressions.xml @@ -1,4 +1,24 @@ + + + + + /* + * + */EOL + (\s|\t)*/\*.*$ + .*\*/(\s|\t)*$ + false + true + false + + + /* + * + */ + #!.* + (\s|\t)*/\*.* + .*\*/(\s|\t)*$ + false + true + false + + \ No newline at end of file diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 7e4d3c45e..ae804c5d5 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -1,19 +1,24 @@ - + 4.0.0 @@ -27,6 +32,10 @@ kcbq-api kafka-connect-bigquery-api + + ${project.parent.basedir} + + org.apache.kafka diff --git a/kcbq-api/src/main/java/com/wepay/kafka/connect/bigquery/api/SchemaRetriever.java b/kcbq-api/src/main/java/com/wepay/kafka/connect/bigquery/api/SchemaRetriever.java index a46c67097..5b7e66306 100644 --- a/kcbq-api/src/main/java/com/wepay/kafka/connect/bigquery/api/SchemaRetriever.java +++ b/kcbq-api/src/main/java/com/wepay/kafka/connect/bigquery/api/SchemaRetriever.java @@ -1,3 +1,22 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.api; import com.google.cloud.bigquery.TableId; diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index 5545e657d..51f555557 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -1,19 +1,24 @@ - + 4.0.0 @@ -27,6 +32,10 @@ kcbq-confluent kafka-connect-bigquery-confluent + + ${project.parent.basedir} + + com.wepay.kcbq diff --git a/kcbq-confluent/src/main/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetriever.java b/kcbq-confluent/src/main/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetriever.java index f85a404f0..420a92e22 100644 --- a/kcbq-confluent/src/main/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetriever.java +++ b/kcbq-confluent/src/main/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetriever.java @@ -1,3 +1,22 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.schemaregistry.schemaretriever; import com.google.cloud.bigquery.TableId; diff --git a/kcbq-confluent/src/main/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetrieverConfig.java b/kcbq-confluent/src/main/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetrieverConfig.java index f43fb411c..069914aaf 100644 --- a/kcbq-confluent/src/main/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetrieverConfig.java +++ b/kcbq-confluent/src/main/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetrieverConfig.java @@ -1,3 +1,22 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.schemaregistry.schemaretriever; import org.apache.kafka.common.config.AbstractConfig; diff --git a/kcbq-confluent/src/test/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetrieverTest.java b/kcbq-confluent/src/test/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetrieverTest.java index 18065ce1d..49a9e9602 100644 --- a/kcbq-confluent/src/test/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetrieverTest.java +++ b/kcbq-confluent/src/test/java/com/wepay/kafka/connect/bigquery/schemaregistry/schemaretriever/SchemaRegistrySchemaRetrieverTest.java @@ -1,3 +1,22 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.schemaregistry.schemaretriever; import static org.junit.Assert.assertEquals; diff --git a/kcbq-confluent/src/test/resources/log4j.properties b/kcbq-confluent/src/test/resources/log4j.properties index 2a5a0233d..94fb72b55 100644 --- a/kcbq-confluent/src/test/resources/log4j.properties +++ b/kcbq-confluent/src/test/resources/log4j.properties @@ -1,3 +1,22 @@ +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. +# +# 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. +# + log4j.rootLogger=INFO, stdout # Send the logs to the console. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 6f364dab3..8ecd8281c 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -1,19 +1,24 @@ - + 4.0.0 @@ -27,6 +32,10 @@ kcbq-connector kafka-connect-bigquery + + ${project.parent.basedir} + + org.apache.kafka diff --git a/kcbq-connector/quickstart/avro-console-producer.sh b/kcbq-connector/quickstart/avro-console-producer.sh index a7fe02118..9065f0cb3 100755 --- a/kcbq-connector/quickstart/avro-console-producer.sh +++ b/kcbq-connector/quickstart/avro-console-producer.sh @@ -1,5 +1,8 @@ #! /usr/bin/env bash -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,6 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# BASE_DIR=`dirname "$0"` diff --git a/kcbq-connector/quickstart/connector.sh b/kcbq-connector/quickstart/connector.sh index 5c9dcecd9..123e9bbe9 100755 --- a/kcbq-connector/quickstart/connector.sh +++ b/kcbq-connector/quickstart/connector.sh @@ -1,5 +1,8 @@ #! /usr/bin/env bash -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,6 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# BASE_DIR="$(cd "$(dirname "$0")" && pwd)" diff --git a/kcbq-connector/quickstart/kafka.sh b/kcbq-connector/quickstart/kafka.sh index 953c0d3f8..2ce3391ab 100755 --- a/kcbq-connector/quickstart/kafka.sh +++ b/kcbq-connector/quickstart/kafka.sh @@ -1,5 +1,8 @@ #! /usr/bin/env bash -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,6 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# BASE_DIR=`dirname "$0"` diff --git a/kcbq-connector/quickstart/properties/connector.properties b/kcbq-connector/quickstart/properties/connector.properties index 05ee82fb1..50c9211ae 100644 --- a/kcbq-connector/quickstart/properties/connector.properties +++ b/kcbq-connector/quickstart/properties/connector.properties @@ -1,4 +1,7 @@ -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -12,6 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# name=bigquery-connector connector.class=com.wepay.kafka.connect.bigquery.BigQuerySinkConnector diff --git a/kcbq-connector/quickstart/properties/standalone.properties b/kcbq-connector/quickstart/properties/standalone.properties index 2aee81055..1450e07cc 100644 --- a/kcbq-connector/quickstart/properties/standalone.properties +++ b/kcbq-connector/quickstart/properties/standalone.properties @@ -1,4 +1,7 @@ -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -12,6 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# bootstrap.servers=localhost:9092 key.converter=io.confluent.connect.avro.AvroConverter diff --git a/kcbq-connector/quickstart/schema-registry.sh b/kcbq-connector/quickstart/schema-registry.sh index 5b5dfd3a6..61735fabc 100755 --- a/kcbq-connector/quickstart/schema-registry.sh +++ b/kcbq-connector/quickstart/schema-registry.sh @@ -1,5 +1,8 @@ #! /usr/bin/env bash -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,6 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# BASE_DIR=`dirname "$0"` diff --git a/kcbq-connector/quickstart/zookeeper.sh b/kcbq-connector/quickstart/zookeeper.sh index ad5a88205..3e5fcbdcc 100755 --- a/kcbq-connector/quickstart/zookeeper.sh +++ b/kcbq-connector/quickstart/zookeeper.sh @@ -1,5 +1,8 @@ #! /usr/bin/env bash -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,6 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# BASE_DIR=`dirname "$0"` diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java index 1013b7a79..97dd30c7f 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; + import com.google.auth.oauth2.GoogleCredentials; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryOptions; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index db8a55e4e..7d2f99bf0 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.TableId; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index b06dff5d6..f43bd722a 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 74085228f..e3fe5bfc5 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -1,3 +1,22 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery; import com.google.cloud.bigquery.BigQuery; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index accb3af7a..3b0bd5d85 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.config; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.config; import com.google.cloud.bigquery.Schema; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConnectorConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConnectorConfig.java index 3d8c8b95a..eded455d5 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConnectorConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConnectorConfig.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.config; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.config; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java index 65b9b056d..482f6b652 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.config; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.config; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverter.java index 65de6b172..6fd57a717 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert; import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java index 74c43f7d2..99e02d99d 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert; import com.wepay.kafka.connect.bigquery.convert.logicaltype.DebeziumLogicalConverters; import com.wepay.kafka.connect.bigquery.convert.logicaltype.KafkaLogicalConverters; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/RecordConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/RecordConverter.java index a3a1bfd1e..73f6c4e7e 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/RecordConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/RecordConverter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert; import org.apache.kafka.connect.sink.SinkRecord; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/SchemaConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/SchemaConverter.java index 985c736f8..8ca2e68b7 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/SchemaConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/SchemaConverter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert; import org.apache.kafka.connect.data.Schema; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQRecordConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQRecordConverter.java index 9cf93cf0c..9178105dc 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQRecordConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQRecordConverter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.kafkadata; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.kafkadata; import com.wepay.kafka.connect.bigquery.convert.BigQueryRecordConverter; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQSchemaConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQSchemaConverter.java index 918fd3c74..0d68cd9e9 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQSchemaConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQSchemaConverter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.kafkadata; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.kafkadata; import com.google.cloud.bigquery.Field; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java index 526f4ab4d..24326789c 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.logicaltype; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.logicaltype; import com.google.cloud.bigquery.Field; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConverters.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConverters.java index 84997d07e..3f8781917 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConverters.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConverters.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.logicaltype; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.logicaltype; import com.google.cloud.bigquery.Field; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalConverterRegistry.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalConverterRegistry.java index 36757de47..b21bcf613 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalConverterRegistry.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalConverterRegistry.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.logicaltype; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.logicaltype; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalTypeConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalTypeConverter.java index 2fe93cc66..891d582a7 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalTypeConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalTypeConverter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.logicaltype; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.logicaltype; import com.google.cloud.bigquery.Field; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java index 40fefd7da..a49c77726 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.exception; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.exception; import com.google.cloud.bigquery.BigQueryError; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/ConversionConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/ConversionConnectException.java index 795ea6749..29e10bd43 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/ConversionConnectException.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/ConversionConnectException.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.exception; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.exception; import org.apache.kafka.connect.errors.ConnectException; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java index 98a11c069..805cd5643 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.exception; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.exception; import org.apache.kafka.connect.errors.ConnectException; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/retrieve/MemorySchemaRetriever.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/retrieve/MemorySchemaRetriever.java index bb6adcb63..a7227ee95 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/retrieve/MemorySchemaRetriever.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/retrieve/MemorySchemaRetriever.java @@ -1,3 +1,22 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.retrieve; import com.google.cloud.bigquery.BigQuery; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java index 4f3e3ee48..28ae9b602 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.utils; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.utils; import com.google.cloud.bigquery.TableId; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/TopicToTableResolver.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/TopicToTableResolver.java index f1db8286b..994109845 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/TopicToTableResolver.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/TopicToTableResolver.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.utils; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.utils; import com.google.cloud.bigquery.TableId; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/Version.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/Version.java index bbcdfae38..8a6c6f4be 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/Version.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/Version.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.utils; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.utils; /** * Utility class for unifying the version of a project. All other references to version number diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/CountDownRunnable.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/CountDownRunnable.java index cbdca4ea6..70edc8a1a 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/CountDownRunnable.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/CountDownRunnable.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.write.batch; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.write.batch; import org.apache.kafka.connect.errors.ConnectException; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java index 909a53709..2b8058856 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.write.batch; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.write.batch; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index aef368450..5fb0a7139 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.write.batch; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.write.batch; import com.google.cloud.bigquery.BigQueryException; import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java index 1031ae166..07d682fee 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.write.row; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.write.row; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryError; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java index f21a8dd7f..c335254b7 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.write.row; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.write.row; import com.google.cloud.bigquery.BigQueryError; import com.google.cloud.bigquery.BigQueryException; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/SimpleBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/SimpleBigQueryWriter.java index b1189473a..8d22af876 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/SimpleBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/SimpleBigQueryWriter.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.write.row; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.write.row; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryError; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java index 2ebf97e6d..65933de07 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index d58a8b076..f36500f7d 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryError; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index 27e5dccd6..bc89accfb 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkConnectorPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkConnectorPropertiesFactory.java index 8da3ca467..c29c25394 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkConnectorPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkConnectorPropertiesFactory.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConnectorConfig; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java index a87e1cb64..42c80f2bd 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java index cda1d6c94..1d09fac3d 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index 7c7327ae6..7547099b9 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.config; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.config; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConnectorConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConnectorConfigTest.java index a27c149e4..e136b99b1 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConnectorConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConnectorConfigTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.config; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.config; import com.wepay.kafka.connect.bigquery.SinkConnectorPropertiesFactory; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java index d989e7ea8..2ca564e00 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.config; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.config; import static org.junit.Assert.fail; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverterTest.java index 7529eec3b..a86bffa85 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQueryRecordConverterTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert; import static org.junit.Assert.assertEquals; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java index 6fb044ba4..5dae59e3c 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert; import static org.junit.Assert.assertEquals; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQRecordConvertTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQRecordConvertTest.java index 66b5c7c40..39c369821 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQRecordConvertTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQRecordConvertTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.kafkadata; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.kafkadata; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQSchemaConverterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQSchemaConverterTest.java index 3fc1970a9..d5764ce99 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQSchemaConverterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/kafkadata/KafkaDataBQSchemaConverterTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.kafkadata; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.kafkadata; import static org.junit.Assert.assertEquals; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConvertersTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConvertersTest.java index ec8f8b6d0..519026715 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConvertersTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConvertersTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.logicaltype; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.logicaltype; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConvertersTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConvertersTest.java index 1808ccfe0..778c68613 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConvertersTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConvertersTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.convert.logicaltype; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.convert.logicaltype; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java index 546c1549f..61d763ee8 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.it; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.it; import static com.google.cloud.bigquery.LegacySQLTypeName.*; import static org.junit.Assert.assertEquals; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java index af981d4ea..64849d0ad 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.it.utils; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.it.utils; import com.google.cloud.bigquery.BigQuery; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/retrieve/MemorySchemaRetrieverTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/retrieve/MemorySchemaRetrieverTest.java index 8cd83e990..83192ddf5 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/retrieve/MemorySchemaRetrieverTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/retrieve/MemorySchemaRetrieverTest.java @@ -1,5 +1,23 @@ -package com.wepay.kafka.connect.bigquery.retrieve; +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.retrieve; import com.google.cloud.bigquery.TableId; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; @@ -11,7 +29,6 @@ import java.util.HashMap; - public class MemorySchemaRetrieverTest { public TableId getTableId(String datasetName, String tableName) { return TableId.of(datasetName, tableName); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java index 1737bfbdf..d24072714 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.utils; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.utils; import com.google.cloud.bigquery.TableId; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/TopicToTableResolverTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/TopicToTableResolverTest.java index bd69219aa..42106a35c 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/TopicToTableResolverTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/TopicToTableResolverTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.utils; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.utils; + import static org.junit.Assert.assertEquals; import com.google.cloud.bigquery.TableId; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index 439c70026..4073c186a 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -1,7 +1,7 @@ -package com.wepay.kafka.connect.bigquery.write.row; - /* - * Copyright 2016 WePay, Inc. + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ * under the License. */ +package com.wepay.kafka.connect.bigquery.write.row; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryError; diff --git a/kcbq-connector/src/test/resources/log4j.properties b/kcbq-connector/src/test/resources/log4j.properties index 2a5a0233d..94fb72b55 100644 --- a/kcbq-connector/src/test/resources/log4j.properties +++ b/kcbq-connector/src/test/resources/log4j.properties @@ -1,3 +1,22 @@ +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. +# +# 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. +# + log4j.rootLogger=INFO, stdout # Send the logs to the console. diff --git a/kcbq-connector/src/test/resources/test.properties b/kcbq-connector/src/test/resources/test.properties new file mode 100644 index 000000000..27b48b485 --- /dev/null +++ b/kcbq-connector/src/test/resources/test.properties @@ -0,0 +1,22 @@ +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. +# +# 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. +# + +keyfile=/Users/chrise/.gcs-creds.json +project=connect-205118 +dataset=confluentBigquery diff --git a/kcbq-connector/test/docker/connect/Dockerfile b/kcbq-connector/test/docker/connect/Dockerfile index 3ad1d8896..ff296f8da 100644 --- a/kcbq-connector/test/docker/connect/Dockerfile +++ b/kcbq-connector/test/docker/connect/Dockerfile @@ -1,4 +1,7 @@ -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,14 +16,6 @@ # specific language governing permissions and limitations # under the License. # -# Builds a docker image for the Kafka-BigQuery Connector. -# Expects links to "kafka" and "schema-registry" containers. -# -# Usage: -# docker build -t kcbq/connect connect -# docker run --name kcbq_test_connect \ -# --link kcbq_test_kafka:kafka --link kcbq_test_schema-registry:schema-registry \ -# kcbq/connect FROM confluent/platform:3.0.0 diff --git a/kcbq-connector/test/docker/connect/connect-docker.sh b/kcbq-connector/test/docker/connect/connect-docker.sh index d0f78fafd..4443f7d38 100755 --- a/kcbq-connector/test/docker/connect/connect-docker.sh +++ b/kcbq-connector/test/docker/connect/connect-docker.sh @@ -1,5 +1,8 @@ #! /usr/bin/env bash -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,6 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# unzip -j -d /usr/share/java/kafka-connect-bigquery /usr/share/java/kafka-connect-bigquery/kcbq.zip 'wepay-kafka-connect-bigquery-*/lib/*.jar' diff --git a/kcbq-connector/test/docker/populate/Dockerfile b/kcbq-connector/test/docker/populate/Dockerfile index 756d418ea..3c9fe5b71 100644 --- a/kcbq-connector/test/docker/populate/Dockerfile +++ b/kcbq-connector/test/docker/populate/Dockerfile @@ -1,4 +1,7 @@ -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,14 +16,6 @@ # specific language governing permissions and limitations # under the License. # -# Populates Kafka and Schema Registry with test data -# Expects links to "kafka" and "schema-registry" containers. -# -# Usage: -# docker build -t kcbq/populate populate -# docker run --name kcbq_test_populate \ -# --link kcbq_test_kafka:kafka --link kcbq_test_schema-registry:schema-registry \ -# kcbq/populate FROM confluent/platform:3.0.0 diff --git a/kcbq-connector/test/docker/populate/populate-docker.sh b/kcbq-connector/test/docker/populate/populate-docker.sh index ee7da76f2..37094b2ee 100755 --- a/kcbq-connector/test/docker/populate/populate-docker.sh +++ b/kcbq-connector/test/docker/populate/populate-docker.sh @@ -1,5 +1,8 @@ #! /usr/bin/env bash -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,6 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# for schema_dir in /tmp/schemas/*; do kafka-avro-console-producer \ diff --git a/kcbq-connector/test/integrationtest.sh b/kcbq-connector/test/integrationtest.sh index 4ea70bef7..76d9b3ad5 100755 --- a/kcbq-connector/test/integrationtest.sh +++ b/kcbq-connector/test/integrationtest.sh @@ -1,5 +1,8 @@ #! /usr/bin/env bash -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -13,6 +16,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# #################################################################################################### # Basic script setup diff --git a/kcbq-connector/test/resources/connector-template.properties b/kcbq-connector/test/resources/connector-template.properties index c37eccb67..5c753c8cb 100644 --- a/kcbq-connector/test/resources/connector-template.properties +++ b/kcbq-connector/test/resources/connector-template.properties @@ -1,4 +1,7 @@ -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -12,6 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# name=bigquery-connector connector.class=com.wepay.kafka.connect.bigquery.BigQuerySinkConnector diff --git a/kcbq-connector/test/resources/standalone-template.properties b/kcbq-connector/test/resources/standalone-template.properties index bf23ba9e6..6ec7dacc7 100644 --- a/kcbq-connector/test/resources/standalone-template.properties +++ b/kcbq-connector/test/resources/standalone-template.properties @@ -1,4 +1,7 @@ -# Copyright 2016 WePay, Inc. +# +# Copyright 2020 Confluent, Inc. +# +# This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -12,6 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +# bootstrap.servers=kafka:9092 key.converter=io.confluent.connect.avro.AvroConverter diff --git a/pom.xml b/pom.xml index adea1235d..1ddc1dd7a 100644 --- a/pom.xml +++ b/pom.xml @@ -1,19 +1,24 @@ - + 4.0.0 @@ -56,6 +61,7 @@ 3.7.1 3.0.0-M4 + ${project.basedir} ${maven.test.skip} @@ -251,6 +257,46 @@ v@{project.version} + + com.mycila + license-maven-plugin + 3.0 + + +Copyright 2020 Confluent, Inc. + +This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + +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. + + + ${main.dir}/config/copyright/custom-header-styles.xml + + + CUSTOM_JAVA_STYLE + JENKINSFILE_STYLE + + + LICENSE.md + *.log + config/checkstyle/google_checks.xml + + + .ci/* + + + From 4aca04e3ca1e7ddc88c8c7ab89c7ba2e3feabccd Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 8 Oct 2020 17:23:40 -0400 Subject: [PATCH 037/190] MINOR: Fix scope for test dependencies (Avro converter, Schema Registry, etc.) (#43) --- pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index 1523c1cd1..616b300b6 100644 --- a/pom.xml +++ b/pom.xml @@ -235,6 +235,7 @@ io.confluent kafka-schema-registry ${confluent.version} + test io.confluent @@ -242,16 +243,19 @@ ${confluent.version} tests test-jar + test io.confluent kafka-connect-avro-converter ${confluent.version} + test io.confluent kafka-avro-serializer ${confluent.version} + test From f92eca69aa0994a47fb5d733aa972d98c9c58d97 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 8 Oct 2020 17:30:17 -0400 Subject: [PATCH 038/190] GH-41: Fix concurrent integration test runs on Jenkins (#42) * GH-41: Fix concurrent integration test runs on Jenkins * GH-41: Read from suffixed tables when verifying data in integration tests * GH-41: Add table sanitization to integration tests * GH-41: Only use RegexRouter in non-upsert delete integration tests --- README.md | 2 +- .../bigquery/integration/BaseConnectorIT.java | 29 +++++++++++++------ .../integration/BigQuerySinkConnectorIT.java | 26 +++++++++++------ .../UpsertDeleteBigQuerySinkConnectorIT.java | 25 ++++++++-------- .../integration/utils/BucketClearer.java | 23 +++++++++++++-- .../integration/utils/TableClearer.java | 13 ++++++++- pom.xml | 27 +++++++++++++---- 7 files changed, 104 insertions(+), 41 deletions(-) diff --git a/README.md b/README.md index d9076fe05..8fc97d8e8 100644 --- a/README.md +++ b/README.md @@ -153,7 +153,7 @@ used. ```bash # (Re)builds the project and runs the integration tests, skipping unit tests to save a bit of time -mvn clean package failsafe:integration-test@embedded-integration-test -Dskip.unit.tests=true +mvn clean package integration-test -Dskip.unit.tests=true ``` ### How Integration Testing Works diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java index f7491417a..5035cc36a 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -42,14 +42,13 @@ import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.bigquery.Schema; import com.google.cloud.bigquery.Table; -import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TableResult; import com.wepay.kafka.connect.bigquery.BigQueryHelper; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.runtime.AbstractStatus; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; @@ -83,6 +82,7 @@ public abstract class BaseConnectorIT { private static final String KEYSOURCE_ENV_VAR = "KCBQ_TEST_KEYSOURCE"; private static final String GCS_BUCKET_ENV_VAR = "KCBQ_TEST_BUCKET"; private static final String GCS_FOLDER_ENV_VAR = "KCBQ_TEST_FOLDER"; + private static final String TEST_NAMESPACE_ENV_VAR = "KCBQ_TEST_TABLE_SUFFIX"; protected static final long OFFSET_COMMIT_INTERVAL_MS = TimeUnit.SECONDS.toMillis(10); protected static final long COMMIT_MAX_DURATION_MS = TimeUnit.MINUTES.toMillis(5); @@ -138,6 +138,8 @@ protected Map baseConnectorProps(int tasksMax) { result.put(BigQuerySinkConfig.KEYFILE_CONFIG, keyFile()); result.put(BigQuerySinkConfig.KEY_SOURCE_CONFIG, keySource()); + result.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); + return result; } @@ -147,13 +149,6 @@ protected BigQuery newBigQuery() { .connect(project(), keyFile()); } - protected void clearPriorTable(BigQuery bigQuery, String table) { - boolean deleted = bigQuery.delete(TableId.of(dataset(), table)); - if (deleted) { - logger.info("Deleted existing test table `{}`.`{}`", dataset(), table); - } - } - protected void waitForCommittedRecords( String connector, String topic, long numRecords, int numTasks ) throws InterruptedException { @@ -332,6 +327,18 @@ protected Optional assertConnectorAndTasksRunning(String connectorName, } } + protected String suffixedTableOrTopic(String tableOrTopic) { + return tableOrTopic + tableSuffix(); + } + + protected String sanitizedTable(String table) { + return FieldNameSanitizer.sanitizeName(table); + } + + protected String suffixedAndSanitizedTable(String table) { + return sanitizedTable(suffixedTableOrTopic(table)); + } + private String readEnvVar(String var) { String result = System.getenv(var); if (result == null) { @@ -369,4 +376,8 @@ protected String gcsBucket() { protected String gcsFolder() { return readEnvVar(GCS_FOLDER_ENV_VAR, BigQuerySinkConfig.GCS_FOLDER_NAME_DEFAULT); } + + protected String tableSuffix() { + return readEnvVar(TEST_NAMESPACE_ENV_VAR, ""); + } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java index aea9f3fc8..9777837bb 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java @@ -158,10 +158,6 @@ public class BigQuerySinkConnectorIT extends BaseConnectorIT { .map(tc -> TEST_CASE_PREFIX + tc) .collect(Collectors.toList()); - private static final Collection TEST_TABLES = TEST_TOPICS.stream() - .map(FieldNameSanitizer::sanitizeName) - .collect(Collectors.toList()); - private RestApp restApp; private String schemaRegistryUrl; private Producer valueProducer; @@ -169,8 +165,11 @@ public class BigQuerySinkConnectorIT extends BaseConnectorIT { @Before public void setup() throws Exception { - BucketClearer.clearBucket(keyFile(), project(), gcsBucket(), keySource()); - TableClearer.clearTables(newBigQuery(), dataset(), TEST_TABLES); + Collection tables = TEST_TOPICS.stream() + .map(this::suffixedAndSanitizedTable) + .collect(Collectors.toSet()); + BucketClearer.clearBucket(keyFile(), project(), gcsBucket(), gcsFolder(), keySource()); + TableClearer.clearTables(newBigQuery(), dataset(), tables); startConnect(); restApp = new RestApp( @@ -271,20 +270,29 @@ private Map connectorProps(int tasksMax) { result.put(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG, "true"); result.put(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG, "true"); - result.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); - result.put(BigQuerySinkConfig.ENABLE_BATCH_CONFIG, "kcbq_test_gcs-load"); + result.put(BigQuerySinkConfig.ENABLE_BATCH_CONFIG, suffixedAndSanitizedTable("kcbq_test_gcs-load")); result.put(BigQuerySinkConfig.BATCH_LOAD_INTERVAL_SEC_CONFIG, "10"); result.put(BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG, gcsBucket()); result.put(BigQuerySinkConfig.GCS_FOLDER_NAME_CONFIG, gcsFolder()); result.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, IdentitySchemaRetriever.class.getName()); + String suffix = tableSuffix(); + if (!suffix.isEmpty()) { + String escapedSuffix = suffix.replaceAll("\\\\", "\\\\\\\\").replaceAll("\\$", "\\\\\\$"); + result.put("transforms", "addSuffix"); + result.put("transforms.addSuffix.type", "org.apache.kafka.connect.transforms.RegexRouter"); + result.put("transforms.addSuffix.regex", "(.*)"); + result.put("transforms.addSuffix.replacement", "$1" + escapedSuffix); + } + return result; } private void verify(String testCase, List> expectedRows) { List> testRows; try { - testRows = readAllRows(newBigQuery(), TEST_CASE_PREFIX + FieldNameSanitizer.sanitizeName(testCase), "row"); + String table = suffixedAndSanitizedTable(TEST_CASE_PREFIX + FieldNameSanitizer.sanitizeName(testCase)); + testRows = readAllRows(newBigQuery(), table, "row"); } catch (InterruptedException e) { throw new RuntimeException(e); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java index 9e2f01d10..5f83e7ef8 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/UpsertDeleteBigQuerySinkConnectorIT.java @@ -21,6 +21,7 @@ import com.google.cloud.bigquery.BigQuery; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.integration.utils.TableClearer; import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.connect.data.Schema; @@ -111,12 +112,12 @@ private Map upsertDeleteProps( @Test public void testUpsert() throws Throwable { // create topic in Kafka - final String topic = "test-upsert"; + final String topic = suffixedTableOrTopic("test-upsert"); // Make sure each task gets to read from at least one partition connect.kafka().createTopic(topic, TASKS_MAX); - final String table = "test_upsert"; - clearPriorTable(bigQuery, table); + final String table = sanitizedTable(topic); + TableClearer.clearTables(bigQuery, dataset(), table); // setup props for the sink connector Map props = baseConnectorProps(TASKS_MAX); @@ -166,12 +167,12 @@ public void testUpsert() throws Throwable { @Test public void testDelete() throws Throwable { // create topic in Kafka - final String topic = "test-delete"; + final String topic = suffixedTableOrTopic("test-delete"); // Make sure each task gets to read from at least one partition connect.kafka().createTopic(topic, TASKS_MAX); - final String table = "test_delete"; - clearPriorTable(bigQuery, table); + final String table = sanitizedTable(topic); + TableClearer.clearTables(bigQuery, dataset(), table); // setup props for the sink connector Map props = baseConnectorProps(TASKS_MAX); @@ -225,12 +226,12 @@ public void testDelete() throws Throwable { @Test public void testUpsertDelete() throws Throwable { // create topic in Kafka - final String topic = "test-upsert-delete"; + final String topic = suffixedTableOrTopic("test-upsert-delete"); // Make sure each task gets to read from at least one partition connect.kafka().createTopic(topic, TASKS_MAX); - final String table = "test_upsert_delete"; - clearPriorTable(bigQuery, table); + final String table = sanitizedTable(topic); + TableClearer.clearTables(bigQuery, dataset(), table); // setup props for the sink connector Map props = baseConnectorProps(TASKS_MAX); @@ -289,11 +290,11 @@ public void testUpsertDeleteHighThroughput() throws Throwable { final int tasksMax = 1; // create topic in Kafka - final String topic = "test-upsert-delete-throughput"; + final String topic = suffixedTableOrTopic("test-upsert-delete-throughput"); connect.kafka().createTopic(topic, numPartitions); - final String table = "test_upsert_delete_throughput"; - clearPriorTable(bigQuery, table); + final String table = sanitizedTable(topic); + TableClearer.clearTables(bigQuery, dataset(), table); // Instantiate the converters we'll use to send records to the connector Converter keyConverter = converter(true); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java index 59e2ba799..4a9fb7301 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java @@ -19,6 +19,7 @@ package com.wepay.kafka.connect.bigquery.integration.utils; +import com.google.api.gax.paging.Page; import com.google.cloud.storage.Blob; import com.google.cloud.storage.Bucket; import com.google.cloud.storage.Storage; @@ -36,15 +37,18 @@ public class BucketClearer { * @param key The GCP credentials to use (can be a filename or a raw JSON string). * @param project The GCP project the bucket belongs to. * @param bucketName The bucket to clear. + * @param folderName The folder to clear (can be empty or null). * @param keySource The key source. If "FILE", then the {@code key} parameter will be treated as a * filename; if "JSON", then {@code key} will be treated as a raw JSON string. */ - public static void clearBucket(String key, String project, String bucketName, String keySource) { + public static void clearBucket( + String key, String project, String bucketName, String folderName, String keySource) { Storage gcs = new GCSBuilder(project).setKey(key).setKeySource(keySource).build(); Bucket bucket = gcs.get(bucketName); if (bucket != null) { - logger.info("Deleting objects in the Bucket {}", bucketName); - for (Blob blob : bucket.list().iterateAll()) { + logger.info("Deleting objects in the {} folder for bucket {}", + humanReadableFolderName(folderName), bucketName); + for (Blob blob : listBlobs(bucket, folderName)) { gcs.delete(blob.getBlobId()); } bucket.delete(); @@ -53,4 +57,17 @@ public static void clearBucket(String key, String project, String bucketName, St logger.info("Bucket {} does not exist", bucketName); } } + + private static String humanReadableFolderName(String folderName) { + return folderName == null || folderName.isEmpty() + ? "root" + : "'" + folderName + "'"; + } + + private static Iterable listBlobs(Bucket bucket, String folderName) { + Page blobListing = folderName == null || folderName.isEmpty() + ? bucket.list() + : bucket.list(Storage.BlobListOption.prefix(folderName)); + return blobListing.iterateAll(); + } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/TableClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/TableClearer.java index d23e81b96..ea708bb19 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/TableClearer.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/TableClearer.java @@ -25,6 +25,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Arrays; import java.util.Collection; import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.table; @@ -32,7 +33,6 @@ public class TableClearer { private static final Logger logger = LoggerFactory.getLogger(TableClearer.class); - // TODO: Might want to add support for table sanitization here /** * Clear out one or more BigQuery tables. Useful in integration testing to provide a clean slate * before creating a connector and writing to those tables. @@ -50,4 +50,15 @@ public static void clearTables(BigQuery bigQuery, String dataset, Collection4.12 3.2.4 + 1.4 2.15 6.18 3.8.1 @@ -329,9 +330,7 @@ under the License. maven-surefire-plugin ${surefire.plugin.version} - - **/*IT.java - + org.apache.kafka.test.IntegrationTest ${skip.unit.tests} @@ -346,9 +345,7 @@ under the License. integration-test - - **/*IT.java - + org.apache.kafka.test.IntegrationTest @@ -419,6 +416,22 @@ under the License. jenkins + + + org.codehaus.mojo + buildnumber-maven-plugin + ${buildnumber.plugin.version} + + + generate-test-suffix + + create + + pre-integration-test + + + + @@ -436,6 +449,8 @@ under the License. connect-205118 jenkinsKcbqIntegrationTesting jenkins-kcbq-integration-testing + -${scmBranch}-${buildNumber}-${timestamp} + ${scmBranch}-${buildNumber}-${timestamp} From 63ecc65784e360746b8af5380ca0f3ebb6f1aea7 Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Fri, 16 Oct 2020 10:08:16 -0700 Subject: [PATCH 039/190] CC-12157: Throw nicer exception on detecting cycle in schema (#47) * CC-12157: Throw nicer exception on detecting cycle in schema * review comments --- .../convert/BigQuerySchemaConverter.java | 33 ++++++++++++++ .../convert/BigQuerySchemaConverterTest.java | 44 +++++++++++++++++++ pom.xml | 2 +- 3 files changed, 78 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java index 2924c7fe0..4354d2692 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java @@ -29,7 +29,9 @@ import com.wepay.kafka.connect.bigquery.exception.ConversionConnectException; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schema.Type; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -103,6 +105,8 @@ public com.google.cloud.bigquery.Schema convertSchema(Schema kafkaConnectSchema) ConversionConnectException("Top-level Kafka Connect schema must be of type 'struct'"); } + throwOnCycle(kafkaConnectSchema, new ArrayList<>()); + List fields = kafkaConnectSchema.fields().stream() .flatMap(kafkaConnectField -> convertField(kafkaConnectField.schema(), kafkaConnectField.name()) @@ -115,6 +119,35 @@ public com.google.cloud.bigquery.Schema convertSchema(Schema kafkaConnectSchema) return com.google.cloud.bigquery.Schema.of(fields); } + private void throwOnCycle(Schema kafkaConnectSchema, List seenSoFar) { + if (PRIMITIVE_TYPE_MAP.containsKey(kafkaConnectSchema.type())) { + return; + } + + if (seenSoFar.contains(kafkaConnectSchema)) { + throw new ConversionConnectException("Kafka Connect schema contains cycle"); + } + + seenSoFar.add(kafkaConnectSchema); + switch(kafkaConnectSchema.type()) { + case ARRAY: + throwOnCycle(kafkaConnectSchema.valueSchema(), seenSoFar); + break; + case MAP: + throwOnCycle(kafkaConnectSchema.keySchema(), seenSoFar); + throwOnCycle(kafkaConnectSchema.valueSchema(), seenSoFar); + break; + case STRUCT: + kafkaConnectSchema.fields().forEach(f -> throwOnCycle(f.schema(), seenSoFar)); + break; + default: + throw new ConversionConnectException( + "Unrecognized schema type: " + kafkaConnectSchema.type() + ); + } + seenSoFar.remove(seenSoFar.size() - 1); + } + private Optional convertField(Schema kafkaConnectSchema, String fieldName) { Optional result; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java index c3f888956..9185fb724 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java @@ -20,10 +20,12 @@ package com.wepay.kafka.connect.bigquery.convert; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.LegacySQLTypeName; +import com.google.common.collect.ImmutableList; import com.wepay.kafka.connect.bigquery.exception.ConversionConnectException; import org.apache.kafka.connect.data.Date; @@ -34,6 +36,8 @@ import org.junit.Test; +import io.confluent.connect.avro.AvroData; + public class BigQuerySchemaConverterTest { @Test(expected = ConversionConnectException.class) @@ -628,6 +632,46 @@ public void testAllFieldsNullable() { com.google.cloud.bigquery.Schema bigQueryTestSchema = new BigQuerySchemaConverter(true).convertSchema(kafkaConnectTestSchema); assertEquals(bigQueryExpectedSchema, bigQueryTestSchema); + } + + @Test + public void testSimpleRecursiveSchemaThrows() { + final String fieldName = "RecursiveField"; + + // Construct Avro schema with recursion since we cannot directly construct Connect schema with cycle + org.apache.avro.Schema recursiveAvroSchema = org.apache.avro.SchemaBuilder + .record("RecursiveItem") + .namespace("com.example") + .fields() + .name(fieldName) + .type().unionOf().nullType().and().type("RecursiveItem").endUnion() + .nullDefault() + .endRecord(); + + Schema connectSchema = new AvroData(100).toConnectSchema(recursiveAvroSchema); + ConversionConnectException e = assertThrows(ConversionConnectException.class, () -> + new BigQuerySchemaConverter(true).convertSchema(connectSchema)); + assertEquals("Kafka Connect schema contains cycle", e.getMessage()); + } + @Test + public void testComplexRecursiveSchemaThrows() { + final String fieldName = "RecursiveField"; + + // Construct Avro schema with recursion since we cannot directly construct Connect schema with cycle + org.apache.avro.Schema recursiveAvroSchema = org.apache.avro.SchemaBuilder + .record("RecursiveItem") + .namespace("com.example") + .fields() + .name(fieldName) + .type() + .array().items() + .map().values().type("RecursiveItem").noDefault() + .endRecord(); + + Schema connectSchema = new AvroData(100).toConnectSchema(recursiveAvroSchema); + ConversionConnectException e = assertThrows(ConversionConnectException.class, () -> + new BigQuerySchemaConverter(true).convertSchema(connectSchema)); + assertEquals("Kafka Connect schema contains cycle", e.getMessage()); } } diff --git a/pom.xml b/pom.xml index 3b0903092..ea4f4ba23 100644 --- a/pom.xml +++ b/pom.xml @@ -47,7 +47,7 @@ 2.5.0 1.7.26 - 4.12 + 4.13 3.2.4 2.15 From 058aad2dbe939cf7f749ab0a5d0d4e37809bf470 Mon Sep 17 00:00:00 2001 From: Nikolay Date: Fri, 16 Oct 2020 10:11:26 -0700 Subject: [PATCH 040/190] Update README.md with some information on migrating configs to 2.x.x (#46) --- README.md | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 1c1e84a1d..f7c57ce79 100644 --- a/README.md +++ b/README.md @@ -48,7 +48,18 @@ save the properties file. Once you get more familiar with the connector, you might want to revisit the `connector.properties` file and experiment with tweaking its settings. - + +#### Migrating to 2.x.x +In accordance with the introduction of schema unionization in version 2.0.0, the following changes +to configs have been introduced and should be made when migrating: +1. `autoUpdateSchemas` has been removed +2. `allowNewBigQueryFields` and `allowBigQueryRequiredFieldRelaxation` have been introduced +3. `allowSchemaUnionization` has been introduced + +Setting `allowNewBigQueryFields` and `allowBigQueryRequiredFieldRelaxation` to `true` while +setting `allowSchemaUnionization` to false results in the same behavior that setting `autoUpdateSchemas` +to `true` used to. + ### Building and Extracting a Confluent Hub archive If you haven't already, move into the repository's top-level directory: From f24c75d5c42af94ef05b0fbb38c44277e8263bd1 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Sat, 17 Oct 2020 09:22:48 -0400 Subject: [PATCH 041/190] MINOR: Catch 'request payload exceeds size' errors from BigQuery and reduce batch size in response (#49) --- .../kafka/connect/bigquery/write/batch/TableWriter.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index 2e4839c1f..6584af53a 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -27,7 +27,6 @@ import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +43,7 @@ public class TableWriter implements Runnable { private static final int BAD_REQUEST_CODE = 400; private static final String INVALID_REASON = "invalid"; + private static final String PAYLOAD_TOO_LARGE_REASON = "Request payload size exceeds the limit:"; private final BigQueryWriter writer; private final PartitionedTableId table; @@ -138,6 +138,10 @@ private static boolean isBatchSizeError(BigQueryException exception) { * todo distinguish this from other invalids (like invalid table schema). */ return true; + } else if (exception.getCode() == BAD_REQUEST_CODE + && exception.getMessage() != null + && exception.getMessage().contains(PAYLOAD_TOO_LARGE_REASON)) { + return true; } return false; } From 202e3fa78b7049eeae820b7ae82e4efcbd911931 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Sun, 18 Oct 2020 19:41:00 -0400 Subject: [PATCH 042/190] GH-50: Remove maybeEnsureExistingTable method (#51) --- .../kafka/connect/bigquery/BigQuerySinkTask.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 4450d1b6e..d9ebb4d59 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -155,15 +155,6 @@ public void flush(Map offsets) { topicPartitionManager.resumeAll(); } - private void maybeEnsureExistingTable(TableId table) { - BigQuery bigQuery = getBigQuery(); - if (bigQuery.getTable(table) == null && !config.getBoolean(config.TABLE_CREATE_CONFIG)) { - throw new BigQueryConnectException("Table '" + table + "' does not exist. " + - "You may want to enable auto table creation by setting " + config.TABLE_CREATE_CONFIG - + "=true in the properties file"); - } - } - @Override public Map preCommit(Map offsets) { if (upsertDelete) { @@ -199,8 +190,6 @@ private PartitionedTableId getRecordTable(SinkRecord record) { TableId intermediateTableId = mergeBatches.intermediateTableFor(baseTableId); // If upsert/delete is enabled, we want to stream into a non-partitioned intermediate table return new PartitionedTableId.Builder(intermediateTableId).build(); - } else { - maybeEnsureExistingTable(baseTableId); } PartitionedTableId.Builder builder = new PartitionedTableId.Builder(baseTableId); From b56fb36b34cf425c78c7c4ee59f96ba9553c9b5e Mon Sep 17 00:00:00 2001 From: atrbgithub <14765982+atrbgithub@users.noreply.github.com> Date: Mon, 23 Nov 2020 18:25:55 +0000 Subject: [PATCH 043/190] 1.6.x backports (#58) * Backport of FMC-463 * Backport of CC-9208 --- .../connect/bigquery/BigQuerySinkTask.java | 5 ++- .../exception/BigQueryConnectException.java | 6 +++ .../write/batch/KCBQThreadPoolExecutor.java | 40 +++++++++++++++++- .../bigquery/write/batch/TableWriter.java | 10 +++-- .../bigquery/BigQuerySinkTaskTest.java | 42 ++++++++++++++++--- .../write/row/BigQueryWriterTest.java | 9 ++-- 6 files changed, 97 insertions(+), 15 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index feeee65fe..118007756 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -186,7 +186,10 @@ private String getRowId(SinkRecord record) { @Override public void put(Collection records) { - logger.info("Putting {} records in the sink.", records.size()); + // Periodically poll for errors here instead of doing a stop-the-world check in flush() + executor.maybeThrowEncounteredErrors(); + + logger.debug("Putting {} records in the sink.", records.size()); // create tableWriters Map tableWriterBuilders = new HashMap<>(); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java index a49c77726..3da4a39ac 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java @@ -62,4 +62,10 @@ private static String formatInsertAllErrors(Map> error } return messageBuilder.toString(); } + + @Override + public String toString() { + return getCause() != null ? + super.toString() + "\nCaused by: " + getCause().getLocalizedMessage() : super.toString(); + } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java index 2e2c28fa4..003dca988 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java @@ -45,7 +45,7 @@ public class KCBQThreadPoolExecutor extends ThreadPoolExecutor { private static final Logger logger = LoggerFactory.getLogger(KCBQThreadPoolExecutor.class); - private ConcurrentHashMap.KeySetView encounteredErrors = + private final ConcurrentHashMap.KeySetView encounteredErrors = ConcurrentHashMap.newKeySet(); /** @@ -92,14 +92,28 @@ public void awaitCurrentTasks() throws InterruptedException, BigQueryConnectExce execute(new CountDownRunnable(countDownLatch)); } countDownLatch.await(); + maybeThrowEncounteredErrors(); if (encounteredErrors.size() > 0) { String errorString = createErrorString(encounteredErrors); - encounteredErrors.clear(); throw new BigQueryConnectException("Some write threads encountered unrecoverable errors: " + errorString + "; See logs for more detail"); } } + /** + * Immediately throw an exception if any unrecoverable errors were encountered by any of the write + * tasks. + * + * @throws BigQueryConnectException if any of the tasks failed. + */ + public void maybeThrowEncounteredErrors() { + if (encounteredErrors.size() > 0) { + String errorString = createErrorString(encounteredErrors); + throw new BigQueryConnectException("Some write threads encountered unrecoverable errors: " + + errorString + "; See logs for more detail"); + } + } + private static String createErrorString(Collection errors) { List exceptionTypeStrings = new ArrayList<>(errors.size()); exceptionTypeStrings.addAll(errors.stream() @@ -107,4 +121,26 @@ private static String createErrorString(Collection errors) { .collect(Collectors.toList())); return String.join(", ", exceptionTypeStrings); } + + private static String createDetailedErrorString(Collection errors) { + List exceptionTypeStrings = new ArrayList<>(errors.size()); + exceptionTypeStrings.addAll(errors.stream() + .map(throwable -> + throwable.getClass().getName() + "\nMessage: " + throwable.getLocalizedMessage()) + .collect(Collectors.toList())); + return String.join(", ", exceptionTypeStrings); + } + + /** + * Checks for BigQuery errors. No-op if there isn't any error. + * + * @throws BigQueryConnectException if there have been any unrecoverable errors when writing to BigQuery. + */ + public void maybeFail() throws BigQueryConnectException { + if (encounteredErrors.size() > 0) { + throw new BigQueryConnectException("Encountered unrecoverable errors: " + + createDetailedErrorString(encounteredErrors) + "; See logs for more detail"); + } + } + } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index 6584af53a..3ff20037b 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -23,6 +23,7 @@ import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; import com.wepay.kafka.connect.bigquery.convert.RecordConverter; +import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; @@ -85,7 +86,10 @@ public void run() { logger.warn("Could not write batch of size {} to BigQuery.", currentBatch.size(), err); if (isBatchSizeError(err)) { failureCount++; - currentBatchSize = getNewBatchSize(currentBatchSize); + currentBatchSize = getNewBatchSize(currentBatchSize, err); + } else { + // Throw exception on write errors such as 403. + throw new BigQueryConnectException("Failed to write to table", err); } } } @@ -105,10 +109,10 @@ public void run() { } - private static int getNewBatchSize(int currentBatchSize) { + private static int getNewBatchSize(int currentBatchSize, Throwable err) { if (currentBatchSize == 1) { // todo correct exception type? - throw new ConnectException("Attempted to reduce batch size below 1."); + throw new BigQueryConnectException("Attempted to reduce batch size below 1.", err); } // round batch size up so we don't end up with a dangling 1 row at the end. return (int) Math.ceil(currentBatchSize / 2.0); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index cbd212e09..fd16921f4 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -299,11 +299,43 @@ public void testPutWhenPartitioningOnMessageTimeWhenNoTimestampType() { TimestampType.NO_TIMESTAMP_TYPE, null))); } - // It's important that the buffer be completely wiped after a call to flush, since any execption - // thrown during flush causes Kafka Connect to not commit the offsets for any records sent to the - // task since the last flush - @Test - public void testBufferClearOnFlushError() { + @Test(expected = BigQueryConnectException.class, timeout = 60000L) + public void testSimplePutException() throws InterruptedException { + final String topic = "test-topic"; + Map properties = propertiesFactory.getProperties(); + properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); + properties.put(BigQuerySinkConfig.DATASETS_CONFIG, ".*=scratch"); + + BigQuery bigQuery = mock(BigQuery.class); + Storage storage = mock(Storage.class); + + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + InsertAllResponse insertAllResponse = mock(InsertAllResponse.class); + when(bigQuery.insertAll(any())).thenReturn(insertAllResponse); + when(insertAllResponse.hasErrors()).thenReturn(true); + when(insertAllResponse.getInsertErrors()).thenReturn(Collections.singletonMap( + 0L, Collections.singletonList(new BigQueryError("no such field", "us-central1", "")))); + + SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); + SchemaManager schemaManager = mock(SchemaManager.class); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + testTask.initialize(sinkTaskContext); + testTask.start(properties); + + testTask.put(Collections.singletonList(spoofSinkRecord(topic))); + while (true) { + Thread.sleep(100); + testTask.put(Collections.emptyList()); + } + } + + + // Since any exception thrown during flush causes Kafka Connect to not commit the offsets for any + // records sent to the task since the last flush. The task should fail, and next flush should + // also throw an error. + @Test(expected = BigQueryConnectException.class) + public void testFlushException() { final String dataset = "scratch"; final String topic = "test_topic"; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index aed44f3f3..4ecbb6961 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -20,6 +20,7 @@ package com.wepay.kafka.connect.bigquery.write.row; import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Matchers.anyObject; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -31,6 +32,7 @@ import com.google.cloud.bigquery.BigQueryException; import com.google.cloud.bigquery.InsertAllRequest; import com.google.cloud.bigquery.InsertAllResponse; +import com.google.cloud.bigquery.Table; import com.google.cloud.storage.Storage; import com.wepay.kafka.connect.bigquery.BigQuerySinkTask; @@ -137,13 +139,15 @@ public void testAutoCreateTables() { verify(bigQuery, times(2)).insertAll(anyObject()); } - @Test + @Test(expected = BigQueryConnectException.class) public void testNonAutoCreateTables() { final String topic = "test_topic"; final String dataset = "scratch"; final Map properties = makeProperties("3", "2000", topic, dataset); BigQuery bigQuery = mock(BigQuery.class); + Table mockTable = mock(Table.class); + when(bigQuery.getTable(any())).thenReturn(mockTable); Map> emptyMap = mock(Map.class); when(emptyMap.isEmpty()).thenReturn(true); @@ -166,9 +170,6 @@ public void testNonAutoCreateTables() { testTask.put( Collections.singletonList(spoofSinkRecord(topic, 0, 0, "some_field", "some_value"))); testTask.flush(Collections.emptyMap()); - - verify(schemaManager, times(0)).createTable(anyObject(), anyObject()); - verify(bigQuery, times(2)).insertAll(anyObject()); } @Test From 506233d7ec41bcc06ed43e919b64f5ba99a6c563 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 23 Nov 2020 18:43:11 +0000 Subject: [PATCH 044/190] [maven-release-plugin] prepare release v1.6.6 --- kcbq-api/pom.xml | 2 +- kcbq-confluent/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 7dce66e53..bdc430be8 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.6-SNAPSHOT + 1.6.6 .. diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index d7ca9031b..075f17936 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.6-SNAPSHOT + 1.6.6 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 92b7e29cd..cf7b6e381 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.6-SNAPSHOT + 1.6.6 .. diff --git a/pom.xml b/pom.xml index ea4f4ba23..ce2bf6cbb 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 1.6.6-SNAPSHOT + 1.6.6 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v1.6.6 From 65eae703f13f0c72d2db514ff82a3bcf16365817 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 23 Nov 2020 18:43:18 +0000 Subject: [PATCH 045/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-confluent/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index bdc430be8..12817236b 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.6 + 1.6.7-SNAPSHOT .. diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index 075f17936..c0f9f24e9 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.6 + 1.6.7-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index cf7b6e381..925405b0b 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.6 + 1.6.7-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index ce2bf6cbb..e01f9954d 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 1.6.6 + 1.6.7-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v1.6.6 + HEAD From dc87644294c15596f028aafcdd309d3485d06164 Mon Sep 17 00:00:00 2001 From: tikimims <39631000+tikimims@users.noreply.github.com> Date: Fri, 8 Jan 2021 18:10:09 -0500 Subject: [PATCH 046/190] Update docs URL path (#66) --- kcbq-connector/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index b3e5c147a..c55786975 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -135,14 +135,14 @@ A sink connector for writing to Google BigQuery, with support for automatic table creation and schema evolution. logos/BigQuery.png - https://docs.confluent.io/current/connect/kafka-connect-bigquery/ + https://docs.confluent.io/kafka-connect-bigquery/current/index.html https://github.com/confluentinc/kafka-connect-bigquery Confluent, Inc. supports WePay's BigQuery connector version 1.1.2 and later, as part of a Confluent Platform subscription. ]]> - https://docs.confluent.io/current/connect/kafka-connect-bigquery/ + https://docs.confluent.io/kafka-connect-bigquery/current/index.html logos/confluent.png wepay From d7246545a57995fc227b168346921c024a529f97 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 19 Jan 2021 13:57:08 -0500 Subject: [PATCH 047/190] GH-68: Continue to check for errors in flush even after stopped (#70) * GH-68: Continue to check for errors in flush even after stopped * GH-68: Add unit test --- .../connect/bigquery/BigQuerySinkTask.java | 3 ++ .../bigquery/BigQuerySinkTaskTest.java | 35 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index d9ebb4d59..3f7cd4b24 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -143,6 +143,9 @@ public void flush(Map offsets) { // Return immediately here since the executor will already be shutdown if (stopped) { + // Still have to check for errors in order to prevent offsets being committed for records that + // we've failed to write + executor.maybeThrowEncounteredErrors(); return; } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 21d9e18fb..f91a755fd 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Matchers.any; @@ -465,6 +466,40 @@ public void testEmptyFlush() { testTask.flush(Collections.emptyMap()); } + @Test + public void testFlushAfterStop() { + Map properties = propertiesFactory.getProperties(); + Storage storage = mock(Storage.class); + + BigQuery bigQuery = mock(BigQuery.class); + when(bigQuery.insertAll(any())) + .thenThrow( + new BigQueryException(400, "Oops", new BigQueryError("invalid", "global", "oops"))); + + SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); + SchemaManager schemaManager = mock(SchemaManager.class); + + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + testTask.initialize(sinkTaskContext); + testTask.start(properties); + + testTask.put(Collections.singletonList(spoofSinkRecord("t"))); + assertThrows( + "first call to flush should fail", + Exception.class, + () -> testTask.flush(Collections.emptyMap())); + assertThrows( + "second call to flush should fail", + Exception.class, + () -> testTask.flush(Collections.emptyMap())); + testTask.stop(); + assertThrows( + "third call to flush (after task stop) should fail", + Exception.class, + () -> testTask.flush(Collections.emptyMap())); + } + @Test public void testBigQuery5XXRetry() { final String topic = "test_topic"; From a0e4cdc9e546510e3f357be53cd03150d2f9a03c Mon Sep 17 00:00:00 2001 From: Mubeen Ahmad Date: Mon, 1 Feb 2021 16:39:20 +0200 Subject: [PATCH 048/190] Time partitioning (#61) * Added time partiitioning support for MONTH and YEAR * Corrected the auth library version * Added caching to BigQuery calls for Table information * Added supportt for BigQuery HOURLY partition type * Fixed up ttests * Fixing integration tests * Removed caffiene and added a Map implementation, synchronized the method to retrieve cache * Fixed up imports * Removed the lookup when creating and updating the table structures * Removed redunadant code, refactored and reformatted code * resolved merge errors from master * removed unused imports * Reverted redundant changes * Timepartitioning not set against a schema when an update operation occurs * Fixed up formatiting * Added partition ttests for update scenario * Formatting changes * Formatting changes * Formatting changes * Rebuild commit * Formatting changes * Added back the missing line Co-authored-by: Mubeen Ahmad --- .../connect/bigquery/BigQuerySinkTask.java | 70 ++++++++- .../kafka/connect/bigquery/SchemaManager.java | 16 +- .../bigquery/utils/PartitionedTableId.java | 52 +++++++ .../bigquery/BigQuerySinkTaskTest.java | 55 ++++--- .../connect/bigquery/SchemaManagerTest.java | 109 +++++++++++++- .../utils/PartitionedTableIdTest.java | 138 ++++++++++++++++++ .../write/row/BigQueryWriterTest.java | 22 ++- .../bigquery/write/row/GCSToBQWriterTest.java | 10 +- pom.xml | 8 +- 9 files changed, 432 insertions(+), 48 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 3f7cd4b24..c8b646905 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -21,7 +21,11 @@ import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; +import com.google.cloud.bigquery.StandardTableDefinition; import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TimePartitioning; +import com.google.cloud.bigquery.TimePartitioning.Type; import com.google.cloud.storage.Bucket; import com.google.cloud.storage.BucketInfo; import com.google.cloud.storage.Storage; @@ -31,7 +35,6 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import com.wepay.kafka.connect.bigquery.utils.SinkRecordConverter; -import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; @@ -108,6 +111,8 @@ public class BigQuerySinkTask extends SinkTask { private final UUID uuid = UUID.randomUUID(); private ScheduledExecutorService loadExecutor; + private Map cache; + /** * Create a new BigquerySinkTask. */ @@ -127,11 +132,13 @@ public BigQuerySinkTask() { * @param testSchemaManager {@link SchemaManager} to use for testing (likely a mock) * @see BigQuerySinkTask#BigQuerySinkTask() */ - public BigQuerySinkTask(BigQuery testBigQuery, SchemaRetriever schemaRetriever, Storage testGcs, SchemaManager testSchemaManager) { + public BigQuerySinkTask(BigQuery testBigQuery, SchemaRetriever schemaRetriever, Storage testGcs, + SchemaManager testSchemaManager, Map testCache) { this.testBigQuery = testBigQuery; this.schemaRetriever = schemaRetriever; this.testGcs = testGcs; this.testSchemaManager = testSchemaManager; + this.cache = testCache; } @Override @@ -197,14 +204,23 @@ private PartitionedTableId getRecordTable(SinkRecord record) { PartitionedTableId.Builder builder = new PartitionedTableId.Builder(baseTableId); if (usePartitionDecorator) { + Table bigQueryTable = retrieveCachedTable(baseTableId); + TimePartitioning timePartitioning = TimePartitioning.of(Type.DAY); + if (bigQueryTable != null) { + StandardTableDefinition standardTableDefinition = bigQueryTable.getDefinition(); + if (standardTableDefinition != null && standardTableDefinition.getTimePartitioning() != null) { + timePartitioning = standardTableDefinition.getTimePartitioning(); + } + } + if (useMessageTimeDatePartitioning) { if (record.timestampType() == TimestampType.NO_TIMESTAMP_TYPE) { throw new ConnectException( "Message has no timestamp type, cannot use message timestamp to partition."); } - builder.setDayPartition(record.timestamp()); + setTimePartitioningForTimestamp(builder, timePartitioning, record.timestamp()); } else { - builder.setDayPartitionForNow(); + setTimePartitioning(builder, timePartitioning); } } @@ -291,6 +307,43 @@ private BigQuery getBigQuery() { return bigQuery.updateAndGet(bq -> bq != null ? bq : newBigQuery()); } + private void setTimePartitioningForTimestamp(PartitionedTableId.Builder builder, TimePartitioning timePartitioning, + Long timestamp) { + switch (timePartitioning.getType()) { + case HOUR: + builder.setHourPartition(timestamp); + break; + case MONTH: + builder.setMonthPartition(timestamp); + break; + case YEAR: + builder.setYearPartition(timestamp); + break; + default: + builder.setDayPartition(timestamp); + } + } + + private void setTimePartitioning(PartitionedTableId.Builder builder, TimePartitioning timePartitioning) { + switch (timePartitioning.getType()) { + case HOUR: + builder.setHourPartitionNow(); + break; + case MONTH: + builder.setMonthPartitionForNow(); + break; + case YEAR: + builder.setYearPartitionForNow(); + break; + default: + builder.setDayPartitionForNow(); + } + } + + private Table retrieveCachedTable(TableId tableId) { + return getCache().computeIfAbsent(tableId, k -> getBigQuery().getTable(tableId)); + } + private BigQuery newBigQuery() { String projectName = config.getString(config.PROJECT_CONFIG); String keyFile = config.getKeyFile(); @@ -378,6 +431,14 @@ private SinkRecordConverter getConverter(BigQuerySinkTaskConfig config) { return new SinkRecordConverter(config, mergeBatches, mergeQueries); } + private synchronized Map getCache() { + if (cache == null) { + cache = new HashMap<>(); + } + + return cache; + } + @Override public void start(Map properties) { logger.trace("task.start()"); @@ -409,6 +470,7 @@ public void start(Map properties) { mergeBatches = new MergeBatches(intermediateTableSuffix); } + cache = getCache(); bigQueryWriter = getBigQueryWriter(); gcsToBQWriter = getGcsWriter(); executor = new KCBQThreadPoolExecutor(config, new LinkedBlockingQueue<>()); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 0bd354e1d..6a3538580 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -213,7 +213,7 @@ public boolean createTable(TableId table, List records) { logger.debug("Skipping create of {} as it should already exist or appear very soon", table(table)); return false; } - TableInfo tableInfo = getTableInfo(table, records); + TableInfo tableInfo = getTableInfo(table, records, true); logger.info("Attempting to create {} with schema {}", table(table), tableInfo.getDefinition().getSchema()); try { @@ -239,7 +239,7 @@ public boolean createTable(TableId table, List records) { */ public void updateSchema(TableId table, List records) { synchronized (lock(tableUpdateLocks, table)) { - TableInfo tableInfo = getTableInfo(table, records); + TableInfo tableInfo = getTableInfo(table, records, false); if (!schemaCache.containsKey(table)) { schemaCache.put(table, readTableSchema(table)); } @@ -260,9 +260,10 @@ public void updateSchema(TableId table, List records) { * Returns the {@link TableInfo} instance of a bigQuery Table * @param table The BigQuery table to return the table info * @param records The sink records used to determine the schema for constructing the table info + * @param createSchema Flag to determine if we are creating a new table schema or updating an existing table schema * @return The resulting BigQuery table information */ - private TableInfo getTableInfo(TableId table, List records) { + private TableInfo getTableInfo(TableId table, List records, Boolean createSchema) { com.google.cloud.bigquery.Schema proposedSchema; String tableDescription; try { @@ -271,7 +272,7 @@ private TableInfo getTableInfo(TableId table, List records) { } catch (BigQueryConnectException exception) { throw new BigQueryConnectException("Failed to unionize schemas of records for the table " + table, exception); } - return constructTableInfo(table, proposedSchema, tableDescription); + return constructTableInfo(table, proposedSchema, tableDescription, createSchema); } @VisibleForTesting @@ -439,7 +440,8 @@ private Map schemaFields(com.google.cloud.bigquery.Schema schema) } // package private for testing. - TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema bigQuerySchema, String tableDescription) { + TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema bigQuerySchema, String tableDescription, + Boolean createSchema) { StandardTableDefinition.Builder builder = StandardTableDefinition.newBuilder() .setSchema(bigQuerySchema); @@ -447,12 +449,12 @@ TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema big // Shameful hack: make the table ingestion time-partitioned here so that the _PARTITIONTIME // pseudocolumn can be queried to filter out rows that are still in the streaming buffer builder.setTimePartitioning(TimePartitioning.of(Type.DAY)); - } else { + } else if (createSchema) { TimePartitioning timePartitioning = TimePartitioning.of(Type.DAY); if (timestampPartitionFieldName.isPresent()) { timePartitioning = timePartitioning.toBuilder().setField(timestampPartitionFieldName.get()).build(); } - + builder.setTimePartitioning(timePartitioning); if (timestampPartitionFieldName.isPresent() && clusteringFieldName.isPresent()) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java index 28ae9b602..e63257691 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java @@ -22,7 +22,10 @@ import com.google.cloud.bigquery.TableId; import java.time.Clock; +import java.time.Instant; import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; import java.time.format.DateTimeFormatter; /** @@ -200,6 +203,19 @@ public Builder setPartition(String partition) { return this; } + public Builder setHourPartition(long utcTime) { + Instant instant = Instant.ofEpochMilli(utcTime); + return setHourPartition(LocalDateTime.ofInstant(instant, ZoneId.of("UTC"))); + } + + public Builder setHourPartition(LocalDateTime localDate) { + return setPartition(dateToHourPartition(localDate)); + } + + public Builder setHourPartitionNow() { + return setHourPartition(LocalDateTime.now(UTC_CLOCK)); + } + public Builder setDayPartition(long utcTime) { return setDayPartition(LocalDate.ofEpochDay(utcTime / MILLIS_IN_DAY)); } @@ -212,6 +228,34 @@ public Builder setDayPartitionForNow() { return setDayPartition(LocalDate.now(UTC_CLOCK)); } + public Builder setMonthPartition(long utcTime) { + return setMonthPartition(LocalDate.ofEpochDay(utcTime / MILLIS_IN_DAY)); + } + + public Builder setMonthPartition(LocalDate localDate) { + return setPartition(dateToMonthPartition(localDate)); + } + + public Builder setMonthPartitionForNow() { + return setMonthPartition(LocalDate.now(UTC_CLOCK)); + } + + public Builder setYearPartition(long utcTime) { + return setYearPartition(LocalDate.ofEpochDay(utcTime / MILLIS_IN_DAY)); + } + + public Builder setYearPartition(LocalDate localDate) { + return setPartition(dateToYearPartition(localDate)); + } + + public Builder setYearPartitionForNow() { + return setYearPartition(LocalDate.now(UTC_CLOCK)); + } + + private String dateToHourPartition(LocalDateTime localDate) { + return localDate.format(DateTimeFormatter.ofPattern("yyyyMMddHH")); + } + /** * @param localDate the localDate of the partition. * @return The String representation of the partition. @@ -220,6 +264,14 @@ private static String dateToDayPartition(LocalDate localDate) { return localDate.format(DateTimeFormatter.BASIC_ISO_DATE); } + private String dateToMonthPartition(LocalDate localDate) { + return localDate.format(DateTimeFormatter.ofPattern("yyyyMM")); + } + + private String dateToYearPartition(LocalDate localDate) { + return localDate.format(DateTimeFormatter.ofPattern("yyyy")); + } + /** * Build the {@link PartitionedTableId}. * diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index f91a755fd..dfa71ff84 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -42,7 +42,6 @@ import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.storage.Storage; -import com.wepay.kafka.connect.bigquery.api.KafkaSchemaRecordType; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; @@ -117,8 +116,9 @@ public void testSimplePut() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -148,8 +148,9 @@ public void testSimplePutWhenSchemaRetrieverIsNotNull() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -168,8 +169,9 @@ public void testEmptyPut() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.start(properties); testTask.put(Collections.emptyList()); @@ -190,8 +192,9 @@ public void testEmptyRecordPut() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.start(properties); SinkRecord emptyRecord = spoofSinkRecord(topic, simpleSchema, null); @@ -221,8 +224,9 @@ public void testPutWhenPartitioningOnMessageTime() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -258,8 +262,9 @@ public void testPutWhenPartitioningIsSetToTrue() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -294,8 +299,9 @@ public void testPutWhenPartitioningIsSetToFalse() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put(Collections.singletonList(spoofSinkRecord(topic, "value", "message text", @@ -330,8 +336,9 @@ public void testPutWhenPartitioningOnMessageTimeWhenNoTimestampType() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -363,6 +370,7 @@ public void testPutWithUpsertDelete() throws Exception { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); Field keyField = Field.of(key, LegacySQLTypeName.STRING); Field valueField = Field.of(value, LegacySQLTypeName.STRING); com.google.cloud.bigquery.Schema intermediateSchema = com.google.cloud.bigquery.Schema.of( @@ -393,7 +401,7 @@ public void testPutWithUpsertDelete() throws Exception { return null; }); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -431,9 +439,10 @@ public void testSimplePutException() throws InterruptedException { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); BigQuerySinkTask testTask = - new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -457,9 +466,10 @@ public void testEmptyFlush() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -478,9 +488,10 @@ public void testFlushAfterStop() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -529,8 +540,9 @@ public void testBigQuery5XXRetry() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put(Collections.singletonList(spoofSinkRecord(topic))); @@ -569,8 +581,9 @@ public void testBigQuery403Retry() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put(Collections.singletonList(spoofSinkRecord(topic))); @@ -606,8 +619,9 @@ public void testBigQueryRetryExceeded() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put(Collections.singletonList(spoofSinkRecord(topic))); @@ -636,9 +650,10 @@ public void testInterruptedException() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); @@ -660,9 +675,10 @@ public void testConfigException() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); BigQuerySinkTask testTask = - new BigQuerySinkTask(mock(BigQuery.class), schemaRetriever, mock(Storage.class), schemaManager); + new BigQuerySinkTask(mock(BigQuery.class), schemaRetriever, mock(Storage.class), schemaManager, cache); testTask.start(badProperties); } catch (ConfigException e) { throw new SinkConfigConnectException(e); @@ -697,9 +713,10 @@ public void testStop() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); Storage storage = mock(Storage.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put(Collections.singletonList(spoofSinkRecord(topic))); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index b65979a57..a2c2b62a1 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -85,7 +85,7 @@ public void testBQTableDescription() { when(mockKafkaSchema.doc()).thenReturn(testDoc); TableInfo tableInfo = schemaManager - .constructTableInfo(tableId, fakeBigQuerySchema, testDoc); + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); Assert.assertEquals("Kafka doc does not match BigQuery table description", testDoc, tableInfo.getDescription()); @@ -103,13 +103,64 @@ public void testTimestampPartitionSet() { when(mockKafkaSchema.doc()).thenReturn(testDoc); TableInfo tableInfo = schemaManager - .constructTableInfo(tableId, fakeBigQuerySchema, testDoc); + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); Assert.assertEquals("Kafka doc does not match BigQuery table description", testDoc, tableInfo.getDescription()); + StandardTableDefinition definition = tableInfo.getDefinition(); + Assert.assertNotNull(definition.getTimePartitioning()); Assert.assertEquals("The field name does not match the field name of time partition", testField.get(), - ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning().getField()); + definition.getTimePartitioning().getField()); + } + + @Test + public void testUpdateTimestampPartitionNull() { + Optional testField = Optional.of("testField"); + SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); + + when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + + TableInfo tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); + + Assert.assertEquals("Kafka doc does not match BigQuery table description", + testDoc, tableInfo.getDescription()); + Assert.assertNull("The time partitioning object should be null", + ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning()); + } + + @Test + public void testUpdateTimestampPartitionNotSet() { + Optional testField = Optional.of("testField"); + SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); + + when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + + TableInfo tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); + + Assert.assertEquals("Kafka doc does not match BigQuery table description", + testDoc, tableInfo.getDescription()); + StandardTableDefinition definition = tableInfo.getDefinition(); + Assert.assertNotNull(definition.getTimePartitioning()); + Assert.assertEquals("The field name does not match the field name of time partition", + testField.get(), + definition.getTimePartitioning().getField()); + + Optional updateField = Optional.of("testUpdateField"); + schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), updateField, Optional.empty()); + + tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); + definition = tableInfo.getDefinition(); + Assert.assertNull("The time partitioning object should be null", + ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning()); } @Test @@ -123,7 +174,7 @@ public void testClusteringPartitionSet() { when(mockKafkaSchema.doc()).thenReturn(testDoc); TableInfo tableInfo = schemaManager - .constructTableInfo(tableId, fakeBigQuerySchema, testDoc); + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); Assert.assertEquals("Kafka doc does not match BigQuery table description", testDoc, tableInfo.getDescription()); @@ -134,6 +185,56 @@ public void testClusteringPartitionSet() { definition.getClustering().getFields()); } + @Test + public void testUpdateClusteringPartitionNull() { + Optional timestampPartitionFieldName = Optional.of("testField"); + Optional> testField = Optional.of(Arrays.asList("column1", "column2")); + SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); + + when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + + TableInfo tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); + + Assert.assertEquals("Kafka doc does not match BigQuery table description", + testDoc, tableInfo.getDescription()); + StandardTableDefinition definition = tableInfo.getDefinition(); + Assert.assertNull("The clustering object should be null", definition.getClustering()); + } + + @Test + public void testUpdateClusteringPartitionNotSet() { + Optional timestampPartitionFieldName = Optional.of("testField"); + Optional> testField = Optional.of(Arrays.asList("column1", "column2")); + SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); + + when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + + TableInfo tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); + + Assert.assertEquals("Kafka doc does not match BigQuery table description", + testDoc, tableInfo.getDescription()); + StandardTableDefinition definition = tableInfo.getDefinition(); + Assert.assertNotNull(definition.getClustering()); + Assert.assertEquals("The field name should not match the field name of time partition", + testField.get(), + definition.getClustering().getFields()); + + Optional> updateTestField = Optional.of(Arrays.asList("column3", "column4")); + schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, updateTestField); + + tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); + definition = tableInfo.getDefinition(); + Assert.assertNull("The clustering object should be null", definition.getClustering()); + } + @Test public void testSuccessfulUpdateWithOnlyRelaxedFields() { com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java index d24072714..dfe8ffc07 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java @@ -26,6 +26,7 @@ import java.time.Instant; import java.time.LocalDate; +import java.time.LocalDateTime; public class PartitionedTableIdTest { @@ -63,6 +64,51 @@ public void testTableIdBuilder() { Assert.assertEquals(tableId, partitionedTableId.getFullTableId()); } + @Test + public void testWithPartitionForHour() { + final String dataset = "dataset"; + final String table = "table"; + final LocalDateTime partitionDate = LocalDateTime.of(2016, 9, 21, 13, 55); + + final PartitionedTableId partitionedTableId = + new PartitionedTableId.Builder(dataset, table).setHourPartition(partitionDate).build(); + + final String expectedPartition = "2016092113"; + + Assert.assertEquals(dataset, partitionedTableId.getDataset()); + Assert.assertEquals(table, partitionedTableId.getBaseTableName()); + Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); + + final TableId expectedBaseTableId = TableId.of(dataset, table); + final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); + + Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); + Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); + } + + @Test + public void testWithEpochTimePartitionForHour() { + final String dataset = "dataset"; + final String table = "table"; + + final long utcTime = 1509007584334L; + + final PartitionedTableId partitionedTableId = + new PartitionedTableId.Builder(dataset, table).setHourPartition(utcTime).build(); + + final String expectedPartition = "2017102608"; + + Assert.assertEquals(dataset, partitionedTableId.getDataset()); + Assert.assertEquals(table, partitionedTableId.getBaseTableName()); + Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); + + final TableId expectedBaseTableId = TableId.of(dataset, table); + final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); + + Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); + Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); + } + @Test public void testWithPartition() { final String dataset = "dataset"; @@ -107,4 +153,96 @@ public void testWithEpochTimePartition() { Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); } + + @Test + public void testWithPartitionForMonth() { + final String dataset = "dataset"; + final String table = "table"; + final LocalDate partitionDate = LocalDate.of(2016, 9, 21); + + final PartitionedTableId partitionedTableId = + new PartitionedTableId.Builder(dataset, table). + setMonthPartition(partitionDate).build(); + + final String expectedPartition = "201609"; + + Assert.assertEquals(dataset, partitionedTableId.getDataset()); + Assert.assertEquals(table, partitionedTableId.getBaseTableName()); + Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); + + final TableId expectedBaseTableId = TableId.of(dataset, table); + final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); + + Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); + Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); + } + + @Test + public void testWithEpochTimePartitionForMonth() { + final String dataset = "dataset"; + final String table = "table"; + + final long utcTime = 1509007584334L; + + final PartitionedTableId partitionedTableId = + new PartitionedTableId.Builder(dataset, table).setMonthPartition(utcTime).build(); + + final String expectedPartition = "201710"; + + Assert.assertEquals(dataset, partitionedTableId.getDataset()); + Assert.assertEquals(table, partitionedTableId.getBaseTableName()); + Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); + + final TableId expectedBaseTableId = TableId.of(dataset, table); + final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); + + Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); + Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); + } + + @Test + public void testWithPartitionForYear() { + final String dataset = "dataset"; + final String table = "table"; + final LocalDate partitionDate = LocalDate.of(2016, 9, 21); + + final PartitionedTableId partitionedTableId = + new PartitionedTableId.Builder(dataset, table). + setYearPartition(partitionDate).build(); + + final String expectedPartition = "2016"; + + Assert.assertEquals(dataset, partitionedTableId.getDataset()); + Assert.assertEquals(table, partitionedTableId.getBaseTableName()); + Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); + + final TableId expectedBaseTableId = TableId.of(dataset, table); + final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); + + Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); + Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); + } + + @Test + public void testWithEpochTimePartitionForYear() { + final String dataset = "dataset"; + final String table = "table"; + + final long utcTime = 1509007584334L; + + final PartitionedTableId partitionedTableId = + new PartitionedTableId.Builder(dataset, table).setYearPartition(utcTime).build(); + + final String expectedPartition = "2017"; + + Assert.assertEquals(dataset, partitionedTableId.getDataset()); + Assert.assertEquals(table, partitionedTableId.getBaseTableName()); + Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); + + final TableId expectedBaseTableId = TableId.of(dataset, table); + final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); + + Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); + Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); + } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index a6a90a423..5c20421d5 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -97,7 +97,9 @@ public void testBigQueryNoFailure() { SchemaManager schemaManager = mock(SchemaManager.class); Storage storage = mock(Storage.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + Map cache = mock(Map.class); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put( @@ -131,7 +133,9 @@ public void testAutoCreateTables() { Storage storage = mock(Storage.class); SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + Map cache = mock(Map.class); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put( @@ -167,7 +171,9 @@ public void testNonAutoCreateTables() { Storage storage = mock(Storage.class); SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + Map cache = mock(Map.class); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put( @@ -216,9 +222,10 @@ public void testBigQueryPartialFailure() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - + Map cache = mock(Map.class); Storage storage = mock(Storage.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put(sinkRecordList); @@ -270,9 +277,10 @@ public void testBigQueryCompleteFailure() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - + Map cache = mock(Map.class); Storage storage = mock(Storage.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put(sinkRecordList); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java index 16748c84e..e1aa29c02 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java @@ -21,7 +21,6 @@ import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.Table; -import com.google.cloud.storage.Blob; import com.google.cloud.storage.BlobInfo; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageException; @@ -73,8 +72,9 @@ public void testGCSNoFailure(){ SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put( @@ -98,12 +98,13 @@ public void testGCSSomeFailures(){ SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); when(storage.create((BlobInfo)anyObject(), (byte[])anyObject())) .thenThrow(new StorageException(500, "internal server error")) // throw first time .thenReturn(null); // return second time. (we don't care about the result.) - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put( @@ -127,11 +128,12 @@ public void testGCSAllFailures(){ SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = mock(Map.class); when(storage.create((BlobInfo)anyObject(), (byte[])anyObject())) .thenThrow(new StorageException(500, "internal server error")); - BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager); + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put( diff --git a/pom.xml b/pom.xml index d6c5fc908..1a398667e 100644 --- a/pom.xml +++ b/pom.xml @@ -37,12 +37,14 @@ 5.5.1 0.6.1 - 0.16.1 - 1.79.0 + 0.21.1 + 1.119.0 + 1.113.4 2.10.2 2.5.0 2.12 1.7.26 + 2.8.6 4.13 3.2.4 @@ -150,7 +152,7 @@ com.google.cloud google-cloud-storage - ${google.cloud.version} + ${google.cloud.storage.version} com.google.auth From 098130052868165d0360f14683c2b07267703999 Mon Sep 17 00:00:00 2001 From: SajanaW Date: Tue, 2 Feb 2021 14:58:54 -0800 Subject: [PATCH 049/190] Make Identity Schema Retriever the default SR (#81) Signed-off-by: SajanaW --- .../connect/bigquery/config/BigQuerySinkConfig.java | 3 ++- .../bigquery/config/BigQuerySinkTaskConfigTest.java | 10 ---------- 2 files changed, 2 insertions(+), 11 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 62cc51f0a..b181b8dd0 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -28,6 +28,7 @@ import com.wepay.kafka.connect.bigquery.convert.RecordConverter; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; +import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.config.ConfigDef; @@ -125,7 +126,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String SCHEMA_RETRIEVER_CONFIG = "schemaRetriever"; private static final ConfigDef.Type SCHEMA_RETRIEVER_TYPE = ConfigDef.Type.CLASS; - private static final Class SCHEMA_RETRIEVER_DEFAULT = null; + private static final Class SCHEMA_RETRIEVER_DEFAULT = IdentitySchemaRetriever.class; private static final ConfigDef.Importance SCHEMA_RETRIEVER_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String SCHEMA_RETRIEVER_DOC = diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java index 511b4f90f..e50dd476e 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java @@ -163,14 +163,4 @@ public void testClusteringFieldNames() { assertTrue(testClusteringPartitionFieldName.isPresent()); assertEquals(expectedClusteringPartitionFieldName, testClusteringPartitionFieldName.get()); } - - @Test(expected = ConfigException.class) - public void testSchemaUpdatesWithoutRetriever() { - Map badConfigProperties = propertiesFactory.getProperties(); - badConfigProperties.remove(BigQuerySinkTaskConfig.SCHEMA_RETRIEVER_CONFIG); - badConfigProperties.put(BigQuerySinkTaskConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG, "true"); - badConfigProperties.put(BigQuerySinkTaskConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG, "true"); - - new BigQuerySinkTaskConfig(badConfigProperties); - } } From 028630b03f058a6cda76b40f11b585394e58fcfe Mon Sep 17 00:00:00 2001 From: Saj Weerawardhena Date: Tue, 2 Feb 2021 23:55:53 +0000 Subject: [PATCH 050/190] [maven-release-plugin] prepare release v2.0.0 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 33c1153b5..b3fff0bc3 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.0-SNAPSHOT + 2.0.0 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index f35548ab6..9543c52ff 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.0-SNAPSHOT + 2.0.0 .. diff --git a/pom.xml b/pom.xml index 60c59d5c7..c36d621ee 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.0-SNAPSHOT + 2.0.0 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.0 From 52c5451683dacfcee054c7d6721f386bad9d05bb Mon Sep 17 00:00:00 2001 From: Saj Weerawardhena Date: Tue, 2 Feb 2021 23:56:00 +0000 Subject: [PATCH 051/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index b3fff0bc3..cd994dbfb 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.0 + 2.0.1-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 9543c52ff..1d64aea9e 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.0 + 2.0.1-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index c36d621ee..f1172d390 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.0 + 2.0.1-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.0 + HEAD From 8c0576fb47bca0c19a69cc79259488b4c26b1fdb Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 10 Feb 2021 15:27:14 -0500 Subject: [PATCH 052/190] GH-80: Add support for configuring time partitioning type on created tables (#84) * GH-80: Add support for configuring time partitioning type on created tables * GH-80: Address review comments --- .../connect/bigquery/BigQuerySinkTask.java | 3 +- .../kafka/connect/bigquery/SchemaManager.java | 11 +++- .../bigquery/config/BigQuerySinkConfig.java | 58 +++++++++++++++++++ .../connect/bigquery/SchemaManagerTest.java | 51 ++++++++++++---- .../config/BigQuerySinkConfigTest.java | 20 +++++++ 5 files changed, 130 insertions(+), 13 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index c8b646905..9c4d7943d 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -366,13 +366,14 @@ private SchemaManager newSchemaManager() { Optional kafkaDataFieldName = config.getKafkaDataFieldName(); Optional timestampPartitionFieldName = config.getTimestampPartitionFieldName(); Optional> clusteringFieldName = config.getClusteringPartitionFieldName(); + TimePartitioning.Type timePartitioningType = config.getTimePartitioningType(); boolean allowNewBQFields = config.getBoolean(config.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); boolean allowReqFieldRelaxation = config.getBoolean(config.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); boolean allowSchemaUnionization = config.getBoolean(config.ALLOW_SCHEMA_UNIONIZATION_CONFIG); return new SchemaManager(schemaRetriever, schemaConverter, getBigQuery(), allowNewBQFields, allowReqFieldRelaxation, allowSchemaUnionization, kafkaKeyFieldName, kafkaDataFieldName, - timestampPartitionFieldName, clusteringFieldName); + timestampPartitionFieldName, clusteringFieldName, timePartitioningType); } private BigQueryWriter getBigQueryWriter() { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 6a3538580..0e257f522 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -67,6 +67,7 @@ public class SchemaManager { private final Optional kafkaDataFieldName; private final Optional timestampPartitionFieldName; private final Optional> clusteringFieldName; + private final TimePartitioning.Type timePartitioningType; private final boolean intermediateTables; private final ConcurrentMap tableCreateLocks; private final ConcurrentMap tableUpdateLocks; @@ -89,6 +90,7 @@ public class SchemaManager { * If set to null, ingestion time-based partitioning will be * used instead. * @param clusteringFieldName + * @param timePartitioningType The time partitioning type (HOUR, DAY, etc.) to use for created tables. */ public SchemaManager( SchemaRetriever schemaRetriever, @@ -100,7 +102,8 @@ public SchemaManager( Optional kafkaKeyFieldName, Optional kafkaDataFieldName, Optional timestampPartitionFieldName, - Optional> clusteringFieldName) { + Optional> clusteringFieldName, + TimePartitioning.Type timePartitioningType) { this( schemaRetriever, schemaConverter, @@ -112,6 +115,7 @@ public SchemaManager( kafkaDataFieldName, timestampPartitionFieldName, clusteringFieldName, + timePartitioningType, false, new ConcurrentHashMap<>(), new ConcurrentHashMap<>(), @@ -129,6 +133,7 @@ private SchemaManager( Optional kafkaDataFieldName, Optional timestampPartitionFieldName, Optional> clusteringFieldName, + TimePartitioning.Type timePartitioningType, boolean intermediateTables, ConcurrentMap tableCreateLocks, ConcurrentMap tableUpdateLocks, @@ -143,6 +148,7 @@ private SchemaManager( this.kafkaDataFieldName = kafkaDataFieldName; this.timestampPartitionFieldName = timestampPartitionFieldName; this.clusteringFieldName = clusteringFieldName; + this.timePartitioningType = timePartitioningType; this.intermediateTables = intermediateTables; this.tableCreateLocks = tableCreateLocks; this.tableUpdateLocks = tableUpdateLocks; @@ -161,6 +167,7 @@ public SchemaManager forIntermediateTables() { kafkaDataFieldName, timestampPartitionFieldName, clusteringFieldName, + timePartitioningType, true, tableCreateLocks, tableUpdateLocks, @@ -450,7 +457,7 @@ TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema big // pseudocolumn can be queried to filter out rows that are still in the streaming buffer builder.setTimePartitioning(TimePartitioning.of(Type.DAY)); } else if (createSchema) { - TimePartitioning timePartitioning = TimePartitioning.of(Type.DAY); + TimePartitioning timePartitioning = TimePartitioning.of(timePartitioningType); if (timestampPartitionFieldName.isPresent()) { timePartitioning = timePartitioning.toBuilder().setField(timestampPartitionFieldName.get()).build(); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index b181b8dd0..619ab6a51 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -21,6 +21,7 @@ import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.TimePartitioning; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.convert.BigQueryRecordConverter; @@ -51,6 +52,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Base class for connector and task configs; contains properties shared between the two of them. @@ -297,6 +300,17 @@ public class BigQuerySinkConfig extends AbstractConfig { "How many records to write to an intermediate table before performing a merge flush, if " + "upsert/delete is enabled. Can be set to -1 to disable record count-based flushing."; + public static final String TIME_PARTITIONING_TYPE_CONFIG = "timePartitioningType"; + private static final ConfigDef.Type TIME_PARTITIONING_TYPE_TYPE = ConfigDef.Type.STRING; + public static final String TIME_PARTITIONING_TYPE_DEFAULT = TimePartitioning.Type.DAY.name().toUpperCase(); + private static final ConfigDef.Importance TIME_PARTITIONING_TYPE_IMPORTANCE = ConfigDef.Importance.LOW; + private static final List TIME_PARTITIONING_TYPES = Stream.of(TimePartitioning.Type.values()) + .map(TimePartitioning.Type::name) + .collect(Collectors.toList()); + private static final String TIME_PARTITIONING_TYPE_DOC = + "The time partitioning type to use when creating tables. " + + "Existing tables will not be altered to use this partitioning type."; + /** * Return a ConfigDef object used to define this config's fields. * @@ -484,6 +498,29 @@ public static ConfigDef getConfig() { MERGE_RECORDS_THRESHOLD_VALIDATOR, MERGE_RECORDS_THRESHOLD_IMPORTANCE, MERGE_RECORDS_THRESHOLD_DOC + ).define( + TIME_PARTITIONING_TYPE_CONFIG, + TIME_PARTITIONING_TYPE_TYPE, + TIME_PARTITIONING_TYPE_DEFAULT, + ConfigDef.CaseInsensitiveValidString.in(TIME_PARTITIONING_TYPES.toArray(new String[0])), + TIME_PARTITIONING_TYPE_IMPORTANCE, + TIME_PARTITIONING_TYPE_DOC, + "", + -1, + ConfigDef.Width.NONE, + TIME_PARTITIONING_TYPE_CONFIG, + new ConfigDef.Recommender() { + @Override + public List validValues(String s, Map map) { + // Construct a new list to transform from List to List + return new ArrayList<>(TIME_PARTITIONING_TYPES); + } + + @Override + public boolean visible(String s, Map map) { + return true; + } + } ); } @@ -665,6 +702,27 @@ public boolean isUpsertDeleteEnabled() { return getBoolean(UPSERT_ENABLED_CONFIG) || getBoolean(DELETE_ENABLED_CONFIG); } + public TimePartitioning.Type getTimePartitioningType() { + return parseTimePartitioningType(getString(TIME_PARTITIONING_TYPE_CONFIG)); + } + + private TimePartitioning.Type parseTimePartitioningType(String rawPartitioningType) { + if (rawPartitioningType == null) { + throw new ConfigException(TIME_PARTITIONING_TYPE_CONFIG, + rawPartitioningType, + "Must be one of " + String.join(", ", TIME_PARTITIONING_TYPES)); + } + + try { + return TimePartitioning.Type.valueOf(rawPartitioningType); + } catch (IllegalArgumentException e) { + throw new ConfigException( + TIME_PARTITIONING_TYPE_CONFIG, + rawPartitioningType, + "Must be one of " + String.join(", ", TIME_PARTITIONING_TYPES)); + } + } + /** * Verifies that a bucket is specified if GCS batch loading is enabled. * @throws ConfigException Exception thrown if no bucket is specified and batch loading is on. diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index a2c2b62a1..775b77fc5 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -31,6 +31,7 @@ import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TableInfo; +import com.google.cloud.bigquery.TimePartitioning; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; @@ -79,7 +80,8 @@ public void testBQTableDescription() { Optional kafkaKeyFieldName = Optional.of("kafkaKey"); Optional kafkaDataFieldName = Optional.of("kafkaData"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, kafkaKeyFieldName, kafkaDataFieldName, Optional.empty(), Optional.empty()); + mockBigQuery, false, false, false, kafkaKeyFieldName, kafkaDataFieldName, Optional.empty(), Optional.empty(), + TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -97,7 +99,8 @@ public void testBQTableDescription() { public void testTimestampPartitionSet() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty(), + TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -109,16 +112,37 @@ public void testTimestampPartitionSet() { testDoc, tableInfo.getDescription()); StandardTableDefinition definition = tableInfo.getDefinition(); Assert.assertNotNull(definition.getTimePartitioning()); + Assert.assertEquals(TimePartitioning.Type.DAY, definition.getTimePartitioning().getType()); Assert.assertEquals("The field name does not match the field name of time partition", testField.get(), definition.getTimePartitioning().getField()); } + @Test + public void testAlternativeTimestampPartitionType() { + SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), + TimePartitioning.Type.HOUR); + + when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + + TableInfo tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); + + Assert.assertEquals("Kafka doc does not match BigQuery table description", + testDoc, tableInfo.getDescription()); + StandardTableDefinition definition = tableInfo.getDefinition(); + Assert.assertNotNull(definition.getTimePartitioning()); + Assert.assertEquals(TimePartitioning.Type.HOUR, definition.getTimePartitioning().getType()); + } + @Test public void testUpdateTimestampPartitionNull() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty(), + TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -136,7 +160,8 @@ public void testUpdateTimestampPartitionNull() { public void testUpdateTimestampPartitionNotSet() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty(), + TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -154,7 +179,8 @@ public void testUpdateTimestampPartitionNotSet() { Optional updateField = Optional.of("testUpdateField"); schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), updateField, Optional.empty()); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), updateField, Optional.empty(), + TimePartitioning.Type.DAY); tableInfo = schemaManager .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); @@ -168,7 +194,8 @@ public void testClusteringPartitionSet() { Optional timestampPartitionFieldName = Optional.of("testField"); Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField, + TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -190,7 +217,8 @@ public void testUpdateClusteringPartitionNull() { Optional timestampPartitionFieldName = Optional.of("testField"); Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField, + TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -209,7 +237,8 @@ public void testUpdateClusteringPartitionNotSet() { Optional timestampPartitionFieldName = Optional.of("testField"); Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField, + TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -227,7 +256,8 @@ public void testUpdateClusteringPartitionNotSet() { Optional> updateTestField = Optional.of(Arrays.asList("column3", "column4")); schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, updateTestField); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, updateTestField, + TimePartitioning.Type.DAY); tableInfo = schemaManager .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); @@ -424,7 +454,8 @@ private SchemaManager createSchemaManager( boolean allowNewFields, boolean allowFieldRelaxation, boolean allowUnionization) { return new SchemaManager(new IdentitySchemaRetriever(), mockSchemaConverter, mockBigQuery, allowNewFields, allowFieldRelaxation, allowUnionization, - Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); + Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), + TimePartitioning.Type.DAY); } private void testGetAndValidateProposedSchema( diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index 92cfff787..a86905d74 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -19,8 +19,10 @@ package com.wepay.kafka.connect.bigquery.config; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.cloud.bigquery.TimePartitioning; import com.wepay.kafka.connect.bigquery.SinkPropertiesFactory; import com.wepay.kafka.connect.bigquery.convert.BigQueryRecordConverter; @@ -79,4 +81,22 @@ public void testInvalidAvroCacheSize() { new BigQuerySinkConfig(badConfigProperties); } + + @Test + public void testValidTimePartitioningTypes() { + Map configProperties = propertiesFactory.getProperties(); + + for (TimePartitioning.Type type : TimePartitioning.Type.values()) { + configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, type.name()); + assertEquals(type, new BigQuerySinkConfig(configProperties).getTimePartitioningType()); + } + } + + @Test(expected = ConfigException.class) + public void testInvalidTimePartitioningType() { + Map configProperties = propertiesFactory.getProperties(); + + configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, "fortnight"); + new BigQuerySinkConfig(configProperties); + } } From 0a2acf8c4b2b0fa00239bf2aa29bb0dc85dad1e2 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 10 Feb 2021 15:29:46 -0500 Subject: [PATCH 053/190] [maven-release-plugin] prepare branch 2.1.x --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1a398667e..6b2ba08d4 100644 --- a/pom.xml +++ b/pom.xml @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + 2.1.x From 03480f57fdd917454a237b407d4fe32e53879034 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 10 Feb 2021 15:29:52 -0500 Subject: [PATCH 054/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index e9fabd274..630218f5f 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0-SNAPSHOT + 2.2.0-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 4460b6966..89b44e8ea 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0-SNAPSHOT + 2.2.0-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 6b2ba08d4..825f5ae80 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0-SNAPSHOT + 2.2.0-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + HEAD From 20adfd180a8665f3af7bdca64d449e261d45d439 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 11 Feb 2021 19:46:09 +0000 Subject: [PATCH 055/190] [maven-release-plugin] prepare release v2.1.0 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index e9fabd274..66330f70b 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0-SNAPSHOT + 2.1.0 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 4460b6966..55a738246 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0-SNAPSHOT + 2.1.0 .. diff --git a/pom.xml b/pom.xml index 6b2ba08d4..d0d1de41e 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0-SNAPSHOT + 2.1.0 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.0 From 798aa6c31e3876c0640cfcbf55292fb0191d99a4 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 11 Feb 2021 19:46:16 +0000 Subject: [PATCH 056/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 66330f70b..aac8e99ed 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0 + 2.1.1-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 55a738246..53ad6ca12 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0 + 2.1.1-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index d0d1de41e..f234e2a6c 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.0 + 2.1.1-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.0 + 2.1.x From 389eae60f20afbf0ec92f4086dc30b0c90762a36 Mon Sep 17 00:00:00 2001 From: VermaSurbhi <76060080+VermaSurbhi@users.noreply.github.com> Date: Fri, 12 Feb 2021 16:46:09 +0000 Subject: [PATCH 057/190] Add partition expiration (#83) * Add partition expiration Co-authored by: Philippa Main on-behalf-of: @autotraderuk Add partition expiration to a table. Partition expiration removes partitions from the table after a specified amount of time. It is an optional field in the connector. * Address review comments Co-authored by: Surbhi Verma Co-authored by: Philippa Main on-behalf-of: @autotraderuk * Clarify documentation of partitionExpirationMs * Reword documentation of partitionExpirationMs * Reword documentation of partitionExpirationMs * Re-align indents Co-authored by: Surbhi Verma Co-authored by: Philippa Main on-behalf-of: @autotraderuk Co-authored-by: Philippa Main --- .../connect/bigquery/BigQuerySinkTask.java | 3 +- .../kafka/connect/bigquery/SchemaManager.java | 15 ++- .../config/BigQuerySinkTaskConfig.java | 34 ++++++- .../connect/bigquery/SchemaManagerTest.java | 92 +++++++++++++++---- .../config/BigQuerySinkTaskConfigTest.java | 32 +++++++ 5 files changed, 148 insertions(+), 28 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 9c4d7943d..a1ce07554 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -365,6 +365,7 @@ private SchemaManager newSchemaManager() { Optional kafkaKeyFieldName = config.getKafkaKeyFieldName(); Optional kafkaDataFieldName = config.getKafkaDataFieldName(); Optional timestampPartitionFieldName = config.getTimestampPartitionFieldName(); + Optional partitionExpiration = config.getPartitionExpirationMs(); Optional> clusteringFieldName = config.getClusteringPartitionFieldName(); TimePartitioning.Type timePartitioningType = config.getTimePartitioningType(); boolean allowNewBQFields = config.getBoolean(config.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); @@ -373,7 +374,7 @@ private SchemaManager newSchemaManager() { return new SchemaManager(schemaRetriever, schemaConverter, getBigQuery(), allowNewBQFields, allowReqFieldRelaxation, allowSchemaUnionization, kafkaKeyFieldName, kafkaDataFieldName, - timestampPartitionFieldName, clusteringFieldName, timePartitioningType); + timestampPartitionFieldName, partitionExpiration, clusteringFieldName, timePartitioningType); } private BigQueryWriter getBigQueryWriter() { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 0e257f522..c2c82b07e 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -66,6 +66,7 @@ public class SchemaManager { private final Optional kafkaKeyFieldName; private final Optional kafkaDataFieldName; private final Optional timestampPartitionFieldName; + private final Optional partitionExpiration; private final Optional> clusteringFieldName; private final TimePartitioning.Type timePartitioningType; private final boolean intermediateTables; @@ -102,6 +103,7 @@ public SchemaManager( Optional kafkaKeyFieldName, Optional kafkaDataFieldName, Optional timestampPartitionFieldName, + Optional partitionExpiration, Optional> clusteringFieldName, TimePartitioning.Type timePartitioningType) { this( @@ -114,6 +116,7 @@ public SchemaManager( kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, + partitionExpiration, clusteringFieldName, timePartitioningType, false, @@ -132,6 +135,7 @@ private SchemaManager( Optional kafkaKeyFieldName, Optional kafkaDataFieldName, Optional timestampPartitionFieldName, + Optional partitionExpiration, Optional> clusteringFieldName, TimePartitioning.Type timePartitioningType, boolean intermediateTables, @@ -147,6 +151,7 @@ private SchemaManager( this.kafkaKeyFieldName = kafkaKeyFieldName; this.kafkaDataFieldName = kafkaDataFieldName; this.timestampPartitionFieldName = timestampPartitionFieldName; + this.partitionExpiration = partitionExpiration; this.clusteringFieldName = clusteringFieldName; this.timePartitioningType = timePartitioningType; this.intermediateTables = intermediateTables; @@ -166,6 +171,7 @@ public SchemaManager forIntermediateTables() { kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, + partitionExpiration, clusteringFieldName, timePartitioningType, true, @@ -457,12 +463,11 @@ TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema big // pseudocolumn can be queried to filter out rows that are still in the streaming buffer builder.setTimePartitioning(TimePartitioning.of(Type.DAY)); } else if (createSchema) { - TimePartitioning timePartitioning = TimePartitioning.of(timePartitioningType); - if (timestampPartitionFieldName.isPresent()) { - timePartitioning = timePartitioning.toBuilder().setField(timestampPartitionFieldName.get()).build(); - } + TimePartitioning.Builder timePartitioningBuilder = TimePartitioning.of(timePartitioningType).toBuilder(); + timestampPartitionFieldName.ifPresent(timePartitioningBuilder::setField); + partitionExpiration.ifPresent(timePartitioningBuilder::setExpirationMs); - builder.setTimePartitioning(timePartitioning); + builder.setTimePartitioning(timePartitioningBuilder.build()); if (timestampPartitionFieldName.isPresent() && clusteringFieldName.isPresent()) { Clustering clustering = Clustering.newBuilder() diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java index 798397eda..da3955274 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java @@ -109,8 +109,18 @@ public class BigQuerySinkTaskConfig extends BigQuerySinkConfig { ConfigDef.Importance.LOW; private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC = "The name of the field in the value that contains the timestamp to partition by in BigQuery" - + " and enable timestamp partitioning for each table. Leave this configuration blank," - + " to enable ingestion time partitioning for each table."; + + " and enable timestamp partitioning for each table. Leave this configuration blank," + + " to enable ingestion time partitioning for each table."; + + public static final String BIGQUERY_PARTITION_EXPIRATION_CONFIG = "partitionExpirationMs"; + private static final ConfigDef.Type BIGQUERY_PARTITION_EXPIRATION_TYPE = ConfigDef.Type.LONG; + private static final String BIGQUERY_PARTITION_EXPIRATION_DEFAULT = null; + private static final ConfigDef.Importance BIGQUERY_PARTITION_EXPIRATION_IMPORTANCE = ConfigDef.Importance.LOW; + private static final String BIGQUERY_PARTITION_EXPIRATION_DOC = + "The amount of time, in milliseconds, after which partitions should be deleted from the tables this " + + "connector creates. If this field is set, all data in partitions in this connector's tables that are " + + "older than the specified partition expiration time will be permanently deleted. " + + "Existing tables will not be altered to use this partition expiration time."; public static final String BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG = "clusteringPartitionFieldNames"; private static final ConfigDef.Type BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE = ConfigDef.Type.LIST; @@ -178,6 +188,12 @@ public static ConfigDef getConfig() { BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC + ).define( + BIGQUERY_PARTITION_EXPIRATION_CONFIG, + BIGQUERY_PARTITION_EXPIRATION_TYPE, + BIGQUERY_PARTITION_EXPIRATION_DEFAULT, + BIGQUERY_PARTITION_EXPIRATION_IMPORTANCE, + BIGQUERY_PARTITION_EXPIRATION_DOC ).define( BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE, @@ -218,6 +234,14 @@ public Optional getTimestampPartitionFieldName() { return Optional.ofNullable(getString(BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG)); } + /** + * Returns the partition expiration in ms. + * @return Long that represents the partition expiration. + */ + public Optional getPartitionExpirationMs() { + return Optional.ofNullable(getLong(BIGQUERY_PARTITION_EXPIRATION_CONFIG)); + } + /** * Returns the field names to use for clustering. * @return List of Strings that represent the field names. @@ -236,6 +260,12 @@ private void checkPartitionConfigs() { + "Use either bigQueryPartitionDecorator OR timestampPartitionFieldName." ); } + getPartitionExpirationMs().ifPresent(partitionExpiration -> { + if (partitionExpiration <= 0) { + throw new ConfigException(BIGQUERY_PARTITION_EXPIRATION_CONFIG, partitionExpiration, + "The partition expiration value must be positive."); + } + }); } /** diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index 775b77fc5..24ab1cb6a 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -80,8 +80,8 @@ public void testBQTableDescription() { Optional kafkaKeyFieldName = Optional.of("kafkaKey"); Optional kafkaDataFieldName = Optional.of("kafkaData"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, kafkaKeyFieldName, kafkaDataFieldName, Optional.empty(), Optional.empty(), - TimePartitioning.Type.DAY); + mockBigQuery, false, false, false, kafkaKeyFieldName, kafkaDataFieldName, + Optional.empty(), Optional.empty(), Optional.empty(), TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -93,14 +93,16 @@ public void testBQTableDescription() { testDoc, tableInfo.getDescription()); Assert.assertNull("Timestamp partition field name is not null", ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning().getField()); + Assert.assertNull("Partition expiration is not null", + ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning().getExpirationMs()); } @Test public void testTimestampPartitionSet() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty(), - TimePartitioning.Type.DAY); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, + Optional.empty(), Optional.empty(), TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -116,13 +118,15 @@ public void testTimestampPartitionSet() { Assert.assertEquals("The field name does not match the field name of time partition", testField.get(), definition.getTimePartitioning().getField()); + Assert.assertNull("Partition expiration is not null", + ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning().getExpirationMs()); } @Test public void testAlternativeTimestampPartitionType() { SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), - TimePartitioning.Type.HOUR); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), Optional.empty(), + Optional.empty(), Optional.empty(), TimePartitioning.Type.HOUR); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -141,8 +145,8 @@ public void testAlternativeTimestampPartitionType() { public void testUpdateTimestampPartitionNull() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty(), - TimePartitioning.Type.DAY); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, + Optional.empty(), Optional.empty(), TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -160,8 +164,8 @@ public void testUpdateTimestampPartitionNull() { public void testUpdateTimestampPartitionNotSet() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty(), - TimePartitioning.Type.DAY); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, + Optional.empty(), Optional.empty(),TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -179,7 +183,7 @@ public void testUpdateTimestampPartitionNotSet() { Optional updateField = Optional.of("testUpdateField"); schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), updateField, Optional.empty(), + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), updateField, Optional.empty(), Optional.empty(), TimePartitioning.Type.DAY); tableInfo = schemaManager @@ -189,13 +193,61 @@ public void testUpdateTimestampPartitionNotSet() { ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning()); } + @Test + public void testPartitionExpirationSetWithoutFieldName() { + Optional testExpirationMs = Optional.of(86400000L); + SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), Optional.empty(), + testExpirationMs, Optional.empty(), TimePartitioning.Type.DAY); + + when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + + TableInfo tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); + + Assert.assertEquals("Kafka doc does not match BigQuery table description", + testDoc, tableInfo.getDescription()); + StandardTableDefinition tableDefinition = (StandardTableDefinition) tableInfo.getDefinition(); + Assert.assertEquals("The partition expiration does not match the expiration in ms", + testExpirationMs.get(), + tableDefinition.getTimePartitioning().getExpirationMs()); + Assert.assertNull("Timestamp partition field name is not null", + tableDefinition.getTimePartitioning().getField()); + } + + @Test + public void testPartitionExpirationSetWithFieldName() { + Optional testExpirationMs = Optional.of(86400000L); + Optional testField = Optional.of("testField"); + SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, + testExpirationMs, Optional.empty(), TimePartitioning.Type.DAY); + + when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + + TableInfo tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); + + Assert.assertEquals("Kafka doc does not match BigQuery table description", + testDoc, tableInfo.getDescription()); + StandardTableDefinition tableDefinition = (StandardTableDefinition) tableInfo.getDefinition(); + Assert.assertEquals("The partition expiration does not match the expiration in ms", + testExpirationMs.get(), + tableDefinition.getTimePartitioning().getExpirationMs()); + Assert.assertEquals("The field name does not match the field name of time partition", + testField.get(), + tableDefinition.getTimePartitioning().getField()); + } + @Test public void testClusteringPartitionSet() { Optional timestampPartitionFieldName = Optional.of("testField"); Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField, - TimePartitioning.Type.DAY); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, + Optional.empty(), testField, TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -217,8 +269,8 @@ public void testUpdateClusteringPartitionNull() { Optional timestampPartitionFieldName = Optional.of("testField"); Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField, - TimePartitioning.Type.DAY); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, + Optional.empty(), testField, TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -237,8 +289,8 @@ public void testUpdateClusteringPartitionNotSet() { Optional timestampPartitionFieldName = Optional.of("testField"); Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField, - TimePartitioning.Type.DAY); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, + Optional.empty(), testField, TimePartitioning.Type.DAY); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -256,8 +308,8 @@ public void testUpdateClusteringPartitionNotSet() { Optional> updateTestField = Optional.of(Arrays.asList("column3", "column4")); schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, updateTestField, - TimePartitioning.Type.DAY); + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, + Optional.empty(), updateTestField, TimePartitioning.Type.DAY); tableInfo = schemaManager .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); @@ -454,7 +506,7 @@ private SchemaManager createSchemaManager( boolean allowNewFields, boolean allowFieldRelaxation, boolean allowUnionization) { return new SchemaManager(new IdentitySchemaRetriever(), mockSchemaConverter, mockBigQuery, allowNewFields, allowFieldRelaxation, allowUnionization, - Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), + Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), TimePartitioning.Type.DAY); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java index e50dd476e..7f2e0cd52 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java @@ -102,6 +102,38 @@ public void testTimestampPartitionFieldName() { assertFalse(testConfig.getBoolean(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG)); } + /** + * Test the default for the partition expiration is not present. + */ + @Test + public void testEmptyPartitionExpirationMs() { + Map configProperties = propertiesFactory.getProperties(); + BigQuerySinkTaskConfig testConfig = new BigQuerySinkTaskConfig(configProperties); + assertFalse(testConfig.getPartitionExpirationMs().isPresent()); + } + + /** + * Test the partition expiration is set correctly for a valid value. + */ + @Test + public void testValidPartitionExpirationMs() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_EXPIRATION_CONFIG, "1"); + BigQuerySinkTaskConfig testConfig = new BigQuerySinkTaskConfig(configProperties); + assertTrue(testConfig.getPartitionExpirationMs().isPresent()); + assertEquals(Optional.of(1L), testConfig.getPartitionExpirationMs()); + } + + /** + * Test the partition expiration being non-positive errors correctly. + */ + @Test (expected = ConfigException.class) + public void testMinimumPartitionExpirationMs() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_EXPIRATION_CONFIG, "0"); + new BigQuerySinkTaskConfig(configProperties); + } + /** * Test the default for the field names is not present. */ From 12900ce894c08f53a95e36d41f3210985ce98854 Mon Sep 17 00:00:00 2001 From: Jinxin Liu Date: Wed, 10 Mar 2021 12:29:35 -0800 Subject: [PATCH 058/190] Prevent NPE thrown at SchemaManager.isFieldRelaxation() by specifying unspecified field mode as NULLABLE (#92) * When retrieving fields from BigQuery schema, if the mode is unspecified, specify it as NULLABLE * added unit test for the new change --- .../kafka/connect/bigquery/SchemaManager.java | 11 +++++++++- .../connect/bigquery/SchemaManagerTest.java | 22 +++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 0bd354e1d..d14fdba1b 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -24,6 +24,7 @@ import com.google.cloud.bigquery.BigQueryException; import com.google.cloud.bigquery.Clustering; import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.Field.Mode; import com.google.cloud.bigquery.LegacySQLTypeName; import com.google.cloud.bigquery.StandardTableDefinition; import com.google.cloud.bigquery.TableId; @@ -365,8 +366,11 @@ private com.google.cloud.bigquery.Schema unionizeSchemas( private void validateSchemaChange( com.google.cloud.bigquery.Schema existingSchema, com.google.cloud.bigquery.Schema proposedSchema) { + logger.trace("Validating schema change. Existing schema: {}; proposed Schema: {}", + existingSchema.toString(), proposedSchema.toString()); Map earliestSchemaFields = schemaFields(existingSchema); Map proposedSchemaFields = schemaFields(proposedSchema); + for (Map.Entry entry : proposedSchemaFields.entrySet()) { if (!earliestSchemaFields.containsKey(entry.getKey())) { if (!isValidFieldAddition(entry.getValue())) { @@ -434,7 +438,12 @@ private String getUnionizedTableDescription(List records) { */ private Map schemaFields(com.google.cloud.bigquery.Schema schema) { Map result = new LinkedHashMap<>(); - schema.getFields().forEach(field -> result.put(field.getName(), field)); + schema.getFields().forEach(field -> { + if (field.getMode() == null) { + field = field.toBuilder().setMode(Field.Mode.NULLABLE).build(); + } + result.put(field.getName(), field); + }); return result; } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index b65979a57..5a29a7782 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -24,6 +24,7 @@ import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.Field.Mode; import com.google.cloud.bigquery.LegacySQLTypeName; import com.google.cloud.bigquery.StandardTableDefinition; import com.google.cloud.bigquery.Table; @@ -319,6 +320,27 @@ public void testUnionizedUpdateWithMultipleSchemas() { testGetAndValidateProposedSchema(schemaManager, existingSchema, newSchemas, expectedSchema); } + @Test + public void FieldsWithUnspecifiedModeShouldNotCauseNpe() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).build() + ); + + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).build() + ); + + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Mode.NULLABLE).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, true); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expectedSchema); + } + private SchemaManager createSchemaManager( boolean allowNewFields, boolean allowFieldRelaxation, boolean allowUnionization) { return new SchemaManager(new IdentitySchemaRetriever(), mockSchemaConverter, mockBigQuery, From 394190dad3ac0ab0d5cf0d0d8af13b00709184b0 Mon Sep 17 00:00:00 2001 From: Jinxin Liu Date: Wed, 10 Mar 2021 21:03:19 +0000 Subject: [PATCH 059/190] [maven-release-plugin] prepare release v2.0.1 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index cd994dbfb..388dd7444 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.1-SNAPSHOT + 2.0.1 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 1d64aea9e..145b1ee17 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.1-SNAPSHOT + 2.0.1 .. diff --git a/pom.xml b/pom.xml index f1172d390..57bc48bc4 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.1-SNAPSHOT + 2.0.1 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.1 From ae26017e600ac5bd5bcab1a0d6af739a4d536654 Mon Sep 17 00:00:00 2001 From: Jinxin Liu Date: Wed, 10 Mar 2021 21:03:24 +0000 Subject: [PATCH 060/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 388dd7444..0515a6cd0 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.1 + 2.0.2-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 145b1ee17..48718af61 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.1 + 2.0.2-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 57bc48bc4..cfa05ffdc 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.1 + 2.0.2-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.1 + HEAD From b38e4ea07a4a9844e59ea96a83765b070b9971c5 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 6 Apr 2021 11:40:36 -0400 Subject: [PATCH 061/190] MINOR: Increase retry period for writes after table updates to 15 minutes (#97) We've had some reports that the connector fails (especially when upsert/delete is enabled) after creating a new table because it keeps getting 404 errors. After bumping this value the errors stopped and the connector was able to work smoothly. --- .../connect/bigquery/write/row/AdaptiveBigQueryWriter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java index 39a94f711..f800694fe 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java @@ -50,8 +50,8 @@ public class AdaptiveBigQueryWriter extends BigQueryWriter { private static final Logger logger = LoggerFactory.getLogger(AdaptiveBigQueryWriter.class); // The maximum number of retries we will attempt to write rows after creating a table or updating a BQ table schema. - private static final int RETRY_LIMIT = 10; - // Wait for about 30s between each retry since both creating table and updating schema take up to 2~3 minutes to take effect. + private static final int RETRY_LIMIT = 30; + // Wait for about 30s between each retry to avoid hammering BigQuery with requests private static final int RETRY_WAIT_TIME = 30000; private final BigQuery bigQuery; From 25a25560066c40764ee9d6322baeb129d024af23 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 6 Apr 2021 15:54:22 +0000 Subject: [PATCH 062/190] [maven-release-plugin] prepare release v2.0.2 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 0515a6cd0..f04a6adf8 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.2-SNAPSHOT + 2.0.2 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 48718af61..71ab3f004 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.2-SNAPSHOT + 2.0.2 .. diff --git a/pom.xml b/pom.xml index cfa05ffdc..35f110dca 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.2-SNAPSHOT + 2.0.2 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.2 From 6a2926285cd08ae2f23f9ded24f849da04b87c46 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 6 Apr 2021 15:54:27 +0000 Subject: [PATCH 063/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index f04a6adf8..5eef2e8b4 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.2 + 2.0.3-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 71ab3f004..bdb4a286f 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.2 + 2.0.3-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 35f110dca..cb175b8aa 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.2 + 2.0.3-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.2 + HEAD From 04546c993db40abe78f24f012ca1c58c3e6a8537 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 6 Apr 2021 15:55:52 +0000 Subject: [PATCH 064/190] [maven-release-plugin] prepare release v2.1.1 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index aac8e99ed..33ba96872 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.1-SNAPSHOT + 2.1.1 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 53ad6ca12..4001f1614 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.1-SNAPSHOT + 2.1.1 .. diff --git a/pom.xml b/pom.xml index f234e2a6c..0bfb85d5d 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.1-SNAPSHOT + 2.1.1 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.1 From 645da1b499899e87c65d014203887555921bb721 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 6 Apr 2021 15:55:57 +0000 Subject: [PATCH 065/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 33ba96872..ed054469f 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.1 + 2.1.2-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 4001f1614..389d111e7 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.1 + 2.1.2-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 0bfb85d5d..d6588746e 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.1 + 2.1.2-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.1 + 2.1.x From ab5b5856670275998b33d5bf8082a91af4ee66e9 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 8 Apr 2021 12:32:10 -0400 Subject: [PATCH 066/190] GH-96: Fix unionization/update logic for REPEATED fields (#99) --- .../kafka/connect/bigquery/SchemaManager.java | 16 +++++-- .../connect/bigquery/SchemaManagerTest.java | 42 ++++++++++++++++++- 2 files changed, 53 insertions(+), 5 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index d14fdba1b..736532663 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -24,7 +24,6 @@ import com.google.cloud.bigquery.BigQueryException; import com.google.cloud.bigquery.Clustering; import com.google.cloud.bigquery.Field; -import com.google.cloud.bigquery.Field.Mode; import com.google.cloud.bigquery.LegacySQLTypeName; import com.google.cloud.bigquery.StandardTableDefinition; import com.google.cloud.bigquery.TableId; @@ -348,7 +347,10 @@ private com.google.cloud.bigquery.Schema unionizeSchemas( firstSchemaFields.forEach((name, firstField) -> { Field secondField = secondSchemaFields.get(name); if (secondField == null) { - unionizedSchemaFields.put(name, firstField.toBuilder().setMode(Field.Mode.NULLABLE).build()); + // Repeated fields are implicitly nullable; no need to set a new mode for them + if (!Field.Mode.REPEATED.equals(firstField.getMode())) { + unionizedSchemaFields.put(name, firstField.toBuilder().setMode(Field.Mode.NULLABLE).build()); + } } else if (isFieldRelaxation(firstField, secondField)) { unionizedSchemaFields.put(name, secondField); } else { @@ -358,7 +360,12 @@ private com.google.cloud.bigquery.Schema unionizeSchemas( secondSchemaFields.forEach((name, secondField) -> { if (!unionizedSchemaFields.containsKey(name)) { - unionizedSchemaFields.put(name, secondField.toBuilder().setMode(Field.Mode.NULLABLE).build()); + if (Field.Mode.REPEATED.equals(secondField.getMode())) { + // Repeated fields are implicitly nullable; no need to set a new mode for them + unionizedSchemaFields.put(name, secondField); + } else { + unionizedSchemaFields.put(name, secondField.toBuilder().setMode(Field.Mode.NULLABLE).build()); + } } }); return com.google.cloud.bigquery.Schema.of(unionizedSchemaFields.values()); @@ -397,6 +404,7 @@ private boolean isFieldRelaxation(Field currentField, Field proposedField) { private boolean isValidFieldAddition(Field newField) { return allowNewBQFields && ( newField.getMode().equals(Field.Mode.NULLABLE) || + newField.getMode().equals(Field.Mode.REPEATED) || (newField.getMode().equals(Field.Mode.REQUIRED) && allowBQRequiredFieldRelaxation)); } @@ -407,7 +415,7 @@ private com.google.cloud.bigquery.Schema relaxFieldsWhereNecessary( Map proposedSchemaFields = schemaFields(proposedSchema); List newSchemaFields = new ArrayList<>(); for (Map.Entry entry : proposedSchemaFields.entrySet()) { - if (!existingSchemaFields.containsKey(entry.getKey())) { + if (!existingSchemaFields.containsKey(entry.getKey()) && !Field.Mode.REPEATED.equals(entry.getValue().getMode())) { newSchemaFields.add(entry.getValue().toBuilder().setMode(Field.Mode.NULLABLE).build()); } else { newSchemaFields.add(entry.getValue()); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index 5a29a7782..81464b7ab 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -49,7 +49,6 @@ import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.stream.Collectors; public class SchemaManagerTest { @@ -260,6 +259,47 @@ public void testSuccessfulUnionizedUpdateWithNewAndRelaxedFields() { testGetAndValidateProposedSchema(schemaManager, existingSchema, disjointSchema, expectedSchema); } + @Test + public void testSuccessfulUnionizedUpdateWithNewRepeatedField() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.REPEATED).build() + ); + + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.REPEATED).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, true); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expectedSchema); + } + + @Test + public void testSuccessfulUpdateWithNewRepeatedField() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.REPEATED).build() + ); + + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.REPEATED).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, false); + + testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expectedSchema); + } @Test(expected = BigQueryConnectException.class) public void testDisallowedUnionizedUpdateWithNewField() { com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( From 394d559bfe7cd42bc10a0ce7d9c090826d36b84d Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 8 Apr 2021 16:46:02 +0000 Subject: [PATCH 067/190] [maven-release-plugin] prepare release v2.0.3 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 5eef2e8b4..b2d8e3473 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.3-SNAPSHOT + 2.0.3 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index bdb4a286f..16181447b 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.3-SNAPSHOT + 2.0.3 .. diff --git a/pom.xml b/pom.xml index cb175b8aa..bfbfc0741 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.3-SNAPSHOT + 2.0.3 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.3 From 5ceb1d331ac8eaf0f4522fd004ee4fbbb1757904 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 8 Apr 2021 16:46:07 +0000 Subject: [PATCH 068/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index b2d8e3473..e9b505a79 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.3 + 2.0.4-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 16181447b..564fef3b8 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.3 + 2.0.4-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index bfbfc0741..41db3dd32 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.3 + 2.0.4-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.3 + HEAD From 172dc5a5f81712eddd578743ed85bd8dea5a083e Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 8 Apr 2021 16:47:35 +0000 Subject: [PATCH 069/190] [maven-release-plugin] prepare release v2.1.2 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index ed054469f..c2a599226 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.2-SNAPSHOT + 2.1.2 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 389d111e7..b882ed1b4 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.2-SNAPSHOT + 2.1.2 .. diff --git a/pom.xml b/pom.xml index d6588746e..480329e08 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.2-SNAPSHOT + 2.1.2 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.2 From af856aa79f67b5e8c1554a6a8457663e46222de1 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 8 Apr 2021 16:47:39 +0000 Subject: [PATCH 070/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index c2a599226..bb8e6b09e 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.2 + 2.1.3-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index b882ed1b4..3509b979e 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.2 + 2.1.3-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 480329e08..96a142925 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.2 + 2.1.3-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.2 + 2.1.x From 6c9919716756585eb771876c500100e28791b65e Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 9 Apr 2021 17:50:00 -0400 Subject: [PATCH 071/190] GH-96 (follow-up): Include REPEATED fields in unionized schemas if missing from later schemas (#101) --- .../com/wepay/kafka/connect/bigquery/SchemaManager.java | 2 ++ .../connect/bigquery/write/row/AdaptiveBigQueryWriter.java | 2 +- .../wepay/kafka/connect/bigquery/SchemaManagerTest.java | 7 +++++-- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 736532663..262d68a6a 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -350,6 +350,8 @@ private com.google.cloud.bigquery.Schema unionizeSchemas( // Repeated fields are implicitly nullable; no need to set a new mode for them if (!Field.Mode.REPEATED.equals(firstField.getMode())) { unionizedSchemaFields.put(name, firstField.toBuilder().setMode(Field.Mode.NULLABLE).build()); + } else { + unionizedSchemaFields.put(name, firstField); } } else if (isFieldRelaxation(firstField, secondField)) { unionizedSchemaFields.put(name, secondField); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java index f800694fe..c5d2d2d60 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java @@ -133,7 +133,7 @@ && onlyContainsInvalidSchemaErrors(writeResponse.getInsertErrors())) { writeResponse = bigQuery.insertAll(request); } catch (BigQueryException exception) { // no-op, we want to keep retrying the insert - logger.trace("insertion failed", exception); + logger.debug("insertion failed", exception); } } else { return writeResponse.getInsertErrors(); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index 81464b7ab..b4c5ee511 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -261,7 +261,7 @@ public void testSuccessfulUnionizedUpdateWithNewAndRelaxedFields() { @Test public void testSuccessfulUnionizedUpdateWithNewRepeatedField() { - com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + com.google.cloud.bigquery.Schema reducedSchema = com.google.cloud.bigquery.Schema.of( Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() ); @@ -277,7 +277,9 @@ public void testSuccessfulUnionizedUpdateWithNewRepeatedField() { SchemaManager schemaManager = createSchemaManager(true, true, true); - testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expectedSchema); + // Unionization should work symmetrically, so test both cases of reduced/expanded as the current/new schemas + testGetAndValidateProposedSchema(schemaManager, reducedSchema, expandedSchema, expectedSchema); + testGetAndValidateProposedSchema(schemaManager, expandedSchema, reducedSchema, expectedSchema); } @Test @@ -300,6 +302,7 @@ public void testSuccessfulUpdateWithNewRepeatedField() { testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expectedSchema); } + @Test(expected = BigQueryConnectException.class) public void testDisallowedUnionizedUpdateWithNewField() { com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( From 5dff101327aeb7c4d00349cb7d5865a58b401fa3 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 9 Apr 2021 21:55:46 +0000 Subject: [PATCH 072/190] [maven-release-plugin] prepare release v2.0.4 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index e9b505a79..20f3c5f34 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.4-SNAPSHOT + 2.0.4 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 564fef3b8..a2bee7802 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.4-SNAPSHOT + 2.0.4 .. diff --git a/pom.xml b/pom.xml index 41db3dd32..2141e93fb 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.4-SNAPSHOT + 2.0.4 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.4 From f9c8b6013d726942253630a4469af093eb220f3d Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 9 Apr 2021 21:55:51 +0000 Subject: [PATCH 073/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 20f3c5f34..b2a83ed9f 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.4 + 2.0.5-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index a2bee7802..aa09c4daa 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.4 + 2.0.5-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 2141e93fb..bd6863daf 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.4 + 2.0.5-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.4 + HEAD From ae35addace2b50421cbccdab01fa832490990e4c Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 9 Apr 2021 21:57:21 +0000 Subject: [PATCH 074/190] [maven-release-plugin] prepare release v2.1.3 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index bb8e6b09e..8174eb4e6 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.3-SNAPSHOT + 2.1.3 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 3509b979e..cede2a7d2 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.3-SNAPSHOT + 2.1.3 .. diff --git a/pom.xml b/pom.xml index 96a142925..289771060 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.3-SNAPSHOT + 2.1.3 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.3 From bc3e7a644f8c83e61a0efaeb1f12d2d0c7ebeda6 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 9 Apr 2021 21:57:25 +0000 Subject: [PATCH 075/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 8174eb4e6..782b172af 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.3 + 2.1.4-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index cede2a7d2..c502b96c7 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.3 + 2.1.4-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 289771060..8d30158ff 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.3 + 2.1.4-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.3 + 2.1.x From 93bd2b338214642c98ebee3ed938a847b89f71b8 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 5 May 2021 14:40:57 -0400 Subject: [PATCH 076/190] GH-105: Allow creation of non-partitioned tables (#106) --- .../connect/bigquery/BigQuerySinkTask.java | 2 +- .../kafka/connect/bigquery/SchemaManager.java | 32 +++++++------- .../bigquery/config/BigQuerySinkConfig.java | 26 +++++------ .../connect/bigquery/SchemaManagerTest.java | 44 +++++++++++++------ .../config/BigQuerySinkConfigTest.java | 10 ++++- 5 files changed, 71 insertions(+), 43 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index a1ce07554..733f7ec5f 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -367,7 +367,7 @@ private SchemaManager newSchemaManager() { Optional timestampPartitionFieldName = config.getTimestampPartitionFieldName(); Optional partitionExpiration = config.getPartitionExpirationMs(); Optional> clusteringFieldName = config.getClusteringPartitionFieldName(); - TimePartitioning.Type timePartitioningType = config.getTimePartitioningType(); + Optional timePartitioningType = config.getTimePartitioningType(); boolean allowNewBQFields = config.getBoolean(config.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); boolean allowReqFieldRelaxation = config.getBoolean(config.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); boolean allowSchemaUnionization = config.getBoolean(config.ALLOW_SCHEMA_UNIONIZATION_CONFIG); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 7d3770ee5..017f62270 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -68,7 +68,7 @@ public class SchemaManager { private final Optional timestampPartitionFieldName; private final Optional partitionExpiration; private final Optional> clusteringFieldName; - private final TimePartitioning.Type timePartitioningType; + private final Optional timePartitioningType; private final boolean intermediateTables; private final ConcurrentMap tableCreateLocks; private final ConcurrentMap tableUpdateLocks; @@ -105,7 +105,7 @@ public SchemaManager( Optional timestampPartitionFieldName, Optional partitionExpiration, Optional> clusteringFieldName, - TimePartitioning.Type timePartitioningType) { + Optional timePartitioningType) { this( schemaRetriever, schemaConverter, @@ -137,7 +137,7 @@ private SchemaManager( Optional timestampPartitionFieldName, Optional partitionExpiration, Optional> clusteringFieldName, - TimePartitioning.Type timePartitioningType, + Optional timePartitioningType, boolean intermediateTables, ConcurrentMap tableCreateLocks, ConcurrentMap tableUpdateLocks, @@ -482,18 +482,20 @@ TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema big // pseudocolumn can be queried to filter out rows that are still in the streaming buffer builder.setTimePartitioning(TimePartitioning.of(Type.DAY)); } else if (createSchema) { - TimePartitioning.Builder timePartitioningBuilder = TimePartitioning.of(timePartitioningType).toBuilder(); - timestampPartitionFieldName.ifPresent(timePartitioningBuilder::setField); - partitionExpiration.ifPresent(timePartitioningBuilder::setExpirationMs); - - builder.setTimePartitioning(timePartitioningBuilder.build()); - - if (timestampPartitionFieldName.isPresent() && clusteringFieldName.isPresent()) { - Clustering clustering = Clustering.newBuilder() - .setFields(clusteringFieldName.get()) - .build(); - builder.setClustering(clustering); - } + timePartitioningType.ifPresent(partitioningType -> { + TimePartitioning.Builder timePartitioningBuilder = TimePartitioning.of(partitioningType).toBuilder(); + timestampPartitionFieldName.ifPresent(timePartitioningBuilder::setField); + partitionExpiration.ifPresent(timePartitioningBuilder::setExpirationMs); + + builder.setTimePartitioning(timePartitioningBuilder.build()); + + if (timestampPartitionFieldName.isPresent() && clusteringFieldName.isPresent()) { + Clustering clustering = Clustering.newBuilder() + .setFields(clusteringFieldName.get()) + .build(); + builder.setClustering(clustering); + } + }); } StandardTableDefinition tableDefinition = builder.build(); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 619ab6a51..b797f7ead 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -36,21 +36,14 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.sink.SinkConnector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.util.AbstractMap; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -303,12 +296,15 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String TIME_PARTITIONING_TYPE_CONFIG = "timePartitioningType"; private static final ConfigDef.Type TIME_PARTITIONING_TYPE_TYPE = ConfigDef.Type.STRING; public static final String TIME_PARTITIONING_TYPE_DEFAULT = TimePartitioning.Type.DAY.name().toUpperCase(); + public static final String TIME_PARTITIONING_TYPE_NONE = "NONE"; private static final ConfigDef.Importance TIME_PARTITIONING_TYPE_IMPORTANCE = ConfigDef.Importance.LOW; - private static final List TIME_PARTITIONING_TYPES = Stream.of(TimePartitioning.Type.values()) - .map(TimePartitioning.Type::name) + private static final List TIME_PARTITIONING_TYPES = Stream.concat( + Stream.of(TimePartitioning.Type.values()).map(TimePartitioning.Type::name), + Stream.of(TIME_PARTITIONING_TYPE_NONE)) .collect(Collectors.toList()); private static final String TIME_PARTITIONING_TYPE_DOC = - "The time partitioning type to use when creating tables. " + "The time partitioning type to use when creating tables, or '" + + TIME_PARTITIONING_TYPE_NONE + "' to create non-partitioned tables. " + "Existing tables will not be altered to use this partitioning type."; /** @@ -702,19 +698,23 @@ public boolean isUpsertDeleteEnabled() { return getBoolean(UPSERT_ENABLED_CONFIG) || getBoolean(DELETE_ENABLED_CONFIG); } - public TimePartitioning.Type getTimePartitioningType() { + public Optional getTimePartitioningType() { return parseTimePartitioningType(getString(TIME_PARTITIONING_TYPE_CONFIG)); } - private TimePartitioning.Type parseTimePartitioningType(String rawPartitioningType) { + private Optional parseTimePartitioningType(String rawPartitioningType) { if (rawPartitioningType == null) { throw new ConfigException(TIME_PARTITIONING_TYPE_CONFIG, rawPartitioningType, "Must be one of " + String.join(", ", TIME_PARTITIONING_TYPES)); } + if (TIME_PARTITIONING_TYPE_NONE.equals(rawPartitioningType)) { + return Optional.empty(); + } + try { - return TimePartitioning.Type.valueOf(rawPartitioningType); + return Optional.of(TimePartitioning.Type.valueOf(rawPartitioningType)); } catch (IllegalArgumentException e) { throw new ConfigException( TIME_PARTITIONING_TYPE_CONFIG, diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index b8994ea9d..2cac7ec38 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -81,7 +81,7 @@ public void testBQTableDescription() { Optional kafkaDataFieldName = Optional.of("kafkaData"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, kafkaKeyFieldName, kafkaDataFieldName, - Optional.empty(), Optional.empty(), Optional.empty(), TimePartitioning.Type.DAY); + Optional.empty(), Optional.empty(), Optional.empty(), Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -102,7 +102,7 @@ public void testTimestampPartitionSet() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, - Optional.empty(), Optional.empty(), TimePartitioning.Type.DAY); + Optional.empty(), Optional.empty(), Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -126,7 +126,7 @@ public void testTimestampPartitionSet() { public void testAlternativeTimestampPartitionType() { SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), Optional.empty(), TimePartitioning.Type.HOUR); + Optional.empty(), Optional.empty(), Optional.of(TimePartitioning.Type.HOUR)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -141,12 +141,30 @@ public void testAlternativeTimestampPartitionType() { Assert.assertEquals(TimePartitioning.Type.HOUR, definition.getTimePartitioning().getType()); } + @Test + public void testNoTimestampPartitionType() { + SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, + mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), Optional.empty(), + Optional.empty(), Optional.empty(), Optional.empty()); + + when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + + TableInfo tableInfo = schemaManager + .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, true); + + Assert.assertEquals("Kafka doc does not match BigQuery table description", + testDoc, tableInfo.getDescription()); + StandardTableDefinition definition = tableInfo.getDefinition(); + Assert.assertNull(definition.getTimePartitioning()); + } + @Test public void testUpdateTimestampPartitionNull() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, - Optional.empty(), Optional.empty(), TimePartitioning.Type.DAY); + Optional.empty(), Optional.empty(), Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -165,7 +183,7 @@ public void testUpdateTimestampPartitionNotSet() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, - Optional.empty(), Optional.empty(),TimePartitioning.Type.DAY); + Optional.empty(), Optional.empty(), Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -184,7 +202,7 @@ public void testUpdateTimestampPartitionNotSet() { Optional updateField = Optional.of("testUpdateField"); schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), updateField, Optional.empty(), Optional.empty(), - TimePartitioning.Type.DAY); + Optional.of(TimePartitioning.Type.DAY)); tableInfo = schemaManager .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); @@ -198,7 +216,7 @@ public void testPartitionExpirationSetWithoutFieldName() { Optional testExpirationMs = Optional.of(86400000L); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), Optional.empty(), - testExpirationMs, Optional.empty(), TimePartitioning.Type.DAY); + testExpirationMs, Optional.empty(), Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -222,7 +240,7 @@ public void testPartitionExpirationSetWithFieldName() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, - testExpirationMs, Optional.empty(), TimePartitioning.Type.DAY); + testExpirationMs, Optional.empty(), Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -247,7 +265,7 @@ public void testClusteringPartitionSet() { Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, - Optional.empty(), testField, TimePartitioning.Type.DAY); + Optional.empty(), testField, Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -270,7 +288,7 @@ public void testUpdateClusteringPartitionNull() { Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, - Optional.empty(), testField, TimePartitioning.Type.DAY); + Optional.empty(), testField, Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -290,7 +308,7 @@ public void testUpdateClusteringPartitionNotSet() { Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, - Optional.empty(), testField, TimePartitioning.Type.DAY); + Optional.empty(), testField, Optional.of(TimePartitioning.Type.DAY)); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -309,7 +327,7 @@ public void testUpdateClusteringPartitionNotSet() { Optional> updateTestField = Optional.of(Arrays.asList("column3", "column4")); schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, - Optional.empty(), updateTestField, TimePartitioning.Type.DAY); + Optional.empty(), updateTestField, Optional.of(TimePartitioning.Type.DAY)); tableInfo = schemaManager .constructTableInfo(tableId, fakeBigQuerySchema, testDoc, false); @@ -572,7 +590,7 @@ private SchemaManager createSchemaManager( return new SchemaManager(new IdentitySchemaRetriever(), mockSchemaConverter, mockBigQuery, allowNewFields, allowFieldRelaxation, allowUnionization, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), - TimePartitioning.Type.DAY); + Optional.of(TimePartitioning.Type.DAY)); } private void testGetAndValidateProposedSchema( diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index a86905d74..06ad832db 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -20,6 +20,7 @@ package com.wepay.kafka.connect.bigquery.config; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import com.google.cloud.bigquery.TimePartitioning; @@ -33,6 +34,7 @@ import org.junit.Test; import java.util.Map; +import java.util.Optional; public class BigQuerySinkConfigTest { private SinkPropertiesFactory propertiesFactory; @@ -88,8 +90,14 @@ public void testValidTimePartitioningTypes() { for (TimePartitioning.Type type : TimePartitioning.Type.values()) { configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, type.name()); - assertEquals(type, new BigQuerySinkConfig(configProperties).getTimePartitioningType()); + Optional timePartitioningType = new BigQuerySinkConfig(configProperties).getTimePartitioningType(); + assertTrue(timePartitioningType.isPresent()); + assertEquals(type, timePartitioningType.get()); } + + configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, BigQuerySinkConfig.TIME_PARTITIONING_TYPE_NONE); + Optional timePartitioningType = new BigQuerySinkConfig(configProperties).getTimePartitioningType(); + assertFalse(timePartitioningType.isPresent()); } @Test(expected = ConfigException.class) From 8f163b06f54caf5299af78417041947cfc6c5eba Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 5 May 2021 15:54:29 -0400 Subject: [PATCH 077/190] MINOR: Remove outdated properties (#107) These properties haven't been used by the connector for a long time; we should remove them from the code base in order to avoid confusing users. --- kcbq-connector/quickstart/properties/connector.properties | 4 ---- 1 file changed, 4 deletions(-) diff --git a/kcbq-connector/quickstart/properties/connector.properties b/kcbq-connector/quickstart/properties/connector.properties index 50c9211ae..9f2d245b5 100644 --- a/kcbq-connector/quickstart/properties/connector.properties +++ b/kcbq-connector/quickstart/properties/connector.properties @@ -30,10 +30,6 @@ autoUpdateSchemas=true schemaRetriever=com.wepay.kafka.connect.bigquery.schemaregistry.schemaretriever.SchemaRegistrySchemaRetriever schemaRegistryLocation=http://localhost:8081 -bufferSize=100000 -maxWriteSize=10000 -tableWriteWait=1000 - ########################################### Fill me in! ########################################### # The name of the BigQuery project to write to project= From 533feafa0ad0cc8979857357d4bf68c16bb40f95 Mon Sep 17 00:00:00 2001 From: "Nathan (Dae Hyun) Nam" Date: Thu, 6 May 2021 09:09:54 -0700 Subject: [PATCH 078/190] Update supportSummary for BigQuery Sink (#108) Update supportSummary for BigQuery Sink based on https://confluentinc.atlassian.net/wiki/spaces/CONNECT/pages/845677094/Support+terms+for+connectors+in+Confluent+Hub --- kcbq-connector/pom.xml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index c502b96c7..2900a8695 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -172,9 +172,7 @@ https://github.com/confluentinc/kafka-connect-bigquery Confluent, Inc. - supports WePay's BigQuery connector version 1.1.2 and later, as part of a Confluent Platform subscription. - ]]> + supported by Confluent as part of a Confluent Platform subscription.]]> https://docs.confluent.io/kafka-connect-bigquery/current/index.html logos/confluent.png From a672fcd88b511de599d9af6c785a9402279b62fc Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 11 May 2021 10:36:49 -0400 Subject: [PATCH 079/190] GH-65: Add support for Kafka logical Time type (#112) --- .../logicaltype/KafkaLogicalConverters.java | 22 ++++++++++++++ .../logicaltype/LogicalTypeConverter.java | 8 ++--- .../KafkaLogicalConvertersTest.java | 29 +++++++++++++++++++ 3 files changed, 55 insertions(+), 4 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConverters.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConverters.java index a6d918c12..6d3685ac5 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConverters.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConverters.java @@ -24,6 +24,7 @@ import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Time; import org.apache.kafka.connect.data.Timestamp; import java.math.BigDecimal; @@ -37,6 +38,7 @@ public class KafkaLogicalConverters { LogicalConverterRegistry.register(Date.LOGICAL_NAME, new DateConverter()); LogicalConverterRegistry.register(Decimal.LOGICAL_NAME, new DecimalConverter()); LogicalConverterRegistry.register(Timestamp.LOGICAL_NAME, new TimestampConverter()); + LogicalConverterRegistry.register(Time.LOGICAL_NAME, new TimeConverter()); } /** @@ -96,4 +98,24 @@ public String convert(Object kafkaConnectObject) { return getBqTimestampFormat().format((java.util.Date) kafkaConnectObject); } } + + + /** + * Class for converting Kafka time logical types to BigQuery time types. + */ + public static class TimeConverter extends LogicalTypeConverter { + /** + * Create a new TimestampConverter. + */ + public TimeConverter() { + super(Time.LOGICAL_NAME, + Schema.Type.INT32, + LegacySQLTypeName.TIME); + } + + @Override + public String convert(Object kafkaConnectObject) { + return getBqTimeFormat().format((java.util.Date) kafkaConnectObject); + } + } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalTypeConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalTypeConverter.java index 1815daede..9adaa330b 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalTypeConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/LogicalTypeConverter.java @@ -88,10 +88,10 @@ protected static SimpleDateFormat getBqTimestampFormat() { return bqTimestampFormat; } - protected static SimpleDateFormat getBQDatetimeFormat() { - SimpleDateFormat bqDateTimeFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS"); - bqDateTimeFormat.setTimeZone(utcTimeZone); - return bqDateTimeFormat; + protected SimpleDateFormat getBqTimeFormat() { + SimpleDateFormat bqTimestampFormat = new SimpleDateFormat("HH:mm:ss.SSS"); + bqTimestampFormat.setTimeZone(utcTimeZone); + return bqTimestampFormat; } protected static SimpleDateFormat getBQDateFormat() { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConvertersTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConvertersTest.java index 43dc1d1ea..5eb72902f 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConvertersTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/KafkaLogicalConvertersTest.java @@ -27,6 +27,7 @@ import com.wepay.kafka.connect.bigquery.convert.logicaltype.KafkaLogicalConverters.DateConverter; import com.wepay.kafka.connect.bigquery.convert.logicaltype.KafkaLogicalConverters.DecimalConverter; import com.wepay.kafka.connect.bigquery.convert.logicaltype.KafkaLogicalConverters.TimestampConverter; +import com.wepay.kafka.connect.bigquery.convert.logicaltype.KafkaLogicalConverters.TimeConverter; import org.apache.kafka.connect.data.Schema; @@ -101,4 +102,32 @@ public void testTimestampConversion() { assertEquals("2017-03-01 22:20:38.808", formattedTimestamp); } + + + @Test + public void testTimeConversion() { + TimeConverter converter = new KafkaLogicalConverters.TimeConverter(); + + assertEquals(LegacySQLTypeName.TIME, converter.getBQSchemaType()); + + try { + converter.checkEncodingType(Schema.Type.INT32); + } catch (Exception ex) { + fail("Expected encoding type check to succeed."); + } + + try { + converter.checkEncodingType(Schema.Type.INT64); + fail("Expected encoding type check to fail"); + } catch (Exception ex) { + // continue + } + + // Can't use the same timestamp here as the one in other tests as the Time type + // should only fall on January 1st, 1970 + Date date = new Date(166838808); + String formattedTimestamp = converter.convert(date); + + assertEquals("22:20:38.808", formattedTimestamp); + } } From 8ca2178e56e4bd196ac076fb8d9906e1876c2c80 Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 8 Jun 2021 10:44:02 -0700 Subject: [PATCH 080/190] Prevent records with null value schema from causing NPE when delete is enabled (#117) * Skip schema conversion if kafka schema is empty. * Test converting tombstone. * Check for null value schema and return empty. * Remove unused imports. * Fix test. * Throw on null value schema if unionzation is not enabled. * Update tests. * Revert unnecessary change. * Update kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java Co-authored-by: Chris Egerton Co-authored-by: Chris Egerton --- .../kafka/connect/bigquery/SchemaManager.java | 10 +++++ .../connect/bigquery/SchemaManagerTest.java | 38 +++++++++++++++++++ .../convert/BigQuerySchemaConverterTest.java | 1 - 3 files changed, 48 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 262d68a6a..4e2bb779f 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -495,6 +495,16 @@ TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema big } private com.google.cloud.bigquery.Schema getBigQuerySchema(Schema kafkaKeySchema, Schema kafkaValueSchema) { + // When converting schema from the last record of a batch, if the record is a tombstone with + // null value schema, only allow it to pass if schema unionization is enabled. + if (kafkaValueSchema == null) { + if (!allowSchemaUnionization) { + throw new BigQueryConnectException( + "Cannot create/update BigQuery table for record with no value schema. " + + "If delete mode is enabled, it may be necessary to enable schema unionization to handle this case."); + } + return com.google.cloud.bigquery.Schema.of(); + } com.google.cloud.bigquery.Schema valueSchema = schemaConverter.convertSchema(kafkaValueSchema); List schemaFields = intermediateTables diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index b4c5ee511..1ac066418 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -32,6 +32,7 @@ import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TableInfo; +import com.google.common.collect.ImmutableList; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; @@ -384,6 +385,43 @@ public void FieldsWithUnspecifiedModeShouldNotCauseNpe() { testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expectedSchema); } + @Test(expected = BigQueryConnectException.class) + public void testDisallowedUnionizedUpdateWithTombstoneRecord() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + Table existingTable = tableWithSchema(existingSchema); + List incomingSinkRecords = ImmutableList.of(recordWithValueSchema(null)); + + when(mockBigQuery.getTable(tableId)).thenReturn(existingTable); + SchemaManager schemaManager = createSchemaManager(false, true, false); + schemaManager.getAndValidateProposedSchema(tableId, incomingSinkRecords); + } + + @Test + public void testAllowedUnionizedUpdateWithTombstoneRecord() { + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() + ); + + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build() + ); + + Table existingTable = tableWithSchema(existingSchema); + SinkRecord tombstone = recordWithValueSchema(null); + List incomingSinkRecords = ImmutableList.of(tombstone); + + when(mockBigQuery.getTable(tableId)).thenReturn(existingTable); + + SchemaManager schemaManager = createSchemaManager(false, true, true); + com.google.cloud.bigquery.Schema proposedSchema = + schemaManager.getAndValidateProposedSchema(tableId, incomingSinkRecords); + + Assert.assertEquals(expectedSchema, proposedSchema); + } + private SchemaManager createSchemaManager( boolean allowNewFields, boolean allowFieldRelaxation, boolean allowUnionization) { return new SchemaManager(new IdentitySchemaRetriever(), mockSchemaConverter, mockBigQuery, diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java index 9185fb724..d9d3ccab4 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java @@ -25,7 +25,6 @@ import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.LegacySQLTypeName; -import com.google.common.collect.ImmutableList; import com.wepay.kafka.connect.bigquery.exception.ConversionConnectException; import org.apache.kafka.connect.data.Date; From 32b3ed0f22676b1849efd48ab98887b3ba9488a7 Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 8 Jun 2021 18:37:48 +0000 Subject: [PATCH 081/190] [maven-release-plugin] prepare release v2.0.5 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index b2a83ed9f..4a58b334c 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.5-SNAPSHOT + 2.0.5 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index aa09c4daa..0cec1b716 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.5-SNAPSHOT + 2.0.5 .. diff --git a/pom.xml b/pom.xml index bd6863daf..156f06a75 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.5-SNAPSHOT + 2.0.5 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.5 From 71bc96759215b90cde9d179f9c8ffc83c31a3535 Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 8 Jun 2021 18:37:54 +0000 Subject: [PATCH 082/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 4a58b334c..e277320da 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.5 + 2.0.6-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 0cec1b716..bb1b9e7ae 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.5 + 2.0.6-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 156f06a75..109182830 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.5 + 2.0.6-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.5 + HEAD From a8d468821ca279b7dc7c3747804611c550ca29ea Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 8 Jun 2021 18:58:15 +0000 Subject: [PATCH 083/190] [maven-release-plugin] prepare release v2.1.4 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 782b172af..4686309ea 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.4-SNAPSHOT + 2.1.4 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 2900a8695..729c12010 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.4-SNAPSHOT + 2.1.4 .. diff --git a/pom.xml b/pom.xml index 8d30158ff..2b4afe145 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.4-SNAPSHOT + 2.1.4 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.4 From 4ed52454627270dc3a88900988a67e28b7d6a7b4 Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 8 Jun 2021 18:58:21 +0000 Subject: [PATCH 084/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 4686309ea..52790b20b 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.4 + 2.1.5-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 729c12010..78c53c1e1 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.4 + 2.1.5-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 2b4afe145..624280feb 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.4 + 2.1.5-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.4 + 2.1.x From eb1c150db51c2c66837cbcf1f1d7b4f481857e10 Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 17 Aug 2021 09:17:45 -0700 Subject: [PATCH 085/190] Skip record with null valueSchema when proposing new BigQuery schema. (#123) * Skip tombstone records when converting to BQ schema, and try using existing if batch are all tombstone. * Guard against null valueSchema when getting unionized table description. * Test records combinations. --- .../kafka/connect/bigquery/SchemaManager.java | 48 +++++++++--- .../connect/bigquery/SchemaManagerTest.java | 74 +++++++++++++------ 2 files changed, 88 insertions(+), 34 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 4e2bb779f..3e84dbd56 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -283,7 +283,16 @@ com.google.cloud.bigquery.Schema getAndValidateProposedSchema( result = getUnionizedSchema(bigQuerySchemas); } else { com.google.cloud.bigquery.Schema existingSchema = readTableSchema(table); - result = convertRecordSchema(records.get(records.size() - 1)); + SinkRecord recordToConvert = getRecordToConvert(records); + if (recordToConvert == null) { + String errorMessage = "Could not convert to BigQuery schema with a batch of tombstone records."; + if (existingSchema == null) { + throw new BigQueryConnectException(errorMessage); + } + logger.debug(errorMessage + " Will fall back to existing schema."); + return existingSchema; + } + result = convertRecordSchema(recordToConvert); if (existingSchema != null) { validateSchemaChange(existingSchema, result); if (allowBQRequiredFieldRelaxation) { @@ -304,11 +313,32 @@ private List getSchemasList(TableId table, Lis List bigQuerySchemas = new ArrayList<>(); Optional.ofNullable(readTableSchema(table)).ifPresent(bigQuerySchemas::add); for (SinkRecord record : records) { + Schema kafkaValueSchema = schemaRetriever.retrieveValueSchema(record); + if (kafkaValueSchema == null) { + continue; + } bigQuerySchemas.add(convertRecordSchema(record)); } return bigQuerySchemas; } + /** + * Gets a regular record from the given batch of SinkRecord for schema conversion. This is needed + * when delete is enabled, because a tombstone record has null value, thus null value schema. + * Converting null value schema to BigQuery schema is not possible. + * @param records List of SinkRecord to look for. + * @return a regular record or null if the whole batch are all tombstone records. + */ + private SinkRecord getRecordToConvert(List records) { + for (int i = records.size() - 1; i >= 0; i--) { + SinkRecord record = records.get(i); + if (schemaRetriever.retrieveValueSchema(record) != null) { + return record; + } + } + return null; + } + private com.google.cloud.bigquery.Schema convertRecordSchema(SinkRecord record) { Schema kafkaValueSchema = schemaRetriever.retrieveValueSchema(record); Schema kafkaKeySchema = kafkaKeyFieldName.isPresent() ? schemaRetriever.retrieveKeySchema(record) : null; @@ -431,10 +461,14 @@ private com.google.cloud.bigquery.Schema relaxFieldsWhereNecessary( * @param records The records used to get the unionized table description * @return The resulting table description */ - private String getUnionizedTableDescription(List records) { + @VisibleForTesting + String getUnionizedTableDescription(List records) { String tableDescription = null; for (SinkRecord record : records) { Schema kafkaValueSchema = schemaRetriever.retrieveValueSchema(record); + if (kafkaValueSchema == null) { + continue; + } tableDescription = kafkaValueSchema.doc() != null ? kafkaValueSchema.doc() : tableDescription; } return tableDescription; @@ -495,16 +529,6 @@ TableInfo constructTableInfo(TableId table, com.google.cloud.bigquery.Schema big } private com.google.cloud.bigquery.Schema getBigQuerySchema(Schema kafkaKeySchema, Schema kafkaValueSchema) { - // When converting schema from the last record of a batch, if the record is a tombstone with - // null value schema, only allow it to pass if schema unionization is enabled. - if (kafkaValueSchema == null) { - if (!allowSchemaUnionization) { - throw new BigQueryConnectException( - "Cannot create/update BigQuery table for record with no value schema. " - + "If delete mode is enabled, it may be necessary to enable schema unionization to handle this case."); - } - return com.google.cloud.bigquery.Schema.of(); - } com.google.cloud.bigquery.Schema valueSchema = schemaConverter.convertSchema(kafkaValueSchema); List schemaFields = intermediateTables diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index 1ac066418..7b63e5464 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -38,8 +38,11 @@ import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; +import java.util.Random; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.junit.Assert; import org.junit.Before; @@ -385,41 +388,62 @@ public void FieldsWithUnspecifiedModeShouldNotCauseNpe() { testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expectedSchema); } - @Test(expected = BigQueryConnectException.class) - public void testDisallowedUnionizedUpdateWithTombstoneRecord() { + @Test + public void testUpdateWithOnlyTombstoneRecordsAndExistingSchema() { com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() ); - Table existingTable = tableWithSchema(existingSchema); - List incomingSinkRecords = ImmutableList.of(recordWithValueSchema(null)); + SchemaManager schemaManager = createSchemaManager(true, false, false); + List incomingSinkRecords = Collections.nCopies(2, recordWithValueSchema(null)); + // Tombstone records are skipped, and existing schema is reused. + testGetAndValidateProposedSchema(schemaManager, existingSchema, + Collections.singletonList(existingSchema), existingSchema, incomingSinkRecords); + } - when(mockBigQuery.getTable(tableId)).thenReturn(existingTable); - SchemaManager schemaManager = createSchemaManager(false, true, false); - schemaManager.getAndValidateProposedSchema(tableId, incomingSinkRecords); + @Test(expected = BigQueryConnectException.class) + public void testUpdateWithOnlyTombstoneRecordsNoExistingSchema() { + SchemaManager schemaManager = createSchemaManager(true, false, false); + List incomingSinkRecords = Collections.nCopies(2, recordWithValueSchema(null)); + testGetAndValidateProposedSchema( + schemaManager, null, Collections.singletonList(null), null, incomingSinkRecords); } @Test - public void testAllowedUnionizedUpdateWithTombstoneRecord() { + public void testUpdateWithRegularAndTombstoneRecords() { com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build() ); - com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( - Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.NULLABLE).build() + com.google.cloud.bigquery.Schema expandedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", LegacySQLTypeName.INTEGER).setMode(Field.Mode.NULLABLE).build() ); - Table existingTable = tableWithSchema(existingSchema); - SinkRecord tombstone = recordWithValueSchema(null); - List incomingSinkRecords = ImmutableList.of(tombstone); - - when(mockBigQuery.getTable(tableId)).thenReturn(existingTable); + SchemaManager schemaManager = createSchemaManager(true, false, false); + // Put tombstone at the end of the batch. + List incomingSinkRecords = ImmutableList.of( + recordWithValueSchema(mockKafkaSchema), recordWithValueSchema(null)); + // Tombstone record is skipped when converting schema. + testGetAndValidateProposedSchema(schemaManager, existingSchema, + Collections.singletonList(expandedSchema), expandedSchema, incomingSinkRecords); + } + @Test + public void testGetUnionizedTableDescriptionFromTombstoneRecord() { SchemaManager schemaManager = createSchemaManager(false, true, true); - com.google.cloud.bigquery.Schema proposedSchema = - schemaManager.getAndValidateProposedSchema(tableId, incomingSinkRecords); + SinkRecord tombstone = recordWithValueSchema(null); + List incomingSinkRecords = ImmutableList.of(tombstone); + Assert.assertNull(schemaManager.getUnionizedTableDescription(incomingSinkRecords)); + } - Assert.assertEquals(expectedSchema, proposedSchema); + @Test + public void testGetUnionizedTableDescriptionFromRegularAndNullRecords() { + SchemaManager schemaManager = createSchemaManager(false, true, true).forIntermediateTables(); + List incomingSinkRecords = ImmutableList.of( + recordWithValueSchema(mockKafkaSchema), recordWithValueSchema(null)); + when(mockKafkaSchema.doc()).thenReturn(testDoc); + Assert.assertNotNull(schemaManager.getUnionizedTableDescription(incomingSinkRecords)); } private SchemaManager createSchemaManager( @@ -443,11 +467,17 @@ private void testGetAndValidateProposedSchema( com.google.cloud.bigquery.Schema existingSchema, List newSchemas, com.google.cloud.bigquery.Schema expectedSchema) { - Table existingTable = existingSchema != null ? tableWithSchema(existingSchema) : null; - - SinkRecord mockSinkRecord = recordWithValueSchema(mockKafkaSchema); - List incomingSinkRecords = Collections.nCopies(newSchemas.size(), mockSinkRecord); + testGetAndValidateProposedSchema(schemaManager, existingSchema, newSchemas, expectedSchema, + Collections.nCopies(newSchemas.size(), recordWithValueSchema(mockKafkaSchema))); + } + private void testGetAndValidateProposedSchema( + SchemaManager schemaManager, + com.google.cloud.bigquery.Schema existingSchema, + List newSchemas, + com.google.cloud.bigquery.Schema expectedSchema, + List incomingSinkRecords) { + Table existingTable = existingSchema != null ? tableWithSchema(existingSchema) : null; when(mockBigQuery.getTable(tableId)).thenReturn(existingTable); OngoingStubbing converterStub = when(mockSchemaConverter.convertSchema(mockKafkaSchema)); From bf9553b5abcabc55cd7bb54c11c0c7dd602d429c Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 17 Aug 2021 21:00:04 +0000 Subject: [PATCH 086/190] [maven-release-plugin] prepare release v2.0.6 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index e277320da..9c52a402b 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.6-SNAPSHOT + 2.0.6 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index bb1b9e7ae..578299783 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.6-SNAPSHOT + 2.0.6 .. diff --git a/pom.xml b/pom.xml index 109182830..6eaece4d6 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.6-SNAPSHOT + 2.0.6 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.6 From 5754b8c77de2a8074c27847c025e53f6ef7436e5 Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 17 Aug 2021 21:00:07 +0000 Subject: [PATCH 087/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 9c52a402b..5dd593154 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.6 + 2.0.7-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 578299783..757d58272 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.6 + 2.0.7-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 6eaece4d6..03c147d89 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.6 + 2.0.7-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.6 + HEAD From 5005951b061b94ed1f85f23fd7792e2599fdffcc Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 17 Aug 2021 21:11:27 +0000 Subject: [PATCH 088/190] [maven-release-plugin] prepare release v2.1.5 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 52790b20b..9b7ab8c6d 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.5-SNAPSHOT + 2.1.5 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 78c53c1e1..328bc95f0 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.5-SNAPSHOT + 2.1.5 .. diff --git a/pom.xml b/pom.xml index 624280feb..171cd387d 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.5-SNAPSHOT + 2.1.5 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.5 From 0bc75421d79769d090b1d6db3d85a66784060e78 Mon Sep 17 00:00:00 2001 From: Yilong Chang Date: Tue, 17 Aug 2021 21:11:29 +0000 Subject: [PATCH 089/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 9b7ab8c6d..e73de3aeb 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.5 + 2.1.6-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 328bc95f0..c8437e451 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.5 + 2.1.6-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 171cd387d..2c9bf4868 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.5 + 2.1.6-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.5 + 2.1.x From e57d616d4b264ea3e7c08b094f3c1551a29c2c2f Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Wed, 25 Aug 2021 09:37:59 -0700 Subject: [PATCH 090/190] CCDB-4187: Recursively unionize subfields (#121) --- .../kafka/connect/bigquery/SchemaManager.java | 71 ++++++++-- .../connect/bigquery/SchemaManagerTest.java | 132 +++++++++++++++++- 2 files changed, 188 insertions(+), 15 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 3e84dbd56..d839b77d3 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -24,6 +24,7 @@ import com.google.cloud.bigquery.BigQueryException; import com.google.cloud.bigquery.Clustering; import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.Field.Mode; import com.google.cloud.bigquery.LegacySQLTypeName; import com.google.cloud.bigquery.StandardTableDefinition; import com.google.cloud.bigquery.TableId; @@ -50,6 +51,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import static com.google.common.base.Preconditions.checkState; + /** * Class for managing Schemas of BigQuery tables (creating and updating). */ @@ -362,13 +365,46 @@ private com.google.cloud.bigquery.Schema getUnionizedSchema(List firstSubFields = subFields(firstField); + Map secondSubFields = subFields(secondField); + Map unionizedSubFields = new LinkedHashMap<>(); + + firstSubFields.forEach((name, firstSubField) -> { + Field secondSubField = secondSubFields.get(name); + unionizedSubFields.put(name, unionizeFields(firstSubField, secondSubField)); + }); + maybeAddToUnionizedFields(secondSubFields, unionizedSubFields); + retBuilder.setType(LegacySQLTypeName.RECORD, + unionizedSubFields.values().toArray(new Field[]{})); + } + return retBuilder.build(); + } + /** * Returns a single unionized BigQuery schema from two BigQuery schemas. * @param firstSchema The first BigQuery schema to unionize * @param secondSchema The second BigQuery schema to unionize * @return The resulting unionized BigQuery schema */ - private com.google.cloud.bigquery.Schema unionizeSchemas( + // VisibleForTesting + com.google.cloud.bigquery.Schema unionizeSchemas( com.google.cloud.bigquery.Schema firstSchema, com.google.cloud.bigquery.Schema secondSchema) { Map firstSchemaFields = schemaFields(firstSchema); Map secondSchemaFields = schemaFields(secondSchema); @@ -383,24 +419,27 @@ private com.google.cloud.bigquery.Schema unionizeSchemas( } else { unionizedSchemaFields.put(name, firstField); } - } else if (isFieldRelaxation(firstField, secondField)) { - unionizedSchemaFields.put(name, secondField); } else { - unionizedSchemaFields.put(name, firstField); + unionizedSchemaFields.put(name, unionizeFields(firstField, secondField)); } }); + maybeAddToUnionizedFields(secondSchemaFields, unionizedSchemaFields); + return com.google.cloud.bigquery.Schema.of(unionizedSchemaFields.values()); + } + + private void maybeAddToUnionizedFields(Map secondSchemaFields, + Map unionizedFields) { secondSchemaFields.forEach((name, secondField) -> { - if (!unionizedSchemaFields.containsKey(name)) { - if (Field.Mode.REPEATED.equals(secondField.getMode())) { + if (!unionizedFields.containsKey(name)) { + if (Mode.REPEATED.equals(secondField.getMode())) { // Repeated fields are implicitly nullable; no need to set a new mode for them - unionizedSchemaFields.put(name, secondField); + unionizedFields.put(name, secondField); } else { - unionizedSchemaFields.put(name, secondField.toBuilder().setMode(Field.Mode.NULLABLE).build()); + unionizedFields.put(name, secondField.toBuilder().setMode(Mode.NULLABLE).build()); } } }); - return com.google.cloud.bigquery.Schema.of(unionizedSchemaFields.values()); } private void validateSchemaChange( @@ -474,6 +513,20 @@ String getUnionizedTableDescription(List records) { return tableDescription; } + private Map subFields(Field parent) { + Map result = new LinkedHashMap<>(); + if (parent == null || parent.getSubFields() == null) { + return result; + } + parent.getSubFields().forEach(field -> { + if (field.getMode() == null) { + field = field.toBuilder().setMode(Mode.NULLABLE).build(); + } + result.put(field.getName(), field); + }); + return result; + } + /** * Returns a dictionary providing lookup of each field in the schema by name. The ordering of the * fields in the schema is preserved in the returned map. diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index 7b63e5464..0edb567bf 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -19,6 +19,7 @@ package com.wepay.kafka.connect.bigquery; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -49,10 +50,12 @@ import org.junit.Test; import org.mockito.stubbing.OngoingStubbing; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; public class SchemaManagerTest { @@ -91,7 +94,7 @@ public void testBQTableDescription() { TableInfo tableInfo = schemaManager .constructTableInfo(tableId, fakeBigQuerySchema, testDoc); - Assert.assertEquals("Kafka doc does not match BigQuery table description", + assertEquals("Kafka doc does not match BigQuery table description", testDoc, tableInfo.getDescription()); Assert.assertNull("Timestamp partition field name is not null", ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning().getField()); @@ -109,9 +112,9 @@ public void testTimestampPartitionSet() { TableInfo tableInfo = schemaManager .constructTableInfo(tableId, fakeBigQuerySchema, testDoc); - Assert.assertEquals("Kafka doc does not match BigQuery table description", + assertEquals("Kafka doc does not match BigQuery table description", testDoc, tableInfo.getDescription()); - Assert.assertEquals("The field name does not match the field name of time partition", + assertEquals("The field name does not match the field name of time partition", testField.get(), ((StandardTableDefinition) tableInfo.getDefinition()).getTimePartitioning().getField()); } @@ -129,11 +132,11 @@ public void testClusteringPartitionSet() { TableInfo tableInfo = schemaManager .constructTableInfo(tableId, fakeBigQuerySchema, testDoc); - Assert.assertEquals("Kafka doc does not match BigQuery table description", + assertEquals("Kafka doc does not match BigQuery table description", testDoc, tableInfo.getDescription()); StandardTableDefinition definition = tableInfo.getDefinition(); Assert.assertNotNull(definition.getClustering()); - Assert.assertEquals("The field name does not match the field name of time partition", + assertEquals("The field name does not match the field name of time partition", testField.get(), definition.getClustering().getFields()); } @@ -491,7 +494,7 @@ private void testGetAndValidateProposedSchema( schemaManager.getAndValidateProposedSchema(tableId, incomingSinkRecords); if (expectedSchema != null) { - Assert.assertEquals(expectedSchema, proposedSchema); + assertEquals(expectedSchema, proposedSchema); } } @@ -510,4 +513,121 @@ private SinkRecord recordWithValueSchema(Schema valueSchema) { when(result.valueSchema()).thenReturn(valueSchema); return result; } + + @Test + public void testUnionizeSchemaNoNestedOrRepeatedRecords() { + com.google.cloud.bigquery.Schema s1 = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.BYTES.name(), LegacySQLTypeName.BYTES), + Field.of(LegacySQLTypeName.STRING.name(), LegacySQLTypeName.STRING), + Field.of(LegacySQLTypeName.DATE.name(), LegacySQLTypeName.DATE) + ); + com.google.cloud.bigquery.Schema s2 = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.TIMESTAMP.name(), LegacySQLTypeName.TIMESTAMP), + Field.of(LegacySQLTypeName.FLOAT.name(), LegacySQLTypeName.FLOAT) + ); + + List expectedFields = new ArrayList<>(); + expectedFields.addAll(s1.getFields()); + expectedFields.addAll(s2.getFields()); + + assertUnion(makeNullable(com.google.cloud.bigquery.Schema.of(expectedFields)), s1, s2); + } + + @Test + public void testUnionizeSchemaWithNestedRecords() { + com.google.cloud.bigquery.Schema s1 = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.of(LegacySQLTypeName.STRING.name(), LegacySQLTypeName.STRING), + Field.of(LegacySQLTypeName.DATE.name(), LegacySQLTypeName.DATE) + )); + com.google.cloud.bigquery.Schema s2 = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.of(LegacySQLTypeName.TIMESTAMP.name(), LegacySQLTypeName.TIMESTAMP) + )); + com.google.cloud.bigquery.Schema expected = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.of(LegacySQLTypeName.STRING.name(), LegacySQLTypeName.STRING), + Field.of(LegacySQLTypeName.DATE.name(), LegacySQLTypeName.DATE), + Field.of(LegacySQLTypeName.TIMESTAMP.name(), LegacySQLTypeName.TIMESTAMP) + ) + ); + assertUnion(makeNullable(expected), s1, s2); + } + + @Test + public void testUnionizeSchemaWithNestedAndRepeatedFields() { + com.google.cloud.bigquery.Schema s1 = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.newBuilder(LegacySQLTypeName.STRING.name(), LegacySQLTypeName.STRING).setMode(Mode.REPEATED).build(), + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.of(LegacySQLTypeName.BYTES.name(), LegacySQLTypeName.BYTES) + ) + ) + ); + com.google.cloud.bigquery.Schema s2 = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.newBuilder(LegacySQLTypeName.DATE.name(), LegacySQLTypeName.DATE).setMode(Mode.REPEATED).build(), + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.of(LegacySQLTypeName.FLOAT.name(), LegacySQLTypeName.FLOAT) + ) + ) + ); + + com.google.cloud.bigquery.Schema expected = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.newBuilder(LegacySQLTypeName.STRING.name(), LegacySQLTypeName.STRING).setMode(Mode.REPEATED).build(), + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.of(LegacySQLTypeName.BYTES.name(), LegacySQLTypeName.BYTES), + Field.of(LegacySQLTypeName.FLOAT.name(), LegacySQLTypeName.FLOAT) + ), + Field.newBuilder(LegacySQLTypeName.DATE.name(), LegacySQLTypeName.DATE).setMode(Mode.REPEATED).build() + ) + ); + assertUnion(makeNullable(expected), s1, s2); + } + + @Test + public void testUnionizeSchemaNestedRelax() { + com.google.cloud.bigquery.Schema s1 = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.newBuilder(LegacySQLTypeName.STRING.name(), LegacySQLTypeName.STRING).setMode(Mode.REQUIRED).build() + ) + ); + com.google.cloud.bigquery.Schema s2 = com.google.cloud.bigquery.Schema.of( + Field.of(LegacySQLTypeName.RECORD.name(), LegacySQLTypeName.RECORD, + Field.newBuilder(LegacySQLTypeName.STRING.name(), LegacySQLTypeName.STRING).setMode(Mode.NULLABLE).build() + ) + ); + assertUnion(makeNullable(s2), s1, s2); + } + + private com.google.cloud.bigquery.Schema makeNullable(com.google.cloud.bigquery.Schema s) { + return com.google.cloud.bigquery.Schema.of( + s.getFields().stream() + .map(this::makeNullable) + .collect(Collectors.toList()) + ); + } + + private Field makeNullable(Field f) { + Field.Builder builder = f.toBuilder(); + if (f.getSubFields() != null) { + List subFields = f.getSubFields().stream() + .map(this::makeNullable) + .collect(Collectors.toList()); + builder.setType(LegacySQLTypeName.RECORD, subFields.toArray(new Field[]{})).build(); + } + return builder + .setMode(f.getMode() == Mode.REPEATED ? Mode.REPEATED : Mode.NULLABLE) + .build(); + } + + private void assertUnion(com.google.cloud.bigquery.Schema expected, + com.google.cloud.bigquery.Schema schema1, com.google.cloud.bigquery.Schema schema2) { + SchemaManager sm = createSchemaManager(true, true, true); + assertEquals( + expected, sm.unionizeSchemas(schema1, schema2) + ); + } + } From c2b66b673812756c0761a2c916572da6e62c2606 Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Wed, 25 Aug 2021 17:01:56 +0000 Subject: [PATCH 091/190] [maven-release-plugin] prepare release v2.0.7 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 5dd593154..ae8552606 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.7-SNAPSHOT + 2.0.7 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 757d58272..bc6015320 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.7-SNAPSHOT + 2.0.7 .. diff --git a/pom.xml b/pom.xml index 03c147d89..12d0af2c0 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.7-SNAPSHOT + 2.0.7 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.7 From de170bf0647b6e39fda342ce7e29d075599e18bc Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Wed, 25 Aug 2021 17:01:58 +0000 Subject: [PATCH 092/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index ae8552606..0567f9e58 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.7 + 2.0.8-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index bc6015320..c26908015 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.7 + 2.0.8-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 12d0af2c0..7ee28cd91 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.7 + 2.0.8-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.7 + HEAD From 06ece834ddcb81d1c1ecf38f923fa3bd57db0257 Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Wed, 25 Aug 2021 17:03:53 +0000 Subject: [PATCH 093/190] [maven-release-plugin] prepare release v2.1.6 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index e73de3aeb..603df317d 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.6-SNAPSHOT + 2.1.6 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index c8437e451..5eacdc4d4 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.6-SNAPSHOT + 2.1.6 .. diff --git a/pom.xml b/pom.xml index 2c9bf4868..896c337fb 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.6-SNAPSHOT + 2.1.6 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.6 From 14db6edf88c485a06b75275f27eec2f4f06a5c32 Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Wed, 25 Aug 2021 17:03:56 +0000 Subject: [PATCH 094/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 603df317d..31986bedb 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.6 + 2.1.7-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 5eacdc4d4..7d106a0f5 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.6 + 2.1.7-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 896c337fb..d32ab68f6 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.6 + 2.1.7-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.6 + 2.1.x From ec09e3e42d3be7cb72e1b525eacc237045d969f0 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 2 Sep 2021 16:20:14 +0000 Subject: [PATCH 095/190] [maven-release-plugin] prepare release v1.6.7 --- kcbq-api/pom.xml | 2 +- kcbq-confluent/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 12817236b..f30c77e6d 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.7-SNAPSHOT + 1.6.7 .. diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index c0f9f24e9..a7722583b 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.7-SNAPSHOT + 1.6.7 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index b4412b492..135f88413 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.7-SNAPSHOT + 1.6.7 .. diff --git a/pom.xml b/pom.xml index e01f9954d..3e88a4461 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 1.6.7-SNAPSHOT + 1.6.7 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v1.6.7 From 0986d116cf3de0a94fa40a8b3dac4964e9e6c8a0 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 2 Sep 2021 16:20:17 +0000 Subject: [PATCH 096/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-confluent/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index f30c77e6d..64748796c 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.7 + 1.6.8-SNAPSHOT .. diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index a7722583b..6dd426d4a 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.7 + 1.6.8-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 135f88413..869c5e63a 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.7 + 1.6.8-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 3e88a4461..efe178081 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 1.6.7 + 1.6.8-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v1.6.7 + HEAD From bc769d38dc4f7dcc94d4baca28789300f45f2e67 Mon Sep 17 00:00:00 2001 From: rerorero Date: Wed, 8 Sep 2021 01:49:46 +0900 Subject: [PATCH 097/190] Use https://packages.confluent.io/ instaed of http (#129) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 825f5ae80..eb691c7b8 100644 --- a/pom.xml +++ b/pom.xml @@ -102,7 +102,7 @@ confluent - http://packages.confluent.io/maven/ + https://packages.confluent.io/maven/ jcenter @@ -113,7 +113,7 @@ confluent - http://packages.confluent.io/maven/ + https://packages.confluent.io/maven/ jcenter From 85b8de31c231b804ad29826ce39cbc18a8143b6b Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 8 Sep 2021 14:02:01 -0400 Subject: [PATCH 098/190] Register TimestampConverter in DebeziumLogicalConverters. (#137) Co-authored-by: Bingqin Zhou --- .../bigquery/convert/logicaltype/DebeziumLogicalConverters.java | 1 + 1 file changed, 1 insertion(+) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java index 1ffef1601..abf35c938 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java @@ -47,6 +47,7 @@ public class DebeziumLogicalConverters { LogicalConverterRegistry.register(MicroTimestamp.SCHEMA_NAME, new MicroTimestampConverter()); LogicalConverterRegistry.register(Time.SCHEMA_NAME, new TimeConverter()); LogicalConverterRegistry.register(ZonedTimestamp.SCHEMA_NAME, new ZonedTimestampConverter()); + LogicalConverterRegistry.register(Timestamp.SCHEMA_NAME, new TimestampConverter()); } private static final int MICROS_IN_SEC = 1000000; From 7232276761912aef0a7c5b093d7625875138b981 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 9 Sep 2021 13:52:38 +0000 Subject: [PATCH 099/190] [maven-release-plugin] prepare release v1.6.8 --- kcbq-api/pom.xml | 2 +- kcbq-confluent/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 64748796c..bb806333d 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.8-SNAPSHOT + 1.6.8 .. diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index 6dd426d4a..88b6260d3 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.8-SNAPSHOT + 1.6.8 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 869c5e63a..c9e864180 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.8-SNAPSHOT + 1.6.8 .. diff --git a/pom.xml b/pom.xml index efe178081..4c5462000 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 1.6.8-SNAPSHOT + 1.6.8 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v1.6.8 From 98c7c625f2f65314beea368b46715405b0dc0bfd Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 9 Sep 2021 13:52:41 +0000 Subject: [PATCH 100/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-confluent/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index bb806333d..b771a52fb 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.8 + 1.6.9-SNAPSHOT .. diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index 88b6260d3..9d58bef55 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.8 + 1.6.9-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index c9e864180..4932c5908 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.8 + 1.6.9-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 4c5462000..675043611 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 1.6.8 + 1.6.9-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v1.6.8 + HEAD From f105b82f5103ee5f0abeebfd70190c843b49d497 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 9 Sep 2021 13:54:14 +0000 Subject: [PATCH 101/190] [maven-release-plugin] prepare release v2.0.8 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 0567f9e58..ba15dd9ce 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.8-SNAPSHOT + 2.0.8 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index c26908015..bffdcd072 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.8-SNAPSHOT + 2.0.8 .. diff --git a/pom.xml b/pom.xml index 7ee28cd91..44fe24e4c 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.8-SNAPSHOT + 2.0.8 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.8 From af217bb4262c780205ee4a761bf8cf2394dfb4b5 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 9 Sep 2021 13:54:16 +0000 Subject: [PATCH 102/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index ba15dd9ce..20ebf0253 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.8 + 2.0.9-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index bffdcd072..c82f7e2bf 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.8 + 2.0.9-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 44fe24e4c..24c0891ee 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.8 + 2.0.9-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.8 + HEAD From ffb194d2af6c3a1446a1fc42b93fb7258580bf0b Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 9 Sep 2021 13:55:33 +0000 Subject: [PATCH 103/190] [maven-release-plugin] prepare release v2.1.7 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 31986bedb..cf217f758 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.7-SNAPSHOT + 2.1.7 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 7d106a0f5..0d525ef9c 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.7-SNAPSHOT + 2.1.7 .. diff --git a/pom.xml b/pom.xml index d32ab68f6..bd1d78598 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.7-SNAPSHOT + 2.1.7 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.7 From 1a6f02139a3d4a89d32f6b3d3eaba90997913aac Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 9 Sep 2021 13:55:36 +0000 Subject: [PATCH 104/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index cf217f758..92a76d8f3 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.7 + 2.1.8-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 0d525ef9c..8edb6665f 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.7 + 2.1.8-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index bd1d78598..622a3cd4c 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.7 + 2.1.8-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.7 + 2.1.x From d2ff27c03a45951961afbe621613ce1a43c46e37 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 22 Sep 2021 10:14:29 -0400 Subject: [PATCH 105/190] GH-144: Isolate error classification logic into single, testable class (#145) * GH-144: Isolate error classification logic into single, testable class * GH-144: Fix failing test, remove unused import --- .../bigquery/write/batch/TableWriter.java | 38 +-- .../write/row/AdaptiveBigQueryWriter.java | 21 +- .../write/row/BigQueryErrorResponses.java | 137 +++++++++++ .../bigquery/write/row/BigQueryWriter.java | 35 +-- .../integration/BigQueryErrorResponsesIT.java | 218 ++++++++++++++++++ .../write/row/BigQueryWriterTest.java | 6 +- 6 files changed, 381 insertions(+), 74 deletions(-) create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index f53a9ab0c..c9538d6bf 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -27,6 +27,7 @@ import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; +import com.wepay.kafka.connect.bigquery.write.row.BigQueryErrorResponses; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; import org.apache.kafka.connect.sink.SinkRecord; @@ -50,10 +51,6 @@ public class TableWriter implements Runnable { private static final Logger logger = LoggerFactory.getLogger(TableWriter.class); - private static final int BAD_REQUEST_CODE = 400; - private static final String INVALID_REASON = "invalid"; - private static final String PAYLOAD_TOO_LARGE_REASON = "Request payload size exceeds the limit:"; - private final BigQueryWriter writer; private final PartitionedTableId table; private final SortedMap rows; @@ -139,30 +136,15 @@ private static int getNewBatchSize(int currentBatchSize, Throwable err) { * size, or false otherwise. */ private static boolean isBatchSizeError(BigQueryException exception) { - if (exception.getCode() == BAD_REQUEST_CODE - && exception.getError() == null - && exception.getReason() == null) { - /* - * 400 with no error or reason represents a request that is more than 10MB. This is not - * documented but is referenced slightly under "Error codes" here: - * https://cloud.google.com/bigquery/quota-policy - * (by decreasing the batch size we can eventually expect to end up with a request under 10MB) - */ - return true; - } else if (exception.getCode() == BAD_REQUEST_CODE - && INVALID_REASON.equals(exception.getReason())) { - /* - * this is the error that the documentation claims google will return if a request exceeds - * 10MB. if this actually ever happens... - * todo distinguish this from other invalids (like invalid table schema). - */ - return true; - } else if (exception.getCode() == BAD_REQUEST_CODE - && exception.getMessage() != null - && exception.getMessage().contains(PAYLOAD_TOO_LARGE_REASON)) { - return true; - } - return false; + /* + * 400 with no error or reason represents a request that is more than 10MB. This is not + * documented but is referenced slightly under "Error codes" here: + * https://cloud.google.com/bigquery/quota-policy + * (by decreasing the batch size we can eventually expect to end up with a request under 10MB) + */ + return BigQueryErrorResponses.isUnspecifiedBadRequestError(exception) + || BigQueryErrorResponses.isRequestTooLargeError(exception) + || BigQueryErrorResponses.isTooManyRowsError(exception); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java index c5d2d2d60..e38920050 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java @@ -40,7 +40,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.SortedMap; /** @@ -76,18 +75,6 @@ public AdaptiveBigQueryWriter(BigQuery bigQuery, this.autoCreateTables = autoCreateTables; } - private boolean isTableMissingSchema(BigQueryException exception) { - // If a table is missing a schema, it will raise a BigQueryException that the input is invalid - // For more information about BigQueryExceptions, see: https://cloud.google.com/bigquery/troubleshooting-errors - return exception.getReason() != null && exception.getReason().equalsIgnoreCase("invalid"); - } - - private boolean isTableNotExistedException(BigQueryException exception) { - // If a table does not exist, it will raise a BigQueryException that the input is notFound - // Referring to Google Cloud Error Codes Doc: https://cloud.google.com/bigquery/docs/error-messages?hl=en - return exception.getCode() == 404; - } - /** * Sends the request to BigQuery, then checks the response to see if any errors have occurred. If * any have, and all errors can be blamed upon invalid columns in the rows sent, attempts to @@ -111,9 +98,9 @@ && onlyContainsInvalidSchemaErrors(writeResponse.getInsertErrors())) { } } catch (BigQueryException exception) { // Should only perform one table creation attempt. - if (isTableNotExistedException(exception) && autoCreateTables) { + if (BigQueryErrorResponses.isNonExistentTableError(exception) && autoCreateTables) { attemptTableCreate(tableId.getBaseTableId(), new ArrayList<>(rows.keySet())); - } else if (isTableMissingSchema(exception)) { + } else if (BigQueryErrorResponses.isTableMissingSchemaError(exception)) { attemptSchemaUpdate(tableId, new ArrayList<>(rows.keySet())); } else { throw exception; @@ -186,9 +173,9 @@ private boolean onlyContainsInvalidSchemaErrors(Map> e boolean invalidSchemaError = false; for (List errorList : errors.values()) { for (BigQueryError error : errorList) { - if (error.getReason().equals("invalid") && (error.getMessage().contains("no such field") || error.getMessage().contains("Missing required field"))) { + if (BigQueryErrorResponses.isMissingRequiredFieldError(error) || BigQueryErrorResponses.isUnrecognizedFieldError(error)) { invalidSchemaError = true; - } else if (!error.getReason().equals("stopped")) { + } else if (!BigQueryErrorResponses.isStoppedError(error)) { /* if some rows are in the old schema format, and others aren't, the old schema * formatted rows will show up as error: stopped. We still want to continue if this is * the case, because these errors don't represent a unique error if there are also diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java new file mode 100644 index 000000000..4e5f2062a --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java @@ -0,0 +1,137 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.write.row; + +import com.google.cloud.bigquery.BigQueryError; +import com.google.cloud.bigquery.BigQueryException; + +import java.util.Optional; +import java.util.function.Function; + +/** + * Handles the logic for classifying BigQuery error responses and determining things like whether they come from an + * invalid schema error, a backend error, etc. This can be used to determine whether a table needs to be created before + * retrying an insert or if a temporary server-side error requires us to retry a request, for example. + */ +public class BigQueryErrorResponses { + + private static final int BAD_REQUEST_CODE = 400; + private static final int FORBIDDEN_CODE = 403; + private static final int NOT_FOUND_CODE = 404; + private static final int INTERNAL_SERVICE_ERROR_CODE = 500; + private static final int BAD_GATEWAY_CODE = 502; + private static final int SERVICE_UNAVAILABLE_CODE = 503; + + private static final String BAD_REQUEST_REASON = "badRequest"; + private static final String INVALID_REASON = "invalid"; + private static final String NOT_FOUND_REASON = "notFound"; + private static final String QUOTA_EXCEEDED_REASON = "quotaExceeded"; + private static final String RATE_LIMIT_EXCEEDED_REASON = "rateLimitExceeded"; + private static final String STOPPED_REASON = "stopped"; + + + public static boolean isNonExistentTableError(BigQueryException exception) { + // If a table does not exist, it will raise a BigQueryException that the input is notFound + // Referring to Google Cloud Error Codes Doc: https://cloud.google.com/bigquery/docs/error-messages?hl=en + return NOT_FOUND_CODE == exception.getCode() + && NOT_FOUND_REASON.equals(exception.getReason()) + && message(exception.getError()).startsWith("Not found: Table "); + } + + public static boolean isTableMissingSchemaError(BigQueryException exception) { + // If a table is missing a schema, it will raise a BigQueryException that the input is invalid + // For more information about BigQueryExceptions, see: https://cloud.google.com/bigquery/troubleshooting-errors + return BAD_REQUEST_CODE == exception.getCode() + && INVALID_REASON.equals(exception.getReason()) + && message(exception.getError()).equals("The destination table has no schema."); + } + + public static boolean isBackendError(BigQueryException exception) { + // backend error: https://cloud.google.com/bigquery/troubleshooting-errors + // for BAD_GATEWAY: https://cloud.google.com/storage/docs/json_api/v1/status-codes + // TODO: possibly this page is inaccurate for bigquery, but the message we are getting + // suggest it's an internal backend error and we should retry, so lets take that at face + // value + return INTERNAL_SERVICE_ERROR_CODE == exception.getCode() + || BAD_GATEWAY_CODE == exception.getCode() + || SERVICE_UNAVAILABLE_CODE == exception.getCode(); + } + + public static boolean isUnspecifiedBadRequestError(BigQueryException exception) { + return BAD_REQUEST_CODE == exception.getCode() + && exception.getError() == null + && exception.getReason() == null; + } + + public static boolean isQuotaExceededError(BigQueryException exception) { + return FORBIDDEN_CODE == exception.getCode() + // TODO: May be able to use exception.getReason() instead of (indirectly) exception.getError().getReason() + // Haven't been able to test yet though, so keeping as-is to avoid breaking anything + && QUOTA_EXCEEDED_REASON.equals(reason(exception.getError())); + } + + public static boolean isRateLimitExceededError(BigQueryException exception) { + return FORBIDDEN_CODE == exception.getCode() + // TODO: May be able to use exception.getReason() instead of (indirectly) exception.getError().getReason() + // Haven't been able to test yet though, so keeping as-is to avoid breaking anything + && RATE_LIMIT_EXCEEDED_REASON.equals(reason(exception.getError())); + } + + public static boolean isRequestTooLargeError(BigQueryException exception) { + return BAD_REQUEST_CODE == exception.getCode() + && BAD_REQUEST_REASON.equals(exception.getReason()) + && message(exception.getError()).startsWith("Request payload size exceeds the limit: "); + } + + public static boolean isTooManyRowsError(BigQueryException exception) { + return BAD_REQUEST_CODE == exception.getCode() + && INVALID_REASON.equalsIgnoreCase(exception.getReason()) + && message(exception.getError()).startsWith("too many rows present in the request"); + } + + public static boolean isUnrecognizedFieldError(BigQueryError error) { + return INVALID_REASON.equals(reason(error)) + && message(error).startsWith("no such field: "); + } + + public static boolean isMissingRequiredFieldError(BigQueryError error) { + return INVALID_REASON.equals(reason(error)) + && message(error).startsWith("Missing required field: "); + } + + public static boolean isStoppedError(BigQueryError error) { + return STOPPED_REASON.equals(reason(error)) + && message(error).equals(""); + } + + private static String reason(BigQueryError error) { + return extractFromError(error, BigQueryError::getReason); + } + + private static String message(BigQueryError error) { + return extractFromError(error, BigQueryError::getMessage); + } + + private static String extractFromError(BigQueryError error, Function extraction) { + return Optional.ofNullable(error) + .map(extraction) + .orElse(""); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java index c562dba68..032ad62d4 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java @@ -43,21 +43,13 @@ */ public abstract class BigQueryWriter { - private static final int FORBIDDEN = 403; - private static final int INTERNAL_SERVICE_ERROR = 500; - private static final int BAD_GATEWAY = 502; - private static final int SERVICE_UNAVAILABLE = 503; - private static final String QUOTA_EXCEEDED_REASON = "quotaExceeded"; - private static final String RATE_LIMIT_EXCEEDED_REASON = "rateLimitExceeded"; - private static final int WAIT_MAX_JITTER = 1000; private static final Logger logger = LoggerFactory.getLogger(BigQueryWriter.class); - private static final Random random = new Random(); - - private int retries; - private long retryWaitMs; + private final int retries; + private final long retryWaitMs; + private final Random random; /** * @param retries the number of times to retry a request if BQ returns an internal service error @@ -68,6 +60,8 @@ public abstract class BigQueryWriter { public BigQueryWriter(int retries, long retryWaitMs) { this.retries = retries; this.retryWaitMs = retryWaitMs; + + this.random = new Random(); } /** @@ -132,26 +126,13 @@ public void writeRows(PartitionedTableId table, } } catch (BigQueryException err) { mostRecentException = err; - if (err.getCode() == INTERNAL_SERVICE_ERROR - || err.getCode() == SERVICE_UNAVAILABLE - || err.getCode() == BAD_GATEWAY) { - // backend error: https://cloud.google.com/bigquery/troubleshooting-errors - /* for BAD_GATEWAY: https://cloud.google.com/storage/docs/json_api/v1/status-codes - todo possibly this page is inaccurate for bigquery, but the message we are getting - suggest it's an internal backend error and we should retry, so lets take that at face - value. */ + if (BigQueryErrorResponses.isBackendError(err)) { logger.warn("BQ backend error: {}, attempting retry", err.getCode()); retryCount++; - } else if (err.getCode() == FORBIDDEN - && err.getError() != null - && QUOTA_EXCEEDED_REASON.equals(err.getReason())) { - // quota exceeded error + } else if (BigQueryErrorResponses.isQuotaExceededError(err)) { logger.warn("Quota exceeded for table {}, attempting retry", table); retryCount++; - } else if (err.getCode() == FORBIDDEN - && err.getError() != null - && RATE_LIMIT_EXCEEDED_REASON.equals(err.getReason())) { - // rate limit exceeded error + } else if (BigQueryErrorResponses.isRateLimitExceededError(err)) { logger.warn("Rate limit exceeded for table {}, attempting retry", table); retryCount++; } else { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java new file mode 100644 index 000000000..0511e06f7 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java @@ -0,0 +1,218 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.integration; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryError; +import com.google.cloud.bigquery.BigQueryException; +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.InsertAllRequest; +import com.google.cloud.bigquery.InsertAllResponse; +import com.google.cloud.bigquery.Schema; +import com.google.cloud.bigquery.StandardSQLTypeName; +import com.google.cloud.bigquery.StandardTableDefinition; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.TableInfo; +import com.wepay.kafka.connect.bigquery.integration.utils.TableClearer; +import com.wepay.kafka.connect.bigquery.write.row.BigQueryErrorResponses; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static com.google.cloud.bigquery.InsertAllRequest.RowToInsert; +import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.table; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class BigQueryErrorResponsesIT extends BaseConnectorIT { + + private static final Logger logger = LoggerFactory.getLogger(BigQueryErrorResponsesIT.class); + + private BigQuery bigQuery; + + @Before + public void setup() { + bigQuery = newBigQuery(); + } + + @Test + public void testWriteToNonExistentTable() { + TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("nonexistent table")); + TableClearer.clearTables(bigQuery, dataset(), table.getTable()); + + try { + bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(Collections.singletonMap("f1", "v1")))); + fail("Should have failed to write to nonexistent table"); + } catch (BigQueryException e) { + logger.debug("Nonexistent table write error", e); + assertTrue(BigQueryErrorResponses.isNonExistentTableError(e)); + } + } + + @Test + public void testWriteToTableWithoutSchema() { + TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("missing schema")); + createOrAssertSchemaMatches(table, Schema.of()); + + try { + bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(Collections.singletonMap("f1", "v1")))); + fail("Should have failed to write to table with no schema"); + } catch (BigQueryException e) { + logger.debug("Table missing schema write error", e); + assertTrue(BigQueryErrorResponses.isTableMissingSchemaError(e)); + } + } + + @Test + public void testWriteWithMissingRequiredFields() { + TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("too many fields")); + Schema schema = Schema.of( + Field.newBuilder("f1", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f2", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build(), + Field.newBuilder("f3", StandardSQLTypeName.BOOL).setMode(Field.Mode.NULLABLE).build() + ); + createOrAssertSchemaMatches(table, schema); + + InsertAllResponse response = bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(Collections.singletonMap("f2", 12L)))); + logger.debug("Write response errors for missing required field: {}", response.getInsertErrors()); + BigQueryError error = assertResponseHasSingleError(response); + assertTrue(BigQueryErrorResponses.isMissingRequiredFieldError(error)); + } + + @Test + public void testWriteWithUnrecognizedFields() { + TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("not enough fields")); + Schema schema = Schema.of( + Field.newBuilder("f1", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + createOrAssertSchemaMatches(table, schema); + + Map row = new HashMap<>(); + row.put("f1", "v1"); + row.put("f2", 12L); + InsertAllResponse response = bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(row))); + logger.debug("Write response errors for unrecognized field: {}", response.getInsertErrors()); + BigQueryError error = assertResponseHasSingleError(response); + assertTrue(BigQueryErrorResponses.isUnrecognizedFieldError(error)); + } + + @Test + public void testStoppedRowsDuringInvalidWrite() { + TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("not enough fields")); + Schema schema = Schema.of( + Field.newBuilder("f1", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + createOrAssertSchemaMatches(table, schema); + + Map row1 = new HashMap<>(); + row1.put("f1", "v1"); + row1.put("f2", 12L); + Map row2 = Collections.singletonMap("f1", "v2"); + InsertAllResponse response = bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(row1), RowToInsert.of(row2))); + logger.debug("Write response errors for unrecognized field and stopped row: {}", response.getInsertErrors()); + assertEquals(2, response.getInsertErrors().size()); + // As long as we have some kind of error on the first row it's fine; we want to be more precise in our assertions about the second row + assertListHasSingleElement(response.getErrorsFor(0)); + BigQueryError secondRowError = assertListHasSingleElement(response.getErrorsFor(1)); + assertTrue(BigQueryErrorResponses.isStoppedError(secondRowError)); + } + + @Test + public void testRequestPayloadTooLarge() { + TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("request payload too large")); + Schema schema = Schema.of( + Field.newBuilder("f1", StandardSQLTypeName.STRING).setMode(Field.Mode.REQUIRED).build() + ); + createOrAssertSchemaMatches(table, schema); + + char[] chars = new char[10 * 1024 * 1024]; + Arrays.fill(chars, '*'); + String columnValue = new String(chars); + try { + bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(Collections.singletonMap("f1", columnValue)))); + fail("Should have failed to write to table with 11MB request"); + } catch (BigQueryException e) { + logger.debug("Large request payload write error", e); + assertTrue(BigQueryErrorResponses.isRequestTooLargeError(e)); + } + } + + @Test + public void testTooManyRows() { + TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("too many rows")); + Schema schema = Schema.of( + Field.newBuilder("f1", StandardSQLTypeName.INT64).setMode(Field.Mode.REQUIRED).build() + ); + createOrAssertSchemaMatches(table, schema); + + Iterable rows = LongStream.range(0, 100_000) + .mapToObj(i -> Collections.singletonMap("f1", i)) + .map(RowToInsert::of) + .collect(Collectors.toList()); + try { + bigQuery.insertAll(InsertAllRequest.of(table, rows)); + fail("Should have failed to write to table with 100,000 rows"); + } catch (BigQueryException e) { + logger.debug("Too many rows write error", e); + assertTrue(BigQueryErrorResponses.isTooManyRowsError(e)); + } + } + + // Some tables can't be deleted, recreated, and written to without getting a temporary error from BigQuery, + // so we just create them once if they don't exist and don't delete them at the end of the test. + // If we detect a table left over (presumably from a prior test), we do a sanity check to make sure that it + // has the expected schema. + private void createOrAssertSchemaMatches(TableId tableId, Schema schema) { + Table table = bigQuery.getTable(tableId); + if (table == null) { + bigQuery.create(TableInfo.newBuilder(tableId, StandardTableDefinition.of(schema)).build()); + } else { + assertEquals( + String.format("Testing %s should be created automatically by tests; please delete the table and re-run this test", table(tableId)), + schema, + table.getDefinition().getSchema() + ); + } + } + + private BigQueryError assertResponseHasSingleError(InsertAllResponse response) { + assertEquals(1, response.getInsertErrors().size()); + Iterator> errorsIterator = response.getInsertErrors().values().iterator(); + assertTrue(errorsIterator.hasNext()); + return assertListHasSingleElement(errorsIterator.next()); + } + + private T assertListHasSingleElement(List list) { + assertEquals(1, list.size()); + return list.get(0); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index a6a90a423..3066df46b 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -122,9 +122,11 @@ public void testAutoCreateTables() { when(insertAllResponse.hasErrors()).thenReturn(false); when(insertAllResponse.getInsertErrors()).thenReturn(emptyMap); - BigQueryException missTableException = new BigQueryException(404, "Table is missing"); + String errorMessage = "Not found: Table project.scratch.test_topic"; + BigQueryError error = new BigQueryError("notFound", "global", errorMessage); + BigQueryException nonExistentTableException = new BigQueryException(404, errorMessage, error); - when(bigQuery.insertAll(anyObject())).thenThrow(missTableException).thenReturn(insertAllResponse); + when(bigQuery.insertAll(anyObject())).thenThrow(nonExistentTableException).thenReturn(insertAllResponse); SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); From 986995e2b8eb35b73a712d959a70b0f0e7f0e8fa Mon Sep 17 00:00:00 2001 From: Xiao Fu Date: Fri, 24 Sep 2021 07:53:36 -0700 Subject: [PATCH 106/190] adding table and column information to help debugging (#152) * adding table and column information to BigQueryConnectionException to help debugging * change log format --- .../connect/bigquery/exception/BigQueryConnectException.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java index 3da4a39ac..847b14542 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java @@ -53,8 +53,9 @@ private static String formatInsertAllErrors(Map> error for (Map.Entry> errorsEntry : errorsMap.entrySet()) { for (BigQueryError error : errorsEntry.getValue()) { messageBuilder.append(String.format( - "%n\t[row index %d]: %s: %s", + "%n\t[row index %d] (location %s, reason: %s): %s", errorsEntry.getKey(), + error.getLocation(), error.getReason(), error.getMessage() )); From d460013d66d8986ad9d3376a2e25d53e1b8cf6b5 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 27 Sep 2021 10:53:30 -0400 Subject: [PATCH 107/190] GH-146: Parameterize BigQuerySinkConnectorIT tests (#147) --- .../bigquery/integration/BaseConnectorIT.java | 5 +- .../integration/BigQuerySinkConnectorIT.java | 140 +++++++++++------- 2 files changed, 89 insertions(+), 56 deletions(-) diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java index 5035cc36a..64db82dbb 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -49,6 +49,7 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.AbstractStatus; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; @@ -116,13 +117,13 @@ protected void startConnect() { protected void stopConnect() { if (kafkaAdminClient != null) { - kafkaAdminClient.close(); + Utils.closeQuietly(kafkaAdminClient, "admin client for embedded Kafka cluster"); kafkaAdminClient = null; } // stop all Connect, Kafka and Zk threads. if (connect != null) { - connect.stop(); + Utils.closeQuietly(connect::stop, "embedded Connect, Kafka, and Zookeeper clusters"); connect = null; } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java index 9777837bb..16866e858 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQuerySinkConnectorIT.java @@ -36,13 +36,18 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.SinkConnectorConfig; import org.apache.kafka.test.IntegrationTest; import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.InputStream; import java.util.ArrayList; @@ -57,17 +62,18 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; +import static com.wepay.kafka.connect.bigquery.integration.BaseConnectorIT.boxByteArray; import static io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG; import static org.junit.Assert.assertEquals; @Category(IntegrationTest.class) -public class BigQuerySinkConnectorIT extends BaseConnectorIT { +@RunWith(Parameterized.class) +public class BigQuerySinkConnectorIT { - private static final Map>> TEST_CASES; - static { - Map>> testCases = new HashMap<>(); + @Parameterized.Parameters + public static Iterable testCases() { + Collection result = new ArrayList<>(); List> expectedGcsLoadRows = new ArrayList<>(); expectedGcsLoadRows.add(Arrays.asList( @@ -103,14 +109,14 @@ public class BigQuerySinkConnectorIT extends BaseConnectorIT { "nineteen", boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) )); - testCases.put("gcs-load", expectedGcsLoadRows); + result.add(new Object[] {"gcs-load", expectedGcsLoadRows}); List> expectedNullsRows = new ArrayList<>(); expectedNullsRows.add(Arrays.asList(1L, "Required string", null, 42L, false)); expectedNullsRows.add(Arrays.asList(2L, "Required string", "Optional string", 89L, null)); expectedNullsRows.add(Arrays.asList(3L, "Required string", null, null, true)); expectedNullsRows.add(Arrays.asList(4L, "Required string", "Optional string", null, null)); - testCases.put("nulls", expectedNullsRows); + result.add(new Object[] {"nulls", expectedNullsRows}); List> expectedMatryoshkaRows = new ArrayList<>(); expectedMatryoshkaRows.add(Arrays.asList( @@ -127,7 +133,7 @@ public class BigQuerySinkConnectorIT extends BaseConnectorIT { "-42" ) )); - testCases.put("matryoshka-dolls", expectedMatryoshkaRows); + result.add(new Object[] {"matryoshka-dolls", expectedMatryoshkaRows}); List> expectedPrimitivesRows = new ArrayList<>(); expectedPrimitivesRows.add(Arrays.asList( @@ -141,52 +147,75 @@ public class BigQuerySinkConnectorIT extends BaseConnectorIT { "forty-two", boxByteArray(new byte[] { 0x0, 0xf, 0x1E, 0x2D, 0x3C, 0x4B, 0x5A, 0x69, 0x78 }) )); - testCases.put("primitives", expectedPrimitivesRows); + result.add(new Object[] {"primitives", expectedPrimitivesRows}); List> expectedLogicalTypesRows = new ArrayList<>(); expectedLogicalTypesRows.add(Arrays.asList(1L, 0L, 0L)); expectedLogicalTypesRows.add(Arrays.asList(2L, 42000000000L, 362880000000L)); expectedLogicalTypesRows.add(Arrays.asList(3L, 1468275102000000L, 1468195200000L)); - testCases.put("logical-types", expectedLogicalTypesRows); + result.add(new Object[] {"logical-types", expectedLogicalTypesRows}); - TEST_CASES = Collections.unmodifiableMap(testCases); + return result; } private static final String TEST_CASE_PREFIX = "kcbq_test_"; - private static final Collection TEST_TOPICS = TEST_CASES.keySet().stream() - .map(tc -> TEST_CASE_PREFIX + tc) - .collect(Collectors.toList()); + // Share a single embedded Connect and Schema Registry cluster for all test cases to keep the runtime down + private static BaseConnectorIT testBase; + private static RestApp schemaRegistry; + private static String schemaRegistryUrl; + + private final String testCase; + private final List> expectedRows; + private final String topic; + private final String table; + private final String connectorName; - private RestApp restApp; - private String schemaRegistryUrl; private Producer valueProducer; private int numRecordsProduced; - @Before - public void setup() throws Exception { - Collection tables = TEST_TOPICS.stream() - .map(this::suffixedAndSanitizedTable) - .collect(Collectors.toSet()); - BucketClearer.clearBucket(keyFile(), project(), gcsBucket(), gcsFolder(), keySource()); - TableClearer.clearTables(newBigQuery(), dataset(), tables); - - startConnect(); - restApp = new RestApp( + public BigQuerySinkConnectorIT(String testCase, List> expectedRows) { + this.testCase = testCase; + this.expectedRows = expectedRows; + + this.topic = TEST_CASE_PREFIX + testCase; + this.table = testBase.suffixedAndSanitizedTable(topic); + this.connectorName = "bigquery-connector-" + testCase; + } + + @BeforeClass + public static void globalSetup() throws Exception { + testBase = new BaseConnectorIT() {}; + testBase.startConnect(); + + schemaRegistry = new RestApp( ClusterTestHarness.choosePort(), null, - connect.kafka().bootstrapServers(), + testBase.connect.kafka().bootstrapServers(), SchemaRegistryConfig.DEFAULT_KAFKASTORE_TOPIC, CompatibilityLevel.BACKWARD.name, true, null); - restApp.start(); + schemaRegistry.start(); - schemaRegistryUrl = restApp.restClient.getBaseUrls().current(); + schemaRegistryUrl = schemaRegistry.restClient.getBaseUrls().current(); + + BucketClearer.clearBucket( + testBase.keyFile(), + testBase.project(), + testBase.gcsBucket(), + testBase.gcsFolder(), + testBase.keySource() + ); + } + + @Before + public void setup() { + TableClearer.clearTables(testBase.newBigQuery(), testBase.dataset(), table); Map producerProps = new HashMap<>(); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, connect.kafka().bootstrapServers()); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, testBase.connect.kafka().bootstrapServers()); valueProducer = new KafkaProducer<>( producerProps, Serdes.ByteArray().serializer(), Serdes.ByteArray().serializer()); @@ -194,33 +223,36 @@ public void setup() throws Exception { } @After - public void close() throws Exception { - if (restApp != null) { - restApp.stop(); + public void cleanup() { + testBase.connect.deleteConnector(connectorName); + } + + @AfterClass + public static void globalCleanup() { + if (schemaRegistry != null) { + Utils.closeQuietly(schemaRegistry::stop, "embedded Schema Registry instance"); } - stopConnect(); + testBase.stopConnect(); } @Test - public void testAll() throws Exception { + public void runTestCase() throws Exception { final int tasksMax = 1; - final String connectorName = "bigquery-connector"; - TEST_CASES.keySet().forEach(this::populate); + populate(); - connect.configureConnector(connectorName, connectorProps(tasksMax)); + testBase.connect.configureConnector(connectorName, connectorProps(tasksMax)); - waitForConnectorToStart(connectorName, tasksMax); + testBase.waitForConnectorToStart(connectorName, tasksMax); - waitForCommittedRecords( - "bigquery-connector", TEST_TOPICS, numRecordsProduced, tasksMax, TimeUnit.MINUTES.toMillis(3)); + testBase.waitForCommittedRecords( + connectorName, Collections.singleton(topic), numRecordsProduced, tasksMax, TimeUnit.MINUTES.toMillis(3)); - TEST_CASES.forEach(this::verify); + verify(); } - private void populate(String testCase) { - String topic = TEST_CASE_PREFIX + testCase; - connect.kafka().createTopic(topic); + private void populate() { + testBase.connect.kafka().createTopic(topic); String testCaseDir = "integration_test_cases/" + testCase + "/"; @@ -251,7 +283,7 @@ private void populate(String testCase) { } private Map connectorProps(int tasksMax) { - Map result = baseConnectorProps(tasksMax); + Map result = testBase.baseConnectorProps(tasksMax); result.put( ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, @@ -266,17 +298,17 @@ private Map connectorProps(int tasksMax) { ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + "." + SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl); - result.put(SinkConnectorConfig.TOPICS_CONFIG, String.join(",", TEST_TOPICS)); + result.put(SinkConnectorConfig.TOPICS_CONFIG, topic); result.put(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG, "true"); result.put(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG, "true"); - result.put(BigQuerySinkConfig.ENABLE_BATCH_CONFIG, suffixedAndSanitizedTable("kcbq_test_gcs-load")); + result.put(BigQuerySinkConfig.ENABLE_BATCH_CONFIG, testBase.suffixedAndSanitizedTable("kcbq_test_gcs-load")); result.put(BigQuerySinkConfig.BATCH_LOAD_INTERVAL_SEC_CONFIG, "10"); - result.put(BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG, gcsBucket()); - result.put(BigQuerySinkConfig.GCS_FOLDER_NAME_CONFIG, gcsFolder()); + result.put(BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG, testBase.gcsBucket()); + result.put(BigQuerySinkConfig.GCS_FOLDER_NAME_CONFIG, testBase.gcsFolder()); result.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, IdentitySchemaRetriever.class.getName()); - String suffix = tableSuffix(); + String suffix = testBase.tableSuffix(); if (!suffix.isEmpty()) { String escapedSuffix = suffix.replaceAll("\\\\", "\\\\\\\\").replaceAll("\\$", "\\\\\\$"); result.put("transforms", "addSuffix"); @@ -288,11 +320,11 @@ private Map connectorProps(int tasksMax) { return result; } - private void verify(String testCase, List> expectedRows) { + private void verify() { List> testRows; try { - String table = suffixedAndSanitizedTable(TEST_CASE_PREFIX + FieldNameSanitizer.sanitizeName(testCase)); - testRows = readAllRows(newBigQuery(), table, "row"); + String table = testBase.suffixedAndSanitizedTable(TEST_CASE_PREFIX + FieldNameSanitizer.sanitizeName(testCase)); + testRows = testBase.readAllRows(testBase.newBigQuery(), table, "row"); } catch (InterruptedException e) { throw new RuntimeException(e); } From a3fa7c449c681c02c81d6963b30cc8a1bfcdf478 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 1 Nov 2021 10:25:57 -0400 Subject: [PATCH 108/190] GH-139: Move all user-visible configuration properties to single configuration class (#140) --- .../bigquery/BigQuerySinkConnector.java | 20 +- .../connect/bigquery/BigQuerySinkTask.java | 40 ++- .../bigquery/config/BigQuerySinkConfig.java | 232 ++++++++++++++++- .../config/BigQuerySinkTaskConfig.java | 239 +----------------- .../bigquery/BigQuerySinkConnectorTest.java | 6 +- .../bigquery/BigQuerySinkTaskTest.java | 27 +- .../SinkConnectorPropertiesFactory.java | 46 ---- .../bigquery/SinkPropertiesFactory.java | 9 +- .../bigquery/SinkTaskPropertiesFactory.java | 49 ---- .../config/BigQuerySinkConfigTest.java | 109 ++++++++ .../config/BigQuerySinkTaskConfigTest.java | 175 ------------- .../write/row/BigQueryWriterTest.java | 15 +- .../bigquery/write/row/GCSToBQWriterTest.java | 12 +- 13 files changed, 395 insertions(+), 584 deletions(-) delete mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkConnectorPropertiesFactory.java delete mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java delete mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index f4ed4eafc..d2fefb8fb 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -22,12 +22,9 @@ import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.TableId; -import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; - import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; -import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; - +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; @@ -47,7 +44,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; /** * A {@link SinkConnector} used to delegate BigQuery data writes to @@ -57,8 +53,6 @@ public class BigQuerySinkConnector extends SinkConnector { private final BigQuery testBigQuery; private final SchemaManager testSchemaManager; - public static final String GCS_BQ_TASK_CONFIG_KEY = "GCSBQTask"; - public BigQuerySinkConnector() { testBigQuery = null; testSchemaManager = null; @@ -84,16 +78,16 @@ public BigQuerySinkConnector() { @Override public ConfigDef config() { logger.trace("connector.config()"); - return config.getConfig(); + return BigQuerySinkConfig.getConfig(); } private BigQuery getBigQuery() { if (testBigQuery != null) { return testBigQuery; } - String projectName = config.getString(config.PROJECT_CONFIG); + String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); String key = config.getKeyFile(); - String keySource = config.getString(config.KEY_SOURCE_CONFIG); + String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); return new BigQueryHelper().setKeySource(keySource).connect(projectName, key); } @@ -104,7 +98,7 @@ private void ensureExistingTables() { if (bigQuery.getTable(tableId) == null) { logger.warn( "You may want to enable auto table creation by setting {}=true in the properties file", - config.TABLE_CREATE_CONFIG); + BigQuerySinkConfig.TABLE_CREATE_CONFIG); throw new BigQueryConnectException("Table '" + tableId + "' does not exist"); } } @@ -123,7 +117,7 @@ public void start(Map properties) { ); } - if (!config.getBoolean(config.TABLE_CREATE_CONFIG)) { + if (!config.getBoolean(BigQuerySinkConfig.TABLE_CREATE_CONFIG)) { ensureExistingTables(); } } @@ -148,7 +142,7 @@ public List> taskConfigs(int maxTasks) { HashMap taskConfig = new HashMap<>(configProperties); if (i == 0 && !config.getList(BigQuerySinkConfig.ENABLE_BATCH_CONFIG).isEmpty()) { // if batch loading is enabled, configure first task to do the GCS -> BQ loading - taskConfig.put(GCS_BQ_TASK_CONFIG_KEY, "true"); + taskConfig.put(BigQuerySinkTaskConfig.GCS_BQ_TASK_CONFIG, "true"); } taskConfigs.add(taskConfig); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 118007756..4f59c07d8 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -171,7 +171,7 @@ private RowToInsert getRecordRow(SinkRecord record) { if (kafkaDataFieldName.isPresent()) { convertedRecord.put(kafkaDataFieldName.get(), KafkaDataBuilder.buildKafkaDataRecord(record)); } - if (config.getBoolean(config.SANITIZE_FIELD_NAME_CONFIG)) { + if (config.getBoolean(BigQuerySinkConfig.SANITIZE_FIELD_NAME_CONFIG)) { convertedRecord = FieldNameSanitizer.replaceInvalidKeys(convertedRecord); } return RowToInsert.of(getRowId(record), convertedRecord); @@ -205,17 +205,17 @@ public void put(Collection records) { if (!tableWriterBuilders.containsKey(table)) { TableWriterBuilder tableWriterBuilder; - if (config.getList(config.ENABLE_BATCH_CONFIG).contains(record.topic())) { + if (config.getList(BigQuerySinkConfig.ENABLE_BATCH_CONFIG).contains(record.topic())) { String topic = record.topic(); String gcsBlobName = topic + "_" + uuid + "_" + Instant.now().toEpochMilli(); - String gcsFolderName = config.getString(config.GCS_FOLDER_NAME_CONFIG); + String gcsFolderName = config.getString(BigQuerySinkConfig.GCS_FOLDER_NAME_CONFIG); if (gcsFolderName != null && !"".equals(gcsFolderName)) { gcsBlobName = gcsFolderName + "/" + gcsBlobName; } tableWriterBuilder = new GCSBatchTableWriter.Builder( gcsToBQWriter, table.getBaseTableId(), - config.getString(config.GCS_BUCKET_NAME_CONFIG), + config.getString(BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG), gcsBlobName, topic, recordConverter); @@ -255,9 +255,9 @@ private BigQuery getBigQuery() { if (testBigQuery != null) { return testBigQuery; } - String projectName = config.getString(config.PROJECT_CONFIG); + String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); String keyFile = config.getKeyFile(); - String keySource = config.getString(config.KEY_SOURCE_CONFIG); + String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); return new BigQueryHelper().setKeySource(keySource).connect(projectName, keyFile); } @@ -277,10 +277,10 @@ private SchemaManager getSchemaManager(BigQuery bigQuery) { } private BigQueryWriter getBigQueryWriter() { - boolean autoUpdateSchemas = config.getBoolean(config.SCHEMA_UPDATE_CONFIG); - boolean autoCreateTables = config.getBoolean(config.TABLE_CREATE_CONFIG); - int retry = config.getInt(config.BIGQUERY_RETRY_CONFIG); - long retryWait = config.getLong(config.BIGQUERY_RETRY_WAIT_CONFIG); + boolean autoUpdateSchemas = config.getBoolean(BigQuerySinkConfig.SCHEMA_UPDATE_CONFIG); + boolean autoCreateTables = config.getBoolean(BigQuerySinkConfig.TABLE_CREATE_CONFIG); + int retry = config.getInt(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG); + long retryWait = config.getLong(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG); BigQuery bigQuery = getBigQuery(); if (autoUpdateSchemas || autoCreateTables) { return new AdaptiveBigQueryWriter(bigQuery, @@ -298,18 +298,18 @@ private Storage getGcs() { if (testGcs != null) { return testGcs; } - String projectName = config.getString(config.PROJECT_CONFIG); + String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); String key = config.getKeyFile(); - String keySource = config.getString(config.KEY_SOURCE_CONFIG); + String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); return new GCSBuilder(projectName).setKey(key).setKeySource(keySource).build(); } private GCSToBQWriter getGcsWriter() { BigQuery bigQuery = getBigQuery(); - int retry = config.getInt(config.BIGQUERY_RETRY_CONFIG); - long retryWait = config.getLong(config.BIGQUERY_RETRY_WAIT_CONFIG); - boolean autoCreateTables = config.getBoolean(config.TABLE_CREATE_CONFIG); + int retry = config.getInt(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG); + long retryWait = config.getLong(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG); + boolean autoCreateTables = config.getBoolean(BigQuerySinkConfig.TABLE_CREATE_CONFIG); // schemaManager shall only be needed for creating table hence do not fetch instance if not // needed. SchemaManager schemaManager = autoCreateTables ? getSchemaManager(bigQuery) : null; @@ -324,8 +324,6 @@ private GCSToBQWriter getGcsWriter() { @Override public void start(Map properties) { logger.trace("task.start()"); - final boolean hasGCSBQTask = - properties.remove(BigQuerySinkConnector.GCS_BQ_TASK_CONFIG_KEY) != null; try { config = new BigQuerySinkTaskConfig(properties); } catch (ConfigException err) { @@ -342,10 +340,10 @@ public void start(Map properties) { executor = new KCBQThreadPoolExecutor(config, new LinkedBlockingQueue<>()); topicPartitionManager = new TopicPartitionManager(); useMessageTimeDatePartitioning = - config.getBoolean(config.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG); + config.getBoolean(BigQuerySinkConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG); usePartitionDecorator = - config.getBoolean(config.BIGQUERY_PARTITION_DECORATOR_CONFIG); - if (hasGCSBQTask) { + config.getBoolean(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG); + if (config.getBoolean(BigQuerySinkTaskConfig.GCS_BQ_TASK_CONFIG)) { startGCSToBQLoadTask(); } } @@ -353,7 +351,7 @@ public void start(Map properties) { private void startGCSToBQLoadTask() { logger.info("Attempting to start GCS Load Executor."); gcsLoadExecutor = Executors.newScheduledThreadPool(1); - String bucketName = config.getString(config.GCS_BUCKET_NAME_CONFIG); + String bucketName = config.getString(BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG); Storage gcs = getGcs(); // get the bucket, or create it if it does not exist. Bucket bucket = gcs.get(bucketName); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 704288f59..c7cc02593 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -54,7 +54,6 @@ * Base class for connector and task configs; contains properties shared between the two of them. */ public class BigQuerySinkConfig extends AbstractConfig { - private static final ConfigDef config; private static final Validator validator = new Validator(); private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkConfig.class); @@ -234,8 +233,105 @@ public class BigQuerySinkConfig extends AbstractConfig { private static final String TABLE_CREATE_DOC = "Automatically create BigQuery tables if they don't already exist"; - static { - config = new ConfigDef() + public static final String SCHEMA_UPDATE_CONFIG = "autoUpdateSchemas"; + private static final ConfigDef.Type SCHEMA_UPDATE_TYPE = ConfigDef.Type.BOOLEAN; + public static final Boolean SCHEMA_UPDATE_DEFAULT = false; + private static final ConfigDef.Importance SCHEMA_UPDATE_IMPORTANCE = ConfigDef.Importance.HIGH; + private static final String SCHEMA_UPDATE_DOC = + "Whether or not to automatically update BigQuery schemas"; + + public static final String THREAD_POOL_SIZE_CONFIG = "threadPoolSize"; + private static final ConfigDef.Type THREAD_POOL_SIZE_TYPE = ConfigDef.Type.INT; + public static final Integer THREAD_POOL_SIZE_DEFAULT = 10; + private static final ConfigDef.Validator THREAD_POOL_SIZE_VALIDATOR = ConfigDef.Range.atLeast(1); + private static final ConfigDef.Importance THREAD_POOL_SIZE_IMPORTANCE = + ConfigDef.Importance.MEDIUM; + private static final String THREAD_POOL_SIZE_DOC = + "The size of the BigQuery write thread pool. This establishes the maximum number of " + + "concurrent writes to BigQuery."; + + public static final String QUEUE_SIZE_CONFIG = "queueSize"; + private static final ConfigDef.Type QUEUE_SIZE_TYPE = ConfigDef.Type.LONG; + // should this even have a default? + public static final Long QUEUE_SIZE_DEFAULT = -1L; + private static final ConfigDef.Validator QUEUE_SIZE_VALIDATOR = ConfigDef.Range.atLeast(-1); + private static final ConfigDef.Importance QUEUE_SIZE_IMPORTANCE = ConfigDef.Importance.HIGH; + private static final String QUEUE_SIZE_DOC = + "The maximum size (or -1 for no maximum size) of the worker queue for bigQuery write " + + "requests before all topics are paused. This is a soft limit; the size of the queue can " + + "go over this before topics are paused. All topics will be resumed once a flush is " + + "requested or the size of the queue drops under half of the maximum size."; + + public static final String BIGQUERY_RETRY_CONFIG = "bigQueryRetry"; + private static final ConfigDef.Type BIGQUERY_RETRY_TYPE = ConfigDef.Type.INT; + public static final Integer BIGQUERY_RETRY_DEFAULT = 0; + private static final ConfigDef.Validator BIGQUERY_RETRY_VALIDATOR = ConfigDef.Range.atLeast(0); + private static final ConfigDef.Importance BIGQUERY_RETRY_IMPORTANCE = + ConfigDef.Importance.MEDIUM; + private static final String BIGQUERY_RETRY_DOC = + "The number of retry attempts that will be made per BigQuery request that fails with a " + + "backend error or a quota exceeded error"; + + public static final String BIGQUERY_RETRY_WAIT_CONFIG = "bigQueryRetryWait"; + private static final ConfigDef.Type BIGQUERY_RETRY_WAIT_CONFIG_TYPE = ConfigDef.Type.LONG; + public static final Long BIGQUERY_RETRY_WAIT_DEFAULT = 1000L; + private static final ConfigDef.Validator BIGQUERY_RETRY_WAIT_VALIDATOR = + ConfigDef.Range.atLeast(0); + private static final ConfigDef.Importance BIGQUERY_RETRY_WAIT_IMPORTANCE = + ConfigDef.Importance.MEDIUM; + private static final String BIGQUERY_RETRY_WAIT_DOC = + "The minimum amount of time, in milliseconds, to wait between BigQuery backend or quota " + + "exceeded error retry attempts."; + + public static final String BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG = + "bigQueryMessageTimePartitioning"; + private static final ConfigDef.Type BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG_TYPE = + ConfigDef.Type.BOOLEAN; + public static final Boolean BIGQUERY_MESSAGE_TIME_PARTITIONING_DEFAULT = false; + private static final ConfigDef.Importance BIGQUERY_MESSAGE_TIME_PARTITIONING_IMPORTANCE = + ConfigDef.Importance.HIGH; + private static final String BIGQUERY_MESSAGE_TIME_PARTITIONING_DOC = + "Whether or not to use the message time when inserting records. " + + "Default uses the connector processing time."; + + public static final String BIGQUERY_PARTITION_DECORATOR_CONFIG = + "bigQueryPartitionDecorator"; + private static final ConfigDef.Type BIGQUERY_PARTITION_DECORATOR_CONFIG_TYPE = + ConfigDef.Type.BOOLEAN; + //This has been set to true to preserve the existing behavior. However, we can set it to false if field based partitioning is used in BigQuery + public static final Boolean BIGQUERY_PARTITION_DECORATOR_DEFAULT = true; + private static final ConfigDef.Importance BIGQUERY_PARTITION_DECORATOR_IMPORTANCE = + ConfigDef.Importance.HIGH; + private static final String BIGQUERY_PARTITION_DECORATOR_DOC = + "Whether or not to append partition decorator to BigQuery table name when inserting records. " + + "Default is true. Setting this to true appends partition decorator to table name (e.g. table$yyyyMMdd depending on the configuration set for bigQueryPartitionDecorator). " + + "Setting this to false bypasses the logic to append the partition decorator and uses raw table name for inserts."; + + public static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG = "timestampPartitionFieldName"; + private static final ConfigDef.Type BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE = ConfigDef.Type.STRING; + private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Importance BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE = + ConfigDef.Importance.LOW; + private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC = + "The name of the field in the value that contains the timestamp to partition by in BigQuery" + + " and enable timestamp partitioning for each table. Leave this configuration blank," + + " to enable ingestion time partitioning for each table."; + + public static final String BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG = "clusteringPartitionFieldNames"; + private static final ConfigDef.Type BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE = ConfigDef.Type.LIST; + private static final List BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT = null; + private static final ConfigDef.Importance BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE = + ConfigDef.Importance.LOW; + private static final String BIGQUERY_CLUSTERING_FIELD_NAMES_DOC = + "List of fields on which data should be clustered by in BigQuery, separated by commas"; + + /** + * Return the ConfigDef object used to define this config's fields. + * + * @return The ConfigDef object used to define this config's fields. + */ + public static ConfigDef getConfig() { + return new ConfigDef() .define( TOPICS_CONFIG, TOPICS_TYPE, @@ -366,6 +462,64 @@ public class BigQuerySinkConfig extends AbstractConfig { TABLE_CREATE_DEFAULT, TABLE_CREATE_IMPORTANCE, TABLE_CREATE_DOC + ).define( + SCHEMA_UPDATE_CONFIG, + SCHEMA_UPDATE_TYPE, + SCHEMA_UPDATE_DEFAULT, + SCHEMA_UPDATE_IMPORTANCE, + SCHEMA_UPDATE_DOC + ).define( + THREAD_POOL_SIZE_CONFIG, + THREAD_POOL_SIZE_TYPE, + THREAD_POOL_SIZE_DEFAULT, + THREAD_POOL_SIZE_VALIDATOR, + THREAD_POOL_SIZE_IMPORTANCE, + THREAD_POOL_SIZE_DOC + ).define( + QUEUE_SIZE_CONFIG, + QUEUE_SIZE_TYPE, + QUEUE_SIZE_DEFAULT, + QUEUE_SIZE_VALIDATOR, + QUEUE_SIZE_IMPORTANCE, + QUEUE_SIZE_DOC + ).define( + BIGQUERY_RETRY_CONFIG, + BIGQUERY_RETRY_TYPE, + BIGQUERY_RETRY_DEFAULT, + BIGQUERY_RETRY_VALIDATOR, + BIGQUERY_RETRY_IMPORTANCE, + BIGQUERY_RETRY_DOC + ).define( + BIGQUERY_RETRY_WAIT_CONFIG, + BIGQUERY_RETRY_WAIT_CONFIG_TYPE, + BIGQUERY_RETRY_WAIT_DEFAULT, + BIGQUERY_RETRY_WAIT_VALIDATOR, + BIGQUERY_RETRY_WAIT_IMPORTANCE, + BIGQUERY_RETRY_WAIT_DOC + ).define( + BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, + BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG_TYPE, + BIGQUERY_MESSAGE_TIME_PARTITIONING_DEFAULT, + BIGQUERY_MESSAGE_TIME_PARTITIONING_IMPORTANCE, + BIGQUERY_MESSAGE_TIME_PARTITIONING_DOC + ).define( + BIGQUERY_PARTITION_DECORATOR_CONFIG, + BIGQUERY_PARTITION_DECORATOR_CONFIG_TYPE, + BIGQUERY_PARTITION_DECORATOR_DEFAULT, + BIGQUERY_PARTITION_DECORATOR_IMPORTANCE, + BIGQUERY_PARTITION_DECORATOR_DOC + ).define( + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC + ).define( + BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, + BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE, + BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT, + BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE, + BIGQUERY_CLUSTERING_FIELD_NAMES_DOC ); } /** @@ -697,24 +851,80 @@ private void checkAutoCreateTables() { } } + private void checkAutoUpdateSchemas() { + Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); + + boolean autoUpdateSchemas = getBoolean(SCHEMA_UPDATE_CONFIG); + if (autoUpdateSchemas && schemaRetriever == null) { + throw new ConfigException( + "Cannot specify automatic table creation without a schema retriever" + ); + } + + if (schemaRetriever == null) { + logger.warn( + "No schema retriever class provided; auto schema updates are impossible" + ); + } + } + /** - * Return the ConfigDef object used to define this config's fields. - * - * @return The ConfigDef object used to define this config's fields. + * Returns the field name to use for timestamp partitioning. + * @return String that represents the field name. */ - public static ConfigDef getConfig() { - return config; + public Optional getTimestampPartitionFieldName() { + return Optional.ofNullable(getString(BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG)); + } + + /** + * Returns the field names to use for clustering. + * @return List of Strings that represent the field names. + */ + public Optional> getClusteringPartitionFieldName() { + return Optional.ofNullable(getList(BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG)); + } + + /** + * Check the validity of table partitioning configs. + */ + private void checkPartitionConfigs() { + if (getTimestampPartitionFieldName().isPresent() && getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { + throw new ConfigException( + "Only one partitioning configuration mode may be specified for the connector. " + + "Use either bigQueryPartitionDecorator OR timestampPartitionFieldName." + ); + } + } + + /** + * Check the validity of table clustering configs. + */ + private void checkClusteringConfigs() { + if (getClusteringPartitionFieldName().isPresent()) { + if (!getTimestampPartitionFieldName().isPresent() && !getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { + throw new ConfigException( + "Clustering field name may be specified only on a partitioned table." + ); + } + if (getClusteringPartitionFieldName().get().size() > 4) { + throw new ConfigException( + "You can only specify up to four clustering field names." + ); + } + } } protected BigQuerySinkConfig(ConfigDef config, Map properties) { super(config, properties); verifyBucketSpecified(); + checkAutoCreateTables(); + checkAutoUpdateSchemas(); + checkPartitionConfigs(); + checkClusteringConfigs(); } public BigQuerySinkConfig(Map properties) { - super(config, properties); - verifyBucketSpecified(); - checkAutoCreateTables(); + this(getConfig(), properties); } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java index 0eb3d9dc5..9a72561be 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfig.java @@ -19,13 +19,7 @@ package com.wepay.kafka.connect.bigquery.config; -import java.util.List; -import java.util.Optional; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Map; @@ -33,238 +27,21 @@ * Class for task-specific configuration properties. */ public class BigQuerySinkTaskConfig extends BigQuerySinkConfig { - private static final ConfigDef config; - private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkTaskConfig.class); - - public static final String SCHEMA_UPDATE_CONFIG = "autoUpdateSchemas"; - private static final ConfigDef.Type SCHEMA_UPDATE_TYPE = ConfigDef.Type.BOOLEAN; - public static final Boolean SCHEMA_UPDATE_DEFAULT = false; - private static final ConfigDef.Importance SCHEMA_UPDATE_IMPORTANCE = ConfigDef.Importance.HIGH; - private static final String SCHEMA_UPDATE_DOC = - "Whether or not to automatically update BigQuery schemas"; - - public static final String THREAD_POOL_SIZE_CONFIG = "threadPoolSize"; - private static final ConfigDef.Type THREAD_POOL_SIZE_TYPE = ConfigDef.Type.INT; - public static final Integer THREAD_POOL_SIZE_DEFAULT = 10; - private static final ConfigDef.Validator THREAD_POOL_SIZE_VALIDATOR = ConfigDef.Range.atLeast(1); - private static final ConfigDef.Importance THREAD_POOL_SIZE_IMPORTANCE = - ConfigDef.Importance.MEDIUM; - private static final String THREAD_POOL_SIZE_DOC = - "The size of the BigQuery write thread pool. This establishes the maximum number of " - + "concurrent writes to BigQuery."; - - public static final String QUEUE_SIZE_CONFIG = "queueSize"; - private static final ConfigDef.Type QUEUE_SIZE_TYPE = ConfigDef.Type.LONG; - // should this even have a default? - public static final Long QUEUE_SIZE_DEFAULT = -1L; - private static final ConfigDef.Validator QUEUE_SIZE_VALIDATOR = ConfigDef.Range.atLeast(-1); - private static final ConfigDef.Importance QUEUE_SIZE_IMPORTANCE = ConfigDef.Importance.HIGH; - private static final String QUEUE_SIZE_DOC = - "The maximum size (or -1 for no maximum size) of the worker queue for bigQuery write " - + "requests before all topics are paused. This is a soft limit; the size of the queue can " - + "go over this before topics are paused. All topics will be resumed once a flush is " - + "requested or the size of the queue drops under half of the maximum size."; - - public static final String BIGQUERY_RETRY_CONFIG = "bigQueryRetry"; - private static final ConfigDef.Type BIGQUERY_RETRY_TYPE = ConfigDef.Type.INT; - public static final Integer BIGQUERY_RETRY_DEFAULT = 0; - private static final ConfigDef.Validator BIGQUERY_RETRY_VALIDATOR = ConfigDef.Range.atLeast(0); - private static final ConfigDef.Importance BIGQUERY_RETRY_IMPORTANCE = - ConfigDef.Importance.MEDIUM; - private static final String BIGQUERY_RETRY_DOC = - "The number of retry attempts that will be made per BigQuery request that fails with a " - + "backend error or a quota exceeded error"; - - public static final String BIGQUERY_RETRY_WAIT_CONFIG = "bigQueryRetryWait"; - private static final ConfigDef.Type BIGQUERY_RETRY_WAIT_CONFIG_TYPE = ConfigDef.Type.LONG; - public static final Long BIGQUERY_RETRY_WAIT_DEFAULT = 1000L; - private static final ConfigDef.Validator BIGQUERY_RETRY_WAIT_VALIDATOR = - ConfigDef.Range.atLeast(0); - private static final ConfigDef.Importance BIGQUERY_RETRY_WAIT_IMPORTANCE = - ConfigDef.Importance.MEDIUM; - private static final String BIGQUERY_RETRY_WAIT_DOC = - "The minimum amount of time, in milliseconds, to wait between BigQuery backend or quota " - + "exceeded error retry attempts."; - - public static final String BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG = - "bigQueryMessageTimePartitioning"; - private static final ConfigDef.Type BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG_TYPE = - ConfigDef.Type.BOOLEAN; - public static final Boolean BIGQUERY_MESSAGE_TIME_PARTITIONING_DEFAULT = false; - private static final ConfigDef.Importance BIGQUERY_MESSAGE_TIME_PARTITIONING_IMPORTANCE = - ConfigDef.Importance.HIGH; - private static final String BIGQUERY_MESSAGE_TIME_PARTITIONING_DOC = - "Whether or not to use the message time when inserting records. " - + "Default uses the connector processing time."; - - public static final String BIGQUERY_PARTITION_DECORATOR_CONFIG = - "bigQueryPartitionDecorator"; - private static final ConfigDef.Type BIGQUERY_PARTITION_DECORATOR_CONFIG_TYPE = - ConfigDef.Type.BOOLEAN; - //This has been set to true to preserve the existing behavior. However, we can set it to false if field based partitioning is used in BigQuery - public static final Boolean BIGQUERY_PARTITION_DECORATOR_DEFAULT = true; - private static final ConfigDef.Importance BIGQUERY_PARTITION_DECORATOR_IMPORTANCE = - ConfigDef.Importance.HIGH; - private static final String BIGQUERY_PARTITION_DECORATOR_DOC = - "Whether or not to append partition decorator to BigQuery table name when inserting records. " - + "Default is true. Setting this to true appends partition decorator to table name (e.g. table$yyyyMMdd depending on the configuration set for bigQueryPartitionDecorator). " - + "Setting this to false bypasses the logic to append the partition decorator and uses raw table name for inserts."; - - public static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG = "timestampPartitionFieldName"; - private static final ConfigDef.Type BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE = ConfigDef.Type.STRING; - private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT = null; - private static final ConfigDef.Importance BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE = - ConfigDef.Importance.LOW; - private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC = - "The name of the field in the value that contains the timestamp to partition by in BigQuery" - + " and enable timestamp partitioning for each table. Leave this configuration blank," - + " to enable ingestion time partitioning for each table."; - - public static final String BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG = "clusteringPartitionFieldNames"; - private static final ConfigDef.Type BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE = ConfigDef.Type.LIST; - private static final List BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT = null; - private static final ConfigDef.Importance BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE = - ConfigDef.Importance.LOW; - private static final String BIGQUERY_CLUSTERING_FIELD_NAMES_DOC = - "List of fields on which data should be clustered by in BigQuery, separated by commas"; - - static { - config = BigQuerySinkConfig.getConfig() - .define( - SCHEMA_UPDATE_CONFIG, - SCHEMA_UPDATE_TYPE, - SCHEMA_UPDATE_DEFAULT, - SCHEMA_UPDATE_IMPORTANCE, - SCHEMA_UPDATE_DOC - ).define( - THREAD_POOL_SIZE_CONFIG, - THREAD_POOL_SIZE_TYPE, - THREAD_POOL_SIZE_DEFAULT, - THREAD_POOL_SIZE_VALIDATOR, - THREAD_POOL_SIZE_IMPORTANCE, - THREAD_POOL_SIZE_DOC - ).define( - QUEUE_SIZE_CONFIG, - QUEUE_SIZE_TYPE, - QUEUE_SIZE_DEFAULT, - QUEUE_SIZE_VALIDATOR, - QUEUE_SIZE_IMPORTANCE, - QUEUE_SIZE_DOC - ).define( - BIGQUERY_RETRY_CONFIG, - BIGQUERY_RETRY_TYPE, - BIGQUERY_RETRY_DEFAULT, - BIGQUERY_RETRY_VALIDATOR, - BIGQUERY_RETRY_IMPORTANCE, - BIGQUERY_RETRY_DOC - ).define( - BIGQUERY_RETRY_WAIT_CONFIG, - BIGQUERY_RETRY_WAIT_CONFIG_TYPE, - BIGQUERY_RETRY_WAIT_DEFAULT, - BIGQUERY_RETRY_WAIT_VALIDATOR, - BIGQUERY_RETRY_WAIT_IMPORTANCE, - BIGQUERY_RETRY_WAIT_DOC - ).define( - BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, - BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG_TYPE, - BIGQUERY_MESSAGE_TIME_PARTITIONING_DEFAULT, - BIGQUERY_MESSAGE_TIME_PARTITIONING_IMPORTANCE, - BIGQUERY_MESSAGE_TIME_PARTITIONING_DOC - ).define( - BIGQUERY_PARTITION_DECORATOR_CONFIG, - BIGQUERY_PARTITION_DECORATOR_CONFIG_TYPE, - BIGQUERY_PARTITION_DECORATOR_DEFAULT, - BIGQUERY_PARTITION_DECORATOR_IMPORTANCE, - BIGQUERY_PARTITION_DECORATOR_DOC - ).define( - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE, - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT, - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE, - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC - ).define( - BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, - BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE, - BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT, - BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE, - BIGQUERY_CLUSTERING_FIELD_NAMES_DOC - ); - } - - private void checkAutoUpdateSchemas() { - Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); - - boolean autoUpdateSchemas = getBoolean(SCHEMA_UPDATE_CONFIG); - if (autoUpdateSchemas && schemaRetriever == null) { - throw new ConfigException( - "Cannot specify automatic table creation without a schema retriever" - ); - } - - if (schemaRetriever == null) { - logger.warn( - "No schema retriever class provided; auto schema updates are impossible" - ); - } - } - /** - * Returns the field name to use for timestamp partitioning. - * @return String that represents the field name. - */ - public Optional getTimestampPartitionFieldName() { - return Optional.ofNullable(getString(BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG)); - } - - /** - * Returns the field names to use for clustering. - * @return List of Strings that represent the field names. - */ - public Optional> getClusteringPartitionFieldName() { - return Optional.ofNullable(getList(BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG)); - } - - /** - * Check the validity of table partitioning configs. - */ - private void checkPartitionConfigs() { - if (getTimestampPartitionFieldName().isPresent() && getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { - throw new ConfigException( - "Only one partitioning configuration mode may be specified for the connector. " - + "Use either bigQueryPartitionDecorator OR timestampPartitionFieldName." - ); - } - } - - /** - * Check the validity of table clustering configs. - */ - private void checkClusteringConfigs() { - if (getClusteringPartitionFieldName().isPresent()) { - if (!getTimestampPartitionFieldName().isPresent() && !getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { - throw new ConfigException( - "Clustering field name may be specified only on a partitioned table." - ); - } - if (getClusteringPartitionFieldName().get().size() > 4) { - throw new ConfigException( - "You can only specify up to four clustering field names." - ); - } - } - } + public static final String GCS_BQ_TASK_CONFIG = "GCSBQTask"; + private static final ConfigDef.Type GCS_BQ_TASK_TYPE = ConfigDef.Type.BOOLEAN; + private static final boolean GCS_BQ_TASK_DEFAULT = false; + private static final ConfigDef.Importance GCS_BQ_TASK_IMPORTANCE = ConfigDef.Importance.LOW; - public static ConfigDef getConfig() { - return config; + private static ConfigDef config() { + return BigQuerySinkConfig.getConfig() + .defineInternal(GCS_BQ_TASK_CONFIG, GCS_BQ_TASK_TYPE, GCS_BQ_TASK_DEFAULT, GCS_BQ_TASK_IMPORTANCE); } /** * @param properties A Map detailing configuration properties and their respective values. */ public BigQuerySinkTaskConfig(Map properties) { - super(config, properties); - checkAutoUpdateSchemas(); - checkPartitionConfigs(); - checkClusteringConfigs(); + super(config(), properties); } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java index 433dc130b..b6558a7fa 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java @@ -53,7 +53,7 @@ import java.util.Map; public class BigQuerySinkConnectorTest { - private static SinkConnectorPropertiesFactory propertiesFactory; + private static SinkPropertiesFactory propertiesFactory; // Would just use Mockito, but can't provide the name of an anonymous class to the config file public static class MockSchemaRetriever implements SchemaRetriever { @@ -75,7 +75,7 @@ public void setLastSeenSchema(TableId table, String topic, Schema schema) { @BeforeClass public static void initializePropertiesFactory() { - propertiesFactory = new SinkConnectorPropertiesFactory(); + propertiesFactory = new SinkPropertiesFactory(); } @Test @@ -127,7 +127,7 @@ public void testTaskConfigs() { @Test public void testConfig() { - assertEquals(BigQuerySinkConfig.getConfig(), new BigQuerySinkConnector().config()); + assertNotNull(new BigQuerySinkConnector().config()); } // Make sure that a config exception is properly translated into a SinkConfigConnectException diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index fd16921f4..9e86c3fe1 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -39,7 +39,6 @@ import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import org.apache.kafka.common.config.ConfigException; @@ -63,11 +62,11 @@ import java.util.concurrent.RejectedExecutionException; public class BigQuerySinkTaskTest { - private static SinkTaskPropertiesFactory propertiesFactory; + private static SinkPropertiesFactory propertiesFactory; @BeforeClass public static void initializePropertiesFactory() { - propertiesFactory = new SinkTaskPropertiesFactory(); + propertiesFactory = new SinkPropertiesFactory(); } @Test @@ -177,7 +176,7 @@ public void testPutWhenPartitioningOnMessageTime() { Map properties = propertiesFactory.getProperties(); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, ".*=scratch"); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, "true"); + properties.put(BigQuerySinkConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, "true"); BigQuery bigQuery = mock(BigQuery.class); Storage storage = mock(Storage.class); @@ -210,8 +209,8 @@ public void testPutWhenPartitioningIsSetToTrue() { Map properties = propertiesFactory.getProperties(); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, ".*=scratch"); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, "true"); + properties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); + properties.put(BigQuerySinkConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, "true"); BigQuery bigQuery = mock(BigQuery.class); Storage storage = mock(Storage.class); @@ -244,7 +243,7 @@ public void testPutWhenPartitioningIsSetToFalse() { Map properties = propertiesFactory.getProperties(); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, ".*=scratch"); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); + properties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); BigQuery bigQuery = mock(BigQuery.class); Storage storage = mock(Storage.class); @@ -278,7 +277,7 @@ public void testPutWhenPartitioningOnMessageTimeWhenNoTimestampType() { Map properties = propertiesFactory.getProperties(); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, ".*=scratch"); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, "true"); + properties.put(BigQuerySinkConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, "true"); BigQuery bigQuery = mock(BigQuery.class); Storage storage = mock(Storage.class); @@ -389,8 +388,8 @@ public void testBigQuery5XXRetry() { final String dataset = "scratch"; Map properties = propertiesFactory.getProperties(); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_CONFIG, "3"); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_WAIT_CONFIG, "2000"); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG, "3"); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG, "2000"); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, String.format(".*=%s", dataset)); @@ -425,8 +424,8 @@ public void testBigQuery403Retry() { final String dataset = "scratch"; Map properties = propertiesFactory.getProperties(); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_CONFIG, "2"); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_WAIT_CONFIG, "2000"); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG, "2"); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG, "2000"); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, String.format(".*=%s", dataset)); @@ -462,8 +461,8 @@ public void testBigQueryRetryExceeded() { final String dataset = "scratch"; Map properties = propertiesFactory.getProperties(); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_CONFIG, "1"); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_WAIT_CONFIG, "2000"); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG, "1"); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG, "2000"); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, String.format(".*=%s", dataset)); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkConnectorPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkConnectorPropertiesFactory.java deleted file mode 100644 index e6ea6a5ee..000000000 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkConnectorPropertiesFactory.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery; - -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; - -import java.util.Map; - -public class SinkConnectorPropertiesFactory extends SinkPropertiesFactory { - @Override - public Map getProperties() { - Map properties = super.getProperties(); - - properties.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "false"); - return properties; - } - - /** - * Make sure that each of the default configuration properties work nicely with the given - * configuration object. - * - * @param config The config object to test - */ - public void testProperties(BigQuerySinkConfig config) { - super.testProperties(config); - - config.getBoolean(config.TABLE_CREATE_CONFIG); - } -} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java index 0d6517c90..3bb981b4e 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java @@ -35,9 +35,9 @@ public Map getProperties() { Map properties = new HashMap<>(); properties.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "false"); + properties.put(BigQuerySinkConfig.SCHEMA_UPDATE_CONFIG, "false"); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, "kcbq-test"); properties.put(BigQuerySinkConfig.PROJECT_CONFIG, "test-project"); - properties.put(BigQuerySinkConfig.DATASETS_CONFIG, ".*=test"); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, "kcbq-test=kcbq-test-table"); properties.put(BigQuerySinkConfig.KEYFILE_CONFIG, "key.json"); @@ -59,14 +59,9 @@ public void testProperties(BigQuerySinkConfig config) { config.getMap(config.DATASETS_CONFIG); config.getMap(config.TOPICS_TO_TABLES_CONFIG); - config.getList(config.TOPICS_CONFIG); - config.getList(config.TOPICS_TO_TABLES_CONFIG); - config.getList(config.DATASETS_CONFIG); - - config.getKeyFile(); config.getString(config.PROJECT_CONFIG); - + config.getKeyFile(); config.getBoolean(config.SANITIZE_TOPICS_CONFIG); config.getInt(config.AVRO_DATA_CACHE_SIZE_CONFIG); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java deleted file mode 100644 index 222d1d881..000000000 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkTaskPropertiesFactory.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery; - -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; - -import java.util.Map; - -public class SinkTaskPropertiesFactory extends SinkPropertiesFactory { - @Override - public Map getProperties() { - Map properties = super.getProperties(); - - properties.put(BigQuerySinkTaskConfig.SCHEMA_UPDATE_CONFIG, "false"); - properties.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "false"); - - return properties; - } - - /** - * Make sure that each of the default configuration properties work nicely with the given - * configuration object. - * - * @param config The config object to test - */ - public void testProperties(BigQuerySinkTaskConfig config) { - super.testProperties(config); - - config.getBoolean(config.SCHEMA_UPDATE_CONFIG); - } -} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index e6ec7bea2..8e1f7328c 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -20,6 +20,7 @@ package com.wepay.kafka.connect.bigquery.config; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -32,8 +33,12 @@ import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.Optional; public class BigQuerySinkConfigTest { private SinkPropertiesFactory propertiesFactory; @@ -208,4 +213,108 @@ public void testInvalidAvroCacheSize() { new BigQuerySinkConfig(badConfigProperties); } + + /** + * Test the default for the field name is not present. + */ + @Test + public void testEmptyTimestampPartitionFieldName() { + Map configProperties = propertiesFactory.getProperties(); + BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); + assertFalse(testConfig.getTimestampPartitionFieldName().isPresent()); + } + + /** + * Test if the field name being non-empty and the decorator default (true) errors correctly. + */ + @Test (expected = ConfigException.class) + public void testTimestampPartitionFieldNameError() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); + new BigQuerySinkConfig(configProperties); + } + + /** + * Test the field name being non-empty and the decorator set to false works correctly. + */ + @Test + public void testTimestampPartitionFieldName() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); + configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); + BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); + assertTrue(testConfig.getTimestampPartitionFieldName().isPresent()); + assertFalse(testConfig.getBoolean(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG)); + } + + /** + * Test the default for the field names is not present. + */ + @Test + public void testEmptyClusteringFieldNames() { + Map configProperties = propertiesFactory.getProperties(); + BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); + assertFalse(testConfig.getClusteringPartitionFieldName().isPresent()); + } + + /** + * Test if the field names being non-empty and the partitioning is not present errors correctly. + */ + @Test (expected = ConfigException.class) + public void testClusteringFieldNamesWithoutTimestampPartitionError() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, null); + configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); + configProperties.put( + BigQuerySinkConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, + "column1,column2" + ); + new BigQuerySinkConfig(configProperties); + } + + /** + * Test if the field names are more than four fields errors correctly. + */ + @Test (expected = ConfigException.class) + public void testClusteringPartitionFieldNamesWithMoreThanFourFieldsError() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); + configProperties.put( + BigQuerySinkConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, + "column1,column2,column3,column4,column5" + ); + new BigQuerySinkConfig(configProperties); + } + + /** + * Test the field names being non-empty and the partitioning field exists works correctly. + */ + @Test + public void testClusteringFieldNames() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); + configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); + configProperties.put( + BigQuerySinkConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, + "column1,column2" + ); + + ArrayList expectedClusteringPartitionFieldName = new ArrayList<>( + Arrays.asList("column1", "column2") + ); + + BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); + Optional> testClusteringPartitionFieldName = testConfig.getClusteringPartitionFieldName(); + assertTrue(testClusteringPartitionFieldName.isPresent()); + assertEquals(expectedClusteringPartitionFieldName, testClusteringPartitionFieldName.get()); + } + + @Test(expected = ConfigException.class) + public void testAutoSchemaUpdateWithoutRetriever() { + Map badConfigProperties = propertiesFactory.getProperties(); + badConfigProperties.remove(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); + badConfigProperties.put(BigQuerySinkConfig.SCHEMA_UPDATE_CONFIG, "true"); + + new BigQuerySinkConfig(badConfigProperties); + } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java deleted file mode 100644 index 3f55adb48..000000000 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkTaskConfigTest.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery.config; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import com.wepay.kafka.connect.bigquery.SinkTaskPropertiesFactory; - -import org.apache.kafka.common.config.ConfigException; - -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -public class BigQuerySinkTaskConfigTest { - private SinkTaskPropertiesFactory propertiesFactory; - - @Before - public void initializePropertiesFactory() { - propertiesFactory = new SinkTaskPropertiesFactory(); - } - - @Test - public void metaTestBasicConfigProperties() { - Map basicConfigProperties = propertiesFactory.getProperties(); - BigQuerySinkTaskConfig config = new BigQuerySinkTaskConfig(basicConfigProperties); - propertiesFactory.testProperties(config); - } - - @Test() - public void testMaxWriteSize() { - // todo: something like this, maybe. - /* - Map badProperties = propertiesFactory.getProperties(); - badProperties.put(BigQuerySinkTaskConfig.MAX_WRITE_CONFIG, "-1"); - - try { - new BigQuerySinkTaskConfig(badProperties); - } catch (ConfigException err) { - fail("Exception encountered before addition of bad configuration field: " + err); - } - - badProperties.put(BigQuerySinkTaskConfig.MAX_WRITE_CONFIG, "0"); - new BigQuerySinkTaskConfig(badProperties); - */ - } - - /** - * Test the default for the field name is not present. - */ - @Test - public void testEmptyTimestampPartitionFieldName() { - Map configProperties = propertiesFactory.getProperties(); - BigQuerySinkTaskConfig testConfig = new BigQuerySinkTaskConfig(configProperties); - assertFalse(testConfig.getTimestampPartitionFieldName().isPresent()); - } - - /** - * Test if the field name being non-empty and the decorator default (true) errors correctly. - */ - @Test (expected = ConfigException.class) - public void testTimestampPartitionFieldNameError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); - new BigQuerySinkTaskConfig(configProperties); - } - - /** - * Test the field name being non-empty and the decorator set to false works correctly. - */ - @Test - public void testTimestampPartitionFieldName() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); - BigQuerySinkTaskConfig testConfig = new BigQuerySinkTaskConfig(configProperties); - assertTrue(testConfig.getTimestampPartitionFieldName().isPresent()); - assertFalse(testConfig.getBoolean(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG)); - } - - /** - * Test the default for the field names is not present. - */ - @Test - public void testEmptyClusteringFieldNames() { - Map configProperties = propertiesFactory.getProperties(); - BigQuerySinkTaskConfig testConfig = new BigQuerySinkTaskConfig(configProperties); - assertFalse(testConfig.getClusteringPartitionFieldName().isPresent()); - } - - /** - * Test if the field names being non-empty and the partitioning is not present errors correctly. - */ - @Test (expected = ConfigException.class) - public void testClusteringFieldNamesWithoutTimestampPartitionError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, null); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); - configProperties.put( - BigQuerySinkTaskConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, - "column1,column2" - ); - new BigQuerySinkTaskConfig(configProperties); - } - - /** - * Test if the field names are more than four fields errors correctly. - */ - @Test (expected = ConfigException.class) - public void testClusteringPartitionFieldNamesWithMoreThanFourFieldsError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); - configProperties.put( - BigQuerySinkTaskConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, - "column1,column2,column3,column4,column5" - ); - new BigQuerySinkTaskConfig(configProperties); - } - - /** - * Test the field names being non-empty and the partitioning field exists works correctly. - */ - @Test - public void testClusteringFieldNames() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); - configProperties.put( - BigQuerySinkTaskConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, - "column1,column2" - ); - - ArrayList expectedClusteringPartitionFieldName = new ArrayList<>( - Arrays.asList("column1", "column2") - ); - - BigQuerySinkTaskConfig testConfig = new BigQuerySinkTaskConfig(configProperties); - Optional> testClusteringPartitionFieldName = testConfig.getClusteringPartitionFieldName(); - assertTrue(testClusteringPartitionFieldName.isPresent()); - assertEquals(expectedClusteringPartitionFieldName, testClusteringPartitionFieldName.get()); - } - - @Test(expected = ConfigException.class) - public void testAutoSchemaUpdateWithoutRetriever() { - Map badConfigProperties = propertiesFactory.getProperties(); - badConfigProperties.remove(BigQuerySinkTaskConfig.SCHEMA_RETRIEVER_CONFIG); - badConfigProperties.put(BigQuerySinkTaskConfig.SCHEMA_UPDATE_CONFIG, "true"); - - new BigQuerySinkTaskConfig(badConfigProperties); - } -} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index 4ecbb6961..d55f377cb 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -37,12 +37,12 @@ import com.wepay.kafka.connect.bigquery.BigQuerySinkTask; import com.wepay.kafka.connect.bigquery.SchemaManager; -import com.wepay.kafka.connect.bigquery.SinkTaskPropertiesFactory; +import com.wepay.kafka.connect.bigquery.SinkPropertiesFactory; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import com.wepay.kafka.connect.bigquery.retrieve.MemorySchemaRetriever; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; @@ -63,11 +63,11 @@ @SuppressWarnings("unchecked") public class BigQueryWriterTest { - private static SinkTaskPropertiesFactory propertiesFactory; + private static SinkPropertiesFactory propertiesFactory; @BeforeClass public static void initializePropertiesFactory() { - propertiesFactory = new SinkTaskPropertiesFactory(); + propertiesFactory = new SinkPropertiesFactory(); } @Test @@ -109,7 +109,8 @@ public void testAutoCreateTables() { final String topic = "test_topic"; final String dataset = "scratch"; final Map properties = makeProperties("3", "2000", topic, dataset); - properties.put(BigQuerySinkTaskConfig.TABLE_CREATE_CONFIG, "true"); + properties.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); + properties.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, MemorySchemaRetriever.class.getName()); BigQuery bigQuery = mock(BigQuery.class); Map> emptyMap = mock(Map.class); @@ -285,8 +286,8 @@ private Map makeProperties(String bigqueryRetry, String topic, String dataset) { Map properties = propertiesFactory.getProperties(); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_CONFIG, bigqueryRetry); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_WAIT_CONFIG, bigqueryRetryWait); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG, bigqueryRetry); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG, bigqueryRetryWait); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, String.format(".*=%s", dataset)); return properties; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java index 546095d35..ca1b4d55b 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java @@ -21,16 +21,14 @@ import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.Table; -import com.google.cloud.storage.Blob; import com.google.cloud.storage.BlobInfo; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageException; import com.wepay.kafka.connect.bigquery.BigQuerySinkTask; import com.wepay.kafka.connect.bigquery.SchemaManager; -import com.wepay.kafka.connect.bigquery.SinkTaskPropertiesFactory; +import com.wepay.kafka.connect.bigquery.SinkPropertiesFactory; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; @@ -52,11 +50,11 @@ public class GCSToBQWriterTest { - private static SinkTaskPropertiesFactory propertiesFactory; + private static SinkPropertiesFactory propertiesFactory; @BeforeClass public static void initializePropertiesFactory() { - propertiesFactory = new SinkTaskPropertiesFactory(); + propertiesFactory = new SinkPropertiesFactory(); } @Test @@ -162,8 +160,8 @@ private Map makeProperties(String bigqueryRetry, String topic, String dataset) { Map properties = propertiesFactory.getProperties(); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_CONFIG, bigqueryRetry); - properties.put(BigQuerySinkTaskConfig.BIGQUERY_RETRY_WAIT_CONFIG, bigqueryRetryWait); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG, bigqueryRetry); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG, bigqueryRetryWait); properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); properties.put(BigQuerySinkConfig.DATASETS_CONFIG, String.format(".*=%s", dataset)); // gcs config From 1a9bfbda66de24ced45b2c20507889d49d15d96e Mon Sep 17 00:00:00 2001 From: Kanthi Date: Tue, 2 Nov 2021 16:34:25 -0400 Subject: [PATCH 109/190] [GH-161] Fixed cast error of Long in debezium time converter (#162) * [GH-161] Fixed cast error of Long in debezium time converter * [GH-161] Fixed cast error of Long in debezium time converter * [GH-161] Fixed test case for TimeConverter --- .../convert/logicaltype/DebeziumLogicalConverters.java | 2 +- .../convert/logicaltype/DebeziumLogicalConvertersTest.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java index abf35c938..19939bf36 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConverters.java @@ -152,7 +152,7 @@ public TimeConverter() { @Override public String convert(Object kafkaConnectObject) { - java.util.Date date = new java.util.Date((Long) kafkaConnectObject); + java.util.Date date = new java.util.Date((Integer) kafkaConnectObject); return getBQTimeFormat().format(date); } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConvertersTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConvertersTest.java index 61ce9d337..059fccbe4 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConvertersTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/logicaltype/DebeziumLogicalConvertersTest.java @@ -40,6 +40,7 @@ public class DebeziumLogicalConvertersTest { //corresponds to March 1 2017, 22:20:38.808(123) UTC // (March 1 2017, 14:20:38.808(123)-8:00) private static final Integer DAYS_TIMESTAMP = 17226; + private static final Integer MILLI_TIMESTAMP_INT = 1488406838; private static final Long MILLI_TIMESTAMP = 1488406838808L; private static final Long MICRO_TIMESTAMP = 1488406838808123L; @@ -103,8 +104,8 @@ public void testTimeConversion() { fail("Expected encoding type check to succeed."); } - String formattedTime = converter.convert(MILLI_TIMESTAMP); - assertEquals("22:20:38.808", formattedTime); + String formattedTime = converter.convert(MILLI_TIMESTAMP_INT); + assertEquals("05:26:46.838", formattedTime); } @Test From 15fe0f5803d8f5c3da7039d1a08ae33bf24265ad Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 12 Nov 2021 19:44:51 +0000 Subject: [PATCH 110/190] [maven-release-plugin] prepare release v1.6.9 --- kcbq-api/pom.xml | 2 +- kcbq-confluent/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index b771a52fb..768bf602d 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.9-SNAPSHOT + 1.6.9 .. diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index 9d58bef55..f17b6b069 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.9-SNAPSHOT + 1.6.9 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 4932c5908..4d2203728 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.9-SNAPSHOT + 1.6.9 .. diff --git a/pom.xml b/pom.xml index 675043611..4dcd7d609 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 1.6.9-SNAPSHOT + 1.6.9 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v1.6.9 From 4c68def60d20bf4f3ee13836d39df69de2fe4be2 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 12 Nov 2021 19:44:54 +0000 Subject: [PATCH 111/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-confluent/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 768bf602d..ce992d2ba 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.9 + 1.6.10-SNAPSHOT .. diff --git a/kcbq-confluent/pom.xml b/kcbq-confluent/pom.xml index f17b6b069..3ab210b3b 100644 --- a/kcbq-confluent/pom.xml +++ b/kcbq-confluent/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.9 + 1.6.10-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 4d2203728..687a17341 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 1.6.9 + 1.6.10-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 4dcd7d609..ff27031ae 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 1.6.9 + 1.6.10-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v1.6.9 + HEAD From 554aa45714bd88eb22ccc4069e1674fb58f626bf Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 12 Nov 2021 19:47:39 +0000 Subject: [PATCH 112/190] [maven-release-plugin] prepare release v2.0.9 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 20ebf0253..565a6001f 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.9-SNAPSHOT + 2.0.9 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index c82f7e2bf..b9c802041 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.9-SNAPSHOT + 2.0.9 .. diff --git a/pom.xml b/pom.xml index 9ace09b36..53db80853 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.9-SNAPSHOT + 2.0.9 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.9 From 6dcf0ba4a5c4b64ea056441dfca8eaebd01d7200 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 12 Nov 2021 19:47:42 +0000 Subject: [PATCH 113/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 565a6001f..954857403 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.9 + 2.0.10-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index b9c802041..6519d5f2d 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.9 + 2.0.10-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 53db80853..e2ddf5122 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.9 + 2.0.10-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.9 + HEAD From f0aca3c2b62b6baba0582f5e523f1b2d54284c8f Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 12 Nov 2021 19:50:21 +0000 Subject: [PATCH 114/190] [maven-release-plugin] prepare release v2.1.8 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 92a76d8f3..77ba4de83 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.8-SNAPSHOT + 2.1.8 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 8edb6665f..e85d07192 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.8-SNAPSHOT + 2.1.8 .. diff --git a/pom.xml b/pom.xml index 27ba96488..2df226a78 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.8-SNAPSHOT + 2.1.8 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.8 From b3de1b48807a9c006ba454961db05cadfbc99a57 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 12 Nov 2021 19:50:24 +0000 Subject: [PATCH 115/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 77ba4de83..c23366e99 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.8 + 2.1.9-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index e85d07192..0e92a5966 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.8 + 2.1.9-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 2df226a78..899a91049 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.8 + 2.1.9-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.8 + 2.1.x From 4e52cece764feb56129cc47ca93317680e585d20 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 15 Nov 2021 15:38:42 -0500 Subject: [PATCH 116/190] MINOR: Handle new "table is deleted" error from BigQuery (#163) * MINOR: Handle new "table is deleted" error from BigQuery Previously, if a table was deleted and recreated in a short time period, some writes to that table ran the risk of being silently dropped by BigQuery. It seems like BigQuery handles that case now by returning a new error for writes to recently-recreated tables with the same 404 code and `notFound` reason, but a slightly different message (something like "Not found: table Table is deleted:
" (the "table Table" part is not a typo; this is what the responses actually look like)). We can handle this new error the same way we handle missing table errors already. It may not be super useful for everyday users of the connector but it makes integration testing (and possibly upsert/delete logic) much easier since we can now wipe out tables left over from previous tests and then spin up connector instances that write to them immediately after. * Add integration test --- .../write/row/BigQueryErrorResponses.java | 3 +- .../integration/BigQueryErrorResponsesIT.java | 50 +++++++++++++++++++ 2 files changed, 52 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java index 4e5f2062a..fd79d8b05 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java @@ -48,11 +48,12 @@ public class BigQueryErrorResponses { public static boolean isNonExistentTableError(BigQueryException exception) { + String message = message(exception.getError()); // If a table does not exist, it will raise a BigQueryException that the input is notFound // Referring to Google Cloud Error Codes Doc: https://cloud.google.com/bigquery/docs/error-messages?hl=en return NOT_FOUND_CODE == exception.getCode() && NOT_FOUND_REASON.equals(exception.getReason()) - && message(exception.getError()).startsWith("Not found: Table "); + && (message.startsWith("Not found: Table ") || message.contains("Table is deleted: ")); } public static boolean isTableMissingSchemaError(BigQueryException exception) { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java index 0511e06f7..bbf430fdc 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java @@ -25,6 +25,7 @@ import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.InsertAllRequest; import com.google.cloud.bigquery.InsertAllResponse; +import com.google.cloud.bigquery.LegacySQLTypeName; import com.google.cloud.bigquery.Schema; import com.google.cloud.bigquery.StandardSQLTypeName; import com.google.cloud.bigquery.StandardTableDefinition; @@ -33,6 +34,7 @@ import com.google.cloud.bigquery.TableInfo; import com.wepay.kafka.connect.bigquery.integration.utils.TableClearer; import com.wepay.kafka.connect.bigquery.write.row.BigQueryErrorResponses; +import org.apache.kafka.test.TestUtils; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -78,6 +80,54 @@ public void testWriteToNonExistentTable() { } } + @Test + public void testWriteToRecreatedTable() throws Exception { + TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("recreated table")); + TableClearer.clearTables(bigQuery, dataset(), table.getTable()); + + Schema schema = Schema.of(Field.of("f1", LegacySQLTypeName.STRING)); + + // Create the table... + bigQuery.create(TableInfo.newBuilder(table, StandardTableDefinition.of(schema)).build()); + + // Make sure that it exists... + TestUtils.waitForCondition( + () -> bigQuery.getTable(table) != null, + 60_000L, + "Table does not appear to exist one minute after issuing create request" + ); + logger.info("Created {} successfully", table(table)); + + // Delete it... + bigQuery.delete(table); + + // Make sure that it's deleted + TestUtils.waitForCondition( + () -> bigQuery.getTable(table) == null, + 60_000L, + "Table still appears to exist one minute after issuing delete request" + ); + logger.info("Deleted {} successfully", table(table)); + + // Recreate it... + bigQuery.create(TableInfo.newBuilder(table, StandardTableDefinition.of(schema)).build()); + + TestUtils.waitForCondition( + () -> { + // Try to write to it... + try { + bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(Collections.singletonMap("f1", "v1")))); + return false; + } catch (BigQueryException e) { + logger.debug("Recreated table write error", e); + return BigQueryErrorResponses.isNonExistentTableError(e); + } + }, + 60_000L, + "Never failed to write to just-recreated table" + ); + } + @Test public void testWriteToTableWithoutSchema() { TableId table = TableId.of(dataset(), suffixedAndSanitizedTable("missing schema")); From d7cbea1649d09626d103f6298a1b96d88a102f92 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 15 Nov 2021 16:24:24 -0500 Subject: [PATCH 117/190] GH-138: Fail faster when attempting to write to non-day-partitioned tables with decorator syntax (#141) --- .../connect/bigquery/BigQuerySinkTask.java | 52 ++- .../bigquery/config/BigQuerySinkConfig.java | 23 +- .../bigquery/utils/PartitionedTableId.java | 52 +-- .../write/row/AdaptiveBigQueryWriter.java | 10 +- .../bigquery/BigQuerySinkTaskTest.java | 69 +++- .../config/BigQuerySinkConfigTest.java | 22 ++ .../bigquery/integration/BaseConnectorIT.java | 9 +- .../integration/TimePartitioningIT.java | 315 ++++++++++++++++++ .../utils/PartitionedTableIdTest.java | 137 -------- .../write/row/BigQueryWriterTest.java | 51 +-- .../bigquery/write/row/GCSToBQWriterTest.java | 8 +- .../src/test/resources/log4j.properties | 2 + 12 files changed, 473 insertions(+), 277 deletions(-) create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/TimePartitioningIT.java diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index af735d59a..c34a31e43 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -38,6 +38,7 @@ import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.utils.SinkRecordConverter; +import com.wepay.kafka.connect.bigquery.utils.TableNameUtils; import com.wepay.kafka.connect.bigquery.utils.Version; import com.wepay.kafka.connect.bigquery.write.batch.GCSBatchTableWriter; import com.wepay.kafka.connect.bigquery.write.batch.KCBQThreadPoolExecutor; @@ -133,7 +134,7 @@ public BigQuerySinkTask() { * @see BigQuerySinkTask#BigQuerySinkTask() */ public BigQuerySinkTask(BigQuery testBigQuery, SchemaRetriever schemaRetriever, Storage testGcs, - SchemaManager testSchemaManager, Map testCache) { + SchemaManager testSchemaManager, Map testCache) { this.testBigQuery = testBigQuery; this.schemaRetriever = schemaRetriever; this.testGcs = testGcs; @@ -218,9 +219,9 @@ private PartitionedTableId getRecordTable(SinkRecord record) { throw new ConnectException( "Message has no timestamp type, cannot use message timestamp to partition."); } - setTimePartitioningForTimestamp(builder, timePartitioning, record.timestamp()); + setTimePartitioningForTimestamp(baseTableId, builder, timePartitioning, record.timestamp()); } else { - setTimePartitioning(builder, timePartitioning); + setTimePartitioning(baseTableId, builder, timePartitioning); } } @@ -307,37 +308,28 @@ private BigQuery getBigQuery() { return bigQuery.updateAndGet(bq -> bq != null ? bq : newBigQuery()); } - private void setTimePartitioningForTimestamp(PartitionedTableId.Builder builder, TimePartitioning timePartitioning, - Long timestamp) { - switch (timePartitioning.getType()) { - case HOUR: - builder.setHourPartition(timestamp); - break; - case MONTH: - builder.setMonthPartition(timestamp); - break; - case YEAR: - builder.setYearPartition(timestamp); - break; - default: - builder.setDayPartition(timestamp); + private void setTimePartitioningForTimestamp( + TableId table, PartitionedTableId.Builder builder, TimePartitioning timePartitioning, Long timestamp + ) { + if (timePartitioning.getType() != Type.DAY) { + throw new ConnectException(String.format( + "Cannot use decorator syntax to write to %s as it is partitioned by %s and not by day", + TableNameUtils.table(table), + timePartitioning.getType().toString().toLowerCase() + )); } + builder.setDayPartition(timestamp); } - private void setTimePartitioning(PartitionedTableId.Builder builder, TimePartitioning timePartitioning) { - switch (timePartitioning.getType()) { - case HOUR: - builder.setHourPartitionNow(); - break; - case MONTH: - builder.setMonthPartitionForNow(); - break; - case YEAR: - builder.setYearPartitionForNow(); - break; - default: - builder.setDayPartitionForNow(); + private void setTimePartitioning(TableId table, PartitionedTableId.Builder builder, TimePartitioning timePartitioning) { + if (timePartitioning.getType() != Type.DAY) { + throw new ConnectException(String.format( + "Cannot use decorator syntax to write to %s as it is partitioned by %s and not by day", + TableNameUtils.table(table), + timePartitioning.getType().toString().toLowerCase() + )); } + builder.setDayPartitionForNow(); } private Table retrieveCachedTable(TableId tableId) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index b2c178768..fc5fd44d0 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -866,7 +866,6 @@ private void verifyBucketSpecified() throws ConfigException { } private void checkAutoCreateTables() { - Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); boolean autoCreateTables = getBoolean(TABLE_CREATE_CONFIG); @@ -911,6 +910,27 @@ public Optional getTimestampPartitionFieldName() { return Optional.ofNullable(getString(BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG)); } + private void checkTimePartitioningConfigs() { + boolean decoratorSyntax = getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG); + boolean createTables = getBoolean(TABLE_CREATE_CONFIG); + + if (!decoratorSyntax || !createTables) { + // The time partitioning type only matters when we're configured to automatically create tables and will write to them with decorator syntax + return; + } + + String rawTimePartitioningType = getString(TIME_PARTITIONING_TYPE_CONFIG); + if (!TimePartitioning.Type.DAY.equals(parseTimePartitioningType(rawTimePartitioningType))) { + throw new ConfigException( + TIME_PARTITIONING_TYPE_CONFIG, + rawTimePartitioningType, + "Tables must be partitioned by DAY when using partition decorator syntax. " + + "Either configure the connector with the DAY time partitioning type, " + + "disable automatic table creation, or disable partition decorator syntax." + ); + } + } + /** * Returns the field names to use for clustering. * @return List of Strings that represent the field names. @@ -956,6 +976,7 @@ protected BigQuerySinkConfig(ConfigDef config, Map properties) { checkBigQuerySchemaUpdateConfigs(); checkPartitionConfigs(); checkClusteringConfigs(); + checkTimePartitioningConfigs(); } public BigQuerySinkConfig(Map properties) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java index e63257691..f38254c1e 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableId.java @@ -30,6 +30,9 @@ /** * A TableId with separate base table name and partition information. + * Note that this class only supports partitioning by day; even though BigQuery supports other time partitioning types + * for tables partitioned by ingestion time, it doesn't support decorator syntax (i.e., appending "$YYYYMMDD" to the + * name of a table being streamed to) for these other time partitioning types. */ public class PartitionedTableId { @@ -203,19 +206,6 @@ public Builder setPartition(String partition) { return this; } - public Builder setHourPartition(long utcTime) { - Instant instant = Instant.ofEpochMilli(utcTime); - return setHourPartition(LocalDateTime.ofInstant(instant, ZoneId.of("UTC"))); - } - - public Builder setHourPartition(LocalDateTime localDate) { - return setPartition(dateToHourPartition(localDate)); - } - - public Builder setHourPartitionNow() { - return setHourPartition(LocalDateTime.now(UTC_CLOCK)); - } - public Builder setDayPartition(long utcTime) { return setDayPartition(LocalDate.ofEpochDay(utcTime / MILLIS_IN_DAY)); } @@ -228,34 +218,6 @@ public Builder setDayPartitionForNow() { return setDayPartition(LocalDate.now(UTC_CLOCK)); } - public Builder setMonthPartition(long utcTime) { - return setMonthPartition(LocalDate.ofEpochDay(utcTime / MILLIS_IN_DAY)); - } - - public Builder setMonthPartition(LocalDate localDate) { - return setPartition(dateToMonthPartition(localDate)); - } - - public Builder setMonthPartitionForNow() { - return setMonthPartition(LocalDate.now(UTC_CLOCK)); - } - - public Builder setYearPartition(long utcTime) { - return setYearPartition(LocalDate.ofEpochDay(utcTime / MILLIS_IN_DAY)); - } - - public Builder setYearPartition(LocalDate localDate) { - return setPartition(dateToYearPartition(localDate)); - } - - public Builder setYearPartitionForNow() { - return setYearPartition(LocalDate.now(UTC_CLOCK)); - } - - private String dateToHourPartition(LocalDateTime localDate) { - return localDate.format(DateTimeFormatter.ofPattern("yyyyMMddHH")); - } - /** * @param localDate the localDate of the partition. * @return The String representation of the partition. @@ -264,14 +226,6 @@ private static String dateToDayPartition(LocalDate localDate) { return localDate.format(DateTimeFormatter.BASIC_ISO_DATE); } - private String dateToMonthPartition(LocalDate localDate) { - return localDate.format(DateTimeFormatter.ofPattern("yyyyMM")); - } - - private String dateToYearPartition(LocalDate localDate) { - return localDate.format(DateTimeFormatter.ofPattern("yyyy")); - } - /** * Build the {@link PartitionedTableId}. * diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java index e38920050..5232ccc32 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/AdaptiveBigQueryWriter.java @@ -119,8 +119,14 @@ && onlyContainsInvalidSchemaErrors(writeResponse.getInsertErrors())) { logger.debug("re-attempting insertion"); writeResponse = bigQuery.insertAll(request); } catch (BigQueryException exception) { - // no-op, we want to keep retrying the insert - logger.debug("insertion failed", exception); + if ((BigQueryErrorResponses.isNonExistentTableError(exception) && autoCreateTables) + || BigQueryErrorResponses.isTableMissingSchemaError(exception) + ) { + // no-op, we want to keep retrying the insert + logger.debug("insertion failed", exception); + } else { + throw exception; + } } } else { return writeResponse.getInsertErrors(); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 848e1a04e..38c57a538 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -36,9 +36,12 @@ import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.InsertAllRequest; import com.google.cloud.bigquery.InsertAllResponse; +import com.google.cloud.bigquery.StandardTableDefinition; import com.google.cloud.bigquery.Table; import com.google.cloud.bigquery.LegacySQLTypeName; import com.google.cloud.bigquery.QueryJobConfiguration; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.TimePartitioning; import com.google.cloud.storage.Storage; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; @@ -65,6 +68,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.RejectedExecutionException; @@ -114,7 +118,7 @@ public void testSimplePut() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -146,7 +150,7 @@ public void testSimplePutWhenSchemaRetrieverIsNotNull() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -167,7 +171,7 @@ public void testEmptyPut() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.start(properties); @@ -190,7 +194,7 @@ public void testEmptyRecordPut() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.start(properties); @@ -222,7 +226,7 @@ public void testPutWhenPartitioningOnMessageTime() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -260,7 +264,7 @@ public void testPutWhenPartitioningIsSetToTrue() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -297,7 +301,7 @@ public void testPutWhenPartitioningIsSetToFalse() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -334,7 +338,7 @@ public void testPutWhenPartitioningOnMessageTimeWhenNoTimestampType() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -368,7 +372,7 @@ public void testPutWithUpsertDelete() throws Exception { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); Field keyField = Field.of(key, LegacySQLTypeName.STRING); Field valueField = Field.of(value, LegacySQLTypeName.STRING); com.google.cloud.bigquery.Schema intermediateSchema = com.google.cloud.bigquery.Schema.of( @@ -437,7 +441,7 @@ public void testSimplePutException() throws InterruptedException { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); @@ -464,7 +468,7 @@ public void testEmptyFlush() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); @@ -486,7 +490,7 @@ public void testFlushAfterStop() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); @@ -538,7 +542,7 @@ public void testBigQuery5XXRetry() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -579,7 +583,7 @@ public void testBigQuery403Retry() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -617,7 +621,7 @@ public void testBigQueryRetryExceeded() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -648,7 +652,7 @@ public void testInterruptedException() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); @@ -673,7 +677,7 @@ public void testConfigException() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(mock(BigQuery.class), schemaRetriever, mock(Storage.class), schemaManager, cache); @@ -683,6 +687,35 @@ public void testConfigException() { } } + @Test(expected = ConnectException.class) + public void testTimePartitioningIncompatibleWithDecoratorSyntax() { + final String topic = "t1"; + final String dataset = "d"; + + Map properties = propertiesFactory.getProperties(); + properties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); + properties.put(BigQuerySinkConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, "true"); + properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); + properties.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, dataset); + + StandardTableDefinition mockTableDefinition = mock(StandardTableDefinition.class); + when(mockTableDefinition.getTimePartitioning()).thenReturn(TimePartitioning.of(TimePartitioning.Type.HOUR)); + Table table = mock(Table.class); + when(table.getDefinition()).thenReturn(mockTableDefinition); + Map tableCache = new HashMap<>(); + tableCache.put(TableId.of(dataset, topic), table); + + Storage storage = mock(Storage.class); + BigQuery bigQuery = mock(BigQuery.class); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, null, storage, null, tableCache); + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + testTask.initialize(sinkTaskContext); + testTask.start(properties); + + testTask.put(Collections.singleton(spoofSinkRecord(topic, "f1", "v1", TimestampType.CREATE_TIME, 1L))); + } + @Test public void testVersion() { assertNotNull(new BigQuerySinkTask().version()); @@ -711,7 +744,7 @@ public void testStop() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); Storage storage = mock(Storage.class); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index 452c74338..2b67fd310 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -20,6 +20,7 @@ package com.wepay.kafka.connect.bigquery.config; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import com.google.cloud.bigquery.TimePartitioning; @@ -187,11 +188,32 @@ public void testClusteringFieldNames() { @Test public void testValidTimePartitioningTypes() { Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); for (TimePartitioning.Type type : TimePartitioning.Type.values()) { configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, type.name()); assertEquals(type, new BigQuerySinkConfig(configProperties).getTimePartitioningType()); } + + configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); + configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, TimePartitioning.Type.DAY.name()); + assertEquals(TimePartitioning.Type.DAY, new BigQuerySinkConfig(configProperties).getTimePartitioningType()); + } + + @Test + public void testInvalidTimePartitioningTypes() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); + configProperties.put(BigQuerySinkTaskConfig.TABLE_CREATE_CONFIG, "true"); + + for (TimePartitioning.Type type : TimePartitioning.Type.values()) { + if (TimePartitioning.Type.DAY.equals(type)) { + continue; + } + + configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, type.name()); + assertThrows(ConfigException.class, () -> new BigQuerySinkConfig(configProperties)); + } } @Test(expected = ConfigException.class) diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java index 64db82dbb..15a47efc6 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -211,7 +211,9 @@ protected List> readAllRows( BigQuery bigQuery, String tableName, String sortColumn) throws InterruptedException { Table table = bigQuery.getTable(dataset(), tableName); - Schema schema = table.getDefinition().getSchema(); + Schema schema = table + .getDefinition() + .getSchema(); TableResult tableResult = bigQuery.query(QueryJobConfiguration.of(String.format( "SELECT * FROM `%s`.`%s` ORDER BY %s ASC", @@ -298,13 +300,12 @@ private List convertRow(List rowSchema, List row) { * @return the time this method discovered the connector has started, in milliseconds past epoch * @throws InterruptedException if this was interrupted */ - protected long waitForConnectorToStart(String name, int numTasks) throws InterruptedException { - TestUtils.waitForCondition( + protected void waitForConnectorToStart(String name, int numTasks) throws InterruptedException { + waitForCondition( () -> assertConnectorAndTasksRunning(name, numTasks).orElse(false), CONNECTOR_STARTUP_DURATION_MS, "Connector tasks did not start in time." ); - return System.currentTimeMillis(); } /** diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/TimePartitioningIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/TimePartitioningIT.java new file mode 100644 index 000000000..a2cc8579d --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/TimePartitioningIT.java @@ -0,0 +1,315 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.integration; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.QueryJobConfiguration; +import com.google.cloud.bigquery.StandardTableDefinition; +import com.google.cloud.bigquery.TableId; +import com.google.cloud.bigquery.TableResult; +import com.google.cloud.bigquery.TimePartitioning; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.integration.utils.TableClearer; +import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig.BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +@Category(IntegrationTest.class) +@RunWith(Parameterized.class) +public class TimePartitioningIT { + + private static final Logger logger = LoggerFactory.getLogger(TimePartitioningIT.class); + + private static final long NUM_RECORDS_PRODUCED = 20; + private static final int TASKS_MAX = 1; + + private static BaseConnectorIT testBase; + + private BigQuery bigQuery; + + private final TimePartitioning.Type partitioningType; + private final boolean usePartitionDecorator; + private final boolean messageTimePartitioning; + private final int testCase; + private final long testStartTime; + private final String connectorName; + + public TimePartitioningIT( + TimePartitioning.Type partitioningType, + boolean usePartitionDecorator, + boolean messageTimePartitioning, + int testCase + ) { + this.partitioningType = partitioningType; + this.usePartitionDecorator = usePartitionDecorator; + this.messageTimePartitioning = messageTimePartitioning; + this.testCase = testCase; + this.testStartTime = System.currentTimeMillis(); + this.connectorName = "kcbq-time-partitioning-test-" + testCase; + } + + @Parameterized.Parameters(name = "{index}: partitioningType: {0}, usePartitionDecorator: {1}, messageTimePartitioning: {2}") + public static Iterable data() { + int testCase = 0; + return Arrays.asList( + new Object[] {TimePartitioning.Type.HOUR, false, false, testCase++ }, + new Object[] {TimePartitioning.Type.DAY, true, true, testCase++ }, + new Object[] {TimePartitioning.Type.DAY, true, false, testCase++ }, + new Object[] {TimePartitioning.Type.DAY, false, false, testCase++ }, + new Object[] {TimePartitioning.Type.MONTH, false, false, testCase++ }, + new Object[] {TimePartitioning.Type.YEAR, false, false, testCase } + ); + } + + @BeforeClass + public static void globalSetup() { + testBase = new BaseConnectorIT() {}; + BigQuery bigQuery = testBase.newBigQuery(); + data().forEach(args -> { + int testCase = (int) args[3]; + TableClearer.clearTables(bigQuery, testBase.dataset(), table(testCase)); + }); + testBase.startConnect(); + } + + @Before + public void setup() { + bigQuery = testBase.newBigQuery(); + } + + @After + public void close() { + bigQuery = null; + testBase.connect.deleteConnector(connectorName); + } + + @AfterClass + public static void globalCleanup() { + testBase.stopConnect(); + } + + private Map partitioningProps() { + Map result = new HashMap<>(); + + // use the JSON converter with schemas enabled + result.put(KEY_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + result.put(VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + + result.put(BIGQUERY_PARTITION_DECORATOR_CONFIG, Boolean.toString(usePartitionDecorator)); + result.put(BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, Boolean.toString(messageTimePartitioning)); + result.put(TIME_PARTITIONING_TYPE_CONFIG, partitioningType.name()); + + return result; + } + + private static String table(int testCase) { + return testBase.suffixedAndSanitizedTable("test-time-partitioning-" + testCase); + } + + @Test + public void testTimePartitioning() throws Throwable { + // create topic in Kafka + final String topic = testBase.suffixedTableOrTopic("test-time-partitioning-" + testCase); + testBase.connect.kafka().createTopic(topic); + + // setup props for the sink connector + Map props = testBase.baseConnectorProps(TASKS_MAX); + props.put(SinkConnectorConfig.TOPICS_CONFIG, topic); + + props.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); + props.put(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG, IdentitySchemaRetriever.class.getName()); + props.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); + + props.putAll(partitioningProps()); + + // start a sink connector + testBase.connect.configureConnector(connectorName, props); + + // wait for tasks to spin up + testBase.waitForConnectorToStart(connectorName, TASKS_MAX); + + // Instantiate the converter we'll use to send records to the connector + Converter valueConverter = converter(); + + // Instantiate the producer we'll use to write records to Kafka + Map producerProps = new HashMap<>(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, testBase.connect.kafka().bootstrapServers()); + Producer valueProducer = new KafkaProducer<>( + producerProps, Serdes.Void().serializer(), Serdes.String().serializer() + ); + + // Send records to Kafka + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + String kafkaValue = value(valueConverter, topic, i); + logger.debug("Sending message with value '{}' to topic '{}'", kafkaValue, topic); + ProducerRecord kafkaRecord = new ProducerRecord<>(topic, null, timestamp((i % 3) - 1), null, kafkaValue); + try { + valueProducer.send(kafkaRecord).get(30, TimeUnit.SECONDS); + } catch (Exception e) { + throw new ConnectException("Failed to produce data to embedded Kafka cluster", e); + } + } + + // wait for tasks to write to BigQuery and commit offsets for their records + testBase.waitForCommittedRecords( + connectorName, + Collections.singleton(topic), + NUM_RECORDS_PRODUCED, + TASKS_MAX, + TimeUnit.MINUTES.toMillis(3) + ); + + String table = table(testCase); + + // Might fail to read from the table for a little bit; keep retrying until it's available + waitForCondition( + () -> { + try { + testBase.readAllRows(bigQuery, table, "i"); + return true; + } catch (RuntimeException e) { + logger.debug("Failed to read rows from table {}", table, e); + return false; + } + }, + TimeUnit.MINUTES.toMillis(5), + "Could not read from table to verify data after connector committed offsets for the expected number of records" + ); + + List> allRows = testBase.readAllRows(bigQuery, table, "i"); + // Just check to make sure we sent the expected number of rows to the table + assertEquals(NUM_RECORDS_PRODUCED, allRows.size()); + + // Ensure that the table was created with the expected time partitioning type + StandardTableDefinition tableDefinition = bigQuery.getTable(TableId.of(testBase.dataset(), table)).getDefinition(); + Optional actualPartitioningType = Optional.ofNullable((tableDefinition).getTimePartitioning()) + .map(TimePartitioning::getType); + assertEquals(Optional.of(partitioningType), actualPartitioningType); + + // Verify that at least one record landed in each of the targeted partitions + if (usePartitionDecorator && messageTimePartitioning) { + for (int i = -1; i < 2; i++) { + long partitionTime = timestamp(i); + TableResult tableResult = bigQuery.query(QueryJobConfiguration.of(String.format( + "SELECT * FROM `%s`.`%s` WHERE _PARTITIONTIME = TIMESTAMP_TRUNC(TIMESTAMP_MILLIS(%d), %s)", + testBase.dataset(), + table, + partitionTime, + partitioningType.toString() + ))); + assertTrue( + "Should have seen at least one row in partition corresponding to timestamp " + partitionTime, + tableResult.getValues().iterator().hasNext() + ); + } + } + } + + private Converter converter() { + Map props = new HashMap<>(); + props.put(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, true); + Converter result = new JsonConverter(); + result.configure(props, false); + return result; + } + + private String value(Converter converter, String topic, long iteration) { + final Schema schema = SchemaBuilder.struct() + .optional() + .field("i", Schema.INT64_SCHEMA) + .field("f1", Schema.STRING_SCHEMA) + .field("f2", Schema.BOOLEAN_SCHEMA) + .field("f3", Schema.FLOAT64_SCHEMA) + .build(); + + final Struct struct = new Struct(schema) + .put("i", iteration) + .put("f1", iteration % 2 == 0 ? "a string" : "another string") + .put("f2", iteration % 3 == 0) + .put("f3", iteration / 39.80); + + return new String(converter.fromConnectData(topic, schema, struct)); + } + + /** + * @param shiftAmount how many partitions forward/backward to shift the timestamp by, + * relative to the partition corresponding to the start of the test + */ + private long timestamp(long shiftAmount) { + long partitionDelta; + switch (partitioningType) { + case HOUR: + partitionDelta = TimeUnit.HOURS.toMillis(1); + break; + case DAY: + partitionDelta = TimeUnit.DAYS.toMillis(1); + break; + case MONTH: + partitionDelta = TimeUnit.DAYS.toMillis(31); + break; + case YEAR: + partitionDelta = TimeUnit.DAYS.toMillis(366); + break; + default: + throw new ConnectException("Unexpected partitioning type: " + partitioningType); + } + + return testStartTime + (shiftAmount * partitionDelta); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java index dfe8ffc07..b6135ffa8 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/utils/PartitionedTableIdTest.java @@ -64,51 +64,6 @@ public void testTableIdBuilder() { Assert.assertEquals(tableId, partitionedTableId.getFullTableId()); } - @Test - public void testWithPartitionForHour() { - final String dataset = "dataset"; - final String table = "table"; - final LocalDateTime partitionDate = LocalDateTime.of(2016, 9, 21, 13, 55); - - final PartitionedTableId partitionedTableId = - new PartitionedTableId.Builder(dataset, table).setHourPartition(partitionDate).build(); - - final String expectedPartition = "2016092113"; - - Assert.assertEquals(dataset, partitionedTableId.getDataset()); - Assert.assertEquals(table, partitionedTableId.getBaseTableName()); - Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); - - final TableId expectedBaseTableId = TableId.of(dataset, table); - final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); - - Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); - Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); - } - - @Test - public void testWithEpochTimePartitionForHour() { - final String dataset = "dataset"; - final String table = "table"; - - final long utcTime = 1509007584334L; - - final PartitionedTableId partitionedTableId = - new PartitionedTableId.Builder(dataset, table).setHourPartition(utcTime).build(); - - final String expectedPartition = "2017102608"; - - Assert.assertEquals(dataset, partitionedTableId.getDataset()); - Assert.assertEquals(table, partitionedTableId.getBaseTableName()); - Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); - - final TableId expectedBaseTableId = TableId.of(dataset, table); - final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); - - Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); - Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); - } - @Test public void testWithPartition() { final String dataset = "dataset"; @@ -153,96 +108,4 @@ public void testWithEpochTimePartition() { Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); } - - @Test - public void testWithPartitionForMonth() { - final String dataset = "dataset"; - final String table = "table"; - final LocalDate partitionDate = LocalDate.of(2016, 9, 21); - - final PartitionedTableId partitionedTableId = - new PartitionedTableId.Builder(dataset, table). - setMonthPartition(partitionDate).build(); - - final String expectedPartition = "201609"; - - Assert.assertEquals(dataset, partitionedTableId.getDataset()); - Assert.assertEquals(table, partitionedTableId.getBaseTableName()); - Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); - - final TableId expectedBaseTableId = TableId.of(dataset, table); - final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); - - Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); - Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); - } - - @Test - public void testWithEpochTimePartitionForMonth() { - final String dataset = "dataset"; - final String table = "table"; - - final long utcTime = 1509007584334L; - - final PartitionedTableId partitionedTableId = - new PartitionedTableId.Builder(dataset, table).setMonthPartition(utcTime).build(); - - final String expectedPartition = "201710"; - - Assert.assertEquals(dataset, partitionedTableId.getDataset()); - Assert.assertEquals(table, partitionedTableId.getBaseTableName()); - Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); - - final TableId expectedBaseTableId = TableId.of(dataset, table); - final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); - - Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); - Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); - } - - @Test - public void testWithPartitionForYear() { - final String dataset = "dataset"; - final String table = "table"; - final LocalDate partitionDate = LocalDate.of(2016, 9, 21); - - final PartitionedTableId partitionedTableId = - new PartitionedTableId.Builder(dataset, table). - setYearPartition(partitionDate).build(); - - final String expectedPartition = "2016"; - - Assert.assertEquals(dataset, partitionedTableId.getDataset()); - Assert.assertEquals(table, partitionedTableId.getBaseTableName()); - Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); - - final TableId expectedBaseTableId = TableId.of(dataset, table); - final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); - - Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); - Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); - } - - @Test - public void testWithEpochTimePartitionForYear() { - final String dataset = "dataset"; - final String table = "table"; - - final long utcTime = 1509007584334L; - - final PartitionedTableId partitionedTableId = - new PartitionedTableId.Builder(dataset, table).setYearPartition(utcTime).build(); - - final String expectedPartition = "2017"; - - Assert.assertEquals(dataset, partitionedTableId.getDataset()); - Assert.assertEquals(table, partitionedTableId.getBaseTableName()); - Assert.assertEquals(table + "$" + expectedPartition, partitionedTableId.getFullTableName()); - - final TableId expectedBaseTableId = TableId.of(dataset, table); - final TableId expectedFullTableId = TableId.of(dataset, table + "$" + expectedPartition); - - Assert.assertEquals(expectedBaseTableId, partitionedTableId.getBaseTableId()); - Assert.assertEquals(expectedFullTableId, partitionedTableId.getFullTableId()); - } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index 87520663d..acd1d0d00 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -33,6 +33,7 @@ import com.google.cloud.bigquery.InsertAllRequest; import com.google.cloud.bigquery.InsertAllResponse; import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableId; import com.google.cloud.storage.Storage; import com.wepay.kafka.connect.bigquery.BigQuerySinkTask; @@ -56,6 +57,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -80,12 +82,9 @@ public void testBigQueryNoFailure() { Table mockTable = mock(Table.class); when(bigQuery.getTable(any())).thenReturn(mockTable); - Map> emptyMap = mock(Map.class); - when(emptyMap.isEmpty()).thenReturn(true); - InsertAllResponse insertAllResponse = mock(InsertAllResponse.class); when(insertAllResponse.hasErrors()).thenReturn(false); - when(insertAllResponse.getInsertErrors()).thenReturn(emptyMap); + when(insertAllResponse.getInsertErrors()).thenReturn(Collections.emptyMap()); //first attempt (success) when(bigQuery.insertAll(anyObject())) @@ -97,7 +96,7 @@ public void testBigQueryNoFailure() { SchemaManager schemaManager = mock(SchemaManager.class); Storage storage = mock(Storage.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -117,12 +116,10 @@ public void testAutoCreateTables() { properties.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); BigQuery bigQuery = mock(BigQuery.class); - Map> emptyMap = mock(Map.class); - when(emptyMap.isEmpty()).thenReturn(true); InsertAllResponse insertAllResponse = mock(InsertAllResponse.class); when(insertAllResponse.hasErrors()).thenReturn(false); - when(insertAllResponse.getInsertErrors()).thenReturn(emptyMap); + when(insertAllResponse.getInsertErrors()).thenReturn(Collections.emptyMap()); String errorMessage = "Not found: Table project.scratch.test_topic"; BigQueryError error = new BigQueryError("notFound", "global", errorMessage); @@ -135,7 +132,7 @@ public void testAutoCreateTables() { Storage storage = mock(Storage.class); SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -158,11 +155,9 @@ public void testNonAutoCreateTables() { Table mockTable = mock(Table.class); when(bigQuery.getTable(any())).thenReturn(mockTable); - Map> emptyMap = mock(Map.class); - when(emptyMap.isEmpty()).thenReturn(true); InsertAllResponse insertAllResponse = mock(InsertAllResponse.class); when(insertAllResponse.hasErrors()).thenReturn(false); - when(insertAllResponse.getInsertErrors()).thenReturn(emptyMap); + when(insertAllResponse.getInsertErrors()).thenReturn(Collections.emptyMap()); BigQueryException missTableException = new BigQueryException(404, "Table is missing"); @@ -173,7 +168,7 @@ public void testNonAutoCreateTables() { Storage storage = mock(Storage.class); SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -188,24 +183,16 @@ public void testBigQueryPartialFailure() { final String topic = "test_topic"; final String dataset = "scratch"; final Map properties = makeProperties("3", "2000", topic, dataset); - final Set failedRowSet = new HashSet<>(); - failedRowSet.add(1L); - - Map> insertErrorMap = mock(Map.class); - when(insertErrorMap.isEmpty()).thenReturn(false); - when(insertErrorMap.size()).thenReturn(1); - when(insertErrorMap.keySet()).thenReturn(failedRowSet); + BigQueryError insertError = new BigQueryError("reason", "location", "message"); + Map> insertErrorMap = Collections.singletonMap(1L, Collections.singletonList(insertError)); InsertAllResponse insertAllResponseWithError = mock(InsertAllResponse.class); when(insertAllResponseWithError.hasErrors()).thenReturn(true); when(insertAllResponseWithError.getInsertErrors()).thenReturn(insertErrorMap); - Map> emptyMap = mock(Map.class); - when(emptyMap.isEmpty()).thenReturn(true); - InsertAllResponse insertAllResponseNoError = mock(InsertAllResponse.class); when(insertAllResponseNoError.hasErrors()).thenReturn(true); - when(insertAllResponseNoError.getInsertErrors()).thenReturn(emptyMap); + when(insertAllResponseNoError.getInsertErrors()).thenReturn(Collections.emptyMap()); BigQuery bigQuery = mock(BigQuery.class); Table mockTable = mock(Table.class); @@ -224,7 +211,7 @@ public void testBigQueryPartialFailure() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); Storage storage = mock(Storage.class); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); @@ -247,21 +234,19 @@ public void testBigQueryCompleteFailure() { final String topic = "test_topic"; final String dataset = "scratch"; final Map properties = makeProperties("3", "2000", topic, dataset); + BigQueryError insertError = new BigQueryError("reason", "location", "message"); - Map> insertErrorMap = mock(Map.class); - when(insertErrorMap.isEmpty()).thenReturn(false); - when(insertErrorMap.size()).thenReturn(2); + Map> insertErrorMap = new HashMap<>(); + insertErrorMap.put(1L, Collections.singletonList(insertError)); + insertErrorMap.put(2L, Collections.singletonList(insertError)); InsertAllResponse insertAllResponseWithError = mock(InsertAllResponse.class); when(insertAllResponseWithError.hasErrors()).thenReturn(true); when(insertAllResponseWithError.getInsertErrors()).thenReturn(insertErrorMap); - Map> emptyMap = mock(Map.class); - when(emptyMap.isEmpty()).thenReturn(true); - InsertAllResponse insertAllResponseNoError = mock(InsertAllResponse.class); when(insertAllResponseNoError.hasErrors()).thenReturn(true); - when(insertAllResponseNoError.getInsertErrors()).thenReturn(emptyMap); + when(insertAllResponseNoError.getInsertErrors()).thenReturn(Collections.emptyMap()); BigQuery bigQuery = mock(BigQuery.class); Table mockTable = mock(Table.class); @@ -279,7 +264,7 @@ public void testBigQueryCompleteFailure() { SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); Storage storage = mock(Storage.class); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java index a0b2ce2b1..d81c48717 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/GCSToBQWriterTest.java @@ -21,6 +21,7 @@ import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableId; import com.google.cloud.storage.BlobInfo; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageException; @@ -41,6 +42,7 @@ import org.junit.Test; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import static org.mockito.Matchers.anyObject; @@ -72,7 +74,7 @@ public void testGCSNoFailure(){ SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); testTask.initialize(sinkTaskContext); @@ -98,7 +100,7 @@ public void testGCSSomeFailures(){ SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); when(storage.create((BlobInfo)anyObject(), (byte[])anyObject())) .thenThrow(new StorageException(500, "internal server error")) // throw first time @@ -128,7 +130,7 @@ public void testGCSAllFailures(){ SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = mock(Map.class); + Map cache = new HashMap<>(); when(storage.create((BlobInfo)anyObject(), (byte[])anyObject())) .thenThrow(new StorageException(500, "internal server error")); diff --git a/kcbq-connector/src/test/resources/log4j.properties b/kcbq-connector/src/test/resources/log4j.properties index 6de3c0539..8e7c8ffef 100644 --- a/kcbq-connector/src/test/resources/log4j.properties +++ b/kcbq-connector/src/test/resources/log4j.properties @@ -32,6 +32,8 @@ log4j.appender.connectAppender.layout.ConversionPattern=${connect.log.pattern} log4j.logger.org.apache.zookeeper=ERROR log4j.logger.org.reflections=ERROR +log4j.logger.com.wepay.kafka.connect.bigquery=DEBUG + # We see a lot of WARN-level messages from this class when a table is created by the connector and # then written to shortly after. No need for that much noise during routine tests log4j.logger.com.wepay.kafka.connect.bigquery.write.batch.TableWriter=ERROR From dbb1f09cef4fe59bb80addfaa239796f16f362a0 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 15 Nov 2021 21:38:21 +0000 Subject: [PATCH 118/190] [maven-release-plugin] prepare release v2.0.10 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 954857403..08f935885 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.10-SNAPSHOT + 2.0.10 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 6519d5f2d..ea4b27fd5 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.10-SNAPSHOT + 2.0.10 .. diff --git a/pom.xml b/pom.xml index e2ddf5122..7c238279f 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.10-SNAPSHOT + 2.0.10 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.10 From 0f1ef13f25c1f5324dfd8a4d9180eec3a83a8be9 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 15 Nov 2021 21:38:24 +0000 Subject: [PATCH 119/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 08f935885..42376d47e 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.10 + 2.0.11-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index ea4b27fd5..43f377040 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.10 + 2.0.11-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 7c238279f..71a820396 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.10 + 2.0.11-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.10 + HEAD From b85d1012a4b266ea56c92a10d559743490ee0bda Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 15 Nov 2021 21:41:42 +0000 Subject: [PATCH 120/190] [maven-release-plugin] prepare release v2.1.9 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index c23366e99..449fc53e9 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.9-SNAPSHOT + 2.1.9 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 0e92a5966..53afe2dd1 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.9-SNAPSHOT + 2.1.9 .. diff --git a/pom.xml b/pom.xml index 899a91049..795973cce 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.9-SNAPSHOT + 2.1.9 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.9 From 1bfb8ccda1f8830ac90e31c9f6585c058e7cb205 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Mon, 15 Nov 2021 21:41:44 +0000 Subject: [PATCH 121/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 449fc53e9..e97431bc7 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.9 + 2.1.10-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 53afe2dd1..c09eb910a 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.9 + 2.1.10-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 795973cce..95f6636b4 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.9 + 2.1.10-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.9 + 2.1.x From d8fc535fc19236d3c3db9c35fb30bf05f37e3669 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 17 Nov 2021 10:15:25 -0500 Subject: [PATCH 122/190] GH-149: Improve error messages on write thread failure (#150) * GH-149: Improve error messages on write thread failure * GH-149: Add more detail to error message for batch reduction error --- .../bigquery/BigQuerySinkConnector.java | 11 +-- .../connect/bigquery/BigQuerySinkTask.java | 11 +-- .../write/batch/KCBQThreadPoolExecutor.java | 68 ++++--------------- .../bigquery/write/batch/TableWriter.java | 20 +++++- 4 files changed, 35 insertions(+), 75 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index d2fefb8fb..8d6cf7ac1 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -107,15 +107,8 @@ private void ensureExistingTables() { @Override public void start(Map properties) { logger.trace("connector.start()"); - try { - configProperties = properties; - config = new BigQuerySinkConfig(properties); - } catch (ConfigException err) { - throw new SinkConfigConnectException( - "Couldn't start BigQuerySinkConnector due to configuration error", - err - ); - } + configProperties = properties; + config = new BigQuerySinkConfig(properties); if (!config.getBoolean(BigQuerySinkConfig.TABLE_CREATE_CONFIG)) { ensureExistingTables(); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 4f59c07d8..a12a93bee 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -187,7 +187,7 @@ private String getRowId(SinkRecord record) { @Override public void put(Collection records) { // Periodically poll for errors here instead of doing a stop-the-world check in flush() - executor.maybeThrowEncounteredErrors(); + executor.maybeThrowEncounteredError(); logger.debug("Putting {} records in the sink.", records.size()); @@ -324,14 +324,7 @@ private GCSToBQWriter getGcsWriter() { @Override public void start(Map properties) { logger.trace("task.start()"); - try { - config = new BigQuerySinkTaskConfig(properties); - } catch (ConfigException err) { - throw new SinkConfigConnectException( - "Couldn't start BigQuerySinkTask due to configuration error", - err - ); - } + config = new BigQuerySinkTaskConfig(properties); bigQueryWriter = getBigQueryWriter(); gcsToBQWriter = getGcsWriter(); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java index 003dca988..491d9b0a1 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KCBQThreadPoolExecutor.java @@ -21,18 +21,16 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import org.apache.kafka.connect.errors.ConnectException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; +import java.util.Optional; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; +import java.util.concurrent.atomic.AtomicReference; /** * ThreadPoolExecutor for writing Rows to BigQuery. @@ -44,9 +42,7 @@ public class KCBQThreadPoolExecutor extends ThreadPoolExecutor { private static final Logger logger = LoggerFactory.getLogger(KCBQThreadPoolExecutor.class); - - private final ConcurrentHashMap.KeySetView encounteredErrors = - ConcurrentHashMap.newKeySet(); + private final AtomicReference encounteredError = new AtomicReference<>(); /** * @param config the {@link BigQuerySinkTaskConfig} @@ -66,11 +62,10 @@ protected void afterExecute(Runnable runnable, Throwable throwable) { super.afterExecute(runnable, throwable); if (throwable != null) { - logger.error("Task failed with {} error: {}", - throwable.getClass().getName(), - throwable.getMessage()); - logger.debug("Error Task Stacktrace:", throwable); - encounteredErrors.add(throwable); + // Log at debug level since this will be shown to the user at error level by the Connect framework if it causes + // the task to fail, and will otherwise just pollute logs and potentially mislead users + logger.debug("A write thread has failed with an unrecoverable error", throwable); + encounteredError.compareAndSet(null, throwable); } } @@ -92,12 +87,7 @@ public void awaitCurrentTasks() throws InterruptedException, BigQueryConnectExce execute(new CountDownRunnable(countDownLatch)); } countDownLatch.await(); - maybeThrowEncounteredErrors(); - if (encounteredErrors.size() > 0) { - String errorString = createErrorString(encounteredErrors); - throw new BigQueryConnectException("Some write threads encountered unrecoverable errors: " - + errorString + "; See logs for more detail"); - } + maybeThrowEncounteredError(); } /** @@ -106,41 +96,9 @@ public void awaitCurrentTasks() throws InterruptedException, BigQueryConnectExce * * @throws BigQueryConnectException if any of the tasks failed. */ - public void maybeThrowEncounteredErrors() { - if (encounteredErrors.size() > 0) { - String errorString = createErrorString(encounteredErrors); - throw new BigQueryConnectException("Some write threads encountered unrecoverable errors: " - + errorString + "; See logs for more detail"); - } - } - - private static String createErrorString(Collection errors) { - List exceptionTypeStrings = new ArrayList<>(errors.size()); - exceptionTypeStrings.addAll(errors.stream() - .map(throwable -> throwable.getClass().getName()) - .collect(Collectors.toList())); - return String.join(", ", exceptionTypeStrings); - } - - private static String createDetailedErrorString(Collection errors) { - List exceptionTypeStrings = new ArrayList<>(errors.size()); - exceptionTypeStrings.addAll(errors.stream() - .map(throwable -> - throwable.getClass().getName() + "\nMessage: " + throwable.getLocalizedMessage()) - .collect(Collectors.toList())); - return String.join(", ", exceptionTypeStrings); - } - - /** - * Checks for BigQuery errors. No-op if there isn't any error. - * - * @throws BigQueryConnectException if there have been any unrecoverable errors when writing to BigQuery. - */ - public void maybeFail() throws BigQueryConnectException { - if (encounteredErrors.size() > 0) { - throw new BigQueryConnectException("Encountered unrecoverable errors: " - + createDetailedErrorString(encounteredErrors) + "; See logs for more detail"); - } + public void maybeThrowEncounteredError() { + Optional.ofNullable(encounteredError.get()).ifPresent(t -> { + throw new BigQueryConnectException("A write thread has failed with an unrecoverable error", t); + }); } - } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index 3ff20037b..53f49e895 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -111,8 +111,24 @@ public void run() { private static int getNewBatchSize(int currentBatchSize, Throwable err) { if (currentBatchSize == 1) { - // todo correct exception type? - throw new BigQueryConnectException("Attempted to reduce batch size below 1.", err); + logger.error("Attempted to reduce batch size below 1"); + throw new BigQueryConnectException( + "Failed to write to BigQuery even after reducing batch size to 1 row at a time. " + + "This can indicate an error in the connector's logic for classifying BigQuery errors, as non-retriable" + + "errors may be being treated as retriable." + + "If that appears to be the case, please report the issue to the project's maintainers and include the " + + "complete stack trace for this error as it appears in the logs. " + + "Alternatively, there may be a record that the connector has read from Kafka that is too large to " + + "write to BigQuery using the streaming insert API, which cannot be addressed with a change to the " + + "connector and will need to be handled externally by optionally writing the record to BigQuery using " + + "another means and then reconfiguring the connector to skip the record. " + + "Finally, streaming insert quotas for BigQuery may be causing insertion failures for the connector; " + + "in that case, please ensure that quotas for maximum rows per second, maximum bytes per second, etc. " + + "are being respected before restarting the connector. " + + "The cause of this exception is the error encountered from BigQuery after the last attempt to write a " + + "batch was made.", + err + ); } // round batch size up so we don't end up with a dangling 1 row at the end. return (int) Math.ceil(currentBatchSize / 2.0); From 5280d0e1417e964e6a961b267896bc053bd34a7d Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 17 Nov 2021 11:13:57 -0500 Subject: [PATCH 123/190] GH-139: Improve preflight config validation (#153) * GH-139: Improve preflight config validation Depends on https://github.com/confluentinc/kafka-connect-bigquery/pull/140 Addresses the remainder of https://github.com/confluentinc/kafka-connect-bigquery/issues/139. All multi-property configuration validation is moved from the constructor of the BigQuerySinkConfig class to a new `BigQuerySinkConfig::validate` method, which is invoked from `BigQuerySinkConnector::validate` and leverages the Connect API for defining custom error messages on a per-property basis instead of throwing exceptions on invalid configurations, which only displays one message at a time and doesn't work well with programmatic UIs. Unit tests are added for all new validation logic. The logic for GCP client construction is also updated to conform to a common interface, which should improve readability and maintainability, and makes it easier to validate credentials for either BigQuery or GCS. Finally, the `SinkConfigConnectException` class is removed as it's not really necessary and doesn't bring anything worthwhile to the code base. * Add config revalidation to connector start method * Simplify error message logic for CredentialsValidator * Add PartitioningModeValidator test case for when decorator syntax is enabled but no timestamp partition field is provided --- .../connect/bigquery/BigQueryHelper.java | 108 ------ .../bigquery/BigQuerySinkConnector.java | 67 +--- .../connect/bigquery/BigQuerySinkTask.java | 19 +- .../kafka/connect/bigquery/GCSBuilder.java | 109 ------ .../connect/bigquery/GcpClientBuilder.java | 145 ++++++++ .../bigquery/config/BigQuerySinkConfig.java | 330 ++++++++---------- .../bigquery/config/CredentialsValidator.java | 117 +++++++ .../bigquery/config/GcsBucketValidator.java | 63 ++++ .../config/MultiPropertyValidator.java | 70 ++++ .../config/PartitioningModeValidator.java | 60 ++++ .../config/SchemaRetrieverValidator.java | 105 ++++++ .../config/TableExistenceValidator.java | 108 ++++++ .../exception/SinkConfigConnectException.java | 40 --- .../bigquery/BigQuerySinkConnectorTest.java | 68 +--- .../bigquery/BigQuerySinkTaskTest.java | 21 -- .../bigquery/SinkPropertiesFactory.java | 18 - .../config/BigQuerySinkConfigTest.java | 44 +-- .../config/CredentialsValidatorTest.java | 69 ++++ .../config/GcsBucketValidatorTest.java | 93 +++++ .../config/MultiPropertyValidatorTest.java | 138 ++++++++ .../config/PartitioningModeValidatorTest.java | 80 +++++ .../config/SchemaRetrieverValidatorTest.java | 107 ++++++ .../config/TableExistenceValidatorTest.java | 160 +++++++++ .../it/BigQueryConnectorIntegrationTest.java | 50 ++- .../bigquery/it/utils/BucketClearer.java | 17 +- .../bigquery/it/utils/TableClearer.java | 16 +- 26 files changed, 1540 insertions(+), 682 deletions(-) delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/SchemaRetrieverValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/TableExistenceValidator.java delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/SchemaRetrieverValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/TableExistenceValidatorTest.java diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java deleted file mode 100644 index f90ea5f3f..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery; - -import com.google.auth.oauth2.GoogleCredentials; -import com.google.cloud.bigquery.BigQuery; -import com.google.cloud.bigquery.BigQueryOptions; - -import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.ByteArrayInputStream; -import java.nio.charset.StandardCharsets; - -/** - * Convenience class for creating a default {@link com.google.cloud.bigquery.BigQuery} instance, - * with or without login credentials. - */ -public class BigQueryHelper { - private static final Logger logger = LoggerFactory.getLogger(BigQueryHelper.class); - private static String keySource; - - /** - * Returns a default {@link BigQuery} instance for the specified project with credentials provided - * in the specified file, which can then be used for creating, updating, and inserting into tables - * from specific datasets. - * - * @param projectName The name of the BigQuery project to work with - * @param key The google credentials JSON key that can be used to provide - * credentials to BigQuery, or null if no authentication should be performed. - * @return The resulting BigQuery object. - */ - public BigQuery connect(String projectName, String key) { - if (key == null) { - return connect(projectName); - } - logger.debug("Attempting to open file {} for service account json key", key); - InputStream credentialsStream; - try { - if (keySource != null && keySource.equals("JSON")) { - credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); - } else { - credentialsStream = new FileInputStream(key); - } - return new - BigQueryOptions.DefaultBigQueryFactory().create( - BigQueryOptions.newBuilder() - .setProjectId(projectName) - .setCredentials(GoogleCredentials.fromStream(credentialsStream)) - .build() - ); - } catch (IOException err) { - throw new BigQueryConnectException("Failed to access json key file", err); - } - } - /** - * Returns a default {@link BigQuery} instance for the specified project with credentials provided - * in the specified file, which can then be used for creating, updating, and inserting into tables - * from specific datasets. - * - * @param keySource The type of key config we can expect. This is either a String - * representation of the Google credentials file, or the path to the Google credentials file. - * @return The resulting BigQuery object. - */ - public BigQueryHelper setKeySource(String keySource) { - this.keySource = keySource; - return this; - } - - /** - * Returns a default {@link BigQuery} instance for the specified project with no authentication - * credentials, which can then be used for creating, updating, and inserting into tables from - * specific datasets. - * - * @param projectName The name of the BigQuery project to work with - * @return The resulting BigQuery object. - */ - public BigQuery connect(String projectName) { - logger.debug("Attempting to access BigQuery without authentication"); - return new BigQueryOptions.DefaultBigQueryFactory().create( - BigQueryOptions.newBuilder() - .setProjectId(projectName) - .build() - ); - } -} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index 8d6cf7ac1..05c912e07 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -19,21 +19,16 @@ package com.wepay.kafka.connect.bigquery; -import com.google.cloud.bigquery.BigQuery; -import com.google.cloud.bigquery.TableId; - import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; -import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; -import com.wepay.kafka.connect.bigquery.utils.TopicToTableResolver; import com.wepay.kafka.connect.bigquery.utils.Version; +import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.sink.SinkConnector; @@ -50,28 +45,9 @@ * {@link org.apache.kafka.connect.sink.SinkTask SinkTasks}. */ public class BigQuerySinkConnector extends SinkConnector { - private final BigQuery testBigQuery; - private final SchemaManager testSchemaManager; - - public BigQuerySinkConnector() { - testBigQuery = null; - testSchemaManager = null; - } - - // For testing purposes only; will never be called by the Kafka Connect framework - BigQuerySinkConnector(BigQuery bigQuery) { - this.testBigQuery = bigQuery; - this.testSchemaManager = null; - } - - // For testing purposes only; will never be called by the Kafka Connect framework - BigQuerySinkConnector(BigQuery bigQuery, SchemaManager schemaManager) { - this.testBigQuery = bigQuery; - this.testSchemaManager = schemaManager; - } - private BigQuerySinkConfig config; - private Map configProperties; + BigQuerySinkConfig config; + Map configProperties; private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkConnector.class); @@ -81,27 +57,16 @@ public ConfigDef config() { return BigQuerySinkConfig.getConfig(); } - private BigQuery getBigQuery() { - if (testBigQuery != null) { - return testBigQuery; - } - String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - String key = config.getKeyFile(); - String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); - return new BigQueryHelper().setKeySource(keySource).connect(projectName, key); - } - - private void ensureExistingTables() { - BigQuery bigQuery = getBigQuery(); - Map topicsToTableIds = TopicToTableResolver.getTopicsToTables(config); - for (TableId tableId : topicsToTableIds.values()) { - if (bigQuery.getTable(tableId) == null) { - logger.warn( - "You may want to enable auto table creation by setting {}=true in the properties file", - BigQuerySinkConfig.TABLE_CREATE_CONFIG); - throw new BigQueryConnectException("Table '" + tableId + "' does not exist"); - } + @Override + public Config validate(Map properties) { + List singlePropertyValidations = config().validate(properties); + // If any of our properties had malformed syntax or failed a validation to ensure, e.g., that it fell within an + // acceptable numeric range, we only report those errors since they prevent us from being able to construct a + // valid BigQuerySinkConfig instance + if (singlePropertyValidations.stream().anyMatch(v -> !v.errorMessages().isEmpty())) { + return new Config(singlePropertyValidations); } + return new BigQuerySinkConfig(properties).validate(); } @Override @@ -109,10 +74,8 @@ public void start(Map properties) { logger.trace("connector.start()"); configProperties = properties; config = new BigQuerySinkConfig(properties); - - if (!config.getBoolean(BigQuerySinkConfig.TABLE_CREATE_CONFIG)) { - ensureExistingTables(); - } + // Revalidate here in case the connector has been upgraded and its old config is no longer valid + config.ensureValid(); } @Override diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index a12a93bee..452024165 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -33,7 +33,6 @@ import com.wepay.kafka.connect.bigquery.convert.KafkaDataBuilder; import com.wepay.kafka.connect.bigquery.convert.RecordConverter; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.utils.TopicToTableResolver; @@ -48,7 +47,6 @@ import com.wepay.kafka.connect.bigquery.write.row.SimpleBigQueryWriter; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; @@ -255,10 +253,9 @@ private BigQuery getBigQuery() { if (testBigQuery != null) { return testBigQuery; } - String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - String keyFile = config.getKeyFile(); - String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); - return new BigQueryHelper().setKeySource(keySource).connect(projectName, keyFile); + return new GcpClientBuilder.BigQueryBuilder() + .withConfig(config) + .build(); } private SchemaManager getSchemaManager(BigQuery bigQuery) { @@ -271,7 +268,7 @@ private SchemaManager getSchemaManager(BigQuery bigQuery) { Optional kafkaKeyFieldName = config.getKafkaKeyFieldName(); Optional kafkaDataFieldName = config.getKafkaDataFieldName(); Optional timestampPartitionFieldName = config.getTimestampPartitionFieldName(); - Optional> clusteringFieldName = config.getClusteringPartitionFieldName(); + Optional> clusteringFieldName = config.getClusteringPartitionFieldNames(); return new SchemaManager(schemaRetriever, schemaConverter, bigQuery, kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, clusteringFieldName); } @@ -298,11 +295,9 @@ private Storage getGcs() { if (testGcs != null) { return testGcs; } - String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - String key = config.getKeyFile(); - String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); - return new GCSBuilder(projectName).setKey(key).setKeySource(keySource).build(); - + return new GcpClientBuilder.GcsBuilder() + .withConfig(config) + .build(); } private GCSToBQWriter getGcsWriter() { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java deleted file mode 100644 index 4a0952b2e..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery; - -import com.google.auth.oauth2.GoogleCredentials; -import com.google.cloud.storage.Storage; -import com.google.cloud.storage.StorageOptions; - -import com.wepay.kafka.connect.bigquery.exception.GCSConnectException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.ByteArrayInputStream; -import java.nio.charset.StandardCharsets; - -/** - * Convenience class for creating a {@link com.google.cloud.storage.Storage} instance - */ -public class GCSBuilder { - private static final Logger logger = LoggerFactory.getLogger(GCSBuilder.class); - - private final String projectName; - private String key; - private String keySource; - - public GCSBuilder(String projectName) { - this.projectName = projectName; - this.key = null; - } - - public GCSBuilder setKeySource(String keySourceType) { - this.keySource = keySourceType; - return this; - } - - public GCSBuilder setKey(String keyFile) { - this.key = keyFile; - return this; - } - public Storage build() { - return connect(projectName, key); - } - - /** - * Returns a default {@link Storage} instance for the specified project with credentials provided - * in the specified file. - * - * @param projectName The name of the GCS project to work with - * @param key The name of a file containing a JSON key that can be used to provide - * credentials to GCS, or null if no authentication should be performed. - * @return The resulting Storage object. - */ - private Storage connect(String projectName, String key) { - if (key == null) { - return connect(projectName); - } - try { - InputStream credentialsStream; - if (keySource != null && keySource.equals("JSON")) { - credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); - } else { - credentialsStream = new FileInputStream(key); - } - return StorageOptions.newBuilder() - .setProjectId(projectName) - .setCredentials(GoogleCredentials.fromStream(credentialsStream)) - .build() - .getService(); - } catch (IOException err) { - throw new GCSConnectException("Failed to access json key file", err); - } - } - - /** - * Returns a default {@link Storage} instance for the specified project with no authentication - * credentials. - * - * @param projectName The name of the GCS project to work with - * @return The resulting Storage object. - */ - private Storage connect(String projectName) { - logger.debug("Attempting to access BigQuery without authentication"); - return StorageOptions.newBuilder() - .setProjectId(projectName) - .build() - .getService(); - } -} - diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java new file mode 100644 index 000000000..5c79fec87 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java @@ -0,0 +1,145 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryOptions; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageOptions; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Objects; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; + +public abstract class GcpClientBuilder { + + public enum KeySource { + FILE, JSON + } + + private static final Logger logger = LoggerFactory.getLogger(GcpClientBuilder.class); + + private String project = null; + private KeySource keySource = null; + private String key = null; + + public GcpClientBuilder withConfig(BigQuerySinkConfig config) { + return withProject(config.getString(PROJECT_CONFIG)) + .withKeySource(config.getKeySource()) + .withKey(config.getKey()); + } + + public GcpClientBuilder withProject(String project) { + Objects.requireNonNull(project, "Project cannot be null"); + this.project = project; + return this; + } + + public GcpClientBuilder withKeySource(KeySource keySource) { + Objects.requireNonNull(keySource, "Key cannot be null"); + this.keySource = keySource; + return this; + } + + public GcpClientBuilder withKey(String key) { + this.key = key; + return this; + } + + public Client build() { + return doBuild(project, credentials()); + } + + private GoogleCredentials credentials() { + if (key == null) { + return null; + } + + Objects.requireNonNull(keySource, "Key source must be defined to build a GCP client"); + Objects.requireNonNull(project, "Project must be defined to build a GCP client"); + + InputStream credentialsStream; + switch (keySource) { + case JSON: + credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); + break; + case FILE: + try { + logger.debug("Attempting to open file {} for service account json key", key); + credentialsStream = new FileInputStream(key); + } catch (IOException e) { + throw new BigQueryConnectException("Failed to access JSON key file", e); + } + break; + default: + throw new IllegalArgumentException("Unexpected value for KeySource enum: " + keySource); + } + + try { + return GoogleCredentials.fromStream(credentialsStream); + } catch (IOException e) { + throw new BigQueryConnectException("Failed to create credentials from input stream", e); + } + } + + protected abstract Client doBuild(String project, GoogleCredentials credentials); + + public static class BigQueryBuilder extends GcpClientBuilder { + @Override + protected BigQuery doBuild(String project, GoogleCredentials credentials) { + BigQueryOptions.Builder builder = BigQueryOptions.newBuilder() + .setProjectId(project); + + if (credentials != null) { + builder.setCredentials(credentials); + } else { + logger.debug("Attempting to access BigQuery without authentication"); + } + + return builder.build().getService(); + } + } + + public static class GcsBuilder extends GcpClientBuilder { + @Override + protected Storage doBuild(String project, GoogleCredentials credentials) { + StorageOptions.Builder builder = StorageOptions.newBuilder() + .setProjectId(project); + + if (credentials != null) { + builder.setCredentials(credentials); + } else { + logger.debug("Attempting to access GCS without authentication"); + } + + return builder.build().getService(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index c7cc02593..32b0b1821 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -21,6 +21,7 @@ import com.google.cloud.bigquery.Schema; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.convert.BigQueryRecordConverter; @@ -29,13 +30,14 @@ import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.Config; +import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkConnector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; @@ -46,17 +48,17 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Base class for connector and task configs; contains properties shared between the two of them. */ public class BigQuerySinkConfig extends AbstractConfig { - private static final Validator validator = new Validator(); - private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkConfig.class); - // Values taken from https://github.com/apache/kafka/blob/1.1.1/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java#L33 public static final String TOPICS_CONFIG = SinkConnector.TOPICS_CONFIG; private static final ConfigDef.Type TOPICS_TYPE = ConfigDef.Type.LIST; @@ -118,6 +120,7 @@ public class BigQuerySinkConfig extends AbstractConfig { private static final ConfigDef.Importance TOPICS_TO_TABLES_IMPORTANCE = ConfigDef.Importance.MEDIUM; public static final Object TOPICS_TO_TABLES_DEFAULT = null; + private static final ConfigDef.Validator TOPICS_TO_TABLES_VALIDATOR = new MapValidator(); private static final String TOPICS_TO_TABLES_DOC = "A list of mappings from topic regexes to table names. Note the regex must include " + "capture groups that are referenced in the format string using placeholders (i.e. $1) " @@ -132,7 +135,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String DATASETS_CONFIG = "datasets"; private static final ConfigDef.Type DATASETS_TYPE = ConfigDef.Type.LIST; private static final Object DATASETS_DEFAULT = ConfigDef.NO_DEFAULT_VALUE; - private static final ConfigDef.Validator DATASETS_VALIDATOR = validator; + private static final ConfigDef.Validator DATASETS_VALIDATOR = new MapValidator(); private static final ConfigDef.Importance DATASETS_IMPORTANCE = ConfigDef.Importance.HIGH; private static final String DATASETS_DOC = "Names for the datasets kafka topics will write to " @@ -155,9 +158,13 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KEY_SOURCE_CONFIG = "keySource"; private static final ConfigDef.Type KEY_SOURCE_TYPE = ConfigDef.Type.STRING; - public static final String KEY_SOURCE_DEFAULT = "FILE"; - private static final ConfigDef.Validator KEY_SOURCE_VALIDATOR = - ConfigDef.ValidString.in("FILE", "JSON"); + public static final String KEY_SOURCE_DEFAULT = GcpClientBuilder.KeySource.FILE.name(); + private static final ConfigDef.Validator KEY_SOURCE_VALIDATOR = ConfigDef.ValidString.in( + Stream.of(GcpClientBuilder.KeySource.values()) + .map(GcpClientBuilder.KeySource::name) + .collect(Collectors.toList()) + .toArray(new String[0]) + ); private static final ConfigDef.Importance KEY_SOURCE_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String KEY_SOURCE_DOC = "Determines whether the keyfile config is the path to the credentials json, or the json itself"; @@ -187,6 +194,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KAFKA_KEY_FIELD_NAME_CONFIG = "kafkaKeyFieldName"; private static final ConfigDef.Type KAFKA_KEY_FIELD_NAME_TYPE = ConfigDef.Type.STRING; public static final String KAFKA_KEY_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator KAFKA_KEY_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance KAFKA_KEY_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String KAFKA_KEY_FIELD_NAME_DOC = "The name of the field of Kafka key. " + "Default to be null, which means Kafka Key Field will not be included."; @@ -194,6 +202,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KAFKA_DATA_FIELD_NAME_CONFIG = "kafkaDataFieldName"; private static final ConfigDef.Type KAFKA_DATA_FIELD_NAME_TYPE = ConfigDef.Type.STRING; public static final String KAFKA_DATA_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator KAFKA_DATA_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance KAFKA_DATA_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String KAFKA_DATA_FIELD_NAME_DOC = "The name of the field of Kafka Data. " + "Default to be null, which means Kafka Data Field will not be included. "; @@ -310,6 +319,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG = "timestampPartitionFieldName"; private static final ConfigDef.Type BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE = ConfigDef.Type.STRING; private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC = @@ -320,6 +330,17 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG = "clusteringPartitionFieldNames"; private static final ConfigDef.Type BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE = ConfigDef.Type.LIST; private static final List BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT = null; + private static final ConfigDef.Validator BIGQUERY_CLUSTERING_FIELD_NAMES_VALIDATOR = (name, value) -> { + if (value == null) { + return; + } + + @SuppressWarnings("unchecked") + List parsedValue = (List) value; + if (parsedValue.size() > 4) { + throw new ConfigException(name, value, "You may only specify up to four clustering field names."); + } + }; private static final ConfigDef.Importance BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE = ConfigDef.Importance.LOW; private static final String BIGQUERY_CLUSTERING_FIELD_NAMES_DOC = @@ -380,6 +401,7 @@ public static ConfigDef getConfig() { TOPICS_TO_TABLES_CONFIG, TOPICS_TO_TABLES_TYPE, TOPICS_TO_TABLES_DEFAULT, + TOPICS_TO_TABLES_VALIDATOR, TOPICS_TO_TABLES_IMPORTANCE, TOPICS_TO_TABLES_DOC ).define( @@ -429,12 +451,14 @@ public static ConfigDef getConfig() { KAFKA_KEY_FIELD_NAME_CONFIG, KAFKA_KEY_FIELD_NAME_TYPE, KAFKA_KEY_FIELD_NAME_DEFAULT, + KAFKA_KEY_FIELD_NAME_VALIDATOR, KAFKA_KEY_FIELD_NAME_IMPORTANCE, KAFKA_KEY_FIELD_NAME_DOC ).define( KAFKA_DATA_FIELD_NAME_CONFIG, KAFKA_DATA_FIELD_NAME_TYPE, KAFKA_DATA_FIELD_NAME_DEFAULT, + KAFKA_DATA_FIELD_NAME_VALIDATOR, KAFKA_DATA_FIELD_NAME_IMPORTANCE, KAFKA_DATA_FIELD_NAME_DOC ).define( @@ -512,133 +536,145 @@ public static ConfigDef getConfig() { BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_VALIDATOR, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC ).define( BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE, BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT, + BIGQUERY_CLUSTERING_FIELD_NAMES_VALIDATOR, BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE, BIGQUERY_CLUSTERING_FIELD_NAMES_DOC ); } - /** - * Throw an exception if the passed-in properties do not constitute a valid sink. - * @param props sink configuration properties - */ - public static void validate(Map props) { - final boolean hasTopicsConfig = hasTopicsConfig(props); - final boolean hasTopicsRegexConfig = hasTopicsRegexConfig(props); - - if (hasTopicsConfig && hasTopicsRegexConfig) { - throw new ConfigException(TOPICS_CONFIG + " and " + TOPICS_REGEX_CONFIG + - " are mutually exclusive options, but both are set."); - } - if (!hasTopicsConfig && !hasTopicsRegexConfig) { - throw new ConfigException("Must configure one of " + - TOPICS_CONFIG + " or " + TOPICS_REGEX_CONFIG); - } - } + private static final List> MULTI_PROPERTY_VALIDATIONS = new ArrayList<>(); + + static { + // Note that order matters here: validations are performed in the order they're added to this list, and if a + // property or any of the properties that it depends on has an error, validation for it gets skipped. + // This comes in handy for things like checking for the existence of tables, which requires valid BigQuery + // credentials. We validate those credentials before checking for tables so that we can safely assume while + // checking for those tables that the credentials are already valid. + MULTI_PROPERTY_VALIDATIONS.add(new CredentialsValidator.BigQueryCredentialsValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new CredentialsValidator.GcsCredentialsValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new TableExistenceValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new SchemaRetrieverValidator.TableCreationValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new SchemaRetrieverValidator.SchemaUpdateValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new GcsBucketValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new PartitioningModeValidator()); + } - public static boolean hasTopicsConfig(Map props) { - String topicsStr = props.get(TOPICS_CONFIG); - return topicsStr != null && !topicsStr.trim().isEmpty(); - } + /** + * Used in conjunction with {@link com.wepay.kafka.connect.bigquery.BigQuerySinkConnector#validate(Map)} to perform + * preflight configuration checks. Simple validations that only require a single property value at a time (such as + * ensuring that boolean properties only contain true/false values, or that values for required properties are + * provided) are handled automatically by the {@link #getConfig() ConfigDef} for this class and optionally-defined + * custom {@link ConfigDef.Validator validators}. Other, more sophisticated validations that require multiple + * property values at a time (such as checking if all of the tables the connector will write to already exist if + * automatic table creation is disabled) are performed manually in a subsequent step. + * + * @return a {@link Config} object containing all errors that the connector was able to detect during preflight + * validation of this configuration; never null + */ + public Config validate() { + List initialValidation = getConfig().validate(originalsStrings()); + Map valuesByName = initialValidation + .stream() + .collect(Collectors.toMap(ConfigValue::name, Function.identity())); + MULTI_PROPERTY_VALIDATIONS.forEach(validator -> { + ConfigValue value = valuesByName.get(validator.propertyName()); + validator.validate(value, this, valuesByName).ifPresent(value::addErrorMessage); + }); + return new Config(initialValidation); + } - public static boolean hasTopicsRegexConfig(Map props) { - String topicsRegexStr = props.get(TOPICS_REGEX_CONFIG); - return topicsRegexStr != null && !topicsRegexStr.trim().isEmpty(); + /** + * Ensure that this config is valid (including multi-property validations performed in {@link #validate()}, and if any errors + * are detected, throw an exception. + * @throws ConnectException if this config is invalid + */ + public void ensureValid() { + Config config = validate(); + List errors = config.configValues().stream() + .filter(v -> !v.errorMessages().isEmpty()) + .map(v -> "For property '" + v.name() + "': " + String.join(",", v.errorMessages())) + .collect(Collectors.toList()); + if (!errors.isEmpty()) { + throw new ConnectException( + "The connector config is invalid and contains the following errors:\n" + + String.join("\n", errors) + ); } + } - @SuppressWarnings("unchecked") - public static class Validator implements ConfigDef.Validator { + public static class MapValidator implements ConfigDef.Validator { @Override public void ensureValid(String name, Object value) { - switch (name) { - case DATASETS_CONFIG: - ensureValidMap(name, (List) value); - break; - case TOPICS_TO_TABLES_CONFIG: - ensureValidMap(name, (List) value); - break; - default: - break; - } - } - - protected static void ensureValidMap(String name, List values) { - if (values == null) { + if (value == null) { return; } - values.forEach((entry) -> parseMapping(entry, name)); + + @SuppressWarnings("unchecked") + List parsedValue = (List) value; + + parsedValue.forEach(BigQuerySinkConfig::parseMapping); } - /** - * Ensures the mapping given is valid, then returns an entry containing its key and value. - * Checks to make sure that the given String adheres to the specified format, and throws - * an exception if it does not. Trims leading and trailing whitespace, and then checks to make - * sure that both Strings are still non-empty. - * - * @param mapping The mapping to parse (should be of the form <key>=<value>) - * @param name The name of the field. Used in error messages. - * @return A Map.Entry containing the parsed key/value pair. - */ - protected static Map.Entry parseMapping(String mapping, String name) { - String[] keyValue = mapping.split("="); - if (keyValue.length != 2) { - throw new ConfigException( - "Invalid mapping for " + name - + " property: '" + mapping - + "' (must follow format '=')" - ); - } + @Override + public String toString() { + return "A list of key-value pairs in the format =, =, ..."; + } + } - String key = keyValue[0].trim(); - if (key.isEmpty()) { - throw new ConfigException( - "Empty key found in mapping '" + mapping - + "' for " + name + " property" - ); - } + /** + * Ensures the mapping given is valid, then returns an entry containing its key and value. + * Checks to make sure that the given String adheres to the specified format, and throws + * an exception if it does not. Trims leading and trailing whitespace, and then checks to make + * sure that both Strings are still non-empty. + * + * @param mapping The mapping to parse (should be of the form <key>=<value>) + * @return A Map.Entry containing the parsed key/value pair. + */ + static Map.Entry parseMapping(String mapping) { + String[] keyValue = mapping.split("="); + if (keyValue.length != 2) { + throw new ConfigException("Invalid mapping '" + mapping + "' (must follow format '=')"); + } - String value = keyValue[1].trim(); - if (value.isEmpty()) { - throw new ConfigException( - "Empty value found in mapping '" + mapping - + "' for " + name + " property" - ); - } + String key = keyValue[0].trim(); + if (key.isEmpty()) { + throw new ConfigException("Invalid mapping '" + mapping + "' (key cannot be empty)"); + } - return new AbstractMap.SimpleEntry<>(key, value); + String value = keyValue[1].trim(); + if (value.isEmpty()) { + throw new ConfigException("Invalid mapping '" + mapping + "' (value cannot be empty)"); } + + return new AbstractMap.SimpleEntry<>(key, value); } /** - * Returns the keyfile + * @return the key, which is (depending on the key source property) either a path to a file or a raw JSON string */ - public String getKeyFile() { + public String getKey() { return Optional.ofNullable(getPassword(KEYFILE_CONFIG)).map(Password::value).orElse(null); } /** - * Parses a config map, which must be provided as a list of Strings of the form - * '<key>=<value>' into a Map. Locates that list, splits its key and value pairs, and - * returns they Map they represent. - * - * @param name The name of the property the mapping is given for. Used in exception messages. - * @return A Map containing the given key and value pairs. + * @return the {@link com.wepay.kafka.connect.bigquery.GcpClientBuilder.KeySource key source type} that dictates how + * the {@link #getKey()} should be be interpreted */ - public Map getMap(String name) { - List assocList = getList(name); - Map configMap = new HashMap<>(); - if (assocList != null) { - for (String mapping : assocList) { - Map.Entry entry = validator.parseMapping(mapping, name); - configMap.put(entry.getKey(), entry.getValue()); - } + public GcpClientBuilder.KeySource getKeySource() { + String rawKeySource = getString(KEY_SOURCE_CONFIG); + try { + return GcpClientBuilder.KeySource.valueOf(rawKeySource); + } catch (IllegalArgumentException e) { + // Should never happen with preflight validation of the key source property + throw new ConnectException("Invalid key source type: " + rawKeySource); } - return configMap; } /** @@ -653,7 +689,7 @@ public List> getSinglePatterns(String property) { List> patternList = new ArrayList<>(); if (propList != null) { for (String propValue : propList) { - Map.Entry mapping = validator.parseMapping(propValue, property); + Map.Entry mapping = parseMapping(propValue); Pattern propPattern = Pattern.compile(mapping.getKey()); Map.Entry patternEntry = new AbstractMap.SimpleEntry<>(propPattern, mapping.getValue()); @@ -775,7 +811,7 @@ public SchemaRetriever getSchemaRetriever() { Class schemaRetrieverClass = userSpecifiedClass.asSubclass(SchemaRetriever.class); - Constructor schemaRetrieverConstructor = null; + Constructor schemaRetrieverConstructor; try { schemaRetrieverConstructor = schemaRetrieverClass.getConstructor(); } catch (NoSuchMethodException nsme) { @@ -785,7 +821,7 @@ public SchemaRetriever getSchemaRetriever() { ); } - SchemaRetriever schemaRetriever = null; + SchemaRetriever schemaRetriever; try { schemaRetriever = schemaRetrieverConstructor.newInstance(); } catch (InstantiationException @@ -804,7 +840,6 @@ public SchemaRetriever getSchemaRetriever() { } /** - * * If the connector is configured to load Kafka data into BigQuery, this config defines * the name of the kafka data field. A structure is created under the field name to contain * kafka data schema including topic, offset, partition and insertTime. @@ -816,7 +851,6 @@ public Optional getKafkaKeyFieldName() { } /** - * * If the connector is configured to load Kafka keys into BigQuery, this config defines * the name of the kafka key field. A structure is created under the field name to contain * a topic's Kafka key schema. @@ -827,47 +861,6 @@ public Optional getKafkaDataFieldName() { return Optional.ofNullable(getString(KAFKA_DATA_FIELD_NAME_CONFIG)); } - /** - * Verifies that a bucket is specified if GCS batch loading is enabled. - * @throws ConfigException Exception thrown if no bucket is specified and batch loading is on. - */ - private void verifyBucketSpecified() throws ConfigException { - // Throw an exception if GCS Batch loading will be used but no bucket is specified - if (getString(GCS_BUCKET_NAME_CONFIG).equals("") - && !getList(ENABLE_BATCH_CONFIG).isEmpty()) { - throw new ConfigException("Batch loading enabled for some topics, but no bucket specified"); - } - } - - private void checkAutoCreateTables() { - - Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); - boolean autoCreateTables = getBoolean(TABLE_CREATE_CONFIG); - - if (autoCreateTables && schemaRetriever == null) { - throw new ConfigException( - "Cannot specify automatic table creation without a schema retriever" - ); - } - } - - private void checkAutoUpdateSchemas() { - Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); - - boolean autoUpdateSchemas = getBoolean(SCHEMA_UPDATE_CONFIG); - if (autoUpdateSchemas && schemaRetriever == null) { - throw new ConfigException( - "Cannot specify automatic table creation without a schema retriever" - ); - } - - if (schemaRetriever == null) { - logger.warn( - "No schema retriever class provided; auto schema updates are impossible" - ); - } - } - /** * Returns the field name to use for timestamp partitioning. * @return String that represents the field name. @@ -880,51 +873,18 @@ public Optional getTimestampPartitionFieldName() { * Returns the field names to use for clustering. * @return List of Strings that represent the field names. */ - public Optional> getClusteringPartitionFieldName() { - return Optional.ofNullable(getList(BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG)); - } - - /** - * Check the validity of table partitioning configs. - */ - private void checkPartitionConfigs() { - if (getTimestampPartitionFieldName().isPresent() && getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { - throw new ConfigException( - "Only one partitioning configuration mode may be specified for the connector. " - + "Use either bigQueryPartitionDecorator OR timestampPartitionFieldName." - ); - } - } - - /** - * Check the validity of table clustering configs. - */ - private void checkClusteringConfigs() { - if (getClusteringPartitionFieldName().isPresent()) { - if (!getTimestampPartitionFieldName().isPresent() && !getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { - throw new ConfigException( - "Clustering field name may be specified only on a partitioned table." - ); - } - if (getClusteringPartitionFieldName().get().size() > 4) { - throw new ConfigException( - "You can only specify up to four clustering field names." - ); - } - } + public Optional> getClusteringPartitionFieldNames() { + return Optional + .ofNullable(getList(BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG)) + // With Java 11 there's Predicate::not, but for now we have to just manually invert the isEmpty check + .filter(l -> !l.isEmpty()); } protected BigQuerySinkConfig(ConfigDef config, Map properties) { super(config, properties); - verifyBucketSpecified(); - checkAutoCreateTables(); - checkAutoUpdateSchemas(); - checkPartitionConfigs(); - checkClusteringConfigs(); } public BigQuerySinkConfig(Map properties) { this(getConfig(), properties); } - } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java new file mode 100644 index 000000000..76007d11a --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java @@ -0,0 +1,117 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.storage.Storage; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEYFILE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEY_SOURCE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; + +public abstract class CredentialsValidator> extends MultiPropertyValidator { + + public CredentialsValidator() { + super(KEYFILE_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + PROJECT_CONFIG, KEY_SOURCE_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + String keyFile = config.getKey(); + if (keyFile == null || keyFile.isEmpty()) { + // No credentials to validate + return Optional.empty(); + } + + try { + clientBuilder() + .withConfig(config) + .build(); + return Optional.empty(); + } catch (RuntimeException e) { + String errorMessage = "An unexpected error occurred while validating credentials for " + gcpService(); + if (e.getMessage() != null) { + errorMessage += ": " + e.getMessage(); + } + return Optional.of(errorMessage); + } + } + + protected abstract String gcpService(); + protected abstract ClientBuilder clientBuilder(); + + public static class BigQueryCredentialsValidator extends CredentialsValidator> { + @Override + public String gcpService() { + return "BigQuery"; + } + + @Override + protected GcpClientBuilder clientBuilder() { + return new GcpClientBuilder.BigQueryBuilder(); + } + } + + public static class GcsCredentialsValidator extends CredentialsValidator> { + + private static final Collection DEPENDENTS; + + static { + List dependents = new ArrayList<>(CredentialsValidator.DEPENDENTS); + dependents.add(ENABLE_BATCH_CONFIG); + dependents.add(GCS_BUCKET_NAME_CONFIG); + DEPENDENTS = Collections.unmodifiableCollection(dependents); + } + + @Override + public Collection dependents() { + return DEPENDENTS; + } + + @Override + public String gcpService() { + return "GCS"; + } + + @Override + protected GcpClientBuilder clientBuilder() { + return new GcpClientBuilder.GcsBuilder(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java new file mode 100644 index 000000000..19141c0c7 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java @@ -0,0 +1,63 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; + +public class GcsBucketValidator extends MultiPropertyValidator { + + public GcsBucketValidator() { + super(GCS_BUCKET_NAME_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + ENABLE_BATCH_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + List batchLoadedTopics = config.getList(ENABLE_BATCH_CONFIG); + if (batchLoadedTopics == null || batchLoadedTopics.isEmpty()) { + // Batch loading is disabled; no need to validate the GCS bucket + return Optional.empty(); + } + + String bucket = config.getString(GCS_BUCKET_NAME_CONFIG); + if (bucket == null || bucket.trim().isEmpty()) { + return Optional.of("When GCS batch loading is enabled, a bucket must be provided"); + } + + // No need to validate that the bucket exists; we create it automatically if it doesn't + + return Optional.empty(); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java new file mode 100644 index 000000000..95b9c2da6 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java @@ -0,0 +1,70 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import org.apache.kafka.common.config.ConfigValue; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +public abstract class MultiPropertyValidator { + + private final String propertyName; + + protected MultiPropertyValidator(String propertyName) { + this.propertyName = propertyName; + } + + public String propertyName() { + return propertyName; + } + + public Optional validate(ConfigValue value, Config config, Map valuesByName) { + // Only perform follow-up validation if the property doesn't already have an error associated with it + if (!value.errorMessages().isEmpty()) { + return Optional.empty(); + } + + boolean dependentsAreValid = dependents().stream() + .map(valuesByName::get) + .filter(Objects::nonNull) + .map(ConfigValue::errorMessages) + .allMatch(List::isEmpty); + // Also ensure that all of the other properties that the validation for this one depends on don't already have errors + if (!dependentsAreValid) { + return Optional.empty(); + } + + try { + return doValidate(config); + } catch (RuntimeException e) { + return Optional.of( + "An unexpected error occurred during validation" + + (e.getMessage() != null ? ": " + e.getMessage() : "") + ); + } + } + + protected abstract Collection dependents(); + protected abstract Optional doValidate(Config config); +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java new file mode 100644 index 000000000..65389e5fd --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java @@ -0,0 +1,60 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG; + +public class PartitioningModeValidator extends MultiPropertyValidator { + public PartitioningModeValidator() { + super(BIGQUERY_PARTITION_DECORATOR_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + if (!config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { + return Optional.empty(); + } + + if (config.getTimestampPartitionFieldName().isPresent()) { + return Optional.of(String.format("Only one partitioning mode may be specified for the connector. " + + "Use either %s OR %s.", + BIGQUERY_PARTITION_DECORATOR_CONFIG, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG + )); + } else { + return Optional.empty(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/SchemaRetrieverValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/SchemaRetrieverValidator.java new file mode 100644 index 000000000..9cb6a3894 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/SchemaRetrieverValidator.java @@ -0,0 +1,105 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.SCHEMA_UPDATE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TABLE_CREATE_CONFIG; + +public abstract class SchemaRetrieverValidator extends MultiPropertyValidator { + + protected SchemaRetrieverValidator(String propertyName) { + super(propertyName); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + SCHEMA_RETRIEVER_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + if (!schemaRetrieverRequired(config)) { + return Optional.empty(); + } + + SchemaRetriever schemaRetriever = config.getSchemaRetriever(); + if (schemaRetriever != null) { + return Optional.empty(); + } else { + return Optional.of(missingSchemaRetrieverMessage()); + } + } + + /** + * @param config the user-provided configuration + * @return whether a schema retriever class is required for the property this validator is responsible for + */ + protected abstract boolean schemaRetrieverRequired(BigQuerySinkConfig config); + + /** + * @return an error message explaining why a schema retriever class is required for the property this validator is + * responsible for + */ + protected abstract String missingSchemaRetrieverMessage(); + + public static class TableCreationValidator extends SchemaRetrieverValidator { + public TableCreationValidator() { + super(TABLE_CREATE_CONFIG); + } + + @Override + protected boolean schemaRetrieverRequired(BigQuerySinkConfig config) { + return config.getBoolean(TABLE_CREATE_CONFIG); + } + + @Override + protected String missingSchemaRetrieverMessage() { + return "A schema retriever class is required when automatic table creation is enabled"; + } + } + + public static class SchemaUpdateValidator extends SchemaRetrieverValidator { + public SchemaUpdateValidator() { + super(SCHEMA_UPDATE_CONFIG); + } + + @Override + protected boolean schemaRetrieverRequired(BigQuerySinkConfig config) { + return config.getBoolean(SCHEMA_UPDATE_CONFIG); + } + + @Override + protected String missingSchemaRetrieverMessage() { + return "A schema retriever class is required when automatic schema updates are enabled"; + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/TableExistenceValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/TableExistenceValidator.java new file mode 100644 index 000000000..149b20380 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/TableExistenceValidator.java @@ -0,0 +1,108 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.TableId; +import com.google.common.annotations.VisibleForTesting; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; +import com.wepay.kafka.connect.bigquery.utils.TopicToTableResolver; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.DATASETS_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEYFILE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEY_SOURCE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TABLE_CREATE_CONFIG; + +public class TableExistenceValidator extends MultiPropertyValidator { + + public TableExistenceValidator() { + super(TABLE_CREATE_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + SANITIZE_TOPICS_CONFIG, + KEY_SOURCE_CONFIG, + KEYFILE_CONFIG, + PROJECT_CONFIG, + DATASETS_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + BigQuery bigQuery; + try { + bigQuery = new GcpClientBuilder.BigQueryBuilder() + .withConfig(config) + .build(); + } catch (RuntimeException e) { + return Optional.of(String.format( + "Failed to construct BigQuery client%s", + e.getMessage() != null ? ": " + e.getMessage() : "" + )); + } + + return doValidate(bigQuery, config); + } + + @VisibleForTesting + Optional doValidate(BigQuery bigQuery, BigQuerySinkConfig config) { + boolean autoCreateTables = config.getBoolean(TABLE_CREATE_CONFIG); + // No need to check if tables already exist if we're allowed to create them ourselves + if (autoCreateTables) { + return Optional.empty(); + } + + List missingTables = missingTables(bigQuery, config); + + if (missingTables.isEmpty()) { + return Optional.empty(); + } + + return Optional.of(String.format( + "Automatic table creation is disabled and the following tables do not appear to exist: %s. " + + "Please either manually create these tables before restarting the connector or enable automatic table " + + "creation by the connector.", + missingTables.stream().map(t -> t.getDataset() + ":" + t.getTable()).collect(Collectors.joining(", ")) + )); + } + + @VisibleForTesting + List missingTables(BigQuery bigQuery, BigQuerySinkConfig config) { + return TopicToTableResolver.getTopicsToTables(config).values().stream() + .filter(t -> bigQuery.getTable(t) == null) + .collect(Collectors.toList()); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java deleted file mode 100644 index 805cd5643..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery.exception; - -import org.apache.kafka.connect.errors.ConnectException; - -/** - * Class for exceptions that occur while attempting to process configuration files, including both - * formatting and logical errors. - */ -public class SinkConfigConnectException extends ConnectException { - public SinkConfigConnectException(String msg) { - super(msg); - } - - public SinkConfigConnectException(String msg, Throwable thr) { - super(msg, thr); - } - - public SinkConfigConnectException(Throwable thr) { - super(thr); - } -} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java index b6558a7fa..06d182694 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java @@ -19,32 +19,13 @@ package com.wepay.kafka.connect.bigquery; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; - -import static org.mockito.Matchers.any; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.Table; import com.google.cloud.bigquery.TableId; - import com.wepay.kafka.connect.bigquery.api.KafkaSchemaRecordType; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; - import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; - -import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; -import org.apache.kafka.common.config.ConfigException; - import org.apache.kafka.connect.data.Schema; - import org.junit.BeforeClass; import org.junit.Test; @@ -52,27 +33,16 @@ import java.util.List; import java.util.Map; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + public class BigQuerySinkConnectorTest { private static SinkPropertiesFactory propertiesFactory; - // Would just use Mockito, but can't provide the name of an anonymous class to the config file - public static class MockSchemaRetriever implements SchemaRetriever { - @Override - public void configure(Map properties) { - // Shouldn't be called - } - - @Override - public Schema retrieveSchema(TableId table, String topic, KafkaSchemaRecordType schemaType) { - // Shouldn't be called - return null; - } - - @Override - public void setLastSeenSchema(TableId table, String topic, Schema schema) { - } - } - @BeforeClass public static void initializePropertiesFactory() { propertiesFactory = new SinkPropertiesFactory(); @@ -87,15 +57,10 @@ public void testTaskClass() { public void testTaskConfigs() { Map properties = propertiesFactory.getProperties(); - Table fakeTable = mock(Table.class); - - BigQuery bigQuery = mock(BigQuery.class); - when(bigQuery.getTable(any(TableId.class))).thenReturn(fakeTable); + BigQuerySinkConnector testConnector = new BigQuerySinkConnector(); - SchemaManager schemaManager = mock(SchemaManager.class); - BigQuerySinkConnector testConnector = new BigQuerySinkConnector(bigQuery, schemaManager); - - testConnector.start(properties); + testConnector.configProperties = properties; + testConnector.config = new BigQuerySinkConfig(properties); for (int i : new int[] { 1, 2, 10, 100 }) { Map expectedProperties = new HashMap<>(properties); @@ -130,19 +95,6 @@ public void testConfig() { assertNotNull(new BigQuerySinkConnector().config()); } - // Make sure that a config exception is properly translated into a SinkConfigConnectException - @Test(expected = SinkConfigConnectException.class) - public void testConfigException() { - try { - Map badProperties = propertiesFactory.getProperties(); - badProperties.remove(BigQuerySinkConfig.TOPICS_CONFIG); - BigQuerySinkConfig.validate(badProperties); - new BigQuerySinkConnector().start(badProperties); - } catch (ConfigException e) { - throw new SinkConfigConnectException(e); - } - } - @Test public void testVersion() { assertNotNull(new BigQuerySinkConnector().version()); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 9e86c3fe1..e5224e32d 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -40,8 +40,6 @@ import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.data.Schema; @@ -520,25 +518,6 @@ public void testInterruptedException() { testTask.flush(Collections.emptyMap()); } - // Make sure that a ConfigException is properly translated into a SinkConfigConnectException - @Test(expected = SinkConfigConnectException.class) - public void testConfigException() { - try { - Map badProperties = propertiesFactory.getProperties(); - badProperties.remove(BigQuerySinkConfig.TOPICS_CONFIG); - BigQuerySinkConfig.validate(badProperties); - - SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); - SchemaManager schemaManager = mock(SchemaManager.class); - - BigQuerySinkTask testTask = - new BigQuerySinkTask(mock(BigQuery.class), schemaRetriever, mock(Storage.class), schemaManager); - testTask.start(badProperties); - } catch (ConfigException e) { - throw new SinkConfigConnectException(e); - } - } - @Test public void testVersion() { assertNotNull(new BigQuerySinkTask().version()); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java index 3bb981b4e..93db65926 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java @@ -47,22 +47,4 @@ public Map getProperties() { return properties; } - - /** - * Make sure that each of the default configuration properties work nicely with the given - * configuration object. - * - * @param config The config object to test - */ - public void testProperties(BigQuerySinkConfig config) { - config.getTopicsToDatasets(); - - config.getMap(config.DATASETS_CONFIG); - config.getMap(config.TOPICS_TO_TABLES_CONFIG); - config.getList(config.TOPICS_CONFIG); - config.getString(config.PROJECT_CONFIG); - config.getKeyFile(); - config.getBoolean(config.SANITIZE_TOPICS_CONFIG); - config.getInt(config.AVRO_DATA_CACHE_SIZE_CONFIG); - } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index 8e1f7328c..4890f3716 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -53,7 +53,11 @@ public void initializePropertiesFactory() { public void metaTestBasicConfigProperties() { Map basicConfigProperties = propertiesFactory.getProperties(); BigQuerySinkConfig config = new BigQuerySinkConfig(basicConfigProperties); - propertiesFactory.testProperties(config); + config.getList(BigQuerySinkConfig.TOPICS_CONFIG); + config.getString(BigQuerySinkConfig.PROJECT_CONFIG); + config.getKey(); + config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG); + config.getInt(BigQuerySinkConfig.AVRO_DATA_CACHE_SIZE_CONFIG); } @Test @@ -224,16 +228,6 @@ public void testEmptyTimestampPartitionFieldName() { assertFalse(testConfig.getTimestampPartitionFieldName().isPresent()); } - /** - * Test if the field name being non-empty and the decorator default (true) errors correctly. - */ - @Test (expected = ConfigException.class) - public void testTimestampPartitionFieldNameError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); - new BigQuerySinkConfig(configProperties); - } - /** * Test the field name being non-empty and the decorator set to false works correctly. */ @@ -254,22 +248,7 @@ public void testTimestampPartitionFieldName() { public void testEmptyClusteringFieldNames() { Map configProperties = propertiesFactory.getProperties(); BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); - assertFalse(testConfig.getClusteringPartitionFieldName().isPresent()); - } - - /** - * Test if the field names being non-empty and the partitioning is not present errors correctly. - */ - @Test (expected = ConfigException.class) - public void testClusteringFieldNamesWithoutTimestampPartitionError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, null); - configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); - configProperties.put( - BigQuerySinkConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, - "column1,column2" - ); - new BigQuerySinkConfig(configProperties); + assertFalse(testConfig.getClusteringPartitionFieldNames().isPresent()); } /** @@ -304,17 +283,8 @@ public void testClusteringFieldNames() { ); BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); - Optional> testClusteringPartitionFieldName = testConfig.getClusteringPartitionFieldName(); + Optional> testClusteringPartitionFieldName = testConfig.getClusteringPartitionFieldNames(); assertTrue(testClusteringPartitionFieldName.isPresent()); assertEquals(expectedClusteringPartitionFieldName, testClusteringPartitionFieldName.get()); } - - @Test(expected = ConfigException.class) - public void testAutoSchemaUpdateWithoutRetriever() { - Map badConfigProperties = propertiesFactory.getProperties(); - badConfigProperties.remove(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); - badConfigProperties.put(BigQuerySinkConfig.SCHEMA_UPDATE_CONFIG, "true"); - - new BigQuerySinkConfig(badConfigProperties); - } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java new file mode 100644 index 000000000..7a55d5ad8 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java @@ -0,0 +1,69 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; +import org.junit.Test; + +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CredentialsValidatorTest { + + @Test + public void testNoCredentialsSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getKey()).thenReturn(null); + + assertEquals( + Optional.empty(), + new CredentialsValidator.BigQueryCredentialsValidator().doValidate(config) + ); + assertEquals( + Optional.empty(), + new CredentialsValidator.GcsCredentialsValidator().doValidate(config) + ); + } + + @Test + public void testFailureToConstructClient() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getKey()).thenReturn("key"); + + @SuppressWarnings("unchecked") + GcpClientBuilder mockClientBuilder = mock(GcpClientBuilder.class); + when(mockClientBuilder.withConfig(eq(config))).thenReturn(mockClientBuilder); + when(mockClientBuilder.build()).thenThrow(new RuntimeException("Provided credentials are invalid")); + + assertNotEquals( + Optional.empty(), + new CredentialsValidator.BigQueryCredentialsValidator().doValidate(config) + ); + assertNotEquals( + Optional.empty(), + new CredentialsValidator.GcsCredentialsValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java new file mode 100644 index 000000000..d46832678 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java @@ -0,0 +1,93 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import org.junit.Test; + +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class GcsBucketValidatorTest { + + @Test + public void testNullBatchLoadingSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(null); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(config) + ); + } + + @Test + public void testEmptyBatchLoadingSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.emptyList()); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(config) + ); + } + + @Test + public void testNullBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(null); + + assertNotEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(config) + ); + } + + @Test + public void testBlankBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(" \t "); + + assertNotEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(config) + ); + } + + @Test + public void testValidBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn("gee_cs"); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java new file mode 100644 index 000000000..205bb56a3 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java @@ -0,0 +1,138 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.config.ConfigValue; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; + +public class MultiPropertyValidatorTest { + + private static class TestValidator extends MultiPropertyValidator { + + private final List dependents; + private final Function> validationFunction; + + public TestValidator(String propertyName, List dependents, Function> validationFunction) { + super(propertyName); + this.dependents = dependents; + this.validationFunction = validationFunction; + } + + @Override + protected Collection dependents() { + return dependents; + } + + @Override + protected Optional doValidate(Config config) { + return validationFunction.apply(config); + } + } + + @Test + public void testExistingErrorSkipsValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Arrays.asList("d1", "d2", "d3"), + o -> { + fail("Validation should have been performed on property that already has an error"); + return null; + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.singletonList("an error")); + + assertEquals( + Optional.empty(), + validator.validate(configValue, null, Collections.emptyMap()) + ); + } + + @Test + public void testDependentErrorSkipsValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Arrays.asList("d1", "d2", "d3"), + o -> { + fail("Validation should have been performed on property whose dependent already has an error"); + return null; + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + Map valuesByName = ImmutableMap.of( + "d1", new ConfigValue("d1", "v1", Collections.emptyList(), Collections.emptyList()), + "d2", new ConfigValue("d2", "v1", Collections.emptyList(), Collections.singletonList("an error")) + ); + + assertEquals( + Optional.empty(), + validator.validate(configValue, null, valuesByName) + ); + } + + @Test + public void testValidationFails() { + Optional expectedError = Optional.of("an error"); + MultiPropertyValidator validator = new TestValidator<>( + "p", + Collections.emptyList(), + o -> expectedError + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + + assertEquals( + expectedError, + validator.validate(configValue, null, Collections.emptyMap()) + ); + } + + @Test + public void testUnexpectedErrorDuringValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Collections.emptyList(), + o -> { + throw new RuntimeException("Some unexpected error"); + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + + assertNotEquals( + Optional.empty(), + validator.validate(configValue, null, Collections.emptyMap()) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java new file mode 100644 index 000000000..a4b79a14c --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java @@ -0,0 +1,80 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import org.junit.Test; + +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PartitioningModeValidatorTest { + + @Test + public void testDisabledDecoratorSyntaxSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(false); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testDecoratorSyntaxWithoutTimestampPartitionFieldName() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.empty()); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testDecoratorSyntaxWithTimestampPartitionFieldName() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.of("f1")); + + assertNotEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testTimestampPartitionFieldNameWithoutDecoratorSyntax() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(false); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.of("f1")); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/SchemaRetrieverValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/SchemaRetrieverValidatorTest.java new file mode 100644 index 000000000..19a1ae28b --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/SchemaRetrieverValidatorTest.java @@ -0,0 +1,107 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; +import org.junit.Test; + +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.SCHEMA_UPDATE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TABLE_CREATE_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class SchemaRetrieverValidatorTest { + + @Test + public void testDisabledTableCreationSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(TABLE_CREATE_CONFIG)).thenReturn(false); + + assertEquals( + Optional.empty(), + new SchemaRetrieverValidator.TableCreationValidator().doValidate(config) + ); + } + + @Test + public void testDisabledSchemaUpdatesSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(TABLE_CREATE_CONFIG)).thenReturn(true); + + assertEquals( + Optional.empty(), + new SchemaRetrieverValidator.SchemaUpdateValidator().doValidate(config) + ); + } + + @Test + public void testTableCreationEnabledWithNoRetriever() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(TABLE_CREATE_CONFIG)).thenReturn(true); + when(config.getSchemaRetriever()).thenReturn(null); + + assertNotEquals( + Optional.empty(), + new SchemaRetrieverValidator.TableCreationValidator().doValidate(config) + ); + } + + @Test + public void testSchemaUpdatesEnabledWithNoRetriever() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(SCHEMA_UPDATE_CONFIG)).thenReturn(true); + when(config.getSchemaRetriever()).thenReturn(null); + + assertNotEquals( + Optional.empty(), + new SchemaRetrieverValidator.SchemaUpdateValidator().doValidate(config) + ); + } + + @Test + public void testTableCreationEnabledWithValidRetriever() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(TABLE_CREATE_CONFIG)).thenReturn(true); + SchemaRetriever mockRetriever = mock(SchemaRetriever.class); + when(config.getSchemaRetriever()).thenReturn(mockRetriever); + + assertEquals( + Optional.empty(), + new SchemaRetrieverValidator.TableCreationValidator().doValidate(config) + ); + } + + @Test + public void testSchemaUpdatesEnabledWithValidRetriever() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(SCHEMA_UPDATE_CONFIG)).thenReturn(true); + SchemaRetriever mockRetriever = mock(SchemaRetriever.class); + when(config.getSchemaRetriever()).thenReturn(mockRetriever); + + assertEquals( + Optional.empty(), + new SchemaRetrieverValidator.SchemaUpdateValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/TableExistenceValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/TableExistenceValidatorTest.java new file mode 100644 index 000000000..fb9ffc32f --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/TableExistenceValidatorTest.java @@ -0,0 +1,160 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.Table; +import com.google.cloud.bigquery.TableId; +import com.google.common.collect.ImmutableMap; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Stream; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TABLE_CREATE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TOPICS_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TableExistenceValidatorTest { + + @Test + public void testMissingTableWithAutoCreationDisabled() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getTopicsToDatasets()) + .thenReturn(ImmutableMap.of( + "t1", "d1", + "t2", "d2" + )); + when(config.getBoolean(eq(TABLE_CREATE_CONFIG))).thenReturn(false); + when(config.getList(TOPICS_CONFIG)).thenReturn(Arrays.asList("t1", "t2")); + + BigQuery bigQuery = bigQuery(TableId.of("d1", "t1")); + + assertNotEquals( + Optional.empty(), + new TableExistenceValidator().doValidate(bigQuery, config) + ); + } + + @Test + public void testEmptyTopicsListWithAutoCreationDisabled() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getTopicsToDatasets()) + .thenReturn(ImmutableMap.of( + "t1", "d1", + "t2", "d2" + )); + when(config.getBoolean(eq(TABLE_CREATE_CONFIG))).thenReturn(false); + when(config.getList(TOPICS_CONFIG)).thenReturn(Collections.emptyList()); + + BigQuery bigQuery = bigQuery(); + + assertEquals( + Optional.empty(), + new TableExistenceValidator().doValidate(bigQuery, config) + ); + } + + @Test + public void testMissingTableWithAutoCreationEnabled() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(eq(TABLE_CREATE_CONFIG))).thenReturn(true); + + assertEquals( + Optional.empty(), + new TableExistenceValidator().doValidate(null, config) + ); + } + + @Test + public void testExactListOfMissingTables() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getTopicsToDatasets()) + .thenReturn(ImmutableMap.of( + "t1", "d1", + "t2", "d2", + "t3", "d1", + "t4", "d2", + "t5", "d1" + )); + when(config.getList(TOPICS_CONFIG)).thenReturn(Arrays.asList("t1", "t2", "t3", "t4", "t5")); + + BigQuery bigQuery = bigQuery( + TableId.of("d1", "t1"), + TableId.of("d3", "t2"), + TableId.of("d2", "t5") + ); + Set expectedMissingTables = new HashSet<>(Arrays.asList( + TableId.of("d2", "t2"), + TableId.of("d1", "t3"), + TableId.of("d2", "t4"), + TableId.of("d1", "t5") + )); + + assertEquals( + expectedMissingTables, + new HashSet<>(new TableExistenceValidator().missingTables(bigQuery, config)) + ); + } + + @Test + public void testExactEmptyListOfMissingTables() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getTopicsToDatasets()) + .thenReturn(ImmutableMap.of( + "t1", "d1", + "t2", "d2", + "t3", "d1", + "t4", "d2", + "t5", "d1" + )); + when(config.getList(TOPICS_CONFIG)).thenReturn(Arrays.asList("t1", "t2", "t3", "t4", "t5")); + + BigQuery bigQuery = bigQuery( + TableId.of("d1", "t1"), + TableId.of("d2", "t2"), + TableId.of("d1", "t3"), + TableId.of("d2", "t4"), + TableId.of("d1", "t5") + ); + + assertEquals( + Collections.emptyList(), + new TableExistenceValidator().missingTables(bigQuery, config) + ); + } + + private static BigQuery bigQuery(TableId... existingTables) { + BigQuery result = mock(BigQuery.class); + Stream.of(existingTables).forEach(table -> { + Table mockTable = mock(Table.class); + when(result.getTable(eq(table))).thenReturn(mockTable); + }); + return result; + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java index 1ce64e5d3..a3287d112 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/BigQueryConnectorIntegrationTest.java @@ -19,16 +19,6 @@ package com.wepay.kafka.connect.bigquery.it; -import static com.google.cloud.bigquery.LegacySQLTypeName.BOOLEAN; -import static com.google.cloud.bigquery.LegacySQLTypeName.BYTES; -import static com.google.cloud.bigquery.LegacySQLTypeName.DATE; -import static com.google.cloud.bigquery.LegacySQLTypeName.FLOAT; -import static com.google.cloud.bigquery.LegacySQLTypeName.INTEGER; -import static com.google.cloud.bigquery.LegacySQLTypeName.STRING; -import static com.google.cloud.bigquery.LegacySQLTypeName.TIMESTAMP; - -import static org.junit.Assert.assertEquals; - import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.Field; import com.google.cloud.bigquery.FieldValue; @@ -36,16 +26,13 @@ import com.google.cloud.bigquery.Schema; import com.google.cloud.bigquery.Table; import com.google.cloud.bigquery.TableResult; - -import com.wepay.kafka.connect.bigquery.BigQueryHelper; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; - +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; +import org.apache.kafka.common.config.ConfigException; import org.junit.BeforeClass; import org.junit.Test; import java.io.FileNotFoundException; import java.io.InputStream; - import java.time.LocalDate; import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; @@ -54,17 +41,24 @@ import java.util.List; import java.util.Properties; +import static com.google.cloud.bigquery.LegacySQLTypeName.BOOLEAN; +import static com.google.cloud.bigquery.LegacySQLTypeName.BYTES; +import static com.google.cloud.bigquery.LegacySQLTypeName.DATE; +import static com.google.cloud.bigquery.LegacySQLTypeName.FLOAT; +import static com.google.cloud.bigquery.LegacySQLTypeName.INTEGER; +import static com.google.cloud.bigquery.LegacySQLTypeName.STRING; +import static com.google.cloud.bigquery.LegacySQLTypeName.TIMESTAMP; +import static org.junit.Assert.assertEquals; + public class BigQueryConnectorIntegrationTest { public static final String TEST_PROPERTIES_FILENAME = "/test.properties"; public static final String KEYFILE_PROPERTY = "keyfile"; public static final String PROJECT_PROPERTY = "project"; public static final String DATASET_PROPERTY = "dataset"; - public static final String KEY_SOURCE_PROPERTY = "keySource"; - private static String keyfile; + private static String key; private static String project; private static String dataset; - private static String keySource; private static BigQuery bigQuery; @@ -87,9 +81,9 @@ private static void initializeTestProperties() throws Exception { Properties properties = new Properties(); properties.load(propertiesFile); - keyfile = properties.getProperty(KEYFILE_PROPERTY); - if (keyfile == null) { - throw new SinkConfigConnectException( + key = properties.getProperty(KEYFILE_PROPERTY); + if (key == null) { + throw new ConfigException( "'" + KEYFILE_PROPERTY + "' property must be specified in test properties file" ); @@ -97,7 +91,7 @@ private static void initializeTestProperties() throws Exception { project = properties.getProperty(PROJECT_PROPERTY); if (project == null) { - throw new SinkConfigConnectException( + throw new ConfigException( "'" + PROJECT_PROPERTY + "' property must be specified in test properties file" ); @@ -105,18 +99,20 @@ private static void initializeTestProperties() throws Exception { dataset = properties.getProperty(DATASET_PROPERTY); if (dataset == null) { - throw new SinkConfigConnectException( + throw new ConfigException( "'" + DATASET_PROPERTY + "' property must be specified in test properties file" ); } - - keySource = properties.getProperty(KEY_SOURCE_PROPERTY); } } - private static void initializeBigQuery() throws Exception { - bigQuery = new BigQueryHelper().setKeySource(keySource).connect(project, keyfile); + private static void initializeBigQuery() { + bigQuery = new GcpClientBuilder.BigQueryBuilder() + .withKeySource(GcpClientBuilder.KeySource.FILE) + .withKey(key) + .withProject(project) + .build(); } private static List boxByteArray(byte[] bytes) { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/BucketClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/BucketClearer.java index ce647d5b2..845be68cc 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/BucketClearer.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/BucketClearer.java @@ -20,27 +20,26 @@ package com.wepay.kafka.connect.bigquery.it.utils; import com.google.cloud.storage.Storage; -import com.wepay.kafka.connect.bigquery.GCSBuilder; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class BucketClearer { private static final Logger logger = LoggerFactory.getLogger(BucketClearer.class); - private static String keySource; /** * Clears tables in the given project and dataset, using a provided JSON service account key. */ public static void main(String[] args) { - if (args.length < 3 || args.length > 4) { + if (args.length != 3) { usage(); - } else if (args.length == 3) { - keySource = "FILE"; - } else { - keySource = args[3]; } - Storage gcs = new GCSBuilder(args[1]).setKey(args[0]).setKeySource(keySource).build(); + Storage gcs = new GcpClientBuilder.GcsBuilder() + .withKeySource(GcpClientBuilder.KeySource.FILE) + .withKey(args[0]) + .withProject(args[1]) + .build(); // if bucket exists, delete it. String bucketName = args[2]; @@ -53,7 +52,7 @@ public static void main(String[] args) { private static void usage() { System.err.println( - "usage: BucketClearer []" + "usage: BucketClearer " ); System.exit(1); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java index 20e7d8fa2..c66606eae 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/it/utils/TableClearer.java @@ -21,16 +21,15 @@ import com.google.cloud.bigquery.BigQuery; -import com.wepay.kafka.connect.bigquery.BigQueryHelper; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class TableClearer { - private static final Logger logger = LoggerFactory.getLogger(TableClearer.class); - private static String keySource; + private static final Logger logger = LoggerFactory.getLogger(TableClearer.class); /** * Clears tables in the given project and dataset, using a provided JSON service account key. @@ -39,9 +38,14 @@ public static void main(String[] args) { if (args.length < 4) { usage(); } - int tablesStart = 3; - BigQuery bigQuery = new BigQueryHelper().connect(args[1], args[0]); - for (int i = tablesStart; i < args.length; i++) { + + BigQuery bigQuery = new GcpClientBuilder.BigQueryBuilder() + .withKeySource(GcpClientBuilder.KeySource.FILE) + .withKey(args[0]) + .withProject(args[1]) + .build(); + + for (int i = 3; i < args.length; i++) { // May be consider using sanitizeTopics property value in future to decide table name // sanitization but as currently we always run test cases with sanitizeTopics value as true // hence sanitize table name prior delete. This is required else it makes test cases flaky. From baaa26084b13c8494ed66998b6b06fce8e682ae4 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 9 Sep 2021 11:35:04 -0400 Subject: [PATCH 124/190] GH-139: Move all user-visible configuration properties to single configuration class --- .../com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java | 1 + .../java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index 2fe315f40..d40c9cee1 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -50,6 +50,7 @@ public ConfigDef config() { return BigQuerySinkConfig.getConfig(); } + @Override public void start(Map properties) { logger.trace("connector.start()"); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 8e081ba31..96e80799d 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -30,7 +30,6 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.utils.SinkRecordConverter; From 04faaba621d6a1f873d0f61166c9a66bd93612e0 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Sun, 19 Sep 2021 07:10:36 -0400 Subject: [PATCH 125/190] GH-139: Improve preflight config validation Depends on https://github.com/confluentinc/kafka-connect-bigquery/pull/140 Addresses the remainder of https://github.com/confluentinc/kafka-connect-bigquery/issues/139. All multi-property configuration validation is moved from the constructor of the BigQuerySinkConfig class to a new `BigQuerySinkConfig::validate` method, which is invoked from `BigQuerySinkConnector::validate` and leverages the Connect API for defining custom error messages on a per-property basis instead of throwing exceptions on invalid configurations, which only displays one message at a time and doesn't work well with programmatic UIs. Unit tests are added for all new validation logic. The logic for GCP client construction is also updated to conform to a common interface, which should improve readability and maintainability, and makes it easier to validate credentials for either BigQuery or GCS. Finally, the `SinkConfigConnectException` class is removed as it's not really necessary and doesn't bring anything worthwhile to the code base. --- .../connect/bigquery/BigQueryHelper.java | 108 ------- .../bigquery/BigQuerySinkConnector.java | 13 + .../connect/bigquery/BigQuerySinkTask.java | 28 +- .../kafka/connect/bigquery/GCSBuilder.java | 109 ------- .../connect/bigquery/GcpClientBuilder.java | 145 +++++++++ .../bigquery/config/BigQuerySinkConfig.java | 274 ++++++++---------- .../bigquery/config/CredentialsValidator.java | 117 ++++++++ .../bigquery/config/GcsBucketValidator.java | 97 +++++++ .../config/MultiPropertyValidator.java | 70 +++++ .../config/PartitioningModeValidator.java | 60 ++++ .../config/UpsertDeleteValidator.java | 102 +++++++ .../exception/SinkConfigConnectException.java | 40 --- .../bigquery/BigQuerySinkConnectorTest.java | 28 +- .../bigquery/BigQuerySinkTaskTest.java | 20 -- .../bigquery/SinkPropertiesFactory.java | 20 -- .../config/BigQuerySinkConfigTest.java | 35 +-- .../config/CredentialsValidatorTest.java | 69 +++++ .../config/GcsBucketValidatorTest.java | 122 ++++++++ .../config/MultiPropertyValidatorTest.java | 138 +++++++++ .../config/PartitioningModeValidatorTest.java | 80 +++++ .../bigquery/integration/BaseConnectorIT.java | 10 +- .../integration/utils/BucketClearer.java | 11 +- 22 files changed, 1174 insertions(+), 522 deletions(-) delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/UpsertDeleteValidator.java delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java deleted file mode 100644 index f90ea5f3f..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery; - -import com.google.auth.oauth2.GoogleCredentials; -import com.google.cloud.bigquery.BigQuery; -import com.google.cloud.bigquery.BigQueryOptions; - -import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.ByteArrayInputStream; -import java.nio.charset.StandardCharsets; - -/** - * Convenience class for creating a default {@link com.google.cloud.bigquery.BigQuery} instance, - * with or without login credentials. - */ -public class BigQueryHelper { - private static final Logger logger = LoggerFactory.getLogger(BigQueryHelper.class); - private static String keySource; - - /** - * Returns a default {@link BigQuery} instance for the specified project with credentials provided - * in the specified file, which can then be used for creating, updating, and inserting into tables - * from specific datasets. - * - * @param projectName The name of the BigQuery project to work with - * @param key The google credentials JSON key that can be used to provide - * credentials to BigQuery, or null if no authentication should be performed. - * @return The resulting BigQuery object. - */ - public BigQuery connect(String projectName, String key) { - if (key == null) { - return connect(projectName); - } - logger.debug("Attempting to open file {} for service account json key", key); - InputStream credentialsStream; - try { - if (keySource != null && keySource.equals("JSON")) { - credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); - } else { - credentialsStream = new FileInputStream(key); - } - return new - BigQueryOptions.DefaultBigQueryFactory().create( - BigQueryOptions.newBuilder() - .setProjectId(projectName) - .setCredentials(GoogleCredentials.fromStream(credentialsStream)) - .build() - ); - } catch (IOException err) { - throw new BigQueryConnectException("Failed to access json key file", err); - } - } - /** - * Returns a default {@link BigQuery} instance for the specified project with credentials provided - * in the specified file, which can then be used for creating, updating, and inserting into tables - * from specific datasets. - * - * @param keySource The type of key config we can expect. This is either a String - * representation of the Google credentials file, or the path to the Google credentials file. - * @return The resulting BigQuery object. - */ - public BigQueryHelper setKeySource(String keySource) { - this.keySource = keySource; - return this; - } - - /** - * Returns a default {@link BigQuery} instance for the specified project with no authentication - * credentials, which can then be used for creating, updating, and inserting into tables from - * specific datasets. - * - * @param projectName The name of the BigQuery project to work with - * @return The resulting BigQuery object. - */ - public BigQuery connect(String projectName) { - logger.debug("Attempting to access BigQuery without authentication"); - return new BigQueryOptions.DefaultBigQueryFactory().create( - BigQueryOptions.newBuilder() - .setProjectId(projectName) - .build() - ); - } -} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index d40c9cee1..ae3ef6f43 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -22,7 +22,9 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.utils.Version; +import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.sink.SinkConnector; import org.slf4j.Logger; @@ -50,6 +52,17 @@ public ConfigDef config() { return BigQuerySinkConfig.getConfig(); } + @Override + public Config validate(Map properties) { + List singlePropertyValidations = config().validate(properties); + // If any of our properties had malformed syntax or failed a validation to ensure, e.g., that it fell within an + // acceptable numeric range, we only report those errors since they prevent us from being able to construct a + // valid BigQuerySinkConfig instance + if (singlePropertyValidations.stream().anyMatch(v -> !v.errorMessages().isEmpty())) { + return new Config(singlePropertyValidations); + } + return new BigQuerySinkConfig(properties).validate(); + } @Override public void start(Map properties) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 96e80799d..71b12b874 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -46,7 +46,6 @@ import com.wepay.kafka.connect.bigquery.write.row.UpsertDeleteBigQueryWriter; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; @@ -170,7 +169,7 @@ public Map preCommit(Map: format or just the format."); + throw new ConnectException(String.format( + "Incorrect regex replacement format in topic name '%s'. " + + "SMT replacement should either produce the : format " + + "or just the format.", + record.topic() + )); } if (sanitize) { @@ -290,10 +293,9 @@ private BigQuery getBigQuery() { } private BigQuery newBigQuery() { - String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - String keyFile = config.getKeyFile(); - String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); - return new BigQueryHelper().setKeySource(keySource).connect(projectName, keyFile); + return new GcpClientBuilder.BigQueryBuilder() + .withConfig(config) + .build(); } private SchemaManager getSchemaManager() { @@ -310,7 +312,7 @@ private SchemaManager newSchemaManager() { Optional kafkaKeyFieldName = config.getKafkaKeyFieldName(); Optional kafkaDataFieldName = config.getKafkaDataFieldName(); Optional timestampPartitionFieldName = config.getTimestampPartitionFieldName(); - Optional> clusteringFieldName = config.getClusteringPartitionFieldName(); + Optional> clusteringFieldName = config.getClusteringPartitionFieldNames(); boolean allowNewBQFields = config.getBoolean(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); boolean allowReqFieldRelaxation = config.getBoolean(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); boolean allowSchemaUnionization = config.getBoolean(BigQuerySinkConfig.ALLOW_SCHEMA_UNIONIZATION_CONFIG); @@ -349,11 +351,9 @@ private Storage getGcs() { if (testGcs != null) { return testGcs; } - String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - String key = config.getKeyFile(); - String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); - return new GCSBuilder(projectName).setKey(key).setKeySource(keySource).build(); - + return new GcpClientBuilder.GcsBuilder() + .withConfig(config) + .build(); } private GCSToBQWriter getGcsWriter() { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java deleted file mode 100644 index 4a0952b2e..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery; - -import com.google.auth.oauth2.GoogleCredentials; -import com.google.cloud.storage.Storage; -import com.google.cloud.storage.StorageOptions; - -import com.wepay.kafka.connect.bigquery.exception.GCSConnectException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.ByteArrayInputStream; -import java.nio.charset.StandardCharsets; - -/** - * Convenience class for creating a {@link com.google.cloud.storage.Storage} instance - */ -public class GCSBuilder { - private static final Logger logger = LoggerFactory.getLogger(GCSBuilder.class); - - private final String projectName; - private String key; - private String keySource; - - public GCSBuilder(String projectName) { - this.projectName = projectName; - this.key = null; - } - - public GCSBuilder setKeySource(String keySourceType) { - this.keySource = keySourceType; - return this; - } - - public GCSBuilder setKey(String keyFile) { - this.key = keyFile; - return this; - } - public Storage build() { - return connect(projectName, key); - } - - /** - * Returns a default {@link Storage} instance for the specified project with credentials provided - * in the specified file. - * - * @param projectName The name of the GCS project to work with - * @param key The name of a file containing a JSON key that can be used to provide - * credentials to GCS, or null if no authentication should be performed. - * @return The resulting Storage object. - */ - private Storage connect(String projectName, String key) { - if (key == null) { - return connect(projectName); - } - try { - InputStream credentialsStream; - if (keySource != null && keySource.equals("JSON")) { - credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); - } else { - credentialsStream = new FileInputStream(key); - } - return StorageOptions.newBuilder() - .setProjectId(projectName) - .setCredentials(GoogleCredentials.fromStream(credentialsStream)) - .build() - .getService(); - } catch (IOException err) { - throw new GCSConnectException("Failed to access json key file", err); - } - } - - /** - * Returns a default {@link Storage} instance for the specified project with no authentication - * credentials. - * - * @param projectName The name of the GCS project to work with - * @return The resulting Storage object. - */ - private Storage connect(String projectName) { - logger.debug("Attempting to access BigQuery without authentication"); - return StorageOptions.newBuilder() - .setProjectId(projectName) - .build() - .getService(); - } -} - diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java new file mode 100644 index 000000000..5c79fec87 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java @@ -0,0 +1,145 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryOptions; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageOptions; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Objects; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; + +public abstract class GcpClientBuilder { + + public enum KeySource { + FILE, JSON + } + + private static final Logger logger = LoggerFactory.getLogger(GcpClientBuilder.class); + + private String project = null; + private KeySource keySource = null; + private String key = null; + + public GcpClientBuilder withConfig(BigQuerySinkConfig config) { + return withProject(config.getString(PROJECT_CONFIG)) + .withKeySource(config.getKeySource()) + .withKey(config.getKey()); + } + + public GcpClientBuilder withProject(String project) { + Objects.requireNonNull(project, "Project cannot be null"); + this.project = project; + return this; + } + + public GcpClientBuilder withKeySource(KeySource keySource) { + Objects.requireNonNull(keySource, "Key cannot be null"); + this.keySource = keySource; + return this; + } + + public GcpClientBuilder withKey(String key) { + this.key = key; + return this; + } + + public Client build() { + return doBuild(project, credentials()); + } + + private GoogleCredentials credentials() { + if (key == null) { + return null; + } + + Objects.requireNonNull(keySource, "Key source must be defined to build a GCP client"); + Objects.requireNonNull(project, "Project must be defined to build a GCP client"); + + InputStream credentialsStream; + switch (keySource) { + case JSON: + credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); + break; + case FILE: + try { + logger.debug("Attempting to open file {} for service account json key", key); + credentialsStream = new FileInputStream(key); + } catch (IOException e) { + throw new BigQueryConnectException("Failed to access JSON key file", e); + } + break; + default: + throw new IllegalArgumentException("Unexpected value for KeySource enum: " + keySource); + } + + try { + return GoogleCredentials.fromStream(credentialsStream); + } catch (IOException e) { + throw new BigQueryConnectException("Failed to create credentials from input stream", e); + } + } + + protected abstract Client doBuild(String project, GoogleCredentials credentials); + + public static class BigQueryBuilder extends GcpClientBuilder { + @Override + protected BigQuery doBuild(String project, GoogleCredentials credentials) { + BigQueryOptions.Builder builder = BigQueryOptions.newBuilder() + .setProjectId(project); + + if (credentials != null) { + builder.setCredentials(credentials); + } else { + logger.debug("Attempting to access BigQuery without authentication"); + } + + return builder.build().getService(); + } + } + + public static class GcsBuilder extends GcpClientBuilder { + @Override + protected Storage doBuild(String project, GoogleCredentials credentials) { + StorageOptions.Builder builder = StorageOptions.newBuilder() + .setProjectId(project); + + if (credentials != null) { + builder.setCredentials(credentials); + } else { + logger.debug("Attempting to access GCS without authentication"); + } + + return builder.build().getService(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 1ac5c664b..91f158be0 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -20,6 +20,7 @@ package com.wepay.kafka.connect.bigquery.config; import com.google.cloud.bigquery.Schema; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.convert.BigQueryRecordConverter; import com.wepay.kafka.connect.bigquery.convert.BigQuerySchemaConverter; @@ -27,26 +28,29 @@ import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.common.config.types.Password; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkConnector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Base class for connector and task configs; contains properties shared between the two of them. */ public class BigQuerySinkConfig extends AbstractConfig { - private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkConfig.class); - // Values taken from https://github.com/apache/kafka/blob/1.1.1/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java#L33 public static final String TOPICS_CONFIG = SinkConnector.TOPICS_CONFIG; private static final ConfigDef.Type TOPICS_TYPE = ConfigDef.Type.LIST; @@ -132,9 +136,13 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KEY_SOURCE_CONFIG = "keySource"; private static final ConfigDef.Type KEY_SOURCE_TYPE = ConfigDef.Type.STRING; - public static final String KEY_SOURCE_DEFAULT = "FILE"; - private static final ConfigDef.Validator KEY_SOURCE_VALIDATOR = - ConfigDef.ValidString.in("FILE", "JSON"); + public static final String KEY_SOURCE_DEFAULT = GcpClientBuilder.KeySource.FILE.name(); + private static final ConfigDef.Validator KEY_SOURCE_VALIDATOR = ConfigDef.ValidString.in( + Stream.of(GcpClientBuilder.KeySource.values()) + .map(GcpClientBuilder.KeySource::name) + .collect(Collectors.toList()) + .toArray(new String[0]) + ); private static final ConfigDef.Importance KEY_SOURCE_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String KEY_SOURCE_DOC = "Determines whether the keyfile config is the path to the credentials json, or the json itself"; @@ -164,6 +172,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KAFKA_KEY_FIELD_NAME_CONFIG = "kafkaKeyFieldName"; private static final ConfigDef.Type KAFKA_KEY_FIELD_NAME_TYPE = ConfigDef.Type.STRING; public static final String KAFKA_KEY_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator KAFKA_KEY_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance KAFKA_KEY_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String KAFKA_KEY_FIELD_NAME_DOC = "The name of the field of Kafka key. " + "Default to be null, which means Kafka Key Field will not be included."; @@ -171,6 +180,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KAFKA_DATA_FIELD_NAME_CONFIG = "kafkaDataFieldName"; private static final ConfigDef.Type KAFKA_DATA_FIELD_NAME_TYPE = ConfigDef.Type.STRING; public static final String KAFKA_DATA_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator KAFKA_DATA_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance KAFKA_DATA_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String KAFKA_DATA_FIELD_NAME_DOC = "The name of the field of Kafka Data. " + "Default to be null, which means Kafka Data Field will not be included. "; @@ -273,7 +283,21 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String MERGE_INTERVAL_MS_CONFIG = "mergeIntervalMs"; private static final ConfigDef.Type MERGE_INTERVAL_MS_TYPE = ConfigDef.Type.LONG; public static final long MERGE_INTERVAL_MS_DEFAULT = 60_000L; - private static final ConfigDef.Validator MERGE_INTERVAL_MS_VALIDATOR = ConfigDef.Range.atLeast(-1); + private static final ConfigDef.Validator MERGE_INTERVAL_MS_VALIDATOR = ConfigDef.LambdaValidator.with( + (name, value) -> { + if (value == null) { + return; + } + long parsedValue = (long) ConfigDef.parseType(name, value, MERGE_INTERVAL_MS_TYPE); + + if (parsedValue == 0) { + throw new ConfigException(name, value, "Cannot be zero"); + } else if (parsedValue < -1) { + throw new ConfigException(name, value, "Cannot be less than -1"); + } + }, + () -> "Either a positive integer or -1 to disable time interval-based merging" + ); private static final ConfigDef.Importance MERGE_INTERVAL_MS_IMPORTANCE = ConfigDef.Importance.LOW; private static final String MERGE_INTERVAL_MS_DOC = "How often (in milliseconds) to perform a merge flush, if upsert/delete is enabled. Can be " @@ -282,7 +306,21 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String MERGE_RECORDS_THRESHOLD_CONFIG = "mergeRecordsThreshold"; private static final ConfigDef.Type MERGE_RECORDS_THRESHOLD_TYPE = ConfigDef.Type.LONG; public static final long MERGE_RECORDS_THRESHOLD_DEFAULT = -1; - private static final ConfigDef.Validator MERGE_RECORDS_THRESHOLD_VALIDATOR = ConfigDef.Range.atLeast(-1); + private static final ConfigDef.Validator MERGE_RECORDS_THRESHOLD_VALIDATOR = ConfigDef.LambdaValidator.with( + (name, value) -> { + if (value == null) { + return; + } + long parsedValue = (long) ConfigDef.parseType(name, value, MERGE_RECORDS_THRESHOLD_TYPE); + + if (parsedValue == 0) { + throw new ConfigException(name, value, "Cannot be zero"); + } else if (parsedValue < -1) { + throw new ConfigException(name, value, "Cannot be less than -1"); + } + }, + () -> "Either a positive integer or -1 to disable throughput-based merging" + ); private static final ConfigDef.Importance MERGE_RECORDS_THRESHOLD_IMPORTANCE = ConfigDef.Importance.LOW; private static final String MERGE_RECORDS_THRESHOLD_DOC = "How many records to write to an intermediate table before performing a merge flush, if " @@ -358,6 +396,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG = "timestampPartitionFieldName"; private static final ConfigDef.Type BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE = ConfigDef.Type.STRING; private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC = @@ -368,6 +407,17 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG = "clusteringPartitionFieldNames"; private static final ConfigDef.Type BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE = ConfigDef.Type.LIST; private static final List BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT = null; + private static final ConfigDef.Validator BIGQUERY_CLUSTERING_FIELD_NAMES_VALIDATOR = (name, value) -> { + if (value == null) { + return; + } + + @SuppressWarnings("unchecked") + List parsedValue = (List) value; + if (parsedValue.size() > 4) { + throw new ConfigException(name, value, "You may only specify up to four clustering field names."); + } + }; private static final ConfigDef.Importance BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE = ConfigDef.Importance.LOW; private static final String BIGQUERY_CLUSTERING_FIELD_NAMES_DOC = @@ -470,12 +520,14 @@ public static ConfigDef getConfig() { KAFKA_KEY_FIELD_NAME_CONFIG, KAFKA_KEY_FIELD_NAME_TYPE, KAFKA_KEY_FIELD_NAME_DEFAULT, + KAFKA_KEY_FIELD_NAME_VALIDATOR, KAFKA_KEY_FIELD_NAME_IMPORTANCE, KAFKA_KEY_FIELD_NAME_DOC ).define( KAFKA_DATA_FIELD_NAME_CONFIG, KAFKA_DATA_FIELD_NAME_TYPE, KAFKA_DATA_FIELD_NAME_DEFAULT, + KAFKA_DATA_FIELD_NAME_VALIDATOR, KAFKA_DATA_FIELD_NAME_IMPORTANCE, KAFKA_DATA_FIELD_NAME_DOC ).define( @@ -604,74 +656,78 @@ public static ConfigDef getConfig() { BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_VALIDATOR, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC ).define( BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE, BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT, + BIGQUERY_CLUSTERING_FIELD_NAMES_VALIDATOR, BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE, BIGQUERY_CLUSTERING_FIELD_NAMES_DOC ); } + private static final List> MULTI_PROPERTY_VALIDATIONS = new ArrayList<>(); + + static { + // Note that order matters here: validations are performed in the order they're added to this list, and if a + // property or any of the properties that it depends on has an error, validation for it gets skipped. + // This comes in handy for things like checking for the existence of tables, which requires valid BigQuery + // credentials. We validate those credentials before checking for tables so that we can safely assume while + // checking for those tables that the credentials are already valid. + MULTI_PROPERTY_VALIDATIONS.add(new CredentialsValidator.BigQueryCredentialsValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new CredentialsValidator.GcsCredentialsValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new GcsBucketValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new PartitioningModeValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new UpsertDeleteValidator.UpsertValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new UpsertDeleteValidator.DeleteValidator()); + } + /** - * Throw an exception if the passed-in properties do not constitute a valid sink. - * @param props sink configuration properties + * Used in conjunction with {@link com.wepay.kafka.connect.bigquery.BigQuerySinkConnector#validate(Map)} to perform + * preflight configuration checks. Simple validations that only require a single property value at a time (such as + * ensuring that boolean properties only contain true/false values, or that values for required properties are + * provided) are handled automatically by the {@link #getConfig() ConfigDef} for this class and optionally-defined + * custom {@link ConfigDef.Validator validators}. Other, more sophisticated validations that require multiple + * property values at a time (such as checking if all of the tables the connector will write to already exist if + * automatic table creation is disabled) are performed manually in a subsequent step. + * + * @return a {@link Config} object containing all errors that the connector was able to detect during preflight + * validation of this configuration; never null */ - public static void validate(Map props) { - final boolean hasTopicsConfig = hasTopicsConfig(props); - final boolean hasTopicsRegexConfig = hasTopicsRegexConfig(props); - - if (hasTopicsConfig && hasTopicsRegexConfig) { - throw new ConfigException(TOPICS_CONFIG + " and " + TOPICS_REGEX_CONFIG + - " are mutually exclusive options, but both are set."); - } - - if (!hasTopicsConfig && !hasTopicsRegexConfig) { - throw new ConfigException("Must configure one of " + - TOPICS_CONFIG + " or " + TOPICS_REGEX_CONFIG); - } - - if (upsertDeleteEnabled(props)) { - if (gcsBatchLoadingEnabled(props)) { - throw new ConfigException("Cannot enable both upsert/delete and GCS batch loading"); - } - - String mergeIntervalStr = Optional.ofNullable(props.get(MERGE_INTERVAL_MS_CONFIG)) - .map(String::trim) - .orElse(Long.toString(MERGE_INTERVAL_MS_DEFAULT)); - String mergeRecordsThresholdStr = Optional.ofNullable(props.get(MERGE_RECORDS_THRESHOLD_CONFIG)) - .map(String::trim) - .orElse(Long.toString(MERGE_RECORDS_THRESHOLD_DEFAULT)); - if ("-1".equals(mergeIntervalStr) && "-1".equals(mergeRecordsThresholdStr)) { - throw new ConfigException(MERGE_INTERVAL_MS_CONFIG + " and " - + MERGE_RECORDS_THRESHOLD_CONFIG + " cannot both be -1"); - } - - if ("0".equals(mergeIntervalStr)) { - throw new ConfigException(MERGE_INTERVAL_MS_CONFIG, mergeIntervalStr, "cannot be zero"); - } - if ("0".equals(mergeRecordsThresholdStr)) { - throw new ConfigException(MERGE_RECORDS_THRESHOLD_CONFIG, mergeRecordsThresholdStr, "cannot be zero"); - } - - String kafkaKeyFieldStr = props.get(KAFKA_KEY_FIELD_NAME_CONFIG); - if (kafkaKeyFieldStr == null || kafkaKeyFieldStr.trim().isEmpty()) { - throw new ConfigException(KAFKA_KEY_FIELD_NAME_CONFIG + " must be specified when " - + UPSERT_ENABLED_CONFIG + " and/or " + DELETE_ENABLED_CONFIG + " are set to true"); - } - } + public Config validate() { + List initialValidation = getConfig().validate(originalsStrings()); + Map valuesByName = initialValidation + .stream() + .collect(Collectors.toMap(ConfigValue::name, Function.identity())); + MULTI_PROPERTY_VALIDATIONS.forEach(validator -> { + ConfigValue value = valuesByName.get(validator.propertyName()); + validator.validate(value, this, valuesByName).ifPresent(value::addErrorMessage); + }); + return new Config(initialValidation); } - public static boolean hasTopicsConfig(Map props) { - String topicsStr = props.get(TOPICS_CONFIG); - return topicsStr != null && !topicsStr.trim().isEmpty(); + /** + * @return the key, which is (depending on the key source property) either a path to a file or a raw JSON string + */ + public String getKey() { + return Optional.ofNullable(getPassword(KEYFILE_CONFIG)).map(Password::value).orElse(null); } - public static boolean hasTopicsRegexConfig(Map props) { - String topicsRegexStr = props.get(TOPICS_REGEX_CONFIG); - return topicsRegexStr != null && !topicsRegexStr.trim().isEmpty(); + /** + * @return the {@link com.wepay.kafka.connect.bigquery.GcpClientBuilder.KeySource key source type} that dictates how + * the {@link #getKey()} should be be interpreted + */ + public GcpClientBuilder.KeySource getKeySource() { + String rawKeySource = getString(KEY_SOURCE_CONFIG); + try { + return GcpClientBuilder.KeySource.valueOf(rawKeySource); + } catch (IllegalArgumentException e) { + // Should never happen with preflight validation of the key source property + throw new ConnectException("Invalid key source type: " + rawKeySource); + } } public static boolean upsertDeleteEnabled(Map props) { @@ -737,7 +793,7 @@ public SchemaRetriever getSchemaRetriever() { Class schemaRetrieverClass = userSpecifiedClass.asSubclass(SchemaRetriever.class); - Constructor schemaRetrieverConstructor = null; + Constructor schemaRetrieverConstructor; try { schemaRetrieverConstructor = schemaRetrieverClass.getConstructor(); } catch (NoSuchMethodException nsme) { @@ -747,7 +803,7 @@ public SchemaRetriever getSchemaRetriever() { ); } - SchemaRetriever schemaRetriever = null; + SchemaRetriever schemaRetriever; try { schemaRetriever = schemaRetrieverConstructor.newInstance(); } catch (InstantiationException @@ -766,7 +822,6 @@ public SchemaRetriever getSchemaRetriever() { } /** - * * If the connector is configured to load Kafka data into BigQuery, this config defines * the name of the kafka data field. A structure is created under the field name to contain * kafka data schema including topic, offset, partition and insertTime. @@ -778,7 +833,6 @@ public Optional getKafkaKeyFieldName() { } /** - * * If the connector is configured to load Kafka keys into BigQuery, this config defines * the name of the kafka key field. A structure is created under the field name to contain * a topic's Kafka key schema. @@ -793,56 +847,6 @@ public boolean isUpsertDeleteEnabled() { return getBoolean(UPSERT_ENABLED_CONFIG) || getBoolean(DELETE_ENABLED_CONFIG); } - /** - * Verifies that a bucket is specified if GCS batch loading is enabled. - * @throws ConfigException Exception thrown if no bucket is specified and batch loading is on. - */ - private void verifyBucketSpecified() throws ConfigException { - // Throw an exception if GCS Batch loading will be used but no bucket is specified - if (getString(GCS_BUCKET_NAME_CONFIG).equals("") - && !getList(ENABLE_BATCH_CONFIG).isEmpty()) { - throw new ConfigException("Batch loading enabled for some topics, but no bucket specified"); - } - } - - private void checkAutoCreateTables() { - - Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); - boolean autoCreateTables = getBoolean(TABLE_CREATE_CONFIG); - - if (autoCreateTables && schemaRetriever == null) { - throw new ConfigException( - "Cannot specify automatic table creation without a schema retriever" - ); - } - } - - private void checkBigQuerySchemaUpdateConfigs() { - boolean allBQFieldsNullable = getBoolean(ALL_BQ_FIELDS_NULLABLE_CONFIG); - boolean allowBQRequiredFieldRelaxation = getBoolean(ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); - if (allBQFieldsNullable && !allowBQRequiredFieldRelaxation) { - throw new ConfigException( - "Conflicting Configs, allBQFieldsNullable can be true only if allowBigQueryFieldRelaxation is true" - ); - } - - Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); - - boolean allowNewBigQueryFields = getBoolean(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); - boolean allowRequiredFieldRelaxation = getBoolean(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); - if ((allowNewBigQueryFields || allowRequiredFieldRelaxation) && schemaRetriever == null) { - throw new ConfigException( - "Cannot perform schema updates without a schema retriever" - ); - } - - if (schemaRetriever == null) { - logger.warn( - "No schema retriever class provided; auto schema updates are impossible" - ); - } - } - /** * Returns the field name to use for timestamp partitioning. * @return String that represents the field name. @@ -855,47 +859,15 @@ public Optional getTimestampPartitionFieldName() { * Returns the field names to use for clustering. * @return List of Strings that represent the field names. */ - public Optional> getClusteringPartitionFieldName() { - return Optional.ofNullable(getList(BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG)); - } - - /** - * Check the validity of table partitioning configs. - */ - private void checkPartitionConfigs() { - if (getTimestampPartitionFieldName().isPresent() && getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { - throw new ConfigException( - "Only one partitioning configuration mode may be specified for the connector. " - + "Use either bigQueryPartitionDecorator OR timestampPartitionFieldName." - ); - } - } - - /** - * Check the validity of table clustering configs. - */ - private void checkClusteringConfigs() { - if (getClusteringPartitionFieldName().isPresent()) { - if (!getTimestampPartitionFieldName().isPresent() && !getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { - throw new ConfigException( - "Clustering field name may be specified only on a partitioned table." - ); - } - if (getClusteringPartitionFieldName().get().size() > 4) { - throw new ConfigException( - "You can only specify up to four clustering field names." - ); - } - } + public Optional> getClusteringPartitionFieldNames() { + return Optional + .ofNullable(getList(BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG)) + // With Java 11 there's Predicate::not, but for now we have to just manually invert the isEmpty check + .filter(l -> !l.isEmpty()); } protected BigQuerySinkConfig(ConfigDef config, Map properties) { super(config, properties); - verifyBucketSpecified(); - checkAutoCreateTables(); - checkBigQuerySchemaUpdateConfigs(); - checkPartitionConfigs(); - checkClusteringConfigs(); } public BigQuerySinkConfig(Map properties) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java new file mode 100644 index 000000000..76007d11a --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java @@ -0,0 +1,117 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.storage.Storage; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEYFILE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEY_SOURCE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; + +public abstract class CredentialsValidator> extends MultiPropertyValidator { + + public CredentialsValidator() { + super(KEYFILE_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + PROJECT_CONFIG, KEY_SOURCE_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + String keyFile = config.getKey(); + if (keyFile == null || keyFile.isEmpty()) { + // No credentials to validate + return Optional.empty(); + } + + try { + clientBuilder() + .withConfig(config) + .build(); + return Optional.empty(); + } catch (RuntimeException e) { + String errorMessage = "An unexpected error occurred while validating credentials for " + gcpService(); + if (e.getMessage() != null) { + errorMessage += ": " + e.getMessage(); + } + return Optional.of(errorMessage); + } + } + + protected abstract String gcpService(); + protected abstract ClientBuilder clientBuilder(); + + public static class BigQueryCredentialsValidator extends CredentialsValidator> { + @Override + public String gcpService() { + return "BigQuery"; + } + + @Override + protected GcpClientBuilder clientBuilder() { + return new GcpClientBuilder.BigQueryBuilder(); + } + } + + public static class GcsCredentialsValidator extends CredentialsValidator> { + + private static final Collection DEPENDENTS; + + static { + List dependents = new ArrayList<>(CredentialsValidator.DEPENDENTS); + dependents.add(ENABLE_BATCH_CONFIG); + dependents.add(GCS_BUCKET_NAME_CONFIG); + DEPENDENTS = Collections.unmodifiableCollection(dependents); + } + + @Override + public Collection dependents() { + return DEPENDENTS; + } + + @Override + public String gcpService() { + return "GCS"; + } + + @Override + protected GcpClientBuilder clientBuilder() { + return new GcpClientBuilder.GcsBuilder(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java new file mode 100644 index 000000000..59d2dbd44 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java @@ -0,0 +1,97 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.storage.Bucket; +import com.google.cloud.storage.Storage; +import com.google.common.annotations.VisibleForTesting; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.AUTO_CREATE_BUCKET_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; + +public class GcsBucketValidator extends MultiPropertyValidator { + + public GcsBucketValidator() { + super(GCS_BUCKET_NAME_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + ENABLE_BATCH_CONFIG, AUTO_CREATE_BUCKET_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + Storage gcs; + try { + gcs = new GcpClientBuilder.GcsBuilder() + .withConfig(config) + .build(); + } catch (RuntimeException e) { + return Optional.of(String.format( + "Failed to construct GCS client%s", + e.getMessage() != null ? ": " + e.getMessage() : "" + )); + } + return doValidate(gcs, config); + } + + @VisibleForTesting + Optional doValidate(Storage gcs, BigQuerySinkConfig config) { + List batchLoadedTopics = config.getList(ENABLE_BATCH_CONFIG); + if (batchLoadedTopics == null || batchLoadedTopics.isEmpty()) { + // Batch loading is disabled; no need to validate the GCS bucket + return Optional.empty(); + } + + String bucketName = config.getString(GCS_BUCKET_NAME_CONFIG); + if (bucketName == null || bucketName.trim().isEmpty()) { + return Optional.of("When GCS batch loading is enabled, a bucket must be provided"); + } + + if (config.getBoolean(AUTO_CREATE_BUCKET_CONFIG)) { + return Optional.empty(); + } + + Bucket bucket = gcs.get(bucketName); + if (bucket == null) { + return Optional.of(String.format( + "Automatic bucket creation is disabled but the GCS bucket %s does not exist. " + + "Please either manually create this table before restarting the connector or enable automatic bucket creation " + + "by the connector", + bucketName + )); + } + + return Optional.empty(); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java new file mode 100644 index 000000000..95b9c2da6 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java @@ -0,0 +1,70 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import org.apache.kafka.common.config.ConfigValue; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +public abstract class MultiPropertyValidator { + + private final String propertyName; + + protected MultiPropertyValidator(String propertyName) { + this.propertyName = propertyName; + } + + public String propertyName() { + return propertyName; + } + + public Optional validate(ConfigValue value, Config config, Map valuesByName) { + // Only perform follow-up validation if the property doesn't already have an error associated with it + if (!value.errorMessages().isEmpty()) { + return Optional.empty(); + } + + boolean dependentsAreValid = dependents().stream() + .map(valuesByName::get) + .filter(Objects::nonNull) + .map(ConfigValue::errorMessages) + .allMatch(List::isEmpty); + // Also ensure that all of the other properties that the validation for this one depends on don't already have errors + if (!dependentsAreValid) { + return Optional.empty(); + } + + try { + return doValidate(config); + } catch (RuntimeException e) { + return Optional.of( + "An unexpected error occurred during validation" + + (e.getMessage() != null ? ": " + e.getMessage() : "") + ); + } + } + + protected abstract Collection dependents(); + protected abstract Optional doValidate(Config config); +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java new file mode 100644 index 000000000..65389e5fd --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java @@ -0,0 +1,60 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG; + +public class PartitioningModeValidator extends MultiPropertyValidator { + public PartitioningModeValidator() { + super(BIGQUERY_PARTITION_DECORATOR_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + if (!config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { + return Optional.empty(); + } + + if (config.getTimestampPartitionFieldName().isPresent()) { + return Optional.of(String.format("Only one partitioning mode may be specified for the connector. " + + "Use either %s OR %s.", + BIGQUERY_PARTITION_DECORATOR_CONFIG, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG + )); + } else { + return Optional.empty(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/UpsertDeleteValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/UpsertDeleteValidator.java new file mode 100644 index 000000000..a20178258 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/UpsertDeleteValidator.java @@ -0,0 +1,102 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.DELETE_ENABLED_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KAFKA_KEY_FIELD_NAME_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.MERGE_INTERVAL_MS_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.MERGE_RECORDS_THRESHOLD_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.UPSERT_ENABLED_CONFIG; + +public abstract class UpsertDeleteValidator extends MultiPropertyValidator { + private UpsertDeleteValidator(String propertyName) { + super(propertyName); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + MERGE_INTERVAL_MS_CONFIG, MERGE_RECORDS_THRESHOLD_CONFIG, KAFKA_KEY_FIELD_NAME_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + if (!modeEnabled(config)) { + return Optional.empty(); + } + + long mergeInterval = config.getLong(MERGE_INTERVAL_MS_CONFIG); + long mergeRecordsThreshold = config.getLong(MERGE_RECORDS_THRESHOLD_CONFIG); + + if (mergeInterval == -1 && mergeRecordsThreshold == -1) { + return Optional.of(String.format( + "%s and %s cannot both be -1", + MERGE_INTERVAL_MS_CONFIG, + MERGE_RECORDS_THRESHOLD_CONFIG + )); + } + + if (!config.getKafkaKeyFieldName().isPresent()) { + return Optional.of(String.format( + "%s must be specified when %s is set to true", + KAFKA_KEY_FIELD_NAME_CONFIG, + propertyName() + )); + } + + return Optional.empty(); + } + + /** + * @param config the user-provided configuration + * @return whether the write mode for the validator (i.e., either upsert or delete) is enabled + */ + protected abstract boolean modeEnabled(BigQuerySinkConfig config); + + public static class UpsertValidator extends UpsertDeleteValidator { + public UpsertValidator() { + super(UPSERT_ENABLED_CONFIG); + } + + @Override + protected boolean modeEnabled(BigQuerySinkConfig config) { + return config.getBoolean(UPSERT_ENABLED_CONFIG); + } + } + + public static class DeleteValidator extends UpsertDeleteValidator { + public DeleteValidator() { + super(DELETE_ENABLED_CONFIG); + } + + @Override + protected boolean modeEnabled(BigQuerySinkConfig config) { + return config.getBoolean(DELETE_ENABLED_CONFIG); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java deleted file mode 100644 index 805cd5643..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery.exception; - -import org.apache.kafka.connect.errors.ConnectException; - -/** - * Class for exceptions that occur while attempting to process configuration files, including both - * formatting and logical errors. - */ -public class SinkConfigConnectException extends ConnectException { - public SinkConfigConnectException(String msg) { - super(msg); - } - - public SinkConfigConnectException(String msg, Throwable thr) { - super(msg, thr); - } - - public SinkConfigConnectException(Throwable thr) { - super(thr); - } -} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java index f90c24b87..c1757bd95 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java @@ -19,20 +19,9 @@ package com.wepay.kafka.connect.bigquery; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; - import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; - -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; - import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; -import org.apache.kafka.common.config.ConfigException; - import org.apache.kafka.connect.data.Schema; - import org.apache.kafka.connect.sink.SinkRecord; import org.junit.BeforeClass; import org.junit.Test; @@ -41,6 +30,10 @@ import java.util.List; import java.util.Map; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; + public class BigQuerySinkConnectorTest { private static SinkPropertiesFactory propertiesFactory; @@ -114,19 +107,6 @@ public void testConfig() { assertNotNull(new BigQuerySinkConnector().config()); } - // Make sure that a config exception is properly translated into a SinkConfigConnectException - @Test(expected = SinkConfigConnectException.class) - public void testConfigException() { - try { - Map badProperties = propertiesFactory.getProperties(); - badProperties.remove(BigQuerySinkConfig.TOPICS_CONFIG); - BigQuerySinkConfig.validate(badProperties); - new BigQuerySinkConnector().start(badProperties); - } catch (ConfigException e) { - throw new SinkConfigConnectException(e); - } - } - @Test public void testVersion() { assertNotNull(new BigQuerySinkConnector().version()); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index e718d9ec7..5f04786a7 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -44,7 +44,6 @@ import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; import org.apache.kafka.common.config.ConfigException; @@ -647,25 +646,6 @@ public void testInterruptedException() { testTask.flush(Collections.emptyMap()); } - // Make sure that a ConfigException is properly translated into a SinkConfigConnectException - @Test(expected = SinkConfigConnectException.class) - public void testConfigException() { - try { - Map badProperties = propertiesFactory.getProperties(); - badProperties.remove(BigQuerySinkConfig.TOPICS_CONFIG); - BigQuerySinkConfig.validate(badProperties); - - SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); - SchemaManager schemaManager = mock(SchemaManager.class); - - BigQuerySinkTask testTask = - new BigQuerySinkTask(mock(BigQuery.class), schemaRetriever, mock(Storage.class), schemaManager); - testTask.start(badProperties); - } catch (ConfigException e) { - throw new SinkConfigConnectException(e); - } - } - @Test public void testVersion() { assertNotNull(new BigQuerySinkTask().version()); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java index da5359419..fde3567c7 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java @@ -49,24 +49,4 @@ public Map getProperties() { return properties; } - - /** - * Make sure that each of the default configuration properties work nicely with the given - * configuration object. - * - * @param config The config object to test - */ - public void testProperties(BigQuerySinkConfig config) { - - config.getList(BigQuerySinkConfig.TOPICS_CONFIG); - config.getString(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG); - - config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - config.getKeyFile(); - config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG); - config.getInt(BigQuerySinkConfig.AVRO_DATA_CACHE_SIZE_CONFIG); - - config.getBoolean(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); - config.getBoolean(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); - } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index ce176e77d..06abc90b0 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -49,7 +49,11 @@ public void initializePropertiesFactory() { public void metaTestBasicConfigProperties() { Map basicConfigProperties = propertiesFactory.getProperties(); BigQuerySinkConfig config = new BigQuerySinkConfig(basicConfigProperties); - propertiesFactory.testProperties(config); + config.getList(BigQuerySinkConfig.TOPICS_CONFIG); + config.getString(BigQuerySinkConfig.PROJECT_CONFIG); + config.getKey(); + config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG); + config.getInt(BigQuerySinkConfig.AVRO_DATA_CACHE_SIZE_CONFIG); } @Test @@ -94,16 +98,6 @@ public void testEmptyTimestampPartitionFieldName() { assertFalse(testConfig.getTimestampPartitionFieldName().isPresent()); } - /** - * Test if the field name being non-empty and the decorator default (true) errors correctly. - */ - @Test (expected = ConfigException.class) - public void testTimestampPartitionFieldNameError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); - new BigQuerySinkConfig(configProperties); - } - /** * Test the field name being non-empty and the decorator set to false works correctly. */ @@ -124,22 +118,7 @@ public void testTimestampPartitionFieldName() { public void testEmptyClusteringFieldNames() { Map configProperties = propertiesFactory.getProperties(); BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); - assertFalse(testConfig.getClusteringPartitionFieldName().isPresent()); - } - - /** - * Test if the field names being non-empty and the partitioning is not present errors correctly. - */ - @Test (expected = ConfigException.class) - public void testClusteringFieldNamesWithoutTimestampPartitionError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, null); - configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); - configProperties.put( - BigQuerySinkConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, - "column1,column2" - ); - new BigQuerySinkConfig(configProperties); + assertFalse(testConfig.getClusteringPartitionFieldNames().isPresent()); } /** @@ -174,7 +153,7 @@ public void testClusteringFieldNames() { ); BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); - Optional> testClusteringPartitionFieldName = testConfig.getClusteringPartitionFieldName(); + Optional> testClusteringPartitionFieldName = testConfig.getClusteringPartitionFieldNames(); assertTrue(testClusteringPartitionFieldName.isPresent()); assertEquals(expectedClusteringPartitionFieldName, testClusteringPartitionFieldName.get()); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java new file mode 100644 index 000000000..7a55d5ad8 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java @@ -0,0 +1,69 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; +import org.junit.Test; + +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CredentialsValidatorTest { + + @Test + public void testNoCredentialsSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getKey()).thenReturn(null); + + assertEquals( + Optional.empty(), + new CredentialsValidator.BigQueryCredentialsValidator().doValidate(config) + ); + assertEquals( + Optional.empty(), + new CredentialsValidator.GcsCredentialsValidator().doValidate(config) + ); + } + + @Test + public void testFailureToConstructClient() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getKey()).thenReturn("key"); + + @SuppressWarnings("unchecked") + GcpClientBuilder mockClientBuilder = mock(GcpClientBuilder.class); + when(mockClientBuilder.withConfig(eq(config))).thenReturn(mockClientBuilder); + when(mockClientBuilder.build()).thenThrow(new RuntimeException("Provided credentials are invalid")); + + assertNotEquals( + Optional.empty(), + new CredentialsValidator.BigQueryCredentialsValidator().doValidate(config) + ); + assertNotEquals( + Optional.empty(), + new CredentialsValidator.GcsCredentialsValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java new file mode 100644 index 000000000..b8c80fee4 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java @@ -0,0 +1,122 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.storage.Bucket; +import com.google.cloud.storage.Storage; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class GcsBucketValidatorTest { + + @Mock + private Storage gcs; + + @Test + public void testNullBatchLoadingSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(null); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testEmptyBatchLoadingSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.emptyList()); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testNullBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(null); + + assertNotEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testBlankBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(" \t "); + + assertNotEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testValidBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + final String bucketName = "gee_cs"; + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(bucketName); + + Bucket bucket = mock(Bucket.class); + when(gcs.get(eq(bucketName))).thenReturn(bucket); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testMissingBucketAndBucketCreationDisabledWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + final String bucketName = "gee_cs"; + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(bucketName); + + when(gcs.get(eq(bucketName))).thenReturn(null); + + assertNotEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java new file mode 100644 index 000000000..205bb56a3 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java @@ -0,0 +1,138 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.config.ConfigValue; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; + +public class MultiPropertyValidatorTest { + + private static class TestValidator extends MultiPropertyValidator { + + private final List dependents; + private final Function> validationFunction; + + public TestValidator(String propertyName, List dependents, Function> validationFunction) { + super(propertyName); + this.dependents = dependents; + this.validationFunction = validationFunction; + } + + @Override + protected Collection dependents() { + return dependents; + } + + @Override + protected Optional doValidate(Config config) { + return validationFunction.apply(config); + } + } + + @Test + public void testExistingErrorSkipsValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Arrays.asList("d1", "d2", "d3"), + o -> { + fail("Validation should have been performed on property that already has an error"); + return null; + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.singletonList("an error")); + + assertEquals( + Optional.empty(), + validator.validate(configValue, null, Collections.emptyMap()) + ); + } + + @Test + public void testDependentErrorSkipsValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Arrays.asList("d1", "d2", "d3"), + o -> { + fail("Validation should have been performed on property whose dependent already has an error"); + return null; + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + Map valuesByName = ImmutableMap.of( + "d1", new ConfigValue("d1", "v1", Collections.emptyList(), Collections.emptyList()), + "d2", new ConfigValue("d2", "v1", Collections.emptyList(), Collections.singletonList("an error")) + ); + + assertEquals( + Optional.empty(), + validator.validate(configValue, null, valuesByName) + ); + } + + @Test + public void testValidationFails() { + Optional expectedError = Optional.of("an error"); + MultiPropertyValidator validator = new TestValidator<>( + "p", + Collections.emptyList(), + o -> expectedError + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + + assertEquals( + expectedError, + validator.validate(configValue, null, Collections.emptyMap()) + ); + } + + @Test + public void testUnexpectedErrorDuringValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Collections.emptyList(), + o -> { + throw new RuntimeException("Some unexpected error"); + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + + assertNotEquals( + Optional.empty(), + validator.validate(configValue, null, Collections.emptyMap()) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java new file mode 100644 index 000000000..a4b79a14c --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java @@ -0,0 +1,80 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import org.junit.Test; + +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PartitioningModeValidatorTest { + + @Test + public void testDisabledDecoratorSyntaxSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(false); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testDecoratorSyntaxWithoutTimestampPartitionFieldName() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.empty()); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testDecoratorSyntaxWithTimestampPartitionFieldName() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.of("f1")); + + assertNotEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testTimestampPartitionFieldNameWithoutDecoratorSyntax() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(false); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.of("f1")); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java index 64db82dbb..963834bd9 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -43,7 +43,7 @@ import com.google.cloud.bigquery.Schema; import com.google.cloud.bigquery.Table; import com.google.cloud.bigquery.TableResult; -import com.wepay.kafka.connect.bigquery.BigQueryHelper; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.apache.kafka.clients.admin.Admin; @@ -145,9 +145,11 @@ protected Map baseConnectorProps(int tasksMax) { } protected BigQuery newBigQuery() { - return new BigQueryHelper() - .setKeySource(keySource()) - .connect(project(), keyFile()); + return new GcpClientBuilder.BigQueryBuilder() + .withKey(keyFile()) + .withKeySource(GcpClientBuilder.KeySource.valueOf(keySource())) + .withProject(project()) + .build(); } protected void waitForCommittedRecords( diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java index 4a9fb7301..26b40e3fe 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java @@ -23,7 +23,7 @@ import com.google.cloud.storage.Blob; import com.google.cloud.storage.Bucket; import com.google.cloud.storage.Storage; -import com.wepay.kafka.connect.bigquery.GCSBuilder; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,9 +41,12 @@ public class BucketClearer { * @param keySource The key source. If "FILE", then the {@code key} parameter will be treated as a * filename; if "JSON", then {@code key} will be treated as a raw JSON string. */ - public static void clearBucket( - String key, String project, String bucketName, String folderName, String keySource) { - Storage gcs = new GCSBuilder(project).setKey(key).setKeySource(keySource).build(); + public static void clearBucket(String key, String project, String bucketName, String folderName, String keySource) { + Storage gcs = new GcpClientBuilder.GcsBuilder() + .withKeySource(GcpClientBuilder.KeySource.valueOf(keySource)) + .withKey(key) + .withProject(project) + .build(); Bucket bucket = gcs.get(bucketName); if (bucket != null) { logger.info("Deleting objects in the {} folder for bucket {}", From dc6097ff06be67ec85f0951448b30c3850cfc44b Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 9 Sep 2021 11:35:04 -0400 Subject: [PATCH 126/190] GH-139: Move all user-visible configuration properties to single configuration class --- .../bigquery/BigQuerySinkConnector.java | 1 + .../connect/bigquery/BigQuerySinkTask.java | 1 - .../bigquery/config/BigQuerySinkConfig.java | 52 +++++++++++++++++++ .../config/BigQuerySinkConfigTest.java | 14 ++--- 4 files changed, 58 insertions(+), 10 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index 2fe315f40..d40c9cee1 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -50,6 +50,7 @@ public ConfigDef config() { return BigQuerySinkConfig.getConfig(); } + @Override public void start(Map properties) { logger.trace("connector.start()"); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 04ac63722..931979075 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -34,7 +34,6 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId; import com.wepay.kafka.connect.bigquery.utils.SinkRecordConverter; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index fc5fd44d0..472a84f59 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -651,6 +651,58 @@ public boolean visible(String s, Map map) { return true; } } + ).define( + THREAD_POOL_SIZE_CONFIG, + THREAD_POOL_SIZE_TYPE, + THREAD_POOL_SIZE_DEFAULT, + THREAD_POOL_SIZE_VALIDATOR, + THREAD_POOL_SIZE_IMPORTANCE, + THREAD_POOL_SIZE_DOC + ).define( + QUEUE_SIZE_CONFIG, + QUEUE_SIZE_TYPE, + QUEUE_SIZE_DEFAULT, + QUEUE_SIZE_VALIDATOR, + QUEUE_SIZE_IMPORTANCE, + QUEUE_SIZE_DOC + ).define( + BIGQUERY_RETRY_CONFIG, + BIGQUERY_RETRY_TYPE, + BIGQUERY_RETRY_DEFAULT, + BIGQUERY_RETRY_VALIDATOR, + BIGQUERY_RETRY_IMPORTANCE, + BIGQUERY_RETRY_DOC + ).define( + BIGQUERY_RETRY_WAIT_CONFIG, + BIGQUERY_RETRY_WAIT_CONFIG_TYPE, + BIGQUERY_RETRY_WAIT_DEFAULT, + BIGQUERY_RETRY_WAIT_VALIDATOR, + BIGQUERY_RETRY_WAIT_IMPORTANCE, + BIGQUERY_RETRY_WAIT_DOC + ).define( + BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, + BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG_TYPE, + BIGQUERY_MESSAGE_TIME_PARTITIONING_DEFAULT, + BIGQUERY_MESSAGE_TIME_PARTITIONING_IMPORTANCE, + BIGQUERY_MESSAGE_TIME_PARTITIONING_DOC + ).define( + BIGQUERY_PARTITION_DECORATOR_CONFIG, + BIGQUERY_PARTITION_DECORATOR_CONFIG_TYPE, + BIGQUERY_PARTITION_DECORATOR_DEFAULT, + BIGQUERY_PARTITION_DECORATOR_IMPORTANCE, + BIGQUERY_PARTITION_DECORATOR_DOC + ).define( + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC + ).define( + BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, + BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE, + BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT, + BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE, + BIGQUERY_CLUSTERING_FIELD_NAMES_DOC ); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index 2b67fd310..3bc905618 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -19,10 +19,6 @@ package com.wepay.kafka.connect.bigquery.config; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - import com.google.cloud.bigquery.TimePartitioning; import com.wepay.kafka.connect.bigquery.SinkPropertiesFactory; import com.wepay.kafka.connect.bigquery.convert.BigQueryRecordConverter; @@ -39,6 +35,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; public class BigQuerySinkConfigTest { @@ -184,18 +181,17 @@ public void testClusteringFieldNames() { assertEquals(expectedClusteringPartitionFieldName, testClusteringPartitionFieldName.get()); } - @Test public void testValidTimePartitioningTypes() { Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); + configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); for (TimePartitioning.Type type : TimePartitioning.Type.values()) { configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, type.name()); assertEquals(type, new BigQuerySinkConfig(configProperties).getTimePartitioningType()); } - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); + configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, TimePartitioning.Type.DAY.name()); assertEquals(TimePartitioning.Type.DAY, new BigQuerySinkConfig(configProperties).getTimePartitioningType()); } @@ -203,8 +199,8 @@ public void testValidTimePartitioningTypes() { @Test public void testInvalidTimePartitioningTypes() { Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkTaskConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); - configProperties.put(BigQuerySinkTaskConfig.TABLE_CREATE_CONFIG, "true"); + configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); + configProperties.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); for (TimePartitioning.Type type : TimePartitioning.Type.values()) { if (TimePartitioning.Type.DAY.equals(type)) { From b180827a89fffac99a15d7f0238752ff37a47520 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Sun, 19 Sep 2021 07:10:36 -0400 Subject: [PATCH 127/190] GH-139: Improve preflight config validation Depends on https://github.com/confluentinc/kafka-connect-bigquery/pull/140 Addresses the remainder of https://github.com/confluentinc/kafka-connect-bigquery/issues/139. All multi-property configuration validation is moved from the constructor of the BigQuerySinkConfig class to a new `BigQuerySinkConfig::validate` method, which is invoked from `BigQuerySinkConnector::validate` and leverages the Connect API for defining custom error messages on a per-property basis instead of throwing exceptions on invalid configurations, which only displays one message at a time and doesn't work well with programmatic UIs. Unit tests are added for all new validation logic. The logic for GCP client construction is also updated to conform to a common interface, which should improve readability and maintainability, and makes it easier to validate credentials for either BigQuery or GCS. Finally, the `SinkConfigConnectException` class is removed as it's not really necessary and doesn't bring anything worthwhile to the code base. --- .../connect/bigquery/BigQueryHelper.java | 108 ------ .../bigquery/BigQuerySinkConnector.java | 13 + .../connect/bigquery/BigQuerySinkTask.java | 28 +- .../kafka/connect/bigquery/GCSBuilder.java | 109 ------ .../connect/bigquery/GcpClientBuilder.java | 145 ++++++++ .../bigquery/config/BigQuerySinkConfig.java | 346 ++++++------------ .../bigquery/config/CredentialsValidator.java | 117 ++++++ .../bigquery/config/GcsBucketValidator.java | 97 +++++ .../config/MultiPropertyValidator.java | 70 ++++ .../config/PartitioningModeValidator.java | 60 +++ .../config/PartitioningTypeValidator.java | 65 ++++ .../config/UpsertDeleteValidator.java | 102 ++++++ .../exception/SinkConfigConnectException.java | 40 -- .../bigquery/BigQuerySinkConnectorTest.java | 28 +- .../bigquery/BigQuerySinkTaskTest.java | 21 -- .../bigquery/SinkPropertiesFactory.java | 20 - .../config/BigQuerySinkConfigTest.java | 56 +-- .../config/CredentialsValidatorTest.java | 69 ++++ .../config/GcsBucketValidatorTest.java | 122 ++++++ .../config/MultiPropertyValidatorTest.java | 138 +++++++ .../config/PartitioningModeValidatorTest.java | 80 ++++ .../config/PartitioningTypeValidatorTest.java | 92 +++++ .../bigquery/integration/BaseConnectorIT.java | 10 +- .../integration/utils/BucketClearer.java | 11 +- 24 files changed, 1329 insertions(+), 618 deletions(-) delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningTypeValidator.java create mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/UpsertDeleteValidator.java delete mode 100644 kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningTypeValidatorTest.java diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java deleted file mode 100644 index f90ea5f3f..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQueryHelper.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery; - -import com.google.auth.oauth2.GoogleCredentials; -import com.google.cloud.bigquery.BigQuery; -import com.google.cloud.bigquery.BigQueryOptions; - -import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.ByteArrayInputStream; -import java.nio.charset.StandardCharsets; - -/** - * Convenience class for creating a default {@link com.google.cloud.bigquery.BigQuery} instance, - * with or without login credentials. - */ -public class BigQueryHelper { - private static final Logger logger = LoggerFactory.getLogger(BigQueryHelper.class); - private static String keySource; - - /** - * Returns a default {@link BigQuery} instance for the specified project with credentials provided - * in the specified file, which can then be used for creating, updating, and inserting into tables - * from specific datasets. - * - * @param projectName The name of the BigQuery project to work with - * @param key The google credentials JSON key that can be used to provide - * credentials to BigQuery, or null if no authentication should be performed. - * @return The resulting BigQuery object. - */ - public BigQuery connect(String projectName, String key) { - if (key == null) { - return connect(projectName); - } - logger.debug("Attempting to open file {} for service account json key", key); - InputStream credentialsStream; - try { - if (keySource != null && keySource.equals("JSON")) { - credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); - } else { - credentialsStream = new FileInputStream(key); - } - return new - BigQueryOptions.DefaultBigQueryFactory().create( - BigQueryOptions.newBuilder() - .setProjectId(projectName) - .setCredentials(GoogleCredentials.fromStream(credentialsStream)) - .build() - ); - } catch (IOException err) { - throw new BigQueryConnectException("Failed to access json key file", err); - } - } - /** - * Returns a default {@link BigQuery} instance for the specified project with credentials provided - * in the specified file, which can then be used for creating, updating, and inserting into tables - * from specific datasets. - * - * @param keySource The type of key config we can expect. This is either a String - * representation of the Google credentials file, or the path to the Google credentials file. - * @return The resulting BigQuery object. - */ - public BigQueryHelper setKeySource(String keySource) { - this.keySource = keySource; - return this; - } - - /** - * Returns a default {@link BigQuery} instance for the specified project with no authentication - * credentials, which can then be used for creating, updating, and inserting into tables from - * specific datasets. - * - * @param projectName The name of the BigQuery project to work with - * @return The resulting BigQuery object. - */ - public BigQuery connect(String projectName) { - logger.debug("Attempting to access BigQuery without authentication"); - return new BigQueryOptions.DefaultBigQueryFactory().create( - BigQueryOptions.newBuilder() - .setProjectId(projectName) - .build() - ); - } -} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java index d40c9cee1..ae3ef6f43 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnector.java @@ -22,7 +22,9 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; import com.wepay.kafka.connect.bigquery.utils.Version; +import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.sink.SinkConnector; import org.slf4j.Logger; @@ -50,6 +52,17 @@ public ConfigDef config() { return BigQuerySinkConfig.getConfig(); } + @Override + public Config validate(Map properties) { + List singlePropertyValidations = config().validate(properties); + // If any of our properties had malformed syntax or failed a validation to ensure, e.g., that it fell within an + // acceptable numeric range, we only report those errors since they prevent us from being able to construct a + // valid BigQuerySinkConfig instance + if (singlePropertyValidations.stream().anyMatch(v -> !v.errorMessages().isEmpty())) { + return new Config(singlePropertyValidations); + } + return new BigQuerySinkConfig(properties).validate(); + } @Override public void start(Map properties) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 931979075..96a1d43f0 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -51,7 +51,6 @@ import com.wepay.kafka.connect.bigquery.write.row.UpsertDeleteBigQueryWriter; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkRecord; @@ -179,7 +178,7 @@ public Map preCommit(Map: format or just the format."); + throw new ConnectException(String.format( + "Incorrect regex replacement format in topic name '%s'. " + + "SMT replacement should either produce the : format " + + "or just the format.", + record.topic() + )); } if (sanitize) { @@ -336,10 +339,9 @@ private Table retrieveCachedTable(TableId tableId) { } private BigQuery newBigQuery() { - String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - String keyFile = config.getKeyFile(); - String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); - return new BigQueryHelper().setKeySource(keySource).connect(projectName, keyFile); + return new GcpClientBuilder.BigQueryBuilder() + .withConfig(config) + .build(); } private SchemaManager getSchemaManager() { @@ -356,7 +358,7 @@ private SchemaManager newSchemaManager() { Optional kafkaKeyFieldName = config.getKafkaKeyFieldName(); Optional kafkaDataFieldName = config.getKafkaDataFieldName(); Optional timestampPartitionFieldName = config.getTimestampPartitionFieldName(); - Optional> clusteringFieldName = config.getClusteringPartitionFieldName(); + Optional> clusteringFieldName = config.getClusteringPartitionFieldNames(); TimePartitioning.Type timePartitioningType = config.getTimePartitioningType(); boolean allowNewBQFields = config.getBoolean(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); boolean allowReqFieldRelaxation = config.getBoolean(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); @@ -396,11 +398,9 @@ private Storage getGcs() { if (testGcs != null) { return testGcs; } - String projectName = config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - String key = config.getKeyFile(); - String keySource = config.getString(BigQuerySinkConfig.KEY_SOURCE_CONFIG); - return new GCSBuilder(projectName).setKey(key).setKeySource(keySource).build(); - + return new GcpClientBuilder.GcsBuilder() + .withConfig(config) + .build(); } private GCSToBQWriter getGcsWriter() { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java deleted file mode 100644 index 4a0952b2e..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GCSBuilder.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery; - -import com.google.auth.oauth2.GoogleCredentials; -import com.google.cloud.storage.Storage; -import com.google.cloud.storage.StorageOptions; - -import com.wepay.kafka.connect.bigquery.exception.GCSConnectException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.ByteArrayInputStream; -import java.nio.charset.StandardCharsets; - -/** - * Convenience class for creating a {@link com.google.cloud.storage.Storage} instance - */ -public class GCSBuilder { - private static final Logger logger = LoggerFactory.getLogger(GCSBuilder.class); - - private final String projectName; - private String key; - private String keySource; - - public GCSBuilder(String projectName) { - this.projectName = projectName; - this.key = null; - } - - public GCSBuilder setKeySource(String keySourceType) { - this.keySource = keySourceType; - return this; - } - - public GCSBuilder setKey(String keyFile) { - this.key = keyFile; - return this; - } - public Storage build() { - return connect(projectName, key); - } - - /** - * Returns a default {@link Storage} instance for the specified project with credentials provided - * in the specified file. - * - * @param projectName The name of the GCS project to work with - * @param key The name of a file containing a JSON key that can be used to provide - * credentials to GCS, or null if no authentication should be performed. - * @return The resulting Storage object. - */ - private Storage connect(String projectName, String key) { - if (key == null) { - return connect(projectName); - } - try { - InputStream credentialsStream; - if (keySource != null && keySource.equals("JSON")) { - credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); - } else { - credentialsStream = new FileInputStream(key); - } - return StorageOptions.newBuilder() - .setProjectId(projectName) - .setCredentials(GoogleCredentials.fromStream(credentialsStream)) - .build() - .getService(); - } catch (IOException err) { - throw new GCSConnectException("Failed to access json key file", err); - } - } - - /** - * Returns a default {@link Storage} instance for the specified project with no authentication - * credentials. - * - * @param projectName The name of the GCS project to work with - * @return The resulting Storage object. - */ - private Storage connect(String projectName) { - logger.debug("Attempting to access BigQuery without authentication"); - return StorageOptions.newBuilder() - .setProjectId(projectName) - .build() - .getService(); - } -} - diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java new file mode 100644 index 000000000..5c79fec87 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java @@ -0,0 +1,145 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryOptions; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageOptions; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Objects; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; + +public abstract class GcpClientBuilder { + + public enum KeySource { + FILE, JSON + } + + private static final Logger logger = LoggerFactory.getLogger(GcpClientBuilder.class); + + private String project = null; + private KeySource keySource = null; + private String key = null; + + public GcpClientBuilder withConfig(BigQuerySinkConfig config) { + return withProject(config.getString(PROJECT_CONFIG)) + .withKeySource(config.getKeySource()) + .withKey(config.getKey()); + } + + public GcpClientBuilder withProject(String project) { + Objects.requireNonNull(project, "Project cannot be null"); + this.project = project; + return this; + } + + public GcpClientBuilder withKeySource(KeySource keySource) { + Objects.requireNonNull(keySource, "Key cannot be null"); + this.keySource = keySource; + return this; + } + + public GcpClientBuilder withKey(String key) { + this.key = key; + return this; + } + + public Client build() { + return doBuild(project, credentials()); + } + + private GoogleCredentials credentials() { + if (key == null) { + return null; + } + + Objects.requireNonNull(keySource, "Key source must be defined to build a GCP client"); + Objects.requireNonNull(project, "Project must be defined to build a GCP client"); + + InputStream credentialsStream; + switch (keySource) { + case JSON: + credentialsStream = new ByteArrayInputStream(key.getBytes(StandardCharsets.UTF_8)); + break; + case FILE: + try { + logger.debug("Attempting to open file {} for service account json key", key); + credentialsStream = new FileInputStream(key); + } catch (IOException e) { + throw new BigQueryConnectException("Failed to access JSON key file", e); + } + break; + default: + throw new IllegalArgumentException("Unexpected value for KeySource enum: " + keySource); + } + + try { + return GoogleCredentials.fromStream(credentialsStream); + } catch (IOException e) { + throw new BigQueryConnectException("Failed to create credentials from input stream", e); + } + } + + protected abstract Client doBuild(String project, GoogleCredentials credentials); + + public static class BigQueryBuilder extends GcpClientBuilder { + @Override + protected BigQuery doBuild(String project, GoogleCredentials credentials) { + BigQueryOptions.Builder builder = BigQueryOptions.newBuilder() + .setProjectId(project); + + if (credentials != null) { + builder.setCredentials(credentials); + } else { + logger.debug("Attempting to access BigQuery without authentication"); + } + + return builder.build().getService(); + } + } + + public static class GcsBuilder extends GcpClientBuilder { + @Override + protected Storage doBuild(String project, GoogleCredentials credentials) { + StorageOptions.Builder builder = StorageOptions.newBuilder() + .setProjectId(project); + + if (credentials != null) { + builder.setCredentials(credentials); + } else { + logger.debug("Attempting to access GCS without authentication"); + } + + return builder.build().getService(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 472a84f59..cecc7f19f 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -20,8 +20,8 @@ package com.wepay.kafka.connect.bigquery.config; import com.google.cloud.bigquery.Schema; - import com.google.cloud.bigquery.TimePartitioning; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.convert.BigQueryRecordConverter; import com.wepay.kafka.connect.bigquery.convert.BigQuerySchemaConverter; @@ -29,12 +29,13 @@ import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.common.config.types.Password; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkConnector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; @@ -43,6 +44,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -50,8 +52,6 @@ * Base class for connector and task configs; contains properties shared between the two of them. */ public class BigQuerySinkConfig extends AbstractConfig { - private static final Logger logger = LoggerFactory.getLogger(BigQuerySinkConfig.class); - // Values taken from https://github.com/apache/kafka/blob/1.1.1/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java#L33 public static final String TOPICS_CONFIG = SinkConnector.TOPICS_CONFIG; private static final ConfigDef.Type TOPICS_TYPE = ConfigDef.Type.LIST; @@ -137,9 +137,13 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KEY_SOURCE_CONFIG = "keySource"; private static final ConfigDef.Type KEY_SOURCE_TYPE = ConfigDef.Type.STRING; - public static final String KEY_SOURCE_DEFAULT = "FILE"; - private static final ConfigDef.Validator KEY_SOURCE_VALIDATOR = - ConfigDef.ValidString.in("FILE", "JSON"); + public static final String KEY_SOURCE_DEFAULT = GcpClientBuilder.KeySource.FILE.name(); + private static final ConfigDef.Validator KEY_SOURCE_VALIDATOR = ConfigDef.ValidString.in( + Stream.of(GcpClientBuilder.KeySource.values()) + .map(GcpClientBuilder.KeySource::name) + .collect(Collectors.toList()) + .toArray(new String[0]) + ); private static final ConfigDef.Importance KEY_SOURCE_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String KEY_SOURCE_DOC = "Determines whether the keyfile config is the path to the credentials json, or the json itself"; @@ -169,6 +173,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KAFKA_KEY_FIELD_NAME_CONFIG = "kafkaKeyFieldName"; private static final ConfigDef.Type KAFKA_KEY_FIELD_NAME_TYPE = ConfigDef.Type.STRING; public static final String KAFKA_KEY_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator KAFKA_KEY_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance KAFKA_KEY_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String KAFKA_KEY_FIELD_NAME_DOC = "The name of the field of Kafka key. " + "Default to be null, which means Kafka Key Field will not be included."; @@ -176,6 +181,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String KAFKA_DATA_FIELD_NAME_CONFIG = "kafkaDataFieldName"; private static final ConfigDef.Type KAFKA_DATA_FIELD_NAME_TYPE = ConfigDef.Type.STRING; public static final String KAFKA_DATA_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator KAFKA_DATA_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance KAFKA_DATA_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String KAFKA_DATA_FIELD_NAME_DOC = "The name of the field of Kafka Data. " + "Default to be null, which means Kafka Data Field will not be included. "; @@ -278,7 +284,21 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String MERGE_INTERVAL_MS_CONFIG = "mergeIntervalMs"; private static final ConfigDef.Type MERGE_INTERVAL_MS_TYPE = ConfigDef.Type.LONG; public static final long MERGE_INTERVAL_MS_DEFAULT = 60_000L; - private static final ConfigDef.Validator MERGE_INTERVAL_MS_VALIDATOR = ConfigDef.Range.atLeast(-1); + private static final ConfigDef.Validator MERGE_INTERVAL_MS_VALIDATOR = ConfigDef.LambdaValidator.with( + (name, value) -> { + if (value == null) { + return; + } + long parsedValue = (long) ConfigDef.parseType(name, value, MERGE_INTERVAL_MS_TYPE); + + if (parsedValue == 0) { + throw new ConfigException(name, value, "Cannot be zero"); + } else if (parsedValue < -1) { + throw new ConfigException(name, value, "Cannot be less than -1"); + } + }, + () -> "Either a positive integer or -1 to disable time interval-based merging" + ); private static final ConfigDef.Importance MERGE_INTERVAL_MS_IMPORTANCE = ConfigDef.Importance.LOW; private static final String MERGE_INTERVAL_MS_DOC = "How often (in milliseconds) to perform a merge flush, if upsert/delete is enabled. Can be " @@ -287,7 +307,21 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String MERGE_RECORDS_THRESHOLD_CONFIG = "mergeRecordsThreshold"; private static final ConfigDef.Type MERGE_RECORDS_THRESHOLD_TYPE = ConfigDef.Type.LONG; public static final long MERGE_RECORDS_THRESHOLD_DEFAULT = -1; - private static final ConfigDef.Validator MERGE_RECORDS_THRESHOLD_VALIDATOR = ConfigDef.Range.atLeast(-1); + private static final ConfigDef.Validator MERGE_RECORDS_THRESHOLD_VALIDATOR = ConfigDef.LambdaValidator.with( + (name, value) -> { + if (value == null) { + return; + } + long parsedValue = (long) ConfigDef.parseType(name, value, MERGE_RECORDS_THRESHOLD_TYPE); + + if (parsedValue == 0) { + throw new ConfigException(name, value, "Cannot be zero"); + } else if (parsedValue < -1) { + throw new ConfigException(name, value, "Cannot be less than -1"); + } + }, + () -> "Either a positive integer or -1 to disable throughput-based merging" + ); private static final ConfigDef.Importance MERGE_RECORDS_THRESHOLD_IMPORTANCE = ConfigDef.Importance.LOW; private static final String MERGE_RECORDS_THRESHOLD_DOC = "How many records to write to an intermediate table before performing a merge flush, if " @@ -363,6 +397,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG = "timestampPartitionFieldName"; private static final ConfigDef.Type BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE = ConfigDef.Type.STRING; private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT = null; + private static final ConfigDef.Validator BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_VALIDATOR = new ConfigDef.NonEmptyString(); private static final ConfigDef.Importance BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE = ConfigDef.Importance.LOW; private static final String BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC = @@ -373,6 +408,17 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG = "clusteringPartitionFieldNames"; private static final ConfigDef.Type BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE = ConfigDef.Type.LIST; private static final List BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT = null; + private static final ConfigDef.Validator BIGQUERY_CLUSTERING_FIELD_NAMES_VALIDATOR = (name, value) -> { + if (value == null) { + return; + } + + @SuppressWarnings("unchecked") + List parsedValue = (List) value; + if (parsedValue.size() > 4) { + throw new ConfigException(name, value, "You may only specify up to four clustering field names."); + } + }; private static final ConfigDef.Importance BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE = ConfigDef.Importance.LOW; private static final String BIGQUERY_CLUSTERING_FIELD_NAMES_DOC = @@ -486,12 +532,14 @@ public static ConfigDef getConfig() { KAFKA_KEY_FIELD_NAME_CONFIG, KAFKA_KEY_FIELD_NAME_TYPE, KAFKA_KEY_FIELD_NAME_DEFAULT, + KAFKA_KEY_FIELD_NAME_VALIDATOR, KAFKA_KEY_FIELD_NAME_IMPORTANCE, KAFKA_KEY_FIELD_NAME_DOC ).define( KAFKA_DATA_FIELD_NAME_CONFIG, KAFKA_DATA_FIELD_NAME_TYPE, KAFKA_DATA_FIELD_NAME_DEFAULT, + KAFKA_DATA_FIELD_NAME_VALIDATOR, KAFKA_DATA_FIELD_NAME_IMPORTANCE, KAFKA_DATA_FIELD_NAME_DOC ).define( @@ -620,12 +668,14 @@ public static ConfigDef getConfig() { BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_VALIDATOR, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE, BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC ).define( BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE, BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT, + BIGQUERY_CLUSTERING_FIELD_NAMES_VALIDATOR, BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE, BIGQUERY_CLUSTERING_FIELD_NAMES_DOC ).define( @@ -651,118 +701,69 @@ public boolean visible(String s, Map map) { return true; } } - ).define( - THREAD_POOL_SIZE_CONFIG, - THREAD_POOL_SIZE_TYPE, - THREAD_POOL_SIZE_DEFAULT, - THREAD_POOL_SIZE_VALIDATOR, - THREAD_POOL_SIZE_IMPORTANCE, - THREAD_POOL_SIZE_DOC - ).define( - QUEUE_SIZE_CONFIG, - QUEUE_SIZE_TYPE, - QUEUE_SIZE_DEFAULT, - QUEUE_SIZE_VALIDATOR, - QUEUE_SIZE_IMPORTANCE, - QUEUE_SIZE_DOC - ).define( - BIGQUERY_RETRY_CONFIG, - BIGQUERY_RETRY_TYPE, - BIGQUERY_RETRY_DEFAULT, - BIGQUERY_RETRY_VALIDATOR, - BIGQUERY_RETRY_IMPORTANCE, - BIGQUERY_RETRY_DOC - ).define( - BIGQUERY_RETRY_WAIT_CONFIG, - BIGQUERY_RETRY_WAIT_CONFIG_TYPE, - BIGQUERY_RETRY_WAIT_DEFAULT, - BIGQUERY_RETRY_WAIT_VALIDATOR, - BIGQUERY_RETRY_WAIT_IMPORTANCE, - BIGQUERY_RETRY_WAIT_DOC - ).define( - BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG, - BIGQUERY_MESSAGE_TIME_PARTITIONING_CONFIG_TYPE, - BIGQUERY_MESSAGE_TIME_PARTITIONING_DEFAULT, - BIGQUERY_MESSAGE_TIME_PARTITIONING_IMPORTANCE, - BIGQUERY_MESSAGE_TIME_PARTITIONING_DOC - ).define( - BIGQUERY_PARTITION_DECORATOR_CONFIG, - BIGQUERY_PARTITION_DECORATOR_CONFIG_TYPE, - BIGQUERY_PARTITION_DECORATOR_DEFAULT, - BIGQUERY_PARTITION_DECORATOR_IMPORTANCE, - BIGQUERY_PARTITION_DECORATOR_DOC - ).define( - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_TYPE, - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DEFAULT, - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_IMPORTANCE, - BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_DOC - ).define( - BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, - BIGQUERY_CLUSTERING_FIELD_NAMES_TYPE, - BIGQUERY_CLUSTERING_FIELD_NAMES_DEFAULT, - BIGQUERY_CLUSTERING_FIELD_NAMES_IMPORTANCE, - BIGQUERY_CLUSTERING_FIELD_NAMES_DOC ); } + private static final List> MULTI_PROPERTY_VALIDATIONS = new ArrayList<>(); + + static { + // Note that order matters here: validations are performed in the order they're added to this list, and if a + // property or any of the properties that it depends on has an error, validation for it gets skipped. + // This comes in handy for things like checking for the existence of tables, which requires valid BigQuery + // credentials. We validate those credentials before checking for tables so that we can safely assume while + // checking for those tables that the credentials are already valid. + MULTI_PROPERTY_VALIDATIONS.add(new CredentialsValidator.BigQueryCredentialsValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new CredentialsValidator.GcsCredentialsValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new GcsBucketValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new PartitioningModeValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new PartitioningTypeValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new UpsertDeleteValidator.UpsertValidator()); + MULTI_PROPERTY_VALIDATIONS.add(new UpsertDeleteValidator.DeleteValidator()); + } + /** - * Throw an exception if the passed-in properties do not constitute a valid sink. - * @param props sink configuration properties + * Used in conjunction with {@link com.wepay.kafka.connect.bigquery.BigQuerySinkConnector#validate(Map)} to perform + * preflight configuration checks. Simple validations that only require a single property value at a time (such as + * ensuring that boolean properties only contain true/false values, or that values for required properties are + * provided) are handled automatically by the {@link #getConfig() ConfigDef} for this class and optionally-defined + * custom {@link ConfigDef.Validator validators}. Other, more sophisticated validations that require multiple + * property values at a time (such as checking if all of the tables the connector will write to already exist if + * automatic table creation is disabled) are performed manually in a subsequent step. + * + * @return a {@link Config} object containing all errors that the connector was able to detect during preflight + * validation of this configuration; never null */ - public static void validate(Map props) { - final boolean hasTopicsConfig = hasTopicsConfig(props); - final boolean hasTopicsRegexConfig = hasTopicsRegexConfig(props); - - if (hasTopicsConfig && hasTopicsRegexConfig) { - throw new ConfigException(TOPICS_CONFIG + " and " + TOPICS_REGEX_CONFIG + - " are mutually exclusive options, but both are set."); - } - - if (!hasTopicsConfig && !hasTopicsRegexConfig) { - throw new ConfigException("Must configure one of " + - TOPICS_CONFIG + " or " + TOPICS_REGEX_CONFIG); - } - - if (upsertDeleteEnabled(props)) { - if (gcsBatchLoadingEnabled(props)) { - throw new ConfigException("Cannot enable both upsert/delete and GCS batch loading"); - } - - String mergeIntervalStr = Optional.ofNullable(props.get(MERGE_INTERVAL_MS_CONFIG)) - .map(String::trim) - .orElse(Long.toString(MERGE_INTERVAL_MS_DEFAULT)); - String mergeRecordsThresholdStr = Optional.ofNullable(props.get(MERGE_RECORDS_THRESHOLD_CONFIG)) - .map(String::trim) - .orElse(Long.toString(MERGE_RECORDS_THRESHOLD_DEFAULT)); - if ("-1".equals(mergeIntervalStr) && "-1".equals(mergeRecordsThresholdStr)) { - throw new ConfigException(MERGE_INTERVAL_MS_CONFIG + " and " - + MERGE_RECORDS_THRESHOLD_CONFIG + " cannot both be -1"); - } - - if ("0".equals(mergeIntervalStr)) { - throw new ConfigException(MERGE_INTERVAL_MS_CONFIG, mergeIntervalStr, "cannot be zero"); - } - if ("0".equals(mergeRecordsThresholdStr)) { - throw new ConfigException(MERGE_RECORDS_THRESHOLD_CONFIG, mergeRecordsThresholdStr, "cannot be zero"); - } - - String kafkaKeyFieldStr = props.get(KAFKA_KEY_FIELD_NAME_CONFIG); - if (kafkaKeyFieldStr == null || kafkaKeyFieldStr.trim().isEmpty()) { - throw new ConfigException(KAFKA_KEY_FIELD_NAME_CONFIG + " must be specified when " - + UPSERT_ENABLED_CONFIG + " and/or " + DELETE_ENABLED_CONFIG + " are set to true"); - } - } + public Config validate() { + List initialValidation = getConfig().validate(originalsStrings()); + Map valuesByName = initialValidation + .stream() + .collect(Collectors.toMap(ConfigValue::name, Function.identity())); + MULTI_PROPERTY_VALIDATIONS.forEach(validator -> { + ConfigValue value = valuesByName.get(validator.propertyName()); + validator.validate(value, this, valuesByName).ifPresent(value::addErrorMessage); + }); + return new Config(initialValidation); } - public static boolean hasTopicsConfig(Map props) { - String topicsStr = props.get(TOPICS_CONFIG); - return topicsStr != null && !topicsStr.trim().isEmpty(); + /** + * @return the key, which is (depending on the key source property) either a path to a file or a raw JSON string + */ + public String getKey() { + return Optional.ofNullable(getPassword(KEYFILE_CONFIG)).map(Password::value).orElse(null); } - public static boolean hasTopicsRegexConfig(Map props) { - String topicsRegexStr = props.get(TOPICS_REGEX_CONFIG); - return topicsRegexStr != null && !topicsRegexStr.trim().isEmpty(); + /** + * @return the {@link com.wepay.kafka.connect.bigquery.GcpClientBuilder.KeySource key source type} that dictates how + * the {@link #getKey()} should be be interpreted + */ + public GcpClientBuilder.KeySource getKeySource() { + String rawKeySource = getString(KEY_SOURCE_CONFIG); + try { + return GcpClientBuilder.KeySource.valueOf(rawKeySource); + } catch (IllegalArgumentException e) { + // Should never happen with preflight validation of the key source property + throw new ConnectException("Invalid key source type: " + rawKeySource); + } } public static boolean upsertDeleteEnabled(Map props) { @@ -828,7 +829,7 @@ public SchemaRetriever getSchemaRetriever() { Class schemaRetrieverClass = userSpecifiedClass.asSubclass(SchemaRetriever.class); - Constructor schemaRetrieverConstructor = null; + Constructor schemaRetrieverConstructor; try { schemaRetrieverConstructor = schemaRetrieverClass.getConstructor(); } catch (NoSuchMethodException nsme) { @@ -838,7 +839,7 @@ public SchemaRetriever getSchemaRetriever() { ); } - SchemaRetriever schemaRetriever = null; + SchemaRetriever schemaRetriever; try { schemaRetriever = schemaRetrieverConstructor.newInstance(); } catch (InstantiationException @@ -857,7 +858,6 @@ public SchemaRetriever getSchemaRetriever() { } /** - * * If the connector is configured to load Kafka data into BigQuery, this config defines * the name of the kafka data field. A structure is created under the field name to contain * kafka data schema including topic, offset, partition and insertTime. @@ -869,7 +869,6 @@ public Optional getKafkaKeyFieldName() { } /** - * * If the connector is configured to load Kafka keys into BigQuery, this config defines * the name of the kafka key field. A structure is created under the field name to contain * a topic's Kafka key schema. @@ -905,55 +904,6 @@ private TimePartitioning.Type parseTimePartitioningType(String rawPartitioningTy } } - /** - * Verifies that a bucket is specified if GCS batch loading is enabled. - * @throws ConfigException Exception thrown if no bucket is specified and batch loading is on. - */ - private void verifyBucketSpecified() throws ConfigException { - // Throw an exception if GCS Batch loading will be used but no bucket is specified - if (getString(GCS_BUCKET_NAME_CONFIG).equals("") - && !getList(ENABLE_BATCH_CONFIG).isEmpty()) { - throw new ConfigException("Batch loading enabled for some topics, but no bucket specified"); - } - } - - private void checkAutoCreateTables() { - Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); - boolean autoCreateTables = getBoolean(TABLE_CREATE_CONFIG); - - if (autoCreateTables && schemaRetriever == null) { - throw new ConfigException( - "Cannot specify automatic table creation without a schema retriever" - ); - } - } - - private void checkBigQuerySchemaUpdateConfigs() { - boolean allBQFieldsNullable = getBoolean(ALL_BQ_FIELDS_NULLABLE_CONFIG); - boolean allowBQRequiredFieldRelaxation = getBoolean(ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); - if (allBQFieldsNullable && !allowBQRequiredFieldRelaxation) { - throw new ConfigException( - "Conflicting Configs, allBQFieldsNullable can be true only if allowBigQueryFieldRelaxation is true" - ); - } - - Class schemaRetriever = getClass(BigQuerySinkConfig.SCHEMA_RETRIEVER_CONFIG); - - boolean allowNewBigQueryFields = getBoolean(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); - boolean allowRequiredFieldRelaxation = getBoolean(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); - if ((allowNewBigQueryFields || allowRequiredFieldRelaxation) && schemaRetriever == null) { - throw new ConfigException( - "Cannot perform schema updates without a schema retriever" - ); - } - - if (schemaRetriever == null) { - logger.warn( - "No schema retriever class provided; auto schema updates are impossible" - ); - } - } - /** * Returns the field name to use for timestamp partitioning. * @return String that represents the field name. @@ -962,73 +912,19 @@ public Optional getTimestampPartitionFieldName() { return Optional.ofNullable(getString(BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG)); } - private void checkTimePartitioningConfigs() { - boolean decoratorSyntax = getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG); - boolean createTables = getBoolean(TABLE_CREATE_CONFIG); - - if (!decoratorSyntax || !createTables) { - // The time partitioning type only matters when we're configured to automatically create tables and will write to them with decorator syntax - return; - } - - String rawTimePartitioningType = getString(TIME_PARTITIONING_TYPE_CONFIG); - if (!TimePartitioning.Type.DAY.equals(parseTimePartitioningType(rawTimePartitioningType))) { - throw new ConfigException( - TIME_PARTITIONING_TYPE_CONFIG, - rawTimePartitioningType, - "Tables must be partitioned by DAY when using partition decorator syntax. " - + "Either configure the connector with the DAY time partitioning type, " - + "disable automatic table creation, or disable partition decorator syntax." - ); - } - } - /** * Returns the field names to use for clustering. * @return List of Strings that represent the field names. */ - public Optional> getClusteringPartitionFieldName() { - return Optional.ofNullable(getList(BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG)); - } - - /** - * Check the validity of table partitioning configs. - */ - private void checkPartitionConfigs() { - if (getTimestampPartitionFieldName().isPresent() && getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { - throw new ConfigException( - "Only one partitioning configuration mode may be specified for the connector. " - + "Use either bigQueryPartitionDecorator OR timestampPartitionFieldName." - ); - } - } - - /** - * Check the validity of table clustering configs. - */ - private void checkClusteringConfigs() { - if (getClusteringPartitionFieldName().isPresent()) { - if (!getTimestampPartitionFieldName().isPresent() && !getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { - throw new ConfigException( - "Clustering field name may be specified only on a partitioned table." - ); - } - if (getClusteringPartitionFieldName().get().size() > 4) { - throw new ConfigException( - "You can only specify up to four clustering field names." - ); - } - } + public Optional> getClusteringPartitionFieldNames() { + return Optional + .ofNullable(getList(BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG)) + // With Java 11 there's Predicate::not, but for now we have to just manually invert the isEmpty check + .filter(l -> !l.isEmpty()); } protected BigQuerySinkConfig(ConfigDef config, Map properties) { super(config, properties); - verifyBucketSpecified(); - checkAutoCreateTables(); - checkBigQuerySchemaUpdateConfigs(); - checkPartitionConfigs(); - checkClusteringConfigs(); - checkTimePartitioningConfigs(); } public BigQuerySinkConfig(Map properties) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java new file mode 100644 index 000000000..76007d11a --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java @@ -0,0 +1,117 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.storage.Storage; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEYFILE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEY_SOURCE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; + +public abstract class CredentialsValidator> extends MultiPropertyValidator { + + public CredentialsValidator() { + super(KEYFILE_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + PROJECT_CONFIG, KEY_SOURCE_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + String keyFile = config.getKey(); + if (keyFile == null || keyFile.isEmpty()) { + // No credentials to validate + return Optional.empty(); + } + + try { + clientBuilder() + .withConfig(config) + .build(); + return Optional.empty(); + } catch (RuntimeException e) { + String errorMessage = "An unexpected error occurred while validating credentials for " + gcpService(); + if (e.getMessage() != null) { + errorMessage += ": " + e.getMessage(); + } + return Optional.of(errorMessage); + } + } + + protected abstract String gcpService(); + protected abstract ClientBuilder clientBuilder(); + + public static class BigQueryCredentialsValidator extends CredentialsValidator> { + @Override + public String gcpService() { + return "BigQuery"; + } + + @Override + protected GcpClientBuilder clientBuilder() { + return new GcpClientBuilder.BigQueryBuilder(); + } + } + + public static class GcsCredentialsValidator extends CredentialsValidator> { + + private static final Collection DEPENDENTS; + + static { + List dependents = new ArrayList<>(CredentialsValidator.DEPENDENTS); + dependents.add(ENABLE_BATCH_CONFIG); + dependents.add(GCS_BUCKET_NAME_CONFIG); + DEPENDENTS = Collections.unmodifiableCollection(dependents); + } + + @Override + public Collection dependents() { + return DEPENDENTS; + } + + @Override + public String gcpService() { + return "GCS"; + } + + @Override + protected GcpClientBuilder clientBuilder() { + return new GcpClientBuilder.GcsBuilder(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java new file mode 100644 index 000000000..59d2dbd44 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidator.java @@ -0,0 +1,97 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.storage.Bucket; +import com.google.cloud.storage.Storage; +import com.google.common.annotations.VisibleForTesting; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.AUTO_CREATE_BUCKET_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; + +public class GcsBucketValidator extends MultiPropertyValidator { + + public GcsBucketValidator() { + super(GCS_BUCKET_NAME_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + ENABLE_BATCH_CONFIG, AUTO_CREATE_BUCKET_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + Storage gcs; + try { + gcs = new GcpClientBuilder.GcsBuilder() + .withConfig(config) + .build(); + } catch (RuntimeException e) { + return Optional.of(String.format( + "Failed to construct GCS client%s", + e.getMessage() != null ? ": " + e.getMessage() : "" + )); + } + return doValidate(gcs, config); + } + + @VisibleForTesting + Optional doValidate(Storage gcs, BigQuerySinkConfig config) { + List batchLoadedTopics = config.getList(ENABLE_BATCH_CONFIG); + if (batchLoadedTopics == null || batchLoadedTopics.isEmpty()) { + // Batch loading is disabled; no need to validate the GCS bucket + return Optional.empty(); + } + + String bucketName = config.getString(GCS_BUCKET_NAME_CONFIG); + if (bucketName == null || bucketName.trim().isEmpty()) { + return Optional.of("When GCS batch loading is enabled, a bucket must be provided"); + } + + if (config.getBoolean(AUTO_CREATE_BUCKET_CONFIG)) { + return Optional.empty(); + } + + Bucket bucket = gcs.get(bucketName); + if (bucket == null) { + return Optional.of(String.format( + "Automatic bucket creation is disabled but the GCS bucket %s does not exist. " + + "Please either manually create this table before restarting the connector or enable automatic bucket creation " + + "by the connector", + bucketName + )); + } + + return Optional.empty(); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java new file mode 100644 index 000000000..95b9c2da6 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidator.java @@ -0,0 +1,70 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import org.apache.kafka.common.config.ConfigValue; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +public abstract class MultiPropertyValidator { + + private final String propertyName; + + protected MultiPropertyValidator(String propertyName) { + this.propertyName = propertyName; + } + + public String propertyName() { + return propertyName; + } + + public Optional validate(ConfigValue value, Config config, Map valuesByName) { + // Only perform follow-up validation if the property doesn't already have an error associated with it + if (!value.errorMessages().isEmpty()) { + return Optional.empty(); + } + + boolean dependentsAreValid = dependents().stream() + .map(valuesByName::get) + .filter(Objects::nonNull) + .map(ConfigValue::errorMessages) + .allMatch(List::isEmpty); + // Also ensure that all of the other properties that the validation for this one depends on don't already have errors + if (!dependentsAreValid) { + return Optional.empty(); + } + + try { + return doValidate(config); + } catch (RuntimeException e) { + return Optional.of( + "An unexpected error occurred during validation" + + (e.getMessage() != null ? ": " + e.getMessage() : "") + ); + } + } + + protected abstract Collection dependents(); + protected abstract Optional doValidate(Config config); +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java new file mode 100644 index 000000000..65389e5fd --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidator.java @@ -0,0 +1,60 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG; + +public class PartitioningModeValidator extends MultiPropertyValidator { + public PartitioningModeValidator() { + super(BIGQUERY_PARTITION_DECORATOR_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + if (!config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)) { + return Optional.empty(); + } + + if (config.getTimestampPartitionFieldName().isPresent()) { + return Optional.of(String.format("Only one partitioning mode may be specified for the connector. " + + "Use either %s OR %s.", + BIGQUERY_PARTITION_DECORATOR_CONFIG, + BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG + )); + } else { + return Optional.empty(); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningTypeValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningTypeValidator.java new file mode 100644 index 000000000..b51548b05 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/PartitioningTypeValidator.java @@ -0,0 +1,65 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.bigquery.TimePartitioning; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TABLE_CREATE_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG; + +public class PartitioningTypeValidator extends MultiPropertyValidator { + public PartitioningTypeValidator() { + super(TIME_PARTITIONING_TYPE_CONFIG); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + BIGQUERY_PARTITION_DECORATOR_CONFIG, TABLE_CREATE_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + if (!config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG) || !config.getBoolean(TABLE_CREATE_CONFIG)) { + return Optional.empty(); + } + + TimePartitioning.Type timePartitioningType = config.getTimePartitioningType(); + + if (!TimePartitioning.Type.DAY.equals(timePartitioningType)) { + return Optional.of( + "Tables must be partitioned by DAY when using partition decorator syntax. " + + "Either configure the connector with the DAY time partitioning type, " + + "disable automatic table creation, or disable partition decorator syntax." + ); + } + + return Optional.empty(); + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/UpsertDeleteValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/UpsertDeleteValidator.java new file mode 100644 index 000000000..a20178258 --- /dev/null +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/UpsertDeleteValidator.java @@ -0,0 +1,102 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.DELETE_ENABLED_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KAFKA_KEY_FIELD_NAME_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.MERGE_INTERVAL_MS_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.MERGE_RECORDS_THRESHOLD_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.UPSERT_ENABLED_CONFIG; + +public abstract class UpsertDeleteValidator extends MultiPropertyValidator { + private UpsertDeleteValidator(String propertyName) { + super(propertyName); + } + + private static final Collection DEPENDENTS = Collections.unmodifiableCollection(Arrays.asList( + MERGE_INTERVAL_MS_CONFIG, MERGE_RECORDS_THRESHOLD_CONFIG, KAFKA_KEY_FIELD_NAME_CONFIG + )); + + @Override + protected Collection dependents() { + return DEPENDENTS; + } + + @Override + protected Optional doValidate(BigQuerySinkConfig config) { + if (!modeEnabled(config)) { + return Optional.empty(); + } + + long mergeInterval = config.getLong(MERGE_INTERVAL_MS_CONFIG); + long mergeRecordsThreshold = config.getLong(MERGE_RECORDS_THRESHOLD_CONFIG); + + if (mergeInterval == -1 && mergeRecordsThreshold == -1) { + return Optional.of(String.format( + "%s and %s cannot both be -1", + MERGE_INTERVAL_MS_CONFIG, + MERGE_RECORDS_THRESHOLD_CONFIG + )); + } + + if (!config.getKafkaKeyFieldName().isPresent()) { + return Optional.of(String.format( + "%s must be specified when %s is set to true", + KAFKA_KEY_FIELD_NAME_CONFIG, + propertyName() + )); + } + + return Optional.empty(); + } + + /** + * @param config the user-provided configuration + * @return whether the write mode for the validator (i.e., either upsert or delete) is enabled + */ + protected abstract boolean modeEnabled(BigQuerySinkConfig config); + + public static class UpsertValidator extends UpsertDeleteValidator { + public UpsertValidator() { + super(UPSERT_ENABLED_CONFIG); + } + + @Override + protected boolean modeEnabled(BigQuerySinkConfig config) { + return config.getBoolean(UPSERT_ENABLED_CONFIG); + } + } + + public static class DeleteValidator extends UpsertDeleteValidator { + public DeleteValidator() { + super(DELETE_ENABLED_CONFIG); + } + + @Override + protected boolean modeEnabled(BigQuerySinkConfig config) { + return config.getBoolean(DELETE_ENABLED_CONFIG); + } + } +} diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java deleted file mode 100644 index 805cd5643..000000000 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/SinkConfigConnectException.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Copyright 2020 Confluent, Inc. - * - * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - * - * 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.wepay.kafka.connect.bigquery.exception; - -import org.apache.kafka.connect.errors.ConnectException; - -/** - * Class for exceptions that occur while attempting to process configuration files, including both - * formatting and logical errors. - */ -public class SinkConfigConnectException extends ConnectException { - public SinkConfigConnectException(String msg) { - super(msg); - } - - public SinkConfigConnectException(String msg, Throwable thr) { - super(msg, thr); - } - - public SinkConfigConnectException(Throwable thr) { - super(thr); - } -} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java index f90c24b87..c1757bd95 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java @@ -19,20 +19,9 @@ package com.wepay.kafka.connect.bigquery; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; - import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; - -import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; - import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; -import org.apache.kafka.common.config.ConfigException; - import org.apache.kafka.connect.data.Schema; - import org.apache.kafka.connect.sink.SinkRecord; import org.junit.BeforeClass; import org.junit.Test; @@ -41,6 +30,10 @@ import java.util.List; import java.util.Map; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; + public class BigQuerySinkConnectorTest { private static SinkPropertiesFactory propertiesFactory; @@ -114,19 +107,6 @@ public void testConfig() { assertNotNull(new BigQuerySinkConnector().config()); } - // Make sure that a config exception is properly translated into a SinkConfigConnectException - @Test(expected = SinkConfigConnectException.class) - public void testConfigException() { - try { - Map badProperties = propertiesFactory.getProperties(); - badProperties.remove(BigQuerySinkConfig.TOPICS_CONFIG); - BigQuerySinkConfig.validate(badProperties); - new BigQuerySinkConnector().start(badProperties); - } catch (ConfigException e) { - throw new SinkConfigConnectException(e); - } - } - @Test public void testVersion() { assertNotNull(new BigQuerySinkConnector().version()); diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 9f69d8542..f7a91fb38 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -47,7 +47,6 @@ import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; -import com.wepay.kafka.connect.bigquery.exception.SinkConfigConnectException; import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; import org.apache.kafka.common.config.ConfigException; @@ -667,26 +666,6 @@ public void testInterruptedException() { testTask.flush(Collections.emptyMap()); } - // Make sure that a ConfigException is properly translated into a SinkConfigConnectException - @Test(expected = SinkConfigConnectException.class) - public void testConfigException() { - try { - Map badProperties = propertiesFactory.getProperties(); - badProperties.remove(BigQuerySinkConfig.TOPICS_CONFIG); - BigQuerySinkConfig.validate(badProperties); - - SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); - SchemaManager schemaManager = mock(SchemaManager.class); - Map cache = new HashMap<>(); - - BigQuerySinkTask testTask = - new BigQuerySinkTask(mock(BigQuery.class), schemaRetriever, mock(Storage.class), schemaManager, cache); - testTask.start(badProperties); - } catch (ConfigException e) { - throw new SinkConfigConnectException(e); - } - } - @Test(expected = ConnectException.class) public void testTimePartitioningIncompatibleWithDecoratorSyntax() { final String topic = "t1"; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java index da5359419..fde3567c7 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SinkPropertiesFactory.java @@ -49,24 +49,4 @@ public Map getProperties() { return properties; } - - /** - * Make sure that each of the default configuration properties work nicely with the given - * configuration object. - * - * @param config The config object to test - */ - public void testProperties(BigQuerySinkConfig config) { - - config.getList(BigQuerySinkConfig.TOPICS_CONFIG); - config.getString(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG); - - config.getString(BigQuerySinkConfig.PROJECT_CONFIG); - config.getKeyFile(); - config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG); - config.getInt(BigQuerySinkConfig.AVRO_DATA_CACHE_SIZE_CONFIG); - - config.getBoolean(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); - config.getBoolean(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); - } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index 3bc905618..9660ad0be 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -51,7 +51,11 @@ public void initializePropertiesFactory() { public void metaTestBasicConfigProperties() { Map basicConfigProperties = propertiesFactory.getProperties(); BigQuerySinkConfig config = new BigQuerySinkConfig(basicConfigProperties); - propertiesFactory.testProperties(config); + config.getList(BigQuerySinkConfig.TOPICS_CONFIG); + config.getString(BigQuerySinkConfig.PROJECT_CONFIG); + config.getKey(); + config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG); + config.getInt(BigQuerySinkConfig.AVRO_DATA_CACHE_SIZE_CONFIG); } @Test @@ -96,16 +100,6 @@ public void testEmptyTimestampPartitionFieldName() { assertFalse(testConfig.getTimestampPartitionFieldName().isPresent()); } - /** - * Test if the field name being non-empty and the decorator default (true) errors correctly. - */ - @Test (expected = ConfigException.class) - public void testTimestampPartitionFieldNameError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, "name"); - new BigQuerySinkConfig(configProperties); - } - /** * Test the field name being non-empty and the decorator set to false works correctly. */ @@ -126,22 +120,7 @@ public void testTimestampPartitionFieldName() { public void testEmptyClusteringFieldNames() { Map configProperties = propertiesFactory.getProperties(); BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); - assertFalse(testConfig.getClusteringPartitionFieldName().isPresent()); - } - - /** - * Test if the field names being non-empty and the partitioning is not present errors correctly. - */ - @Test (expected = ConfigException.class) - public void testClusteringFieldNamesWithoutTimestampPartitionError() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.BIGQUERY_TIMESTAMP_PARTITION_FIELD_NAME_CONFIG, null); - configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); - configProperties.put( - BigQuerySinkConfig.BIGQUERY_CLUSTERING_FIELD_NAMES_CONFIG, - "column1,column2" - ); - new BigQuerySinkConfig(configProperties); + assertFalse(testConfig.getClusteringPartitionFieldNames().isPresent()); } /** @@ -176,7 +155,7 @@ public void testClusteringFieldNames() { ); BigQuerySinkConfig testConfig = new BigQuerySinkConfig(configProperties); - Optional> testClusteringPartitionFieldName = testConfig.getClusteringPartitionFieldName(); + Optional> testClusteringPartitionFieldName = testConfig.getClusteringPartitionFieldNames(); assertTrue(testClusteringPartitionFieldName.isPresent()); assertEquals(expectedClusteringPartitionFieldName, testClusteringPartitionFieldName.get()); } @@ -184,32 +163,11 @@ public void testClusteringFieldNames() { @Test public void testValidTimePartitioningTypes() { Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "false"); for (TimePartitioning.Type type : TimePartitioning.Type.values()) { configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, type.name()); assertEquals(type, new BigQuerySinkConfig(configProperties).getTimePartitioningType()); } - - configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); - configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, TimePartitioning.Type.DAY.name()); - assertEquals(TimePartitioning.Type.DAY, new BigQuerySinkConfig(configProperties).getTimePartitioningType()); - } - - @Test - public void testInvalidTimePartitioningTypes() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG, "true"); - configProperties.put(BigQuerySinkConfig.TABLE_CREATE_CONFIG, "true"); - - for (TimePartitioning.Type type : TimePartitioning.Type.values()) { - if (TimePartitioning.Type.DAY.equals(type)) { - continue; - } - - configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, type.name()); - assertThrows(ConfigException.class, () -> new BigQuerySinkConfig(configProperties)); - } } @Test(expected = ConfigException.class) diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java new file mode 100644 index 000000000..7a55d5ad8 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java @@ -0,0 +1,69 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; +import org.junit.Test; + +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CredentialsValidatorTest { + + @Test + public void testNoCredentialsSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getKey()).thenReturn(null); + + assertEquals( + Optional.empty(), + new CredentialsValidator.BigQueryCredentialsValidator().doValidate(config) + ); + assertEquals( + Optional.empty(), + new CredentialsValidator.GcsCredentialsValidator().doValidate(config) + ); + } + + @Test + public void testFailureToConstructClient() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getKey()).thenReturn("key"); + + @SuppressWarnings("unchecked") + GcpClientBuilder mockClientBuilder = mock(GcpClientBuilder.class); + when(mockClientBuilder.withConfig(eq(config))).thenReturn(mockClientBuilder); + when(mockClientBuilder.build()).thenThrow(new RuntimeException("Provided credentials are invalid")); + + assertNotEquals( + Optional.empty(), + new CredentialsValidator.BigQueryCredentialsValidator().doValidate(config) + ); + assertNotEquals( + Optional.empty(), + new CredentialsValidator.GcsCredentialsValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java new file mode 100644 index 000000000..b8c80fee4 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/GcsBucketValidatorTest.java @@ -0,0 +1,122 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.storage.Bucket; +import com.google.cloud.storage.Storage; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Collections; +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.ENABLE_BATCH_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.GCS_BUCKET_NAME_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class GcsBucketValidatorTest { + + @Mock + private Storage gcs; + + @Test + public void testNullBatchLoadingSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(null); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testEmptyBatchLoadingSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.emptyList()); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testNullBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(null); + + assertNotEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testBlankBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(" \t "); + + assertNotEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testValidBucketWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + final String bucketName = "gee_cs"; + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(bucketName); + + Bucket bucket = mock(Bucket.class); + when(gcs.get(eq(bucketName))).thenReturn(bucket); + + assertEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } + + @Test + public void testMissingBucketAndBucketCreationDisabledWithBatchLoading() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + final String bucketName = "gee_cs"; + when(config.getList(ENABLE_BATCH_CONFIG)).thenReturn(Collections.singletonList("t1")); + when(config.getString(GCS_BUCKET_NAME_CONFIG)).thenReturn(bucketName); + + when(gcs.get(eq(bucketName))).thenReturn(null); + + assertNotEquals( + Optional.empty(), + new GcsBucketValidator().doValidate(gcs, config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java new file mode 100644 index 000000000..205bb56a3 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/MultiPropertyValidatorTest.java @@ -0,0 +1,138 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.config.ConfigValue; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; + +public class MultiPropertyValidatorTest { + + private static class TestValidator extends MultiPropertyValidator { + + private final List dependents; + private final Function> validationFunction; + + public TestValidator(String propertyName, List dependents, Function> validationFunction) { + super(propertyName); + this.dependents = dependents; + this.validationFunction = validationFunction; + } + + @Override + protected Collection dependents() { + return dependents; + } + + @Override + protected Optional doValidate(Config config) { + return validationFunction.apply(config); + } + } + + @Test + public void testExistingErrorSkipsValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Arrays.asList("d1", "d2", "d3"), + o -> { + fail("Validation should have been performed on property that already has an error"); + return null; + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.singletonList("an error")); + + assertEquals( + Optional.empty(), + validator.validate(configValue, null, Collections.emptyMap()) + ); + } + + @Test + public void testDependentErrorSkipsValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Arrays.asList("d1", "d2", "d3"), + o -> { + fail("Validation should have been performed on property whose dependent already has an error"); + return null; + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + Map valuesByName = ImmutableMap.of( + "d1", new ConfigValue("d1", "v1", Collections.emptyList(), Collections.emptyList()), + "d2", new ConfigValue("d2", "v1", Collections.emptyList(), Collections.singletonList("an error")) + ); + + assertEquals( + Optional.empty(), + validator.validate(configValue, null, valuesByName) + ); + } + + @Test + public void testValidationFails() { + Optional expectedError = Optional.of("an error"); + MultiPropertyValidator validator = new TestValidator<>( + "p", + Collections.emptyList(), + o -> expectedError + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + + assertEquals( + expectedError, + validator.validate(configValue, null, Collections.emptyMap()) + ); + } + + @Test + public void testUnexpectedErrorDuringValidation() { + MultiPropertyValidator validator = new TestValidator<>( + "p", + Collections.emptyList(), + o -> { + throw new RuntimeException("Some unexpected error"); + } + ); + + ConfigValue configValue = new ConfigValue("p", "v", Collections.emptyList(), Collections.emptyList()); + + assertNotEquals( + Optional.empty(), + validator.validate(configValue, null, Collections.emptyMap()) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java new file mode 100644 index 000000000..a4b79a14c --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningModeValidatorTest.java @@ -0,0 +1,80 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import org.junit.Test; + +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PartitioningModeValidatorTest { + + @Test + public void testDisabledDecoratorSyntaxSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(false); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testDecoratorSyntaxWithoutTimestampPartitionFieldName() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.empty()); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testDecoratorSyntaxWithTimestampPartitionFieldName() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.of("f1")); + + assertNotEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } + + @Test + public void testTimestampPartitionFieldNameWithoutDecoratorSyntax() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(false); + when(config.getTimestampPartitionFieldName()).thenReturn(Optional.of("f1")); + + assertEquals( + Optional.empty(), + new PartitioningModeValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningTypeValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningTypeValidatorTest.java new file mode 100644 index 000000000..57c006ddb --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/PartitioningTypeValidatorTest.java @@ -0,0 +1,92 @@ +/* + * Copyright 2020 Confluent, Inc. + * + * This software contains code derived from the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + * + * 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.wepay.kafka.connect.bigquery.config; + +import com.google.cloud.bigquery.TimePartitioning; +import org.junit.Test; + +import java.util.Optional; + +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.BIGQUERY_PARTITION_DECORATOR_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TABLE_CREATE_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PartitioningTypeValidatorTest { + + @Test + public void testDisabledDecoratorSyntaxSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(false); + when(config.getBoolean(TABLE_CREATE_CONFIG)).thenReturn(true); + + assertEquals( + Optional.empty(), + new PartitioningTypeValidator().doValidate(config) + ); + } + + @Test + public void testDisabledTableCreationSkipsValidation() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getBoolean(TABLE_CREATE_CONFIG)).thenReturn(false); + + assertEquals( + Optional.empty(), + new PartitioningTypeValidator().doValidate(config) + ); + } + + @Test + public void testNonDayTimePartitioningWithTableCreationAndDecoratorSyntax() { + // TODO: This can be refactored into programmatically-generated test cases once we start using JUnit 5 + for (TimePartitioning.Type timePartitioningType : TimePartitioning.Type.values()) { + if (TimePartitioning.Type.DAY.equals(timePartitioningType)) { + continue; + } + + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getBoolean(TABLE_CREATE_CONFIG)).thenReturn(true); + when(config.getTimePartitioningType()).thenReturn(timePartitioningType); + + assertNotEquals( + Optional.empty(), + new PartitioningTypeValidator().doValidate(config) + ); + } + } + + @Test + public void testDayTimePartitioningWithTableCreationAndDecoratorSyntax() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getBoolean(BIGQUERY_PARTITION_DECORATOR_CONFIG)).thenReturn(true); + when(config.getBoolean(TABLE_CREATE_CONFIG)).thenReturn(true); + when(config.getTimePartitioningType()).thenReturn(TimePartitioning.Type.DAY); + + assertEquals( + Optional.empty(), + new PartitioningTypeValidator().doValidate(config) + ); + } +} diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java index 15a47efc6..6a34c02f5 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -43,7 +43,7 @@ import com.google.cloud.bigquery.Schema; import com.google.cloud.bigquery.Table; import com.google.cloud.bigquery.TableResult; -import com.wepay.kafka.connect.bigquery.BigQueryHelper; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.apache.kafka.clients.admin.Admin; @@ -145,9 +145,11 @@ protected Map baseConnectorProps(int tasksMax) { } protected BigQuery newBigQuery() { - return new BigQueryHelper() - .setKeySource(keySource()) - .connect(project(), keyFile()); + return new GcpClientBuilder.BigQueryBuilder() + .withKey(keyFile()) + .withKeySource(GcpClientBuilder.KeySource.valueOf(keySource())) + .withProject(project()) + .build(); } protected void waitForCommittedRecords( diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java index 4a9fb7301..26b40e3fe 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/utils/BucketClearer.java @@ -23,7 +23,7 @@ import com.google.cloud.storage.Blob; import com.google.cloud.storage.Bucket; import com.google.cloud.storage.Storage; -import com.wepay.kafka.connect.bigquery.GCSBuilder; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,9 +41,12 @@ public class BucketClearer { * @param keySource The key source. If "FILE", then the {@code key} parameter will be treated as a * filename; if "JSON", then {@code key} will be treated as a raw JSON string. */ - public static void clearBucket( - String key, String project, String bucketName, String folderName, String keySource) { - Storage gcs = new GCSBuilder(project).setKey(key).setKeySource(keySource).build(); + public static void clearBucket(String key, String project, String bucketName, String folderName, String keySource) { + Storage gcs = new GcpClientBuilder.GcsBuilder() + .withKeySource(GcpClientBuilder.KeySource.valueOf(keySource)) + .withKey(key) + .withProject(project) + .build(); Bucket bucket = gcs.get(bucketName); if (bucket != null) { logger.info("Deleting objects in the {} folder for bucket {}", From 34e6157cab9524f89e44dcb0d3e0a78a33cdf6ad Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 8 Dec 2021 17:55:55 +0000 Subject: [PATCH 128/190] [maven-release-plugin] prepare release v2.0.11 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 42376d47e..eb8615f5b 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.11-SNAPSHOT + 2.0.11 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 43f377040..c7202acc7 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.11-SNAPSHOT + 2.0.11 .. diff --git a/pom.xml b/pom.xml index 71a820396..e61f7c1ed 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.11-SNAPSHOT + 2.0.11 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.11 From e0816c83e0b7aac769ca0c6f12b239d5cc41dfb7 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 8 Dec 2021 17:55:58 +0000 Subject: [PATCH 129/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index eb8615f5b..b1e4b0bbe 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.11 + 2.0.12-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index c7202acc7..3c7fdba64 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.11 + 2.0.12-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index e61f7c1ed..71f2d4b63 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.11 + 2.0.12-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.11 + HEAD From e78f99a485dc8764ce8518d08d4ee0582fd545de Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 8 Dec 2021 17:59:10 +0000 Subject: [PATCH 130/190] [maven-release-plugin] prepare release v2.1.10 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index e97431bc7..a91511af5 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.10-SNAPSHOT + 2.1.10 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index c09eb910a..6b362334f 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.10-SNAPSHOT + 2.1.10 .. diff --git a/pom.xml b/pom.xml index 95f6636b4..70ca1bbad 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.10-SNAPSHOT + 2.1.10 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.10 From 607f4635e3779c468859285b0fed2d56f0611ecc Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 8 Dec 2021 17:59:13 +0000 Subject: [PATCH 131/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index a91511af5..af6aa3ecb 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.10 + 2.1.11-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 6b362334f..d259a3377 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.10 + 2.1.11-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 70ca1bbad..d48a5da1c 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.10 + 2.1.11-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.10 + 2.1.x From e84930032ca7ad42cc1de0130a84bf0f4b802478 Mon Sep 17 00:00:00 2001 From: danidelvalle Date: Thu, 30 Dec 2021 16:05:56 +0100 Subject: [PATCH 132/190] Support Application Default Credentials (ADC) to find credentials automatically in GCP (#172) * Support Application Default Credentials (ADC) to allow finding credentials automatically if deployed in GCP Co-authored-by: Daniel del Valle Co-authored-by: Chris Egerton --- Jenkinsfile | 3 +- README.md | 7 +- .../connect/bigquery/GcpClientBuilder.java | 11 ++- .../bigquery/config/BigQuerySinkConfig.java | 6 +- .../bigquery/config/CredentialsValidator.java | 11 ++- .../config/CredentialsValidatorTest.java | 13 ++++ .../bigquery/integration/BaseConnectorIT.java | 3 +- .../integration/GcpClientBuilderIT.java | 67 +++++++++++++++++++ 8 files changed, 108 insertions(+), 13 deletions(-) create mode 100644 kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/GcpClientBuilderIT.java diff --git a/Jenkinsfile b/Jenkinsfile index 504cad7ae..72dcb6a5c 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -23,6 +23,7 @@ common { publish = false downStreamValidate = false secret_file_list = [ - ['gcp/kcbq', 'creds', '/tmp/creds.json', 'KCBQ_TEST_KEYFILE'] + ['gcp/kcbq', 'creds', '/tmp/creds.json', 'KCBQ_TEST_KEYFILE'], + ['gcp/kcbq', 'creds', '/tmp/creds.json', 'GOOGLE_APPLICATION_CREDENTIALS'] ] } diff --git a/README.md b/README.md index f7c57ce79..37c978239 100644 --- a/README.md +++ b/README.md @@ -156,14 +156,9 @@ You must supply the following environment variables in order to run the tests: - `$KCBQ_TEST_PROJECT`: The name of the BigQuery project to use for the test - `$KCBQ_TEST_DATASET`: The name of the BigQuery dataset to use for the test -- `$KCBQ_TEST_KEYFILE`: The key (either file or raw contents) used to authenticate with BigQuery -during the test +- `$KCBQ_TEST_KEYFILE`: The key file used to authenticate with BigQuery during the test - `$KCBQ_TEST_BUCKET`: The name of the GCS bucket to use (for testing the GCS batch loading feature) -Optionally, the `$KCBQ_TEST_KEYSOURCE` variable can be supplied to specify whether the value of -`$KCBQ_TEST_KEYFILE` is a path to a key file (if set to `FILE`) or the raw contents of a key file -(if set to `JSON`). The default is `FILE`. - The `$KCBQ_TEST_FOLDER` variable can be supplied to specify which subfolder of the GCS bucket should be used when testing the GCS batch loading feature; if not supplied, the top-level folder will be used. diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java index 5c79fec87..bc5e035c5 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java @@ -41,7 +41,7 @@ public abstract class GcpClientBuilder { public enum KeySource { - FILE, JSON + FILE, JSON, APPLICATION_DEFAULT } private static final Logger logger = LoggerFactory.getLogger(GcpClientBuilder.class); @@ -78,7 +78,7 @@ public Client build() { } private GoogleCredentials credentials() { - if (key == null) { + if (key == null && keySource != KeySource.APPLICATION_DEFAULT) { return null; } @@ -98,6 +98,13 @@ private GoogleCredentials credentials() { throw new BigQueryConnectException("Failed to access JSON key file", e); } break; + case APPLICATION_DEFAULT: + try { + logger.debug("Attempting to use application default credentials"); + return GoogleCredentials.getApplicationDefault(); + } catch (IOException e) { + throw new BigQueryConnectException("Failed to create Application Default Credentials", e); + } default: throw new IllegalArgumentException("Unexpected value for KeySource enum: " + keySource); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 93b1003f2..7cb239b53 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -147,7 +147,11 @@ public class BigQuerySinkConfig extends AbstractConfig { ); private static final ConfigDef.Importance KEY_SOURCE_IMPORTANCE = ConfigDef.Importance.MEDIUM; private static final String KEY_SOURCE_DOC = - "Determines whether the keyfile config is the path to the credentials json, or the json itself"; + "Determines whether the " + KEYFILE_CONFIG + " config is the path to the credentials json file " + + "or the raw json of the key itself. " + + "If set to " + GcpClientBuilder.KeySource.APPLICATION_DEFAULT.name() + ", the " + + KEYFILE_CONFIG + " should not be provided and the connector will use any GCP " + + "application default credentials that it can find on the Connect worker for authentication."; public static final String SANITIZE_TOPICS_CONFIG = "sanitizeTopics"; private static final ConfigDef.Type SANITIZE_TOPICS_TYPE = ConfigDef.Type.BOOLEAN; diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java index 76007d11a..0aa8d6d89 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidator.java @@ -35,6 +35,7 @@ import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEYFILE_CONFIG; import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.KEY_SOURCE_CONFIG; import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; +import static com.wepay.kafka.connect.bigquery.GcpClientBuilder.KeySource; public abstract class CredentialsValidator> extends MultiPropertyValidator { @@ -54,7 +55,15 @@ protected Collection dependents() { @Override protected Optional doValidate(BigQuerySinkConfig config) { String keyFile = config.getKey(); - if (keyFile == null || keyFile.isEmpty()) { + KeySource keySource = config.getKeySource(); + + if (keySource == KeySource.APPLICATION_DEFAULT && keyFile != null && !keyFile.isEmpty()) { + String errorMessage = KEYFILE_CONFIG + " should not be provided if " + KEY_SOURCE_CONFIG + + " is " + KeySource.APPLICATION_DEFAULT; + return Optional.of(errorMessage); + } + + if ((keyFile == null || keyFile.isEmpty()) && config.getKeySource() != GcpClientBuilder.KeySource.APPLICATION_DEFAULT) { // No credentials to validate return Optional.empty(); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java index 7a55d5ad8..95ba68d54 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/CredentialsValidatorTest.java @@ -26,6 +26,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -66,4 +67,16 @@ public void testFailureToConstructClient() { new CredentialsValidator.GcsCredentialsValidator().doValidate(config) ); } + + @Test + public void testKeyShouldNotBeProvidedIfUsingApplicationDefaultCredentials() { + BigQuerySinkConfig config = mock(BigQuerySinkConfig.class); + when(config.getKey()).thenReturn("key"); + when(config.getKeySource()).thenReturn(GcpClientBuilder.KeySource.APPLICATION_DEFAULT); + + assertTrue( + new CredentialsValidator.BigQueryCredentialsValidator().doValidate(config) + .get().contains("should not be provided") + ); + } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java index 6a34c02f5..4fe2cbcf2 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -80,7 +80,6 @@ public abstract class BaseConnectorIT { private static final String KEYFILE_ENV_VAR = "KCBQ_TEST_KEYFILE"; private static final String PROJECT_ENV_VAR = "KCBQ_TEST_PROJECT"; private static final String DATASET_ENV_VAR = "KCBQ_TEST_DATASET"; - private static final String KEYSOURCE_ENV_VAR = "KCBQ_TEST_KEYSOURCE"; private static final String GCS_BUCKET_ENV_VAR = "KCBQ_TEST_BUCKET"; private static final String GCS_FOLDER_ENV_VAR = "KCBQ_TEST_FOLDER"; private static final String TEST_NAMESPACE_ENV_VAR = "KCBQ_TEST_TABLE_SUFFIX"; @@ -370,7 +369,7 @@ protected String dataset() { } protected String keySource() { - return readEnvVar(KEYSOURCE_ENV_VAR, BigQuerySinkConfig.KEY_SOURCE_DEFAULT); + return BigQuerySinkConfig.KEY_SOURCE_DEFAULT; } protected String gcsBucket() { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/GcpClientBuilderIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/GcpClientBuilderIT.java new file mode 100644 index 000000000..64693e3f5 --- /dev/null +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/GcpClientBuilderIT.java @@ -0,0 +1,67 @@ +package com.wepay.kafka.connect.bigquery.integration; + +import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.DatasetId; +import com.google.cloud.storage.Storage; +import com.wepay.kafka.connect.bigquery.GcpClientBuilder; +import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; +import org.apache.kafka.test.IntegrationTest; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Map; + +@Category(IntegrationTest.class) +public class GcpClientBuilderIT extends BaseConnectorIT { + + private BigQuerySinkConfig connectorProps(GcpClientBuilder.KeySource keySource) throws IOException { + Map properties = baseConnectorProps(1); + properties.put(BigQuerySinkConfig.KEY_SOURCE_CONFIG, keySource.name()); + + if (keySource == GcpClientBuilder.KeySource.APPLICATION_DEFAULT) { + properties.put(BigQuerySinkConfig.KEYFILE_CONFIG, null); + } + else if (keySource == GcpClientBuilder.KeySource.JSON){ + // actually keyFile is the path to the credentials file, so we convert it to the json string + String credentialsJsonString = new String(Files.readAllBytes(Paths.get(keyFile())), StandardCharsets.UTF_8); + properties.put(BigQuerySinkConfig.KEYFILE_CONFIG, credentialsJsonString); + } + + return new BigQuerySinkConfig(properties); + } + + /** + * Construct the BigQuery and Storage clients and perform some basic operations to check they are operational. + * @param keySource the key Source to use + * @throws IOException + */ + private void testClients(GcpClientBuilder.KeySource keySource) throws IOException { + BigQuerySinkConfig config = connectorProps(keySource); + + BigQuery bigQuery = new GcpClientBuilder.BigQueryBuilder().withConfig(config).build(); + Storage storage = new GcpClientBuilder.GcsBuilder().withConfig(config).build(); + + bigQuery.listTables(DatasetId.of(dataset())); + storage.get(gcsBucket()); + } + + @Test + public void testApplicationDefaultCredentials() throws IOException { + testClients(GcpClientBuilder.KeySource.APPLICATION_DEFAULT); + } + + @Test + public void testFile() throws IOException { + testClients(GcpClientBuilder.KeySource.FILE); + } + + @Test + public void testJson() throws IOException { + testClients(GcpClientBuilder.KeySource.JSON); + } + +} From e1b710e21a7e496140d6e16126379944e6bbb11a Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 30 Dec 2021 13:31:00 -0500 Subject: [PATCH 133/190] [maven-release-plugin] prepare branch 2.2.x --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9f0a05b33..bd579110c 100644 --- a/pom.xml +++ b/pom.xml @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + 2.2.x From 810c867c88d0ca72e7c2a93282be04a0b1432cef Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 30 Dec 2021 13:31:03 -0500 Subject: [PATCH 134/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 630218f5f..116f7a382 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0-SNAPSHOT + 2.3.0-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 7024ebb78..430285925 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0-SNAPSHOT + 2.3.0-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index bd579110c..1dc72c983 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0-SNAPSHOT + 2.3.0-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.2.x + HEAD From b3d3100a09f4508f43ade5991ff17dd2e8fdc9f1 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 30 Dec 2021 18:37:25 +0000 Subject: [PATCH 135/190] [maven-release-plugin] prepare release v2.2.0 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 630218f5f..72692d9ab 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0-SNAPSHOT + 2.2.0 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 7024ebb78..e1c2616ce 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0-SNAPSHOT + 2.2.0 .. diff --git a/pom.xml b/pom.xml index bd579110c..6a07f4336 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0-SNAPSHOT + 2.2.0 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.2.x + v2.2.0 From 78849bebdc844a7265b0fb083d2a677efece25ed Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 30 Dec 2021 18:37:28 +0000 Subject: [PATCH 136/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 72692d9ab..5c42592f3 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0 + 2.2.1-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index e1c2616ce..16c92ae32 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0 + 2.2.1-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 6a07f4336..74bbfb824 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.0 + 2.2.1-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.2.0 + 2.2.x From ace4bf4360d5d5e235747aad4b44ef6ead012412 Mon Sep 17 00:00:00 2001 From: danidelvalle Date: Tue, 25 Jan 2022 17:43:53 +0100 Subject: [PATCH 137/190] Support keywords by enclosing column identifiers by backtick characters (#174) * Enclose column identifiers with backtick characters at MergeQueries --- .../kafka/connect/bigquery/MergeQueries.java | 35 ++++++++------- .../connect/bigquery/MergeQueriesTest.java | 44 +++++++++---------- 2 files changed, 41 insertions(+), 38 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java index 3942a28b3..d8a251841 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/MergeQueries.java @@ -187,11 +187,11 @@ SELECT ARRAY_AGG( ) ON ``.=src.key WHEN MATCHED AND src.value IS NOT NULL - THEN UPDATE SET =src.value.[, =src.value....] + THEN UPDATE SET ``=src.value.[, ``=src.value....] WHEN MATCHED AND src.value IS NULL THEN DELETE WHEN NOT MATCHED AND src.value IS NOT NULL - THEN INSERT (, [_PARTITIONTIME, ][, ]) + THEN INSERT (``, [_PARTITIONTIME, ]``[, ``]) VALUES ( src.key, [CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP),] @@ -226,14 +226,15 @@ private String upsertDeleteMergeFlushQuery( + ") " + "ON `" + destinationTable.getTable() + "`." + keyFieldName + "=src." + key + " " + "WHEN MATCHED AND src." + value + " IS NOT NULL " - + "THEN UPDATE SET " + valueColumns.stream().map(col -> col + "=src." + value + "." + col).collect(Collectors.joining(", ")) + " " + + "THEN UPDATE SET " + valueColumns.stream().map(col -> "`" + col + "`=src." + value + "." + col).collect(Collectors.joining(", ")) + " " + "WHEN MATCHED AND src." + value + " IS NULL " + "THEN DELETE " + "WHEN NOT MATCHED AND src." + value + " IS NOT NULL " - + "THEN INSERT (" - + keyFieldName + ", " + + "THEN INSERT (`" + + keyFieldName + "`, " + partitionTimePseudoColumn() - + String.join(", ", valueColumns) + ") " + + "`" + + String.join("`, `", valueColumns) + "`) " + "VALUES (" + "src." + key + ", " + partitionTimeValue() @@ -255,9 +256,9 @@ SELECT ARRAY_AGG( ) ON ``.=src.key WHEN MATCHED - THEN UPDATE SET =src.value.[, ...] + THEN UPDATE SET ``=src.value.[, ``=src.value....] WHEN NOT MATCHED - THEN INSERT (`, [_PARTITIONTIME, ]``[, ``]) VALUES ( src.key, [CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP),] @@ -292,12 +293,13 @@ private String upsertMergeFlushQuery( + ") " + "ON `" + destinationTable.getTable() + "`." + keyFieldName + "=src." + key + " " + "WHEN MATCHED " - + "THEN UPDATE SET " + valueColumns.stream().map(col -> col + "=src." + value + "." + col).collect(Collectors.joining(", ")) + " " + + "THEN UPDATE SET " + valueColumns.stream().map(col -> "`" + col + "`=src." + value + "." + col).collect(Collectors.joining(", ")) + " " + "WHEN NOT MATCHED " - + "THEN INSERT (" - + keyFieldName + ", " + + "THEN INSERT (`" + + keyFieldName + "`, " + partitionTimePseudoColumn() - + String.join(", ", valueColumns) + ") " + + "`" + + String.join("`, `", valueColumns) + "`) " + "VALUES (" + "src." + key + ", " + partitionTimeValue() @@ -350,7 +352,7 @@ RIGHT JOIN ( WHEN MATCHED THEN DELETE WHEN NOT MATCHED AND src.value IS NOT NULL - THEN INSERT (, [_PARTITIONTIME, ][, ]) + THEN INSERT (``, [_PARTITIONTIME, ]``[, ``]) VALUES ( src.key, [CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP),] @@ -397,10 +399,11 @@ private String deleteMergeFlushQuery( + "WHEN MATCHED " + "THEN DELETE " + "WHEN NOT MATCHED AND src." + value + " IS NOT NULL " - + "THEN INSERT (" - + keyFieldName + ", " + + "THEN INSERT (`" + + keyFieldName + "`, " + partitionTimePseudoColumn() - + String.join(", ", valueColumns) + ") " + + "`" + + String.join("`, `", valueColumns) + "`) " + "VALUES (" + "src." + key + ", " + partitionTimeValue() diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java index 9d865bfd7..293b7736a 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/MergeQueriesTest.java @@ -123,12 +123,12 @@ public void testUpsertQueryWithPartitionTime() { + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key " + "WHEN MATCHED " - + "THEN UPDATE SET f1=src.value.f1, f2=src.value.f2, f3=src.value.f3, f4=src.value.f4 " + + "THEN UPDATE SET `f1`=src.value.f1, `f2`=src.value.f2, `f3`=src.value.f3, `f4`=src.value.f4 " + "WHEN NOT MATCHED " - + "THEN INSERT (" - + KEY + ", " + + "THEN INSERT (`" + + KEY + "`, " + "_PARTITIONTIME, " - + "f1, f2, f3, f4) " + + "`f1`, `f2`, `f3`, `f4`) " + "VALUES (" + "src.key, " + "CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP), " @@ -149,11 +149,11 @@ public void testUpsertQueryWithoutPartitionTime() { + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key " + "WHEN MATCHED " - + "THEN UPDATE SET f1=src.value.f1, f2=src.value.f2, f3=src.value.f3, f4=src.value.f4 " + + "THEN UPDATE SET `f1`=src.value.f1, `f2`=src.value.f2, `f3`=src.value.f3, `f4`=src.value.f4 " + "WHEN NOT MATCHED " - + "THEN INSERT (" - + KEY + ", " - + "f1, f2, f3, f4) " + + "THEN INSERT (`" + + KEY + "`, " + + "`f1`, `f2`, `f3`, `f4`) " + "VALUES (" + "src.key, " + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" @@ -191,10 +191,10 @@ public void testDeleteQueryWithPartitionTime() { + "WHEN MATCHED " + "THEN DELETE " + "WHEN NOT MATCHED AND src.value IS NOT NULL " - + "THEN INSERT (" - + KEY + ", " + + "THEN INSERT (`" + + KEY + "`, " + "_PARTITIONTIME, " - + "f1, f2, f3, f4) " + + "`f1`, `f2`, `f3`, `f4`) " + "VALUES (" + "src.key, " + "CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP), " @@ -233,9 +233,9 @@ public void testDeleteQueryWithoutPartitionTime() { + "WHEN MATCHED " + "THEN DELETE " + "WHEN NOT MATCHED AND src.value IS NOT NULL " - + "THEN INSERT (" - + KEY + ", " - + "f1, f2, f3, f4) " + + "THEN INSERT (`" + + KEY + "`, " + + "`f1`, `f2`, `f3`, `f4`) " + "VALUES (" + "src.key, " + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" @@ -255,14 +255,14 @@ public void testUpsertDeleteQueryWithPartitionTime() { + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key " + "WHEN MATCHED AND src.value IS NOT NULL " - + "THEN UPDATE SET f1=src.value.f1, f2=src.value.f2, f3=src.value.f3, f4=src.value.f4 " + + "THEN UPDATE SET `f1`=src.value.f1, `f2`=src.value.f2, `f3`=src.value.f3, `f4`=src.value.f4 " + "WHEN MATCHED AND src.value IS NULL " + "THEN DELETE " + "WHEN NOT MATCHED AND src.value IS NOT NULL " - + "THEN INSERT (" - + KEY + ", " + + "THEN INSERT (`" + + KEY + "`, " + "_PARTITIONTIME, " - + "f1, f2, f3, f4) " + + "`f1`, `f2`, `f3`, `f4`) " + "VALUES (" + "src.key, " + "CAST(CAST(DATE(src.partitionTime) AS DATE) AS TIMESTAMP), " @@ -283,13 +283,13 @@ public void testUpsertDeleteQueryWithoutPartitionTime() { + "GROUP BY key.k1, key.k2.nested_k1.doubly_nested_k, key.k2.nested_k2)) " + "ON `" + DESTINATION_TABLE.getTable() + "`." + KEY + "=src.key " + "WHEN MATCHED AND src.value IS NOT NULL " - + "THEN UPDATE SET f1=src.value.f1, f2=src.value.f2, f3=src.value.f3, f4=src.value.f4 " + + "THEN UPDATE SET `f1`=src.value.f1, `f2`=src.value.f2, `f3`=src.value.f3, `f4`=src.value.f4 " + "WHEN MATCHED AND src.value IS NULL " + "THEN DELETE " + "WHEN NOT MATCHED AND src.value IS NOT NULL " - + "THEN INSERT (" - + KEY + ", " - + "f1, f2, f3, f4) " + + "THEN INSERT (`" + + KEY + "`, " + + "`f1`, `f2`, `f3`, `f4`) " + "VALUES (" + "src.key, " + "src.value.f1, src.value.f2, src.value.f3, src.value.f4" From 722b3fcaf9072602f53ff213bc8ef59c1693b4a7 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 25 Jan 2022 17:04:37 +0000 Subject: [PATCH 138/190] [maven-release-plugin] prepare release v2.0.12 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index b1e4b0bbe..4f555a627 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.12-SNAPSHOT + 2.0.12 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 3c7fdba64..62ea44efe 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.12-SNAPSHOT + 2.0.12 .. diff --git a/pom.xml b/pom.xml index 71f2d4b63..479968db5 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.12-SNAPSHOT + 2.0.12 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.12 From 83b1c43746c91c9934ed04bd193ee21bba275f93 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 25 Jan 2022 17:04:40 +0000 Subject: [PATCH 139/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 4f555a627..01a703438 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.12 + 2.0.13-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 62ea44efe..65465362a 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.12 + 2.0.13-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 479968db5..904f7558a 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.12 + 2.0.13-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.12 + HEAD From 01bca8b1aa0f5e31a5ec73aef7e60a392284ef40 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 25 Jan 2022 17:08:02 +0000 Subject: [PATCH 140/190] [maven-release-plugin] prepare release v2.1.11 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index af6aa3ecb..e131d2a71 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.11-SNAPSHOT + 2.1.11 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index d259a3377..fc8092923 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.11-SNAPSHOT + 2.1.11 .. diff --git a/pom.xml b/pom.xml index d48a5da1c..f2d3c56ee 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.11-SNAPSHOT + 2.1.11 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.11 From 1e8f8019b6af9df027c68d967a1578ce38da9e9d Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 25 Jan 2022 17:08:05 +0000 Subject: [PATCH 141/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index e131d2a71..ed05fcc7d 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.11 + 2.1.12-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index fc8092923..6fdf34b93 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.11 + 2.1.12-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index f2d3c56ee..eff342a42 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.11 + 2.1.12-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.11 + 2.1.x From 83904b0e6290ecac2e495a82d4fd95aab7568716 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 25 Jan 2022 17:11:34 +0000 Subject: [PATCH 142/190] [maven-release-plugin] prepare release v2.2.1 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 5c42592f3..33353890f 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.1-SNAPSHOT + 2.2.1 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 16c92ae32..b95825373 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.1-SNAPSHOT + 2.2.1 .. diff --git a/pom.xml b/pom.xml index 74bbfb824..7dc5de584 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.1-SNAPSHOT + 2.2.1 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.2.x + v2.2.1 From 5d22ac3133e8bcccd4f66709fbc833ccd7dfa9db Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Tue, 25 Jan 2022 17:11:37 +0000 Subject: [PATCH 143/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 33353890f..b01efb571 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.1 + 2.2.2-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index b95825373..f92f805fd 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.1 + 2.2.2-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 7dc5de584..fb73f26ff 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.1 + 2.2.2-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.2.1 + 2.2.x From 25ec713258ca319e42bcc103a012d487faacbeb8 Mon Sep 17 00:00:00 2001 From: lanyao77 <93687783+lanyao77@users.noreply.github.com> Date: Tue, 25 Jan 2022 11:20:27 -0800 Subject: [PATCH 144/190] Add exception message (#170) * Add overloaded function for BigQueryException * remove original exception api and update BigQueryWriterTest for added exception message. --- .../bigquery/exception/BigQueryConnectException.java | 8 ++++---- .../kafka/connect/bigquery/write/row/BigQueryWriter.java | 4 ++-- .../connect/bigquery/write/row/BigQueryWriterTest.java | 9 ++++++--- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java index 847b14542..38561917f 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/exception/BigQueryConnectException.java @@ -43,13 +43,13 @@ public BigQueryConnectException(Throwable thr) { super(thr); } - public BigQueryConnectException(Map> errors) { - super(formatInsertAllErrors(errors)); + public BigQueryConnectException(String tableInfo, Map> errors) { + super(formatInsertAllErrors(tableInfo, errors)); } - private static String formatInsertAllErrors(Map> errorsMap) { + private static String formatInsertAllErrors(String tableInfo, Map> errorsMap) { StringBuilder messageBuilder = new StringBuilder(); - messageBuilder.append("table insertion failed for the following rows:"); + messageBuilder.append(String.format("table: %s insertion failed for the following rows:", tableInfo)); for (Map.Entry> errorsEntry : errorsMap.entrySet()) { for (BigQueryError error : errorsEntry.getValue()) { messageBuilder.append(String.format( diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java index 032ad62d4..afe83c1b4 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java @@ -118,11 +118,11 @@ public void writeRows(PartitionedTableId table, rows.size() - failedRowsMap.size(), failedRowsMap.size()); // update insert rows and retry in case of partial failure rows = getFailedRows(rows, failedRowsMap.keySet(), table); - mostRecentException = new BigQueryConnectException(failedRowsMap); + mostRecentException = new BigQueryConnectException(table.toString(), failedRowsMap); retryCount++; } else { // throw an exception in case of complete failure - throw new BigQueryConnectException(failedRowsMap); + throw new BigQueryConnectException(table.toString(), failedRowsMap); } } catch (BigQueryException err) { mostRecentException = err; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index acd1d0d00..875d85bce 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -20,6 +20,8 @@ package com.wepay.kafka.connect.bigquery.write.row; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Matchers.anyObject; import static org.mockito.Mockito.mock; @@ -229,7 +231,7 @@ public void testBigQueryPartialFailure() { assertEquals("test_topic-1-1", varArgs.getAllValues().get(1).getRows().get(0).getId()); } - @Test(expected = BigQueryConnectException.class) + @Test public void testBigQueryCompleteFailure() { final String topic = "test_topic"; final String dataset = "scratch"; @@ -271,9 +273,10 @@ public void testBigQueryCompleteFailure() { testTask.initialize(sinkTaskContext); testTask.start(properties); testTask.put(sinkRecordList); - testTask.flush(Collections.emptyMap()); + Exception expectedEx = assertThrows(BigQueryConnectException.class, + () -> testTask.flush(Collections.emptyMap())); + assertTrue(expectedEx.getMessage().contains("test_topic")); } - /** * Utility method for making and retrieving properties based on provided parameters. * @param bigqueryRetry The number of retries. From 045fcd12321d90b260c2ef3d8fcbebff8c63959b Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 26 Jan 2022 12:01:53 -0500 Subject: [PATCH 145/190] GH-164: Remove dependency on ConfigDef.CaseInsensitiveValidString (#165) --- .../connect/bigquery/config/BigQuerySinkConfig.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index cecc7f19f..366f54f57 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -682,7 +682,14 @@ public static ConfigDef getConfig() { TIME_PARTITIONING_TYPE_CONFIG, TIME_PARTITIONING_TYPE_TYPE, TIME_PARTITIONING_TYPE_DEFAULT, - ConfigDef.CaseInsensitiveValidString.in(TIME_PARTITIONING_TYPES.toArray(new String[0])), + (name, value) -> { + if (value == null) { + return; + } + String[] validStrings = TIME_PARTITIONING_TYPES.stream().map(String::toLowerCase).toArray(String[]::new); + String lowercaseValue = ((String) value).toLowerCase(); + ConfigDef.ValidString.in(validStrings).ensureValid(name, lowercaseValue); + }, TIME_PARTITIONING_TYPE_IMPORTANCE, TIME_PARTITIONING_TYPE_DOC, "", From f7e25b863378da8747eeab73bcfab7914db6e26a Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Fri, 28 Jan 2022 07:52:37 +0800 Subject: [PATCH 146/190] CCDB-4518: Sanitize field names when converting to BQ schema (#177) * CCDB-4518: Sanitize field names when converting to BQ schema * Apply suggestions from code review Co-authored-by: Chris Egerton * review comments * remove IT, more testcases in SchemaManagerTest * remote whitespace only change Co-authored-by: Chris Egerton --- .../connect/bigquery/BigQuerySinkTask.java | 2 + .../kafka/connect/bigquery/SchemaManager.java | 24 ++- .../bigquery/config/BigQuerySinkConfig.java | 4 +- .../convert/BigQuerySchemaConverter.java | 15 +- .../connect/bigquery/SchemaManagerTest.java | 150 ++++++++++++++++-- .../convert/BigQuerySchemaConverterTest.java | 23 +++ 6 files changed, 200 insertions(+), 18 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 71b12b874..304dc11e6 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -316,8 +316,10 @@ private SchemaManager newSchemaManager() { boolean allowNewBQFields = config.getBoolean(BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG); boolean allowReqFieldRelaxation = config.getBoolean(BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG); boolean allowSchemaUnionization = config.getBoolean(BigQuerySinkConfig.ALLOW_SCHEMA_UNIONIZATION_CONFIG); + boolean sanitizeFieldNames = config.getBoolean(BigQuerySinkConfig.SANITIZE_FIELD_NAME_CONFIG); return new SchemaManager(schemaRetriever, schemaConverter, getBigQuery(), allowNewBQFields, allowReqFieldRelaxation, allowSchemaUnionization, + sanitizeFieldNames, kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, clusteringFieldName); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index d839b77d3..4a083c8be 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -37,6 +37,7 @@ import com.wepay.kafka.connect.bigquery.convert.KafkaDataBuilder; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import com.wepay.kafka.connect.bigquery.utils.TableNameUtils; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.sink.SinkRecord; @@ -66,6 +67,7 @@ public class SchemaManager { private final boolean allowNewBQFields; private final boolean allowBQRequiredFieldRelaxation; private final boolean allowSchemaUnionization; + private final boolean sanitizeFieldNames; private final Optional kafkaKeyFieldName; private final Optional kafkaDataFieldName; private final Optional timestampPartitionFieldName; @@ -83,6 +85,7 @@ public class SchemaManager { * @param allowNewBQFields If set to true, allows new fields to be added to BigQuery Schema. * @param allowBQRequiredFieldRelaxation If set to true, allows changing field mode from REQUIRED to NULLABLE * @param allowSchemaUnionization If set to true, allows existing and new schemas to be unionized + * @param sanitizeFieldNames If true, sanitizes field names to adhere to BigQuery column name restrictions * @param kafkaKeyFieldName The name of kafka key field to be used in BigQuery. * If set to null, Kafka Key Field will not be included in BigQuery. * @param kafkaDataFieldName The name of kafka data field to be used in BigQuery. @@ -100,6 +103,7 @@ public SchemaManager( boolean allowNewBQFields, boolean allowBQRequiredFieldRelaxation, boolean allowSchemaUnionization, + boolean sanitizeFieldNames, Optional kafkaKeyFieldName, Optional kafkaDataFieldName, Optional timestampPartitionFieldName, @@ -111,6 +115,7 @@ public SchemaManager( allowNewBQFields, allowBQRequiredFieldRelaxation, allowSchemaUnionization, + sanitizeFieldNames, kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, @@ -128,6 +133,7 @@ private SchemaManager( boolean allowNewBQFields, boolean allowBQRequiredFieldRelaxation, boolean allowSchemaUnionization, + boolean sanitizeFieldNames, Optional kafkaKeyFieldName, Optional kafkaDataFieldName, Optional timestampPartitionFieldName, @@ -142,6 +148,7 @@ private SchemaManager( this.allowNewBQFields = allowNewBQFields; this.allowBQRequiredFieldRelaxation = allowBQRequiredFieldRelaxation; this.allowSchemaUnionization = allowSchemaUnionization; + this.sanitizeFieldNames = sanitizeFieldNames; this.kafkaKeyFieldName = kafkaKeyFieldName; this.kafkaDataFieldName = kafkaDataFieldName; this.timestampPartitionFieldName = timestampPartitionFieldName; @@ -160,6 +167,7 @@ public SchemaManager forIntermediateTables() { allowNewBQFields, allowBQRequiredFieldRelaxation, allowSchemaUnionization, + sanitizeFieldNames, kafkaKeyFieldName, kafkaDataFieldName, timestampPartitionFieldName, @@ -603,7 +611,9 @@ private List getIntermediateSchemaFields(com.google.cloud.bigquery.Schema List valueFields = new ArrayList<>(valueSchema.getFields()); if (kafkaDataFieldName.isPresent()) { - Field kafkaDataField = KafkaDataBuilder.buildKafkaDataField(kafkaDataFieldName.get()); + String dataFieldName = sanitizeFieldNames ? + FieldNameSanitizer.sanitizeName(kafkaDataFieldName.get()) : kafkaDataFieldName.get(); + Field kafkaDataField = KafkaDataBuilder.buildKafkaDataField(dataFieldName); valueFields.add(kafkaDataField); } @@ -645,14 +655,20 @@ private List getRegularSchemaFields(com.google.cloud.bigquery.Schema valu List result = new ArrayList<>(valueSchema.getFields()); if (kafkaDataFieldName.isPresent()) { - Field kafkaDataField = KafkaDataBuilder.buildKafkaDataField(kafkaDataFieldName.get()); + String dataFieldName = sanitizeFieldNames ? + FieldNameSanitizer.sanitizeName(kafkaDataFieldName.get()) : kafkaDataFieldName.get(); + Field kafkaDataField = KafkaDataBuilder.buildKafkaDataField(dataFieldName); result.add(kafkaDataField); } if (kafkaKeyFieldName.isPresent()) { com.google.cloud.bigquery.Schema keySchema = schemaConverter.convertSchema(kafkaKeySchema); - Field kafkaKeyField = Field.newBuilder(kafkaKeyFieldName.get(), LegacySQLTypeName.RECORD, keySchema.getFields()) - .setMode(Field.Mode.NULLABLE).build(); + String keyFieldName = sanitizeFieldNames ? + FieldNameSanitizer.sanitizeName(kafkaKeyFieldName.get()) : kafkaKeyFieldName.get(); + Field kafkaKeyField = Field.newBuilder( + keyFieldName, + LegacySQLTypeName.RECORD, + keySchema.getFields()).setMode(Field.Mode.NULLABLE).build(); result.add(kafkaKeyField); } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 91f158be0..76ce183f5 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -754,7 +754,9 @@ public String getKeyFile() { * @return a {@link SchemaConverter} for BigQuery. */ public SchemaConverter getSchemaConverter() { - return new BigQuerySchemaConverter(getBoolean(ALL_BQ_FIELDS_NULLABLE_CONFIG)); + return new BigQuerySchemaConverter( + getBoolean(ALL_BQ_FIELDS_NULLABLE_CONFIG), + getBoolean(SANITIZE_FIELD_NAME_CONFIG)); } /** diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java index f98125980..731670044 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverter.java @@ -28,8 +28,8 @@ import com.wepay.kafka.connect.bigquery.convert.logicaltype.LogicalTypeConverter; import com.wepay.kafka.connect.bigquery.exception.ConversionConnectException; +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Schema.Type; import java.util.ArrayList; import java.util.HashMap; @@ -84,9 +84,16 @@ public class BigQuerySchemaConverter implements SchemaConverter convertField(Schema ka String fieldName) { Optional result; Schema.Type kafkaConnectSchemaType = kafkaConnectSchema.type(); + if (sanitizeFieldNames) { + fieldName = FieldNameSanitizer.sanitizeName(fieldName); + } + if (LogicalConverterRegistry.isRegisteredLogicalType(kafkaConnectSchema.name())) { result = Optional.of(convertLogical(kafkaConnectSchema, fieldName)); } else if (PRIMITIVE_TYPE_MAP.containsKey(kafkaConnectSchemaType)) { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java index 0edb567bf..37c2d53a4 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/SchemaManagerTest.java @@ -20,7 +20,10 @@ package com.wepay.kafka.connect.bigquery; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import com.google.cloud.bigquery.BigQuery; @@ -35,11 +38,14 @@ import com.google.common.collect.ImmutableList; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; +import com.wepay.kafka.connect.bigquery.convert.BigQuerySchemaConverter; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; import java.util.Random; + +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; @@ -48,6 +54,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.ArgumentCaptor; import org.mockito.stubbing.OngoingStubbing; import java.util.ArrayList; @@ -86,7 +93,7 @@ public void testBQTableDescription() { Optional kafkaKeyFieldName = Optional.of("kafkaKey"); Optional kafkaDataFieldName = Optional.of("kafkaData"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, kafkaKeyFieldName, kafkaDataFieldName, Optional.empty(), Optional.empty()); + mockBigQuery, false, false, false, false, kafkaKeyFieldName, kafkaDataFieldName, Optional.empty(), Optional.empty()); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -104,7 +111,7 @@ public void testBQTableDescription() { public void testTimestampPartitionSet() { Optional testField = Optional.of("testField"); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); + mockBigQuery, false, false, false, false, Optional.empty(), Optional.empty(), testField, Optional.empty()); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -124,7 +131,7 @@ public void testClusteringPartitionSet() { Optional timestampPartitionFieldName = Optional.of("testField"); Optional> testField = Optional.of(Arrays.asList("column1", "column2")); SchemaManager schemaManager = new SchemaManager(mockSchemaRetriever, mockSchemaConverter, - mockBigQuery, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); + mockBigQuery, false, false, false, false, Optional.empty(), Optional.empty(), timestampPartitionFieldName, testField); when(mockSchemaConverter.convertSchema(mockKafkaSchema)).thenReturn(fakeBigQuerySchema); when(mockKafkaSchema.doc()).thenReturn(testDoc); @@ -391,6 +398,89 @@ public void FieldsWithUnspecifiedModeShouldNotCauseNpe() { testGetAndValidateProposedSchema(schemaManager, existingSchema, expandedSchema, expectedSchema); } + @Test + public void testFieldNamesSanitizedNoExistingSchema() { + BigQuerySchemaConverter converter = new BigQuerySchemaConverter(false, true); + + Schema kafkaSchema = SchemaBuilder.struct() + .field("f 1", Schema.BOOLEAN_SCHEMA) + .field("f 2", Schema.INT32_SCHEMA) + .build(); + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f_1", LegacySQLTypeName.BOOLEAN).setMode(Mode.REQUIRED).build(), + Field.newBuilder("f_2", LegacySQLTypeName.INTEGER).setMode(Mode.REQUIRED).build() + ); + + SchemaManager schemaManager = createSchemaManager(false, false, false, true, converter); + testGetAndValidateProposedSchema(schemaManager, null, + null, expectedSchema, + Collections.singletonList(recordWithValueSchema(kafkaSchema))); + } + + @Test + public void testFieldNameSanitizedNewFields() { + BigQuerySchemaConverter converter = new BigQuerySchemaConverter(false, true); + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Mode.REQUIRED).build() + ); + Schema kafkaSchema = SchemaBuilder.struct() + .field("f1", Schema.BOOLEAN_SCHEMA) + .field("f 1", Schema.BOOLEAN_SCHEMA) + .field("f 2", Schema.INT32_SCHEMA) + .build(); + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Mode.REQUIRED).build(), + Field.newBuilder("f_1", LegacySQLTypeName.BOOLEAN).setMode(Mode.NULLABLE).build(), + Field.newBuilder("f_2", LegacySQLTypeName.INTEGER).setMode(Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, false, true, converter); + testGetAndValidateProposedSchema(schemaManager, existingSchema, + null, expectedSchema, + Collections.singletonList(recordWithValueSchema(kafkaSchema))); + } + + @Test + public void testFieldNamesSanitizedUnionizedFields() { + BigQuerySchemaConverter converter = new BigQuerySchemaConverter(false, true); + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Mode.REQUIRED).build() + ); + Schema kafkaSchema = SchemaBuilder.struct() + .field("f 1", Schema.BOOLEAN_SCHEMA) + .field("f 2", Schema.INT32_SCHEMA) + .build(); + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f1", LegacySQLTypeName.BOOLEAN).setMode(Mode.NULLABLE).build(), + Field.newBuilder("f_1", LegacySQLTypeName.BOOLEAN).setMode(Mode.NULLABLE).build(), + Field.newBuilder("f_2", LegacySQLTypeName.INTEGER).setMode(Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, true, true, converter); + testGetAndValidateProposedSchema(schemaManager, existingSchema, + null, expectedSchema, + Collections.singletonList(recordWithValueSchema(kafkaSchema))); + } + + @Test + public void testFieldNamesSanitizedFieldRelaxation() { + BigQuerySchemaConverter converter = new BigQuerySchemaConverter(false, true); + com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f_1", LegacySQLTypeName.BOOLEAN).setMode(Mode.REQUIRED).build() + ); + Schema kafkaSchema = SchemaBuilder.struct() + .field("f 1", Schema.OPTIONAL_BOOLEAN_SCHEMA) + .build(); + com.google.cloud.bigquery.Schema expectedSchema = com.google.cloud.bigquery.Schema.of( + Field.newBuilder("f_1", LegacySQLTypeName.BOOLEAN).setMode(Mode.NULLABLE).build() + ); + + SchemaManager schemaManager = createSchemaManager(true, true, false, true, converter); + testGetAndValidateProposedSchema(schemaManager, existingSchema, + null, expectedSchema, + Collections.singletonList(recordWithValueSchema(kafkaSchema))); + } + @Test public void testUpdateWithOnlyTombstoneRecordsAndExistingSchema() { com.google.cloud.bigquery.Schema existingSchema = com.google.cloud.bigquery.Schema.of( @@ -451,8 +541,14 @@ public void testGetUnionizedTableDescriptionFromRegularAndNullRecords() { private SchemaManager createSchemaManager( boolean allowNewFields, boolean allowFieldRelaxation, boolean allowUnionization) { - return new SchemaManager(new IdentitySchemaRetriever(), mockSchemaConverter, mockBigQuery, - allowNewFields, allowFieldRelaxation, allowUnionization, + return createSchemaManager( + allowNewFields, allowFieldRelaxation, allowUnionization, false, mockSchemaConverter); + } + + private SchemaManager createSchemaManager( + boolean allowNewFields, boolean allowFieldRelaxation, boolean allowUnionization, boolean sanitizeFieldNames, SchemaConverter schemaConverter) { + return new SchemaManager(new IdentitySchemaRetriever(), schemaConverter, mockBigQuery, + allowNewFields, allowFieldRelaxation, allowUnionization, sanitizeFieldNames, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); } @@ -482,12 +578,15 @@ private void testGetAndValidateProposedSchema( List incomingSinkRecords) { Table existingTable = existingSchema != null ? tableWithSchema(existingSchema) : null; when(mockBigQuery.getTable(tableId)).thenReturn(existingTable); - OngoingStubbing converterStub = - when(mockSchemaConverter.convertSchema(mockKafkaSchema)); - for (com.google.cloud.bigquery.Schema newSchema : newSchemas) { - // The converter will return the schemas in the order that they are provided to it with the - // call to "thenReturn" - converterStub = converterStub.thenReturn(newSchema); + + if (newSchemas != null) { + OngoingStubbing converterStub = + when(mockSchemaConverter.convertSchema(mockKafkaSchema)); + for (com.google.cloud.bigquery.Schema newSchema : newSchemas) { + // The converter will return the schemas in the order that they are provided to it with the + // call to "thenReturn" + converterStub = converterStub.thenReturn(newSchema); + } } com.google.cloud.bigquery.Schema proposedSchema = @@ -601,6 +700,35 @@ public void testUnionizeSchemaNestedRelax() { assertUnion(makeNullable(s2), s1, s2); } + @Test + public void testFieldNameSanitizedOnCreateTable() { + Schema embeddedStructWithInvalidFieldName = SchemaBuilder.struct() + .field("embedded-invalid", Schema.INT32_SCHEMA) + .build(); + Schema schemaWithInvalidFieldNames = SchemaBuilder.struct() + .field("1st field", Schema.BOOLEAN_SCHEMA) + .field("second-field", Schema.STRING_SCHEMA) + .field("embedded", embeddedStructWithInvalidFieldName) + .build(); + + List incomingSinkRecords = Collections.nCopies(2, recordWithValueSchema(schemaWithInvalidFieldNames)); + BigQuerySchemaConverter converter = new BigQuerySchemaConverter(true, true); + + SchemaManager schemaManager = createSchemaManager(false, false, false, true, converter); + schemaManager.createTable(tableId, incomingSinkRecords); + + ArgumentCaptor tableInfoCaptor = ArgumentCaptor.forClass(TableInfo.class); + + verify(mockBigQuery).create(tableInfoCaptor.capture()); + + com.google.cloud.bigquery.Schema actualSchema = tableInfoCaptor.getValue().getDefinition().getSchema(); + for (org.apache.kafka.connect.data.Field field : schemaWithInvalidFieldNames.fields()) { + String sanitizedName = FieldNameSanitizer.sanitizeName(field.name()); + assertEquals(sanitizedName, actualSchema.getFields().get(sanitizedName).getName()); + } + assertEquals("embedded_invalid", actualSchema.getFields().get("embedded").getSubFields().get(0).getName()); + } + private com.google.cloud.bigquery.Schema makeNullable(com.google.cloud.bigquery.Schema s) { return com.google.cloud.bigquery.Schema.of( s.getFields().stream() diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java index d9d3ccab4..307ac34ea 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/convert/BigQuerySchemaConverterTest.java @@ -27,6 +27,7 @@ import com.wepay.kafka.connect.bigquery.exception.ConversionConnectException; +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; @@ -421,6 +422,28 @@ public void testStringArray() { assertEquals(bigQueryExpectedSchema, bigQueryTestSchema); } + @Test + public void testFieldNameSanitized() { + final String fieldName = "String Array"; + com.google.cloud.bigquery.Schema bigQueryExpectedSchema = + com.google.cloud.bigquery.Schema.of( + com.google.cloud.bigquery.Field.newBuilder( + FieldNameSanitizer.sanitizeName(fieldName), + LegacySQLTypeName.STRING + ).setMode(com.google.cloud.bigquery.Field.Mode.REPEATED).build() + ); + + Schema kafkaConnectArraySchema = SchemaBuilder.array(Schema.STRING_SCHEMA).build(); + Schema kafkaConnectTestSchema = SchemaBuilder + .struct() + .field(fieldName, kafkaConnectArraySchema) + .build(); + + com.google.cloud.bigquery.Schema bigQueryTestSchema = + new BigQuerySchemaConverter(false, true).convertSchema(kafkaConnectTestSchema); + assertEquals(bigQueryExpectedSchema, bigQueryTestSchema); + } + @Test public void testBytes() { final String fieldName = "Bytes"; From 9ae88dd46cd7172176311c392c2f518894201c10 Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Fri, 28 Jan 2022 20:07:22 +0000 Subject: [PATCH 147/190] [maven-release-plugin] prepare release v2.0.13 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 01a703438..18a469d78 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.13-SNAPSHOT + 2.0.13 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 65465362a..9d424af47 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.13-SNAPSHOT + 2.0.13 .. diff --git a/pom.xml b/pom.xml index 904f7558a..206fbc632 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.13-SNAPSHOT + 2.0.13 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.13 From 523b12aff190b0aea5e8e15c64e4236a403cc397 Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Fri, 28 Jan 2022 20:07:25 +0000 Subject: [PATCH 148/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 18a469d78..549a91551 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.13 + 2.0.14-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 9d424af47..a090b79b2 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.13 + 2.0.14-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 206fbc632..c4f420d89 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.13 + 2.0.14-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.13 + HEAD From cf305e938fef9d5e6d7cba2f0f2672d310799c00 Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Fri, 28 Jan 2022 20:10:55 +0000 Subject: [PATCH 149/190] [maven-release-plugin] prepare release v2.1.12 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index ed05fcc7d..2c3d167f8 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.12-SNAPSHOT + 2.1.12 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 6fdf34b93..eaf233d2f 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.12-SNAPSHOT + 2.1.12 .. diff --git a/pom.xml b/pom.xml index eff342a42..89bfd3595 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.12-SNAPSHOT + 2.1.12 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.12 From 1f5d469f6eb78012b724fc1a72086f9c529d670f Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Fri, 28 Jan 2022 20:10:57 +0000 Subject: [PATCH 150/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 2c3d167f8..3a27fd7ab 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.12 + 2.1.13-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index eaf233d2f..867378ede 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.12 + 2.1.13-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 89bfd3595..51345b9b8 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.12 + 2.1.13-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.12 + 2.1.x From 6588299bbc7246bd89d046f82d0ed3f5560e2a06 Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Fri, 28 Jan 2022 20:14:20 +0000 Subject: [PATCH 151/190] [maven-release-plugin] prepare release v2.2.2 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index b01efb571..18c5a6e4e 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.2-SNAPSHOT + 2.2.2 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index f92f805fd..bc24955cd 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.2-SNAPSHOT + 2.2.2 .. diff --git a/pom.xml b/pom.xml index fb73f26ff..05903bcbf 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.2-SNAPSHOT + 2.2.2 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.2.x + v2.2.2 From 576623c996b70d9e9dd33d6c651c546c90906e9d Mon Sep 17 00:00:00 2001 From: Nigel Liang Date: Fri, 28 Jan 2022 20:14:22 +0000 Subject: [PATCH 152/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 18c5a6e4e..54c706137 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.2 + 2.2.3-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index bc24955cd..8b496a3a4 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.2 + 2.2.3-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 05903bcbf..ec55b1f9e 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.2 + 2.2.3-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.2.2 + 2.2.x From b798418d554a93b9bd36c059bc468a4079347727 Mon Sep 17 00:00:00 2001 From: Luke Young <91491244+lyoung-confluent@users.noreply.github.com> Date: Fri, 1 Apr 2022 10:54:43 -0700 Subject: [PATCH 153/190] Introduce Pull Request Reviewers (#182) --- .github/CODEOWNERS | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 .github/CODEOWNERS diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS new file mode 100644 index 000000000..740acdeff --- /dev/null +++ b/.github/CODEOWNERS @@ -0,0 +1,2 @@ +# See go/codeowners - automatically generated for confluentinc/kafka-connect-bigquery: +* @confluentinc/connect From 51ffef1aff9e0a5aee9778ada18cb7f23b16d0ee Mon Sep 17 00:00:00 2001 From: ypmahajan <42435723+ypmahajan@users.noreply.github.com> Date: Thu, 28 Apr 2022 09:36:58 +0530 Subject: [PATCH 154/190] Fixed test and increased jenkins timeout (#201) --- Jenkinsfile | 1 + .../connect/bigquery/integration/BigQueryErrorResponsesIT.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/Jenkinsfile b/Jenkinsfile index 504cad7ae..488ffa5f1 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -25,4 +25,5 @@ common { secret_file_list = [ ['gcp/kcbq', 'creds', '/tmp/creds.json', 'KCBQ_TEST_KEYFILE'] ] + timeoutHours = 2 } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java index bbf430fdc..7bacb2e5b 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java @@ -117,7 +117,7 @@ public void testWriteToRecreatedTable() throws Exception { // Try to write to it... try { bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(Collections.singletonMap("f1", "v1")))); - return false; + return true; } catch (BigQueryException e) { logger.debug("Recreated table write error", e); return BigQueryErrorResponses.isNonExistentTableError(e); From 3a025b469262249f20da5e3aa8a4f65b41043bcf Mon Sep 17 00:00:00 2001 From: ypmahajan <42435723+ypmahajan@users.noreply.github.com> Date: Tue, 3 May 2022 16:49:22 +0530 Subject: [PATCH 155/190] Added user-agent header for GCP calls (#196) * Added user-agent header for GCP calls * Addressed review comments * Addressed PR comments * Fixed test and added timeout for Jenkins job --- .../connect/bigquery/GcpClientBuilder.java | 44 +++++++++++++++---- .../bigquery/config/BigQuerySinkConfig.java | 14 ++++++ .../bigquery/BigQuerySinkTaskTest.java | 17 ++++++- .../config/BigQuerySinkConfigTest.java | 18 ++++++++ .../bigquery/integration/BaseConnectorIT.java | 1 + .../write/row/BigQueryWriterTest.java | 2 +- 6 files changed, 86 insertions(+), 10 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java index bc5e035c5..a24a897a4 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/GcpClientBuilder.java @@ -19,13 +19,17 @@ package com.wepay.kafka.connect.bigquery; +import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.api.gax.rpc.HeaderProvider; import com.google.auth.oauth2.GoogleCredentials; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryOptions; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageOptions; +import com.google.common.annotations.VisibleForTesting; import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; +import com.wepay.kafka.connect.bigquery.utils.Version; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,6 +40,9 @@ import java.nio.charset.StandardCharsets; import java.util.Objects; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_DEFAULT; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_TYPES; import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.PROJECT_CONFIG; public abstract class GcpClientBuilder { @@ -45,7 +52,9 @@ public enum KeySource { } private static final Logger logger = LoggerFactory.getLogger(GcpClientBuilder.class); - + private static final String USER_AGENT_HEADER_KEY = "user-agent"; + private static final String USER_AGENT_HEADER_FORMAT = "%s (GPN: Confluent;) Google BigQuery Sink/%s"; + private HeaderProvider headerProvider = null; private String project = null; private KeySource keySource = null; private String key = null; @@ -53,7 +62,8 @@ public enum KeySource { public GcpClientBuilder withConfig(BigQuerySinkConfig config) { return withProject(config.getString(PROJECT_CONFIG)) .withKeySource(config.getKeySource()) - .withKey(config.getKey()); + .withKey(config.getKey()) + .withUserAgent(config.getString(CONNECTOR_RUNTIME_PROVIDER_CONFIG)); } public GcpClientBuilder withProject(String project) { @@ -73,8 +83,19 @@ public GcpClientBuilder withKey(String key) { return this; } + public GcpClientBuilder withUserAgent(String userAgent) { + if (!CONNECTOR_RUNTIME_PROVIDER_TYPES.contains(userAgent)) { + logger.warn(String.format("Invalid Kafka runtime provider value received. Provider : %s. Defaulting to %s", + userAgent, CONNECTOR_RUNTIME_PROVIDER_DEFAULT)); + userAgent = CONNECTOR_RUNTIME_PROVIDER_DEFAULT; + } + this.headerProvider = FixedHeaderProvider.create(USER_AGENT_HEADER_KEY, + String.format(USER_AGENT_HEADER_FORMAT, userAgent, Version.version())); + return this; + } + public Client build() { - return doBuild(project, credentials()); + return doBuild(project, credentials(), headerProvider); } private GoogleCredentials credentials() { @@ -116,13 +137,14 @@ private GoogleCredentials credentials() { } } - protected abstract Client doBuild(String project, GoogleCredentials credentials); + protected abstract Client doBuild(String project, GoogleCredentials credentials, HeaderProvider userAgent); public static class BigQueryBuilder extends GcpClientBuilder { @Override - protected BigQuery doBuild(String project, GoogleCredentials credentials) { + protected BigQuery doBuild(String project, GoogleCredentials credentials, HeaderProvider headerProvider) { BigQueryOptions.Builder builder = BigQueryOptions.newBuilder() - .setProjectId(project); + .setProjectId(project) + .setHeaderProvider(headerProvider); if (credentials != null) { builder.setCredentials(credentials); @@ -136,9 +158,10 @@ protected BigQuery doBuild(String project, GoogleCredentials credentials) { public static class GcsBuilder extends GcpClientBuilder { @Override - protected Storage doBuild(String project, GoogleCredentials credentials) { + protected Storage doBuild(String project, GoogleCredentials credentials, HeaderProvider headerProvider) { StorageOptions.Builder builder = StorageOptions.newBuilder() - .setProjectId(project); + .setProjectId(project) + .setHeaderProvider(headerProvider); if (credentials != null) { builder.setCredentials(credentials); @@ -149,4 +172,9 @@ protected Storage doBuild(String project, GoogleCredentials credentials) { return builder.build().getService(); } } + + @VisibleForTesting + HeaderProvider getHeaderProvider() { + return this.headerProvider; + } } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 748c98ca2..67d570677 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -458,6 +458,15 @@ public class BigQuerySinkConfig extends AbstractConfig { + "older than the specified partition expiration time will be permanently deleted. " + "Existing tables will not be altered to use this partition expiration time."; + //This config determines where the connector is hosted (Confluent Cloud or Confluent Platform). + //This is not enforced and defaulted to "Confluent Platform". Currently, it is only used for user-agent tracking in GCP. + public static final String CONNECTOR_RUNTIME_PROVIDER_CONFIG = "runtimeProvider"; + private static final ConfigDef.Type CONNECTOR_RUNTIME_PROVIDER_TYPE = ConfigDef.Type.STRING; + public static final String CONNECTOR_RUNTIME_PROVIDER_DEFAULT = "Confluent Platform"; + private static final ConfigDef.Importance CONNECTOR_RUNTIME_PROVIDER_IMPORTANCE = ConfigDef.Importance.LOW; + public static final List CONNECTOR_RUNTIME_PROVIDER_TYPES = Stream.of("Confluent Platform", "Confluent Cloud") + .collect(Collectors.toList()); + /** * Return the ConfigDef object used to define this config's fields. * @@ -738,6 +747,11 @@ public boolean visible(String s, Map map) { BIGQUERY_PARTITION_EXPIRATION_VALIDATOR, BIGQUERY_PARTITION_EXPIRATION_IMPORTANCE, BIGQUERY_PARTITION_EXPIRATION_DOC + ).defineInternal( + CONNECTOR_RUNTIME_PROVIDER_CONFIG, + CONNECTOR_RUNTIME_PROVIDER_TYPE, + CONNECTOR_RUNTIME_PROVIDER_DEFAULT, + CONNECTOR_RUNTIME_PROVIDER_IMPORTANCE ); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index f7a91fb38..462a661e1 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -19,6 +19,8 @@ package com.wepay.kafka.connect.bigquery; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_DEFAULT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThrows; @@ -48,7 +50,6 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.data.Schema; @@ -739,6 +740,20 @@ public void testStop() { testTask.put(Collections.singletonList(spoofSinkRecord(topic))); } + @Test + public void testKafkaProviderConfigInvalidValue() { + Map configProperties = propertiesFactory.getProperties(); + String testKafkaProvider = "testProvider"; + configProperties.put(CONNECTOR_RUNTIME_PROVIDER_CONFIG, testKafkaProvider); + BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); + + GcpClientBuilder clientBuilder = new GcpClientBuilder.BigQueryBuilder().withConfig(config); + assertTrue(clientBuilder.getHeaderProvider().getHeaders().get("user-agent").contains(CONNECTOR_RUNTIME_PROVIDER_DEFAULT)); + + GcpClientBuilder storageBuilder = new GcpClientBuilder.GcsBuilder().withConfig(config); + assertTrue(storageBuilder.getHeaderProvider().getHeaders().get("user-agent").contains(CONNECTOR_RUNTIME_PROVIDER_DEFAULT)); + } + /** * Utility method for spoofing SinkRecords that should be passed to SinkTask.put() * @param topic The topic of the record. diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index b75653d78..6cfd7324e 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -33,6 +33,8 @@ import java.util.Map; import java.util.Optional; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_CONFIG; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_DEFAULT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -214,4 +216,20 @@ public void testInvalidTimePartitioningType() { configProperties.put(BigQuerySinkConfig.TIME_PARTITIONING_TYPE_CONFIG, "fortnight"); new BigQuerySinkConfig(configProperties); } + + @Test + public void testKafkaProviderConfigDefaultValue() { + Map configProperties = propertiesFactory.getProperties(); + BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); + assertEquals(CONNECTOR_RUNTIME_PROVIDER_DEFAULT, config.getString(CONNECTOR_RUNTIME_PROVIDER_CONFIG)); + } + + @Test + public void testKafkaProviderConfig() { + Map configProperties = propertiesFactory.getProperties(); + String testKafkaProvider = "testProvider"; + configProperties.put(CONNECTOR_RUNTIME_PROVIDER_CONFIG, testKafkaProvider); + BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); + assertEquals(testKafkaProvider, config.getString(CONNECTOR_RUNTIME_PROVIDER_CONFIG)); + } } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java index 4fe2cbcf2..e27b6e5c3 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BaseConnectorIT.java @@ -148,6 +148,7 @@ protected BigQuery newBigQuery() { .withKey(keyFile()) .withKeySource(GcpClientBuilder.KeySource.valueOf(keySource())) .withProject(project()) + .withUserAgent("ITTest-user-agent") .build(); } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java index 875d85bce..d96273ca5 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriterTest.java @@ -275,7 +275,7 @@ public void testBigQueryCompleteFailure() { testTask.put(sinkRecordList); Exception expectedEx = assertThrows(BigQueryConnectException.class, () -> testTask.flush(Collections.emptyMap())); - assertTrue(expectedEx.getMessage().contains("test_topic")); + assertTrue(expectedEx.getCause().getMessage().contains("test_topic")); } /** * Utility method for making and retrieving properties based on provided parameters. From a7730c3732f277a1b79afb6e489fc912aa793ccd Mon Sep 17 00:00:00 2001 From: ypmahajan Date: Wed, 4 May 2022 11:39:14 +0530 Subject: [PATCH 156/190] [maven-release-plugin] prepare branch 2.3.x --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1dc72c983..ae544bc5c 100644 --- a/pom.xml +++ b/pom.xml @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + 2.3.x From 0b78030d61504198619197238fa02ab906f165e7 Mon Sep 17 00:00:00 2001 From: ypmahajan Date: Wed, 4 May 2022 11:39:21 +0530 Subject: [PATCH 157/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 116f7a382..9e7b43608 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0-SNAPSHOT + 2.4.0-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 430285925..d438102b1 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0-SNAPSHOT + 2.4.0-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index ae544bc5c..541d544b1 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0-SNAPSHOT + 2.4.0-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.3.x + HEAD From 8f37b50f962324d3888d0b8a111f217a231b7309 Mon Sep 17 00:00:00 2001 From: Yash Mahajan Date: Wed, 4 May 2022 13:25:47 +0000 Subject: [PATCH 158/190] [maven-release-plugin] prepare release v2.3.0 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 116f7a382..92e97b0db 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0-SNAPSHOT + 2.3.0 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 430285925..3a4cc5a98 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0-SNAPSHOT + 2.3.0 .. diff --git a/pom.xml b/pom.xml index ae544bc5c..e927929ea 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0-SNAPSHOT + 2.3.0 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.3.x + v2.3.0 From 03858c71f50b30daa42630150a498d17b5dbff8e Mon Sep 17 00:00:00 2001 From: Yash Mahajan Date: Wed, 4 May 2022 13:25:49 +0000 Subject: [PATCH 159/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 92e97b0db..82399624d 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0 + 2.3.1-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 3a4cc5a98..b4b463c64 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0 + 2.3.1-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index e927929ea..11347aa8f 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.0 + 2.3.1-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.3.0 + 2.3.x From a74abf44ce7e34119d21b5c10a8b85ec539146f9 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Wed, 18 May 2022 16:34:24 -0700 Subject: [PATCH 160/190] Retry on IOExceptions thrown during table writes (#183) Signed-off-by: Greg Harris Co-authored-by: Rikus Grobler --- .../bigquery/write/batch/TableWriter.java | 5 ++++- .../write/row/BigQueryErrorResponses.java | 8 +++++++- .../bigquery/write/row/BigQueryWriter.java | 3 +++ .../integration/BigQueryErrorResponsesIT.java | 19 +++++++++++++++++-- .../src/test/resources/log4j.properties | 3 ++- 5 files changed, 33 insertions(+), 5 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java index fe390650f..104488c4e 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriter.java @@ -94,7 +94,10 @@ public void run() { currentIndex += currentBatchSize; successCount++; } catch (BigQueryException err) { - logger.warn("Could not write batch of size {} to BigQuery.", currentBatchList.size(), err); + logger.warn( + "Could not write batch of size {} to BigQuery. " + + "Error code: {}, underlying error (if present): {}", + currentBatchList.size(), err.getCode(), err.getError(), err); if (isBatchSizeError(err)) { failureCount++; currentBatchSize = getNewBatchSize(currentBatchSize, err); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java index fd79d8b05..433821b59 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java @@ -22,6 +22,7 @@ import com.google.cloud.bigquery.BigQueryError; import com.google.cloud.bigquery.BigQueryException; +import java.io.IOException; import java.util.Optional; import java.util.function.Function; @@ -40,7 +41,7 @@ public class BigQueryErrorResponses { private static final int SERVICE_UNAVAILABLE_CODE = 503; private static final String BAD_REQUEST_REASON = "badRequest"; - private static final String INVALID_REASON = "invalid"; + private static final String INVALID_REASON = "invalid"; private static final String NOT_FOUND_REASON = "notFound"; private static final String QUOTA_EXCEEDED_REASON = "quotaExceeded"; private static final String RATE_LIMIT_EXCEEDED_REASON = "rateLimitExceeded"; @@ -107,6 +108,11 @@ public static boolean isTooManyRowsError(BigQueryException exception) { && message(exception.getError()).startsWith("too many rows present in the request"); } + public static boolean isIOError(BigQueryException error) { + return BigQueryException.UNKNOWN_CODE == error.getCode() + && error.getCause() instanceof IOException; + } + public static boolean isUnrecognizedFieldError(BigQueryError error) { return INVALID_REASON.equals(reason(error)) && message(error).startsWith("no such field: "); diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java index 032ad62d4..4af870447 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryWriter.java @@ -135,6 +135,9 @@ public void writeRows(PartitionedTableId table, } else if (BigQueryErrorResponses.isRateLimitExceededError(err)) { logger.warn("Rate limit exceeded for table {}, attempting retry", table); retryCount++; + } else if (BigQueryErrorResponses.isIOError(err)){ + logger.warn("IO Exception: {}, attempting retry", err.getCause().getMessage()); + retryCount++; } else { throw err; } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java index 7bacb2e5b..17ccd89f4 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/integration/BigQueryErrorResponsesIT.java @@ -109,6 +109,21 @@ public void testWriteToRecreatedTable() throws Exception { ); logger.info("Deleted {} successfully", table(table)); + TestUtils.waitForCondition( + () -> { + // Try to write to it... + try { + bigQuery.insertAll(InsertAllRequest.of(table, RowToInsert.of(Collections.singletonMap("f1", "v1")))); + return false; + } catch (BigQueryException e) { + logger.debug("Deleted table write error", e); + return BigQueryErrorResponses.isNonExistentTableError(e); + } + }, + 60_000L, + "Never failed to write to just-deleted table" + ); + // Recreate it... bigQuery.create(TableInfo.newBuilder(table, StandardTableDefinition.of(schema)).build()); @@ -120,11 +135,11 @@ public void testWriteToRecreatedTable() throws Exception { return true; } catch (BigQueryException e) { logger.debug("Recreated table write error", e); - return BigQueryErrorResponses.isNonExistentTableError(e); + return false; } }, 60_000L, - "Never failed to write to just-recreated table" + "Never succeeded to write to just-recreated table" ); } diff --git a/kcbq-connector/src/test/resources/log4j.properties b/kcbq-connector/src/test/resources/log4j.properties index 6de3c0539..bb50dc2fd 100644 --- a/kcbq-connector/src/test/resources/log4j.properties +++ b/kcbq-connector/src/test/resources/log4j.properties @@ -36,4 +36,5 @@ log4j.logger.org.reflections=ERROR # then written to shortly after. No need for that much noise during routine tests log4j.logger.com.wepay.kafka.connect.bigquery.write.batch.TableWriter=ERROR # Logs a message at INFO on every http request -log4j.logger.org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster=WARN \ No newline at end of file +log4j.logger.org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster=WARN +log4j.logger.com.wepay.kafka.connect.bigquery.integration.BigQueryErrorResponsesIT=DEBUG From 409c3fafa53df0227b8f8f61df48274ff49cf5bb Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Fri, 20 May 2022 15:08:37 -0700 Subject: [PATCH 161/190] Retry on IOExceptions thrown during getTable called in put() (#184) Signed-off-by: Greg Harris --- .../connect/bigquery/BigQuerySinkTask.java | 18 ++++++++++- .../bigquery/BigQuerySinkTaskTest.java | 30 +++++++++++++++++++ 2 files changed, 47 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 0ab51773b..d7ed675dc 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -20,6 +20,7 @@ package com.wepay.kafka.connect.bigquery; import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryException; import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; import com.google.cloud.bigquery.StandardTableDefinition; import com.google.cloud.bigquery.Table; @@ -45,14 +46,17 @@ import com.wepay.kafka.connect.bigquery.write.batch.TableWriter; import com.wepay.kafka.connect.bigquery.write.batch.TableWriterBuilder; import com.wepay.kafka.connect.bigquery.write.row.AdaptiveBigQueryWriter; +import com.wepay.kafka.connect.bigquery.write.row.BigQueryErrorResponses; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; import com.wepay.kafka.connect.bigquery.write.row.GCSToBQWriter; import com.wepay.kafka.connect.bigquery.write.row.SimpleBigQueryWriter; import com.wepay.kafka.connect.bigquery.write.row.UpsertDeleteBigQueryWriter; +import java.io.IOException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; import org.slf4j.Logger; @@ -335,7 +339,19 @@ private void setTimePartitioning(TableId table, PartitionedTableId.Builder build } private Table retrieveCachedTable(TableId tableId) { - return getCache().computeIfAbsent(tableId, k -> getBigQuery().getTable(tableId)); + return getCache().computeIfAbsent(tableId, this::retrieveTable); + } + + private Table retrieveTable(TableId tableId) { + try { + return getBigQuery().getTable(tableId); + } catch (BigQueryException e) { + if (BigQueryErrorResponses.isIOError(e)) { + throw new RetriableException("Failed to retrieve information for table " + tableId, e); + } else { + throw e; + } + } } private BigQuery newBigQuery() { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index f7a91fb38..29360ef5d 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -48,6 +48,7 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; +import java.net.SocketTimeoutException; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; @@ -55,6 +56,7 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; @@ -512,6 +514,34 @@ public void testFlushAfterStop() { () -> testTask.flush(Collections.emptyMap())); } + @Test(expected = RetriableException.class) + public void testBigQueryReadTimeout() { + final String topic = "test_topic"; + final String dataset = "scratch"; + + Map properties = propertiesFactory.getProperties(); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG, "3"); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG, "2000"); + properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); + properties.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, dataset); + + BigQuery bigQuery = mock(BigQuery.class); + when(bigQuery.getTable(any())).thenThrow(new BigQueryException(new SocketTimeoutException("mock timeout"))); + + Storage storage = mock(Storage.class); + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + + SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); + SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = new HashMap<>(); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); + testTask.initialize(sinkTaskContext); + testTask.start(properties); + testTask.put(Collections.singletonList(spoofSinkRecord(topic))); + testTask.flush(Collections.emptyMap()); + } + @Test public void testBigQuery5XXRetry() { final String topic = "test_topic"; From 03dd14a66bb1ad98fb3259c71eae69cfce1da1fb Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 15:12:21 -0700 Subject: [PATCH 162/190] Retry on IOExceptions thrown during getTable called in put() (#184) Manual Merge (#208) Signed-off-by: Greg Harris --- .../connect/bigquery/BigQuerySinkTask.java | 18 ++++++++++- .../bigquery/BigQuerySinkTaskTest.java | 31 +++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index e00c3e843..51fcaaa03 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -20,6 +20,7 @@ package com.wepay.kafka.connect.bigquery; import com.google.cloud.bigquery.BigQuery; +import com.google.cloud.bigquery.BigQueryException; import com.google.cloud.bigquery.InsertAllRequest.RowToInsert; import com.google.cloud.bigquery.StandardTableDefinition; import com.google.cloud.bigquery.Table; @@ -45,14 +46,17 @@ import com.wepay.kafka.connect.bigquery.write.batch.TableWriter; import com.wepay.kafka.connect.bigquery.write.batch.TableWriterBuilder; import com.wepay.kafka.connect.bigquery.write.row.AdaptiveBigQueryWriter; +import com.wepay.kafka.connect.bigquery.write.row.BigQueryErrorResponses; import com.wepay.kafka.connect.bigquery.write.row.BigQueryWriter; import com.wepay.kafka.connect.bigquery.write.row.GCSToBQWriter; import com.wepay.kafka.connect.bigquery.write.row.SimpleBigQueryWriter; import com.wepay.kafka.connect.bigquery.write.row.UpsertDeleteBigQueryWriter; +import java.io.IOException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; import org.slf4j.Logger; @@ -335,7 +339,19 @@ private void setTimePartitioning(TableId table, PartitionedTableId.Builder build } private Table retrieveCachedTable(TableId tableId) { - return getCache().computeIfAbsent(tableId, k -> getBigQuery().getTable(tableId)); + return getCache().computeIfAbsent(tableId, this::retrieveTable); + } + + private Table retrieveTable(TableId tableId) { + try { + return getBigQuery().getTable(tableId); + } catch (BigQueryException e) { + if (BigQueryErrorResponses.isIOError(e)) { + throw new RetriableException("Failed to retrieve information for table " + tableId, e); + } else { + throw e; + } + } } private BigQuery newBigQuery() { diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 462a661e1..9b240f35f 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -50,12 +50,15 @@ import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig; import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException; import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches; +import java.net.SocketTimeoutException; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; @@ -513,6 +516,34 @@ public void testFlushAfterStop() { () -> testTask.flush(Collections.emptyMap())); } + @Test(expected = RetriableException.class) + public void testBigQueryReadTimeout() { + final String topic = "test_topic"; + final String dataset = "scratch"; + + Map properties = propertiesFactory.getProperties(); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_CONFIG, "3"); + properties.put(BigQuerySinkConfig.BIGQUERY_RETRY_WAIT_CONFIG, "2000"); + properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); + properties.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, dataset); + + BigQuery bigQuery = mock(BigQuery.class); + when(bigQuery.getTable(any())).thenThrow(new BigQueryException(new SocketTimeoutException("mock timeout"))); + + Storage storage = mock(Storage.class); + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + + SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); + SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = new HashMap<>(); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); + testTask.initialize(sinkTaskContext); + testTask.start(properties); + testTask.put(Collections.singletonList(spoofSinkRecord(topic))); + testTask.flush(Collections.emptyMap()); + } + @Test public void testBigQuery5XXRetry() { final String topic = "test_topic"; From 8ff2e125afdb958a70d4ed92e6f0fc36991058c4 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 22:19:38 +0000 Subject: [PATCH 163/190] [maven-release-plugin] prepare release v2.0.14 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 549a91551..f6b922c8d 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.14-SNAPSHOT + 2.0.14 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index a090b79b2..06ab08f00 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.14-SNAPSHOT + 2.0.14 .. diff --git a/pom.xml b/pom.xml index c4f420d89..866696c7f 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.14-SNAPSHOT + 2.0.14 pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + v2.0.14 From a7384ff6d15395fc29ecb4486355191114264488 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 22:19:40 +0000 Subject: [PATCH 164/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index f6b922c8d..8c7dee09e 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.14 + 2.0.15-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 06ab08f00..0062fca67 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.0.14 + 2.0.15-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 866696c7f..a68a1077c 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.0.14 + 2.0.15-SNAPSHOT pom @@ -79,7 +79,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.0.14 + HEAD From 92edaf0c70213bd28aafed0d599e0ae62a81e310 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 22:22:16 +0000 Subject: [PATCH 165/190] [maven-release-plugin] prepare release v2.1.13 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 3a27fd7ab..38cab3e82 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.13-SNAPSHOT + 2.1.13 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 867378ede..ca3f392d0 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.13-SNAPSHOT + 2.1.13 .. diff --git a/pom.xml b/pom.xml index 51345b9b8..d70e36b01 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.13-SNAPSHOT + 2.1.13 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.1.x + v2.1.13 From b7ba07767547fe02660cfc784b01f7d89f5b6512 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 22:22:19 +0000 Subject: [PATCH 166/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 38cab3e82..e25c78cbb 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.13 + 2.1.14-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index ca3f392d0..57a674239 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.1.13 + 2.1.14-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index d70e36b01..cf8aa439f 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.1.13 + 2.1.14-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.1.13 + 2.1.x From 6c8737b6f2635730d5c40b9ecb2ef6342ea9d0f8 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 22:24:54 +0000 Subject: [PATCH 167/190] [maven-release-plugin] prepare release v2.2.3 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 54c706137..f31348044 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.3-SNAPSHOT + 2.2.3 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 8b496a3a4..eb9b69044 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.3-SNAPSHOT + 2.2.3 .. diff --git a/pom.xml b/pom.xml index ec55b1f9e..9bb5729c4 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.3-SNAPSHOT + 2.2.3 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.2.x + v2.2.3 From 7f37cd3b0c1c47ac08fe278a4ee6c9944c301924 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 22:24:56 +0000 Subject: [PATCH 168/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index f31348044..3d5a63415 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.3 + 2.2.4-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index eb9b69044..01b64085c 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.2.3 + 2.2.4-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 9bb5729c4..ecd1a32cb 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.2.3 + 2.2.4-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.2.3 + 2.2.x From f6abce7be3af8ac01f95207f8588d2e4f7076425 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 22:28:42 +0000 Subject: [PATCH 169/190] [maven-release-plugin] prepare release v2.3.1 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 82399624d..c26b9dff1 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.1-SNAPSHOT + 2.3.1 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index b4b463c64..fe1b4ba4a 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.1-SNAPSHOT + 2.3.1 .. diff --git a/pom.xml b/pom.xml index 11347aa8f..6d9a90a47 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.1-SNAPSHOT + 2.3.1 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.3.x + v2.3.1 From 199116e7bc8f6c400d6fce9918cb1119853c6831 Mon Sep 17 00:00:00 2001 From: Greg Harris Date: Mon, 23 May 2022 22:28:44 +0000 Subject: [PATCH 170/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index c26b9dff1..b84cb21d4 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.1 + 2.3.2-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index fe1b4ba4a..1b69190cf 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.1 + 2.3.2-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 6d9a90a47..20df0d3dd 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.1 + 2.3.2-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.3.1 + 2.3.x From 7280e8f2974946123f71184f8d107fe2b1365655 Mon Sep 17 00:00:00 2001 From: Aakash Shah Date: Thu, 9 Jun 2022 14:28:36 -0700 Subject: [PATCH 171/190] RCCA-7127: Adjust error identification to include multiple missing fields (#207) * RCCA-7127: Adjust error identification to include multiple missing fields Signed-off-by: Aakash Shah --- .../connect/bigquery/write/row/BigQueryErrorResponses.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java index 433821b59..fc1635401 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/BigQueryErrorResponses.java @@ -120,7 +120,7 @@ public static boolean isUnrecognizedFieldError(BigQueryError error) { public static boolean isMissingRequiredFieldError(BigQueryError error) { return INVALID_REASON.equals(reason(error)) - && message(error).startsWith("Missing required field: "); + && message(error).startsWith("Missing required field"); } public static boolean isStoppedError(BigQueryError error) { From 1f5abaf222971fb50df573fefb52b1df33913cd8 Mon Sep 17 00:00:00 2001 From: Yash Mahajan Date: Tue, 14 Jun 2022 13:33:49 +0000 Subject: [PATCH 172/190] [maven-release-plugin] prepare release v2.3.2 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index b84cb21d4..3188603a4 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.2-SNAPSHOT + 2.3.2 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 1b69190cf..1f136a682 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.2-SNAPSHOT + 2.3.2 .. diff --git a/pom.xml b/pom.xml index 20df0d3dd..7938a5508 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.2-SNAPSHOT + 2.3.2 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.3.x + v2.3.2 From f5c9e9a6134e74bb1541697c0ce63af8eda76faf Mon Sep 17 00:00:00 2001 From: Yash Mahajan Date: Tue, 14 Jun 2022 13:33:53 +0000 Subject: [PATCH 173/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 3188603a4..a1d7eeaa2 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.2 + 2.3.3-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 1f136a682..94958d37a 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.2 + 2.3.3-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 7938a5508..c6e7b5875 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.2 + 2.3.3-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.3.2 + 2.3.x From 56aa0cf7ce9c3909bcc3bf9284408f873d5c7f1f Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Thu, 4 Aug 2022 10:22:34 +0530 Subject: [PATCH 174/190] CCDB-4929: Fixing "Code scanning"/CodeQL warnings. (#229) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 1ddc1dd7a..b24ffcb6d 100644 --- a/pom.xml +++ b/pom.xml @@ -104,7 +104,7 @@ confluent - http://packages.confluent.io/maven/ + https://packages.confluent.io/maven/ jcenter @@ -115,7 +115,7 @@ confluent - http://packages.confluent.io/maven/ + https://packages.confluent.io/maven/ jcenter From dd4a8a9e4e5ef9c845b85df5912a9202bff56320 Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Tue, 9 Aug 2022 22:50:13 +0530 Subject: [PATCH 175/190] CCDB-4929: Adding error messages for union operation failure cases. (#225) CCDB-4929: Adding error messages for union operation failure cases. --- README.md | 20 +++++++++++++++++++ .../kafka/connect/bigquery/SchemaManager.java | 8 ++++++-- 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 8fc97d8e8..684b8a719 100644 --- a/README.md +++ b/README.md @@ -149,6 +149,26 @@ The `$KCBQ_TEST_FOLDER` variable can be supplied to specify which subfolder of t be used when testing the GCS batch loading feature; if not supplied, the top-level folder will be used. +### Adding new GCP Credentials & BigQuery DataSet +This section is optional in case one wants to use a different GCP project and generate new creds for that +- **Create a GCP Service Account:** Follow instructions from https://cloud.google.com/iam/docs/creating-managing-service-accounts e.g. +``` +gcloud iam service-accounts create kcbq-test --description="service account key for bigquery sink integration test" --display-name="kcbq-test" +``` +- **Create Service Account Keys:** Follow instructions from https://cloud.google.com/iam/docs/creating-managing-service-account-keys e.g. +``` +gcloud iam service-accounts keys create /tmp/creds.json --iam-account=kcbq-test@.iam.gserviceaccount.com +``` +- **Give BigQuery & Storage Admin Permissions to Service Account:** + - Open https://console.cloud.google.com/iam-admin/iam?project= + - Click on Add and enter New Principal as created above e.g. `kcbq-test@.iam.gserviceaccount.com` + - Add following 2 roles from "Select a role" drop down menu: + - BigQuery -> BigQuery Admin + - Cloud Storage -> Storage Admin +- **Add a BigQuery DataSet into the Project:** + - Open https://console.cloud.google.com/bigquery?project= + - Click on the 3 vertical dots against the project name and click on "Create dataset" and follow the steps there. + ### Running the Integration Tests ```bash diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java index 4a083c8be..015ff13ea 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/SchemaManager.java @@ -382,8 +382,12 @@ private Field unionizeFields(Field firstField, Field secondField) { } } - checkState(firstField.getName().equals(secondField.getName())); - checkState(firstField.getType() == secondField.getType()); + checkState(firstField.getName().equals(secondField.getName()), + String.format("Cannot perform union operation on two fields having different names. " + + "Field names are '%s' and '%s'.", firstField.getName(), secondField.getName())); + checkState(firstField.getType() == secondField.getType(), + String.format("Cannot perform union operation on two fields having different datatypes. " + + "Field name is '%s' and datatypes are '%s' and '%s'.", firstField.getName(), firstField.getType(), secondField.getType())); Field.Builder retBuilder = firstField.toBuilder(); if (isFieldRelaxation(firstField, secondField)) { From b9bc6e9a3726133a5ab44a3a232567c2e6bb6d33 Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Thu, 25 Aug 2022 03:23:30 +0000 Subject: [PATCH 176/190] [maven-release-plugin] prepare release v2.3.3 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index a1d7eeaa2..0da9d7c2f 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.3-SNAPSHOT + 2.3.3 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 94958d37a..2284ff295 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.3-SNAPSHOT + 2.3.3 .. diff --git a/pom.xml b/pom.xml index c6e7b5875..1699cbb23 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.3-SNAPSHOT + 2.3.3 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.3.x + v2.3.3 From 7d8a1806cd26d274030a0c8e7599970fd5270468 Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Thu, 25 Aug 2022 03:23:35 +0000 Subject: [PATCH 177/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 0da9d7c2f..894f01a7f 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.3 + 2.3.4-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 2284ff295..5055a5992 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.3 + 2.3.4-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 1699cbb23..e333a323b 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.3 + 2.3.4-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.3.3 + 2.3.x From 898f2396d3edcafaa88aac4ddbd66e16b9e6f231 Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Tue, 30 Aug 2022 09:42:01 +0530 Subject: [PATCH 178/190] Attempting to fix these CVEs: CCDB-4936, CCDB-4836, CCDB-4835. (#216) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 541d544b1..d6269ca01 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ 5.5.1 0.6.1 0.21.1 - 1.119.0 + 2.10.9 1.113.4 2.10.2 2.5.0 From a9243275f00b54a2133154ad0e0b55319ed94438 Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Wed, 31 Aug 2022 14:36:07 +0530 Subject: [PATCH 179/190] Fixing CVEs: CCDB-4936, CCDB-4836 & CCDB-4835. (#231) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e333a323b..758c9f723 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ 5.5.1 0.6.1 0.21.1 - 1.119.0 + 2.10.9 1.113.4 2.10.2 2.5.0 From 103c0598ddc27da21a026dd8b64296fea97ddc2e Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Wed, 31 Aug 2022 09:29:22 +0000 Subject: [PATCH 180/190] [maven-release-plugin] prepare release v2.3.4 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 894f01a7f..5310c4242 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.4-SNAPSHOT + 2.3.4 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 5055a5992..f29dcc45b 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.4-SNAPSHOT + 2.3.4 .. diff --git a/pom.xml b/pom.xml index 758c9f723..173e4823d 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.4-SNAPSHOT + 2.3.4 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.3.x + v2.3.4 From 54265fd208f7198304a774fa22bd1dc7c7c9f53b Mon Sep 17 00:00:00 2001 From: Manasjyoti Sharma Date: Wed, 31 Aug 2022 09:29:28 +0000 Subject: [PATCH 181/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 5310c4242..ffd8fe164 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.4 + 2.3.5-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index f29dcc45b..419e8a823 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.3.4 + 2.3.5-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index 173e4823d..366b9452b 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.3.4 + 2.3.5-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.3.4 + 2.3.x From fc858dcecafd3381a29e135db1e00bb696965e7a Mon Sep 17 00:00:00 2001 From: Sagar Rao Date: Tue, 30 Aug 2022 15:32:39 +0530 Subject: [PATCH 182/190] CCDB-5048: Adding topic2Table map support initial commit --- .../connect/bigquery/BigQuerySinkTask.java | 124 ++++++++++++++---- .../bigquery/config/BigQuerySinkConfig.java | 14 ++ 2 files changed, 115 insertions(+), 23 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 51fcaaa03..b539901d5 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -54,6 +54,7 @@ import java.io.IOException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.RetriableException; @@ -63,13 +64,7 @@ import org.slf4j.LoggerFactory; import java.time.Instant; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; +import java.util.*; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; @@ -77,6 +72,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG; import static com.wepay.kafka.connect.bigquery.utils.TableNameUtils.intTable; /** @@ -115,6 +111,7 @@ public class BigQuerySinkTask extends SinkTask { private ScheduledExecutorService loadExecutor; private Map cache; + private Map topic2TableMap; /** * Create a new BigquerySinkTask. @@ -183,25 +180,42 @@ public Map preCommit(Map: format " - + "or just the format.", - record.topic() - )); - } + String[] smtReplacement = record.topic().split(":"); - if (sanitize) { - tableName = FieldNameSanitizer.sanitizeName(tableName); + if (smtReplacement.length == 2) { + dataset = smtReplacement[0]; + tableName = smtReplacement[1]; + } else if (smtReplacement.length == 1) { + tableName = smtReplacement[0]; + } else { + throw new ConnectException(String.format( + "Incorrect regex replacement format in topic name '%s'. " + + "SMT replacement should either produce the : format " + + "or just the format.", + record.topic() + )); + } + if (sanitize) { + tableName = FieldNameSanitizer.sanitizeName(tableName); + } } + + // TODO: Order of execution of topic/table name modifications => + // regex router SMT modifies topic name in sinkrecord. + // It could be either : separated or not. + + // should we use topic2table map with sanitize table name? doesn't make sense. + + // we use table name from above to sanitize table name further. + + TableId baseTableId = TableId.of(dataset, tableName); if (upsertDelete) { TableId intermediateTableId = mergeBatches.intermediateTableFor(baseTableId); @@ -492,6 +506,7 @@ public void start(Map properties) { } recordConverter = getConverter(config); + topic2TableMap = parseTopic2TableMapConfig(config.getString(TOPIC2TABLE_MAP_CONFIG)); } private void startGCSToBQLoadTask() { @@ -521,6 +536,69 @@ private void startGCSToBQLoadTask() { loadExecutor.scheduleAtFixedRate(loadRunnable, intervalSec, intervalSec, TimeUnit.SECONDS); } + private Map parseTopic2TableMapConfig(String topic2TableMapString) { + if (topic2TableMapString.isEmpty()) { + return null; + } + + Map topic2TableMap = new HashMap<>(); + + for (String str : topic2TableMapString.split(",")) { + String[] tt = str.split(":"); + + if (tt.length != 2) { + throw new ConfigException( + TOPIC2TABLE_MAP_CONFIG, + topic2TableMapString, + "One of the topic to table mappings has an invalid format." + ); + } + + String topic = tt[0].trim(); + String table = tt[1].trim(); + + if (topic.isEmpty() || table.isEmpty()) { + throw new ConfigException( + TOPIC2TABLE_MAP_CONFIG, + topic2TableMapString, + "One of the topic to table mappings has an invalid format." + ); + } + + if (topic2TableMap.containsKey(topic)) { + throw new ConfigException( + TOPIC2TABLE_MAP_CONFIG, + topic2TableMapString, + String.format( + "The topic name %s is duplicated. Topic names cannot be duplicated.", + topic + ) + ); + } + + if (topic2TableMap.containsValue(table)) { + throw new ConfigException( + TOPIC2TABLE_MAP_CONFIG, + topic2TableMapString, + String.format( + "The table name %s is duplicated. Table names cannot be duplicated.", + table + ) + ); + } + if (sanitize) { + table = FieldNameSanitizer.sanitizeName(table); + } + topic2TableMap.put(topic, table); + } + + if (topic2TableMap.isEmpty()) { + return null; + } + return topic2TableMap; + } + + private void maybeStartMergeFlushTask() { long intervalMs = config.getLong(BigQuerySinkConfig.MERGE_INTERVAL_MS_CONFIG); if (intervalMs == -1) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index 67d570677..a3204af0b 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -162,6 +162,14 @@ public class BigQuerySinkConfig extends AbstractConfig { "Whether to automatically sanitize topic names before using them as table names;" + " if not enabled topic names will be used directly as table names"; + public static final String TOPIC2TABLE_MAP_CONFIG = "topic2TableMap"; + private static final ConfigDef.Type TOPIC2TABLE_MAP_TYPE = ConfigDef.Type.STRING; + public static final String TOPIC2TABLE_MAP_DEFAULT = ""; + private static final ConfigDef.Importance TOPIC2TABLE_MAP_IMPORTANCE = ConfigDef.Importance.LOW; + public static final String TOPIC2TABLE_MAP_DOC = "Map of topics to tables (optional). " + + "Format: comma-separated tuples, e.g. :,:,... " + + "Note that topic name should not be modified using regex SMT if using this option."; + public static final String SANITIZE_FIELD_NAME_CONFIG = "sanitizeFieldNames"; private static final ConfigDef.Type SANITIZE_FIELD_NAME_TYPE = ConfigDef.Type.BOOLEAN; public static final Boolean SANITIZE_FIELD_NAME_DEFAULT = false; @@ -555,6 +563,12 @@ public static ConfigDef getConfig() { SANITIZE_TOPICS_IMPORTANCE, SANITIZE_TOPICS_DOC ).define( + TOPIC2TABLE_MAP_CONFIG, + TOPIC2TABLE_MAP_TYPE, + TOPIC2TABLE_MAP_DEFAULT, + TOPIC2TABLE_MAP_IMPORTANCE, + TOPIC2TABLE_MAP_DOC + ).define( SANITIZE_FIELD_NAME_CONFIG, SANITIZE_FIELD_NAME_TYPE, SANITIZE_FIELD_NAME_DEFAULT, From 88edb042e6b5d6a48d945072351a4cf2077b3a95 Mon Sep 17 00:00:00 2001 From: Sagar Rao Date: Tue, 6 Sep 2022 11:26:10 +0530 Subject: [PATCH 183/190] CCDB-5048: Adding table2topic map config to BQ sink connector --- .../connect/bigquery/BigQuerySinkTask.java | 53 +---------- .../bigquery/config/BigQuerySinkConfig.java | 87 +++++++++++++++++-- .../config/BigQuerySinkConfigTest.java | 74 ++++++++++++++-- 3 files changed, 153 insertions(+), 61 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index b539901d5..a197d0ba7 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -506,7 +506,7 @@ public void start(Map properties) { } recordConverter = getConverter(config); - topic2TableMap = parseTopic2TableMapConfig(config.getString(TOPIC2TABLE_MAP_CONFIG)); + topic2TableMap = config.getTopic2TableMap(sanitize).orElse(null); } private void startGCSToBQLoadTask() { @@ -540,60 +540,13 @@ private Map parseTopic2TableMapConfig(String topic2TableMapStrin if (topic2TableMapString.isEmpty()) { return null; } - Map topic2TableMap = new HashMap<>(); - + // It's already validated, so we can just populate the map for (String str : topic2TableMapString.split(",")) { String[] tt = str.split(":"); - - if (tt.length != 2) { - throw new ConfigException( - TOPIC2TABLE_MAP_CONFIG, - topic2TableMapString, - "One of the topic to table mappings has an invalid format." - ); - } - String topic = tt[0].trim(); String table = tt[1].trim(); - - if (topic.isEmpty() || table.isEmpty()) { - throw new ConfigException( - TOPIC2TABLE_MAP_CONFIG, - topic2TableMapString, - "One of the topic to table mappings has an invalid format." - ); - } - - if (topic2TableMap.containsKey(topic)) { - throw new ConfigException( - TOPIC2TABLE_MAP_CONFIG, - topic2TableMapString, - String.format( - "The topic name %s is duplicated. Topic names cannot be duplicated.", - topic - ) - ); - } - - if (topic2TableMap.containsValue(table)) { - throw new ConfigException( - TOPIC2TABLE_MAP_CONFIG, - topic2TableMapString, - String.format( - "The table name %s is duplicated. Table names cannot be duplicated.", - table - ) - ); - } - if (sanitize) { - table = FieldNameSanitizer.sanitizeName(table); - } - topic2TableMap.put(topic, table); - } - - if (topic2TableMap.isEmpty()) { - return null; + topic2TableMap.put(topic, sanitize ? FieldNameSanitizer.sanitizeName(table) : table); } return topic2TableMap; } diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index a3204af0b..bde26cc00 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -28,6 +28,7 @@ import com.wepay.kafka.connect.bigquery.convert.RecordConverter; import com.wepay.kafka.connect.bigquery.convert.SchemaConverter; import com.wepay.kafka.connect.bigquery.retrieve.IdentitySchemaRetriever; +import com.wepay.kafka.connect.bigquery.utils.FieldNameSanitizer; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; @@ -40,11 +41,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; +import java.util.*; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -168,7 +165,65 @@ public class BigQuerySinkConfig extends AbstractConfig { private static final ConfigDef.Importance TOPIC2TABLE_MAP_IMPORTANCE = ConfigDef.Importance.LOW; public static final String TOPIC2TABLE_MAP_DOC = "Map of topics to tables (optional). " + "Format: comma-separated tuples, e.g. :,:,... " + - "Note that topic name should not be modified using regex SMT if using this option."; + "Note that topic name should not be modified using regex SMT while using this option." + + "Also note that if sanitizeTopics is true, then the table names supplied in this config " + + "would be sanitized according to the same rules"; + private static final ConfigDef.Validator TOPIC2TABLE_MAP_VALIDATOR = (name, value) -> { + String topic2TableMapString = (String) ConfigDef.parseType(name, value, TOPIC2TABLE_MAP_TYPE); + + if (topic2TableMapString.isEmpty()) { + return; + } + + Map topic2TableMap = new HashMap<>(); + + for (String str : topic2TableMapString.split(",")) { + String[] tt = str.split(":"); + + if (tt.length != 2) { + throw new ConfigException( + name, + topic2TableMapString, + "One of the topic to table mappings has an invalid format." + ); + } + + String topic = tt[0].trim(); + String table = tt[1].trim(); + + if (topic.isEmpty() || table.isEmpty()) { + throw new ConfigException( + name, + topic2TableMapString, + "One of the topic to table mappings has an invalid format." + ); + } + + if (topic2TableMap.containsKey(topic)) { + throw new ConfigException( + name, + name, + String.format( + "The topic name %s is duplicated. Topic names cannot be duplicated.", + topic + ) + ); + } + + if (topic2TableMap.containsValue(table)) { + throw new ConfigException( + name, + topic2TableMapString, + String.format( + "The table name %s is duplicated. Table names cannot be duplicated.", + table + ) + ); + } + topic2TableMap.put(topic, table); + } + }; + public static final String SANITIZE_FIELD_NAME_CONFIG = "sanitizeFieldNames"; private static final ConfigDef.Type SANITIZE_FIELD_NAME_TYPE = ConfigDef.Type.BOOLEAN; @@ -566,6 +621,7 @@ public static ConfigDef getConfig() { TOPIC2TABLE_MAP_CONFIG, TOPIC2TABLE_MAP_TYPE, TOPIC2TABLE_MAP_DEFAULT, + TOPIC2TABLE_MAP_VALIDATOR, TOPIC2TABLE_MAP_IMPORTANCE, TOPIC2TABLE_MAP_DOC ).define( @@ -954,6 +1010,10 @@ public Optional getTimePartitioningType() { return parseTimePartitioningType(getString(TIME_PARTITIONING_TYPE_CONFIG)); } + public Optional> getTopic2TableMap(boolean sanitize) { + return Optional.ofNullable(parseTopic2TableMapConfig(getString(TOPIC2TABLE_MAP_CONFIG), sanitize)); + } + private Optional parseTimePartitioningType(String rawPartitioningType) { if (rawPartitioningType == null) { throw new ConfigException(TIME_PARTITIONING_TYPE_CONFIG, @@ -975,6 +1035,21 @@ private Optional parseTimePartitioningType(String rawPart } } + private Map parseTopic2TableMapConfig(String topic2TableMapString, boolean sanitize) { + if (topic2TableMapString.isEmpty()) { + return null; + } + Map topic2TableMap = new HashMap<>(); + // It's already validated, so we can just populate the map + for (String str : topic2TableMapString.split(",")) { + String[] tt = str.split(":"); + String topic = tt[0].trim(); + String table = tt[1].trim(); + topic2TableMap.put(topic, sanitize ? FieldNameSanitizer.sanitizeName(table) : table); + } + return topic2TableMap.isEmpty() ? null : topic2TableMap; + } + /** * Returns the partition expiration in ms. * @return Long that represents the partition expiration. diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index 6cfd7324e..cab387e6b 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -27,11 +27,7 @@ import org.junit.Before; import org.junit.Test; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; +import java.util.*; import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_CONFIG; import static com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig.CONNECTOR_RUNTIME_PROVIDER_DEFAULT; @@ -232,4 +228,72 @@ public void testKafkaProviderConfig() { BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); assertEquals(testKafkaProvider, config.getString(CONNECTOR_RUNTIME_PROVIDER_CONFIG)); } + + @Test (expected = ConfigException.class) + public void testTopic2TableInvalidFormat() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, "topic:"); + new BigQuerySinkConfig(configProperties); + } + + @Test (expected = ConfigException.class) + public void testTopic2TableDuplicateTopic() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, "topic:table, topic:table2"); + new BigQuerySinkConfig(configProperties); + } + + @Test (expected = ConfigException.class) + public void testTopic2TableDuplicateTable() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, "topic:table, topic2:table"); + new BigQuerySinkConfig(configProperties); + } + + @Test (expected = ConfigException.class) + public void testTopic2TableSemicolonOnly() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, ":"); + new BigQuerySinkConfig(configProperties); + } + + @Test + public void testValidTopic2TableMap() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, "topic:table, topic2:table2"); + BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); + Map topic2TableMap = new HashMap<>(); + topic2TableMap.put("topic", "table"); + topic2TableMap.put("topic2", "table2"); + assertEquals(topic2TableMap, config.getTopic2TableMap(config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG)).get()); + } + + @Test + public void testTopic2TableEmptyString() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, ""); + BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); + assertFalse(config.getTopic2TableMap(config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG)).isPresent()); + } + + @Test + public void testTopic2TableCommaOnly() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, ","); + BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); + assertFalse(config.getTopic2TableMap(config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG)).isPresent()); + } + + @Test + public void testTopicNameShouldGetSanitizedIfSanitizeFlagTrue() { + Map configProperties = propertiesFactory.getProperties(); + configProperties.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); + configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, "topic:table#badname, topic2:2table2"); + BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); + Map topic2TableMap = new HashMap<>(); + topic2TableMap.put("topic", "table_badname"); + topic2TableMap.put("topic2", "_2table2"); + assertEquals(topic2TableMap, config.getTopic2TableMap(config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG)).get()); + } + } From c5509f721797cd227c72805a35e7db1ab7d4dac2 Mon Sep 17 00:00:00 2001 From: Sagar Rao Date: Fri, 9 Sep 2022 15:22:28 +0530 Subject: [PATCH 184/190] Review comments addressed --- .../connect/bigquery/BigQuerySinkTask.java | 2 +- .../bigquery/config/BigQuerySinkConfig.java | 11 +++++------ .../config/BigQuerySinkConfigTest.java | 18 +++--------------- 3 files changed, 9 insertions(+), 22 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index a197d0ba7..3edd09e17 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -506,7 +506,7 @@ public void start(Map properties) { } recordConverter = getConverter(config); - topic2TableMap = config.getTopic2TableMap(sanitize).orElse(null); + topic2TableMap = config.getTopic2TableMap().orElse(null); } private void startGCSToBQLoadTask() { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index bde26cc00..d9de8e626 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -166,8 +166,7 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String TOPIC2TABLE_MAP_DOC = "Map of topics to tables (optional). " + "Format: comma-separated tuples, e.g. :,:,... " + "Note that topic name should not be modified using regex SMT while using this option." + - "Also note that if sanitizeTopics is true, then the table names supplied in this config " + - "would be sanitized according to the same rules"; + "Also note that SANITIZE_TOPICS_CONFIG would be ignored if this config is set."; private static final ConfigDef.Validator TOPIC2TABLE_MAP_VALIDATOR = (name, value) -> { String topic2TableMapString = (String) ConfigDef.parseType(name, value, TOPIC2TABLE_MAP_TYPE); @@ -1010,8 +1009,8 @@ public Optional getTimePartitioningType() { return parseTimePartitioningType(getString(TIME_PARTITIONING_TYPE_CONFIG)); } - public Optional> getTopic2TableMap(boolean sanitize) { - return Optional.ofNullable(parseTopic2TableMapConfig(getString(TOPIC2TABLE_MAP_CONFIG), sanitize)); + public Optional> getTopic2TableMap() { + return Optional.ofNullable(parseTopic2TableMapConfig(getString(TOPIC2TABLE_MAP_CONFIG))); } private Optional parseTimePartitioningType(String rawPartitioningType) { @@ -1035,7 +1034,7 @@ private Optional parseTimePartitioningType(String rawPart } } - private Map parseTopic2TableMapConfig(String topic2TableMapString, boolean sanitize) { + private Map parseTopic2TableMapConfig(String topic2TableMapString) { if (topic2TableMapString.isEmpty()) { return null; } @@ -1045,7 +1044,7 @@ private Map parseTopic2TableMapConfig(String topic2TableMapStrin String[] tt = str.split(":"); String topic = tt[0].trim(); String table = tt[1].trim(); - topic2TableMap.put(topic, sanitize ? FieldNameSanitizer.sanitizeName(table) : table); + topic2TableMap.put(topic, table); } return topic2TableMap.isEmpty() ? null : topic2TableMap; } diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java index cab387e6b..cb61dd640 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfigTest.java @@ -265,7 +265,7 @@ public void testValidTopic2TableMap() { Map topic2TableMap = new HashMap<>(); topic2TableMap.put("topic", "table"); topic2TableMap.put("topic2", "table2"); - assertEquals(topic2TableMap, config.getTopic2TableMap(config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG)).get()); + assertEquals(topic2TableMap, config.getTopic2TableMap().get()); } @Test @@ -273,7 +273,7 @@ public void testTopic2TableEmptyString() { Map configProperties = propertiesFactory.getProperties(); configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, ""); BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); - assertFalse(config.getTopic2TableMap(config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG)).isPresent()); + assertFalse(config.getTopic2TableMap().isPresent()); } @Test @@ -281,19 +281,7 @@ public void testTopic2TableCommaOnly() { Map configProperties = propertiesFactory.getProperties(); configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, ","); BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); - assertFalse(config.getTopic2TableMap(config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG)).isPresent()); - } - - @Test - public void testTopicNameShouldGetSanitizedIfSanitizeFlagTrue() { - Map configProperties = propertiesFactory.getProperties(); - configProperties.put(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG, "true"); - configProperties.put(BigQuerySinkConfig.TOPIC2TABLE_MAP_CONFIG, "topic:table#badname, topic2:2table2"); - BigQuerySinkConfig config = new BigQuerySinkConfig(configProperties); - Map topic2TableMap = new HashMap<>(); - topic2TableMap.put("topic", "table_badname"); - topic2TableMap.put("topic2", "_2table2"); - assertEquals(topic2TableMap, config.getTopic2TableMap(config.getBoolean(BigQuerySinkConfig.SANITIZE_TOPICS_CONFIG)).get()); + assertFalse(config.getTopic2TableMap().isPresent()); } } From 80ab3f4f9f407d3db603972ae18264c47a5ece55 Mon Sep 17 00:00:00 2001 From: Sagar Rao Date: Tue, 13 Sep 2022 20:25:03 +0530 Subject: [PATCH 185/190] Review comments addressed --- .../connect/bigquery/BigQuerySinkTask.java | 22 +------------------ .../bigquery/config/BigQuerySinkConfig.java | 4 +++- 2 files changed, 4 insertions(+), 22 deletions(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 3edd09e17..7489607a8 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -181,11 +181,7 @@ private PartitionedTableId getRecordTable(SinkRecord record) { String tableName; String dataset = config.getString(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG); if (topic2TableMap != null) { - // TODO what if the map doesn't contain the topic name? - // We don't need an explicit call to sanitize here because if the - // sanitize flag is true, then the topic2TableMap would already contain - // sanitized table names. - tableName = topic2TableMap.get(record.topic()); + tableName = topic2TableMap.getOrDefault(record.topic(), record.topic()); } else { String[] smtReplacement = record.topic().split(":"); @@ -536,22 +532,6 @@ private void startGCSToBQLoadTask() { loadExecutor.scheduleAtFixedRate(loadRunnable, intervalSec, intervalSec, TimeUnit.SECONDS); } - private Map parseTopic2TableMapConfig(String topic2TableMapString) { - if (topic2TableMapString.isEmpty()) { - return null; - } - Map topic2TableMap = new HashMap<>(); - // It's already validated, so we can just populate the map - for (String str : topic2TableMapString.split(",")) { - String[] tt = str.split(":"); - String topic = tt[0].trim(); - String table = tt[1].trim(); - topic2TableMap.put(topic, sanitize ? FieldNameSanitizer.sanitizeName(table) : table); - } - return topic2TableMap; - } - - private void maybeStartMergeFlushTask() { long intervalMs = config.getLong(BigQuerySinkConfig.MERGE_INTERVAL_MS_CONFIG); if (intervalMs == -1) { diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java index d9de8e626..2c6ca01f9 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/config/BigQuerySinkConfig.java @@ -166,7 +166,9 @@ public class BigQuerySinkConfig extends AbstractConfig { public static final String TOPIC2TABLE_MAP_DOC = "Map of topics to tables (optional). " + "Format: comma-separated tuples, e.g. :,:,... " + "Note that topic name should not be modified using regex SMT while using this option." + - "Also note that SANITIZE_TOPICS_CONFIG would be ignored if this config is set."; + "Also note that SANITIZE_TOPICS_CONFIG would be ignored if this config is set." + + "Lastly, if the topic2table map doesn't contain the topic for a record, a table" + + " with the same name as the topic name would be created"; private static final ConfigDef.Validator TOPIC2TABLE_MAP_VALIDATOR = (name, value) -> { String topic2TableMapString = (String) ConfigDef.parseType(name, value, TOPIC2TABLE_MAP_TYPE); From 8eaa11ca9d7807949f7cb71083b27f938bed33bf Mon Sep 17 00:00:00 2001 From: Confluent Jenkins Bot Date: Mon, 19 Sep 2022 08:39:45 +0000 Subject: [PATCH 186/190] [maven-release-plugin] prepare branch 2.4.x --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d6269ca01..ca643dfc0 100644 --- a/pom.xml +++ b/pom.xml @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - HEAD + 2.4.x From 13fb4929a399f2488f4e4dd4ce27bd361e7b38b7 Mon Sep 17 00:00:00 2001 From: Confluent Jenkins Bot Date: Mon, 19 Sep 2022 08:39:50 +0000 Subject: [PATCH 187/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 9e7b43608..1b46a5291 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0-SNAPSHOT + 2.5.0-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index d438102b1..3d0b5978e 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0-SNAPSHOT + 2.5.0-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index ca643dfc0..402e32f0f 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0-SNAPSHOT + 2.5.0-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.4.x + HEAD From 9dd2fac6f4788e56b18e7547b95243d1861bbacd Mon Sep 17 00:00:00 2001 From: Sagar Rao Date: Mon, 19 Sep 2022 09:01:28 +0000 Subject: [PATCH 188/190] [maven-release-plugin] prepare release v2.4.0 --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 9e7b43608..35f66bef0 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0-SNAPSHOT + 2.4.0 .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index d438102b1..3319211cd 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0-SNAPSHOT + 2.4.0 .. diff --git a/pom.xml b/pom.xml index ca643dfc0..a49e23c6a 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0-SNAPSHOT + 2.4.0 pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - 2.4.x + v2.4.0 From 02667024ee7872d58b9f1f632cfc546a03ff3d73 Mon Sep 17 00:00:00 2001 From: Sagar Rao Date: Mon, 19 Sep 2022 09:01:33 +0000 Subject: [PATCH 189/190] [maven-release-plugin] prepare for next development iteration --- kcbq-api/pom.xml | 2 +- kcbq-connector/pom.xml | 2 +- pom.xml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kcbq-api/pom.xml b/kcbq-api/pom.xml index 35f66bef0..b14b5d244 100644 --- a/kcbq-api/pom.xml +++ b/kcbq-api/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0 + 2.4.1-SNAPSHOT .. diff --git a/kcbq-connector/pom.xml b/kcbq-connector/pom.xml index 3319211cd..f63ab1650 100644 --- a/kcbq-connector/pom.xml +++ b/kcbq-connector/pom.xml @@ -25,7 +25,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0 + 2.4.1-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index a49e23c6a..ea42338be 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ com.wepay.kcbq kcbq-parent - 2.4.0 + 2.4.1-SNAPSHOT pom @@ -81,7 +81,7 @@ scm:git:git://github.com/confluentinc/kafka-connect-bigquery.git scm:git:git@github.com:confluentinc/kafka-connect-bigquery.git https://github.com/confluentinc/kafka-connect-bigquery - v2.4.0 + 2.4.x From 70144318f4750940f6802f8238720315f319db1d Mon Sep 17 00:00:00 2001 From: Bhagyashree - Date: Fri, 9 Dec 2022 17:29:10 +0530 Subject: [PATCH 190/190] adding more uniqueness to filename --- .../connect/bigquery/BigQuerySinkTask.java | 3 +- .../bigquery/BigQuerySinkTaskTest.java | 41 +++++++++++++++++++ 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java index 7489607a8..39f9b59f5 100644 --- a/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java +++ b/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTask.java @@ -261,7 +261,8 @@ public void put(Collection records) { TableWriterBuilder tableWriterBuilder; if (config.getList(BigQuerySinkConfig.ENABLE_BATCH_CONFIG).contains(record.topic())) { String topic = record.topic(); - String gcsBlobName = topic + "_" + uuid + "_" + Instant.now().toEpochMilli(); + long offset = record.kafkaOffset(); + String gcsBlobName = topic + "_" + uuid + "_" + Instant.now().toEpochMilli() + "_" + offset; String gcsFolderName = config.getString(BigQuerySinkConfig.GCS_FOLDER_NAME_CONFIG); if (gcsFolderName != null && !"".equals(gcsFolderName)) { gcsBlobName = gcsFolderName + "/" + gcsBlobName; diff --git a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java index 9b240f35f..7c1faa609 100644 --- a/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java +++ b/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkTaskTest.java @@ -44,6 +44,7 @@ import com.google.cloud.bigquery.QueryJobConfiguration; import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TimePartitioning; +import com.google.cloud.storage.BlobInfo; import com.google.cloud.storage.Storage; import com.wepay.kafka.connect.bigquery.api.SchemaRetriever; @@ -77,6 +78,8 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class BigQuerySinkTaskTest { private static SinkTaskPropertiesFactory propertiesFactory; @@ -132,6 +135,44 @@ public void testSimplePut() { verify(bigQuery, times(1)).insertAll(any(InsertAllRequest.class)); } + @Test + public void testPutForGCSToBQ() { + final String topic = "test-topic"; + final int repeats = 20; + Map properties = propertiesFactory.getProperties(); + properties.put(BigQuerySinkConfig.TOPICS_CONFIG, topic); + properties.put(BigQuerySinkConfig.DEFAULT_DATASET_CONFIG, "scratch"); + properties.put(BigQuerySinkConfig.ENABLE_BATCH_CONFIG, "test-topic"); + + BigQuery bigQuery = mock(BigQuery.class); + Table mockTable = mock(Table.class); + when(bigQuery.getTable(any())).thenReturn(mockTable); + + Storage storage = mock(Storage.class); + + SinkTaskContext sinkTaskContext = mock(SinkTaskContext.class); + InsertAllResponse insertAllResponse = mock(InsertAllResponse.class); + + when(bigQuery.insertAll(anyObject())).thenReturn(insertAllResponse); + when(insertAllResponse.hasErrors()).thenReturn(false); + + SchemaRetriever schemaRetriever = mock(SchemaRetriever.class); + SchemaManager schemaManager = mock(SchemaManager.class); + Map cache = new HashMap<>(); + + BigQuerySinkTask testTask = new BigQuerySinkTask(bigQuery, schemaRetriever, storage, schemaManager, cache); + testTask.initialize(sinkTaskContext); + testTask.start(properties); + + IntStream.range(0,repeats).forEach(i -> testTask.put(Collections.singletonList(spoofSinkRecord(topic)))); + + ArgumentCaptor blobInfo = ArgumentCaptor.forClass(BlobInfo.class); + testTask.flush(Collections.emptyMap()); + + verify(storage, times(repeats)).create(blobInfo.capture(), (byte[])anyObject()); + assertEquals(repeats, blobInfo.getAllValues().stream().map(info -> info.getBlobId().getName()).collect(Collectors.toSet()).size()); + + } @Test public void testSimplePutWhenSchemaRetrieverIsNotNull() { final String topic = "test-topic";