currentBatch() {
42 | int size = Math.min(records.size() - batchStart, batchSize);
43 | return records.subList(batchStart, batchStart + size);
44 | }
45 |
46 | public void advanceToNextBatch() {
47 | batchStart += batchSize;
48 | }
49 |
50 | public void reduceBatchSize() {
51 | if (batchSize <= 1) {
52 | throw new IllegalStateException("Cannot reduce batch size any further");
53 | }
54 | batchSize /= 2;
55 | }
56 |
57 | public boolean completed() {
58 | return batchStart >= records.size();
59 | }
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/CountDownRunnable.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.batch;
25 |
26 | import java.util.concurrent.CountDownLatch;
27 | import org.apache.kafka.connect.errors.ConnectException;
28 |
29 | /**
30 | * A Runnable that counts down, and then waits for the countdown to be finished.
31 | */
32 | public class CountDownRunnable implements Runnable {
33 |
34 | private CountDownLatch countDownLatch;
35 |
36 | public CountDownRunnable(CountDownLatch countDownLatch) {
37 | this.countDownLatch = countDownLatch;
38 | }
39 |
40 | @Override
41 | public void run() {
42 | countDownLatch.countDown();
43 | try {
44 | /*
45 | * Hog this thread until ALL threads are finished counting down. This is needed so that
46 | * this thread doesn't start processing another countdown. If countdown tasks are holding onto
47 | * all the threads, then we know that nothing that went in before the countdown is still
48 | * processing.
49 | */
50 | countDownLatch.await();
51 | } catch (InterruptedException err) {
52 | throw new ConnectException("Thread interrupted while waiting for countdown.", err);
53 | }
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/KcbqThreadPoolExecutor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.batch;
25 |
26 | import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig;
27 | import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException;
28 | import com.wepay.kafka.connect.bigquery.exception.ExpectedInterruptException;
29 | import java.util.Optional;
30 | import java.util.concurrent.BlockingQueue;
31 | import java.util.concurrent.CountDownLatch;
32 | import java.util.concurrent.ThreadFactory;
33 | import java.util.concurrent.ThreadPoolExecutor;
34 | import java.util.concurrent.TimeUnit;
35 | import java.util.concurrent.atomic.AtomicReference;
36 | import org.slf4j.Logger;
37 | import org.slf4j.LoggerFactory;
38 |
39 | /**
40 | * ThreadPoolExecutor for writing Rows to BigQuery.
41 | *
42 | * Keeps track of the number of threads actively writing for each topic.
43 | * Keeps track of the number of failed threads in each batch of requests.
44 | */
45 | public class KcbqThreadPoolExecutor extends ThreadPoolExecutor {
46 |
47 | private static final Logger logger = LoggerFactory.getLogger(KcbqThreadPoolExecutor.class);
48 |
49 | private final AtomicReference encounteredError = new AtomicReference<>();
50 |
51 | /**
52 | * @param config the {@link BigQuerySinkTaskConfig}
53 | * @param workQueue the queue for storing tasks.
54 | */
55 | public KcbqThreadPoolExecutor(
56 | BigQuerySinkTaskConfig config,
57 | BlockingQueue workQueue,
58 | ThreadFactory threadFactory
59 | ) {
60 | super(
61 | config.getInt(BigQuerySinkTaskConfig.THREAD_POOL_SIZE_CONFIG),
62 | config.getInt(BigQuerySinkTaskConfig.THREAD_POOL_SIZE_CONFIG),
63 | // the following line is irrelevant because the core and max thread counts are the same.
64 | 1, TimeUnit.SECONDS,
65 | workQueue,
66 | threadFactory
67 | );
68 | }
69 |
70 | @Override
71 | protected void afterExecute(Runnable runnable, Throwable throwable) {
72 | super.afterExecute(runnable, throwable);
73 |
74 | if (throwable != null && !(throwable instanceof ExpectedInterruptException)) {
75 | // Log at debug level since this will be shown to the user at error level by the Connect framework if it causes
76 | // the task to fail, and will otherwise just pollute logs and potentially mislead users
77 | logger.debug("A write thread has failed with an unrecoverable error", throwable);
78 | encounteredError.compareAndSet(null, throwable);
79 | }
80 | }
81 |
82 | /**
83 | * Wait for all the currently queued tasks to complete, and then return.
84 | *
85 | * @throws BigQueryConnectException if any of the tasks failed.
86 | * @throws InterruptedException if interrupted while waiting.
87 | */
88 | public void awaitCurrentTasks() throws InterruptedException, BigQueryConnectException {
89 | /*
90 | * create CountDownRunnables equal to the number of threads in the pool and add them to the
91 | * queue. Then wait for all CountDownRunnables to complete. This way we can be sure that all
92 | * tasks added before this method was called are complete.
93 | */
94 | int maximumPoolSize = getMaximumPoolSize();
95 | CountDownLatch countDownLatch = new CountDownLatch(maximumPoolSize);
96 | for (int i = 0; i < maximumPoolSize; i++) {
97 | execute(new CountDownRunnable(countDownLatch));
98 | }
99 | countDownLatch.await();
100 | maybeThrowEncounteredError();
101 | }
102 |
103 | /**
104 | * Immediately throw an exception if any unrecoverable errors were encountered by any of the write
105 | * tasks.
106 | *
107 | * @throws BigQueryConnectException if any of the tasks failed.
108 | */
109 | public void maybeThrowEncounteredError() {
110 | Optional.ofNullable(encounteredError.get()).ifPresent(t -> {
111 | throw new BigQueryConnectException("A write thread has failed with an unrecoverable error", t);
112 | });
113 | }
114 | }
115 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/batch/TableWriterBuilder.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.batch;
25 |
26 | import com.google.cloud.bigquery.TableId;
27 | import org.apache.kafka.connect.sink.SinkRecord;
28 |
29 | /**
30 | * Interface for building a {@link TableWriter} or TableWriterGCS.
31 | */
32 | public interface TableWriterBuilder {
33 |
34 | /**
35 | * Add a record to the builder.
36 | *
37 | * @param sinkRecord the row to add.
38 | * @param table the table the row will be written to.
39 | */
40 | void addRow(SinkRecord sinkRecord, TableId table);
41 |
42 | /**
43 | * Create a {@link TableWriter} from this builder.
44 | *
45 | * @return a TableWriter containing the given writer, table, topic, and all added rows.
46 | */
47 | Runnable build();
48 | }
49 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/SimpleBigQueryWriter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.row;
25 |
26 | import com.google.cloud.bigquery.BigQuery;
27 | import com.google.cloud.bigquery.BigQueryError;
28 | import com.google.cloud.bigquery.InsertAllRequest;
29 | import com.google.cloud.bigquery.InsertAllResponse;
30 | import com.wepay.kafka.connect.bigquery.ErrantRecordHandler;
31 | import com.wepay.kafka.connect.bigquery.config.BigQuerySinkConfig;
32 | import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId;
33 | import com.wepay.kafka.connect.bigquery.utils.Time;
34 | import java.util.HashMap;
35 | import java.util.List;
36 | import java.util.Map;
37 | import java.util.SortedMap;
38 | import org.apache.kafka.connect.sink.SinkRecord;
39 | import org.slf4j.Logger;
40 | import org.slf4j.LoggerFactory;
41 |
42 | /**
43 | * A simple BigQueryWriter implementation. Sends the request to BigQuery, and throws an exception if
44 | * any errors occur as a result.
45 | */
46 | public class SimpleBigQueryWriter extends BigQueryWriter {
47 | private static final Logger logger = LoggerFactory.getLogger(SimpleBigQueryWriter.class);
48 |
49 | private final BigQuery bigQuery;
50 |
51 | /**
52 | * @param bigQuery The object used to send write requests to BigQuery.
53 | * @param retry How many retries to make in the event of a 500/503 error.
54 | * @param retryWait How long to wait in between retries.
55 | * @param errantRecordHandler Used to handle errant records
56 | * @param time used to wait during backoff periods
57 | */
58 | public SimpleBigQueryWriter(BigQuery bigQuery, int retry, long retryWait, ErrantRecordHandler errantRecordHandler, Time time) {
59 | super(retry, retryWait, errantRecordHandler, time);
60 | this.bigQuery = bigQuery;
61 | }
62 |
63 | /**
64 | * Sends the request to BigQuery, and return a map of insertErrors in case of partial failure.
65 | * Throws an exception if any other errors occur as a result of doing so.
66 | *
67 | * @see BigQueryWriter#performWriteRequest(PartitionedTableId, SortedMap)
68 | */
69 | @Override
70 | public Map> performWriteRequest(PartitionedTableId tableId,
71 | SortedMap rows) {
72 | InsertAllRequest request = createInsertAllRequest(tableId, rows.values());
73 | InsertAllResponse writeResponse = bigQuery.insertAll(request);
74 | if (writeResponse.hasErrors()) {
75 | logger.warn(
76 | "You may want to enable schema updates by specifying "
77 | + "{}=true or {}=true in the properties file",
78 | BigQuerySinkConfig.ALLOW_NEW_BIGQUERY_FIELDS_CONFIG, BigQuerySinkConfig.ALLOW_BIGQUERY_REQUIRED_FIELD_RELAXATION_CONFIG
79 | );
80 | return writeResponse.getInsertErrors();
81 | } else {
82 | logger.debug("table insertion completed with no reported errors");
83 | return new HashMap<>();
84 | }
85 | }
86 | }
87 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/row/UpsertDeleteBigQueryWriter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.row;
25 |
26 | import com.google.cloud.bigquery.BigQuery;
27 | import com.google.cloud.bigquery.BigQueryException;
28 | import com.google.cloud.bigquery.TableId;
29 | import com.wepay.kafka.connect.bigquery.ErrantRecordHandler;
30 | import com.wepay.kafka.connect.bigquery.SchemaManager;
31 | import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException;
32 | import com.wepay.kafka.connect.bigquery.utils.PartitionedTableId;
33 | import com.wepay.kafka.connect.bigquery.utils.Time;
34 | import java.util.List;
35 | import java.util.Map;
36 | import org.apache.kafka.connect.sink.SinkRecord;
37 |
38 | public class UpsertDeleteBigQueryWriter extends AdaptiveBigQueryWriter {
39 |
40 | private final SchemaManager schemaManager;
41 | private final boolean autoCreateTables;
42 | private final Map intermediateToDestinationTables;
43 |
44 | /**
45 | * @param bigQuery Used to send write requests to BigQuery.
46 | * @param schemaManager Used to update BigQuery tables.
47 | * @param retry How many retries to make in the event of a 500/503 error.
48 | * @param retryWait How long to wait in between retries.
49 | * @param autoCreateTables Whether destination tables should be automatically created
50 | * @param intermediateToDestinationTables A mapping used to determine the destination table for
51 | * given intermediate tables; used for create/update
52 | * operations in order to propagate them to the destination
53 | * table
54 | * @param errantRecordHandler Used to handle errant records
55 | * @param time used to wait during backoff periods
56 | */
57 | public UpsertDeleteBigQueryWriter(BigQuery bigQuery,
58 | SchemaManager schemaManager,
59 | int retry,
60 | long retryWait,
61 | boolean autoCreateTables,
62 | Map intermediateToDestinationTables,
63 | ErrantRecordHandler errantRecordHandler,
64 | Time time) {
65 | // Hardcode autoCreateTables to true in the superclass so that intermediate tables will be
66 | // automatically created
67 | // The super class will handle all of the logic for writing to, creating, and updating
68 | // intermediate tables; this class will handle logic for creating/updating the destination table
69 | super(bigQuery, schemaManager.forIntermediateTables(), retry, retryWait, true, errantRecordHandler, time);
70 | this.schemaManager = schemaManager;
71 | this.autoCreateTables = autoCreateTables;
72 | this.intermediateToDestinationTables = intermediateToDestinationTables;
73 | }
74 |
75 | @Override
76 | protected void attemptSchemaUpdate(PartitionedTableId tableId, List records) {
77 | // Update the intermediate table here...
78 | super.attemptSchemaUpdate(tableId, records);
79 | try {
80 | // ... and update the destination table here
81 | schemaManager.updateSchema(intermediateToDestinationTables.get(tableId.getBaseTableId()), records);
82 | } catch (BigQueryException exception) {
83 | throw new BigQueryConnectException(
84 | "Failed to update destination table schema for: " + tableId.getBaseTableId(), exception);
85 | }
86 | }
87 |
88 | @Override
89 | protected void attemptTableCreate(TableId tableId, List records) {
90 | // Create the intermediate table here...
91 | super.attemptTableCreate(tableId, records);
92 | if (autoCreateTables) {
93 | try {
94 | // ... and create or update the destination table here, if it doesn't already exist and auto
95 | // table creation is enabled
96 | schemaManager.createOrUpdateTable(intermediateToDestinationTables.get(tableId), records);
97 | } catch (BigQueryException exception) {
98 | throw new BigQueryConnectException(
99 | "Failed to create table " + tableId, exception);
100 | }
101 | }
102 | }
103 | }
104 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/storage/ConvertedRecord.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.storage;
25 |
26 | import org.apache.kafka.connect.sink.SinkRecord;
27 | import org.json.JSONObject;
28 |
29 | public class ConvertedRecord {
30 |
31 | private final SinkRecord original;
32 | private final JSONObject converted;
33 |
34 | public ConvertedRecord(SinkRecord original, JSONObject converted) {
35 | this.original = original;
36 | this.converted = converted;
37 | }
38 |
39 | public SinkRecord original() {
40 | return original;
41 | }
42 |
43 | public JSONObject converted() {
44 | return converted;
45 | }
46 |
47 | }
48 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/storage/StorageApiBatchModeHandler.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.storage;
25 |
26 | import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig;
27 | import java.util.List;
28 | import java.util.Map;
29 | import org.apache.kafka.clients.consumer.OffsetAndMetadata;
30 | import org.apache.kafka.common.TopicPartition;
31 | import org.slf4j.Logger;
32 | import org.slf4j.LoggerFactory;
33 |
34 | /**
35 | * Handles all operations related to Batch Storage Write API
36 | */
37 | public class StorageApiBatchModeHandler {
38 |
39 | private static final Logger logger = LoggerFactory.getLogger(StorageApiBatchModeHandler.class);
40 | private final StorageWriteApiBatchApplicationStream streamApi;
41 |
42 | public StorageApiBatchModeHandler(StorageWriteApiBatchApplicationStream streamApi, BigQuerySinkTaskConfig config) {
43 | this.streamApi = streamApi;
44 | }
45 |
46 | /**
47 | * Used by the scheduler to commit all eligible streams and create new active
48 | * streams.
49 | */
50 | public void refreshStreams() {
51 | logger.trace("Storage Write API commit stream attempt by scheduler");
52 | streamApi.refreshStreams();
53 | }
54 |
55 | /**
56 | * Saves the offsets assigned to a particular stream on a table. This is required to commit offsets sequentially
57 | * even if the execution takes place in parallel at different times.
58 | *
59 | * @param tableName Name of tha table in project/dataset/tablename format
60 | * @param rows Records which would be written to table {tableName} sent to define schema if table creation is
61 | * attempted
62 | * @return Returns the streamName on which offsets are updated
63 | */
64 | public String updateOffsetsOnStream(
65 | String tableName,
66 | List rows) {
67 | logger.trace("Updating offsets on current stream of table {}", tableName);
68 | return this.streamApi.updateOffsetsOnStream(tableName, rows);
69 | }
70 |
71 | /**
72 | * Gets offsets which are committed on BigQuery table.
73 | *
74 | * @return Returns Map of topic, partition, offset mapping
75 | */
76 | public Map getCommitableOffsets() {
77 | logger.trace("Getting list of commitable offsets for batch mode");
78 | return this.streamApi.getCommitableOffsets();
79 | }
80 |
81 | }
82 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/storage/StreamState.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.storage;
25 |
26 | /**
27 | * Enums for Stream states
28 | */
29 | public enum StreamState {
30 | CREATED,
31 | APPEND,
32 | FINALISED,
33 | COMMITTED,
34 | INACTIVE
35 | }
36 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/java/com/wepay/kafka/connect/bigquery/write/storage/StreamWriter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery.write.storage;
25 |
26 | import com.google.api.core.ApiFuture;
27 | import com.google.cloud.bigquery.storage.v1.AppendRowsResponse;
28 | import com.google.protobuf.Descriptors;
29 | import java.io.IOException;
30 | import org.json.JSONArray;
31 |
32 | public interface StreamWriter {
33 |
34 | /**
35 | * Write the provided rows
36 | *
37 | * @param rows the rows to write; may not be null
38 | * @return the response from BigQuery for the write attempt
39 | */
40 | ApiFuture appendRows(
41 | JSONArray rows
42 | ) throws Descriptors.DescriptorValidationException, IOException;
43 |
44 | /**
45 | * Invoked if the underlying stream appears to be closed. Implementing classes
46 | * should respond by re-initialize the underlying stream.
47 | */
48 | void refresh();
49 |
50 | /**
51 | * Invoked when all rows have either been written to BigQuery or intentionally
52 | * discarded (e.g., reported to an {@link com.wepay.kafka.connect.bigquery.ErrantRecordHandler}).
53 | */
54 | void onSuccess();
55 |
56 | String streamName();
57 |
58 | }
59 |
--------------------------------------------------------------------------------
/kcbq-connector/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector:
--------------------------------------------------------------------------------
1 | com.wepay.kafka.connect.bigquery.BigQuerySinkConnector
2 |
--------------------------------------------------------------------------------
/kcbq-connector/src/test/java/com/wepay/kafka/connect/bigquery/BigQuerySinkConnectorTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Copyright 2022 Aiven Oy and
3 | * bigquery-connector-for-apache-kafka project contributors
4 | *
5 | * This software contains code derived from the Confluent BigQuery
6 | * Kafka Connector, Copyright Confluent, Inc, which in turn
7 | * contains code derived from the WePay BigQuery Kafka Connector,
8 | * Copyright WePay, Inc.
9 | *
10 | * Licensed under the Apache License, Version 2.0 (the "License");
11 | * you may not use this file except in compliance with the License.
12 | * You may obtain a copy of the License at
13 | *
14 | * http://www.apache.org/licenses/LICENSE-2.0
15 | *
16 | * Unless required by applicable law or agreed to in writing,
17 | * software distributed under the License is distributed on an
18 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
19 | * KIND, either express or implied. See the License for the
20 | * specific language governing permissions and limitations
21 | * under the License.
22 | */
23 |
24 | package com.wepay.kafka.connect.bigquery;
25 |
26 | import static org.junit.jupiter.api.Assertions.assertEquals;
27 | import static org.junit.jupiter.api.Assertions.assertNotNull;
28 | import static org.junit.jupiter.api.Assertions.assertNotSame;
29 |
30 | import com.wepay.kafka.connect.bigquery.api.SchemaRetriever;
31 | import com.wepay.kafka.connect.bigquery.config.BigQuerySinkTaskConfig;
32 | import java.util.HashMap;
33 | import java.util.List;
34 | import java.util.Map;
35 | import org.apache.kafka.connect.data.Schema;
36 | import org.apache.kafka.connect.sink.SinkRecord;
37 | import org.junit.jupiter.api.BeforeAll;
38 | import org.junit.jupiter.api.Test;
39 |
40 | public class BigQuerySinkConnectorTest {
41 | private static SinkPropertiesFactory propertiesFactory;
42 |
43 | @BeforeAll
44 | public static void initializePropertiesFactory() {
45 | propertiesFactory = new SinkPropertiesFactory();
46 | }
47 |
48 | @Test
49 | public void testTaskClass() {
50 | assertEquals(BigQuerySinkTask.class, new BigQuerySinkConnector().taskClass());
51 | }
52 |
53 | @Test
54 | public void testTaskConfigs() {
55 | Map properties = propertiesFactory.getProperties();
56 |
57 | BigQuerySinkConnector testConnector = new BigQuerySinkConnector();
58 |
59 | testConnector.start(properties);
60 |
61 | for (int i : new int[]{1, 2, 10, 100}) {
62 | Map expectedProperties = new HashMap<>(properties);
63 | List