forked from confluentinc/kafka-connect-bigquery
-
Notifications
You must be signed in to change notification settings - Fork 0
/
MergeQueriesTest.java
445 lines (400 loc) · 18.5 KB
/
MergeQueriesTest.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
/*
* 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;
import com.google.cloud.bigquery.BigQueryError;
import com.google.cloud.bigquery.BigQueryException;
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.google.cloud.bigquery.TableResult;
import com.wepay.kafka.connect.bigquery.exception.BigQueryConnectException;
import com.wepay.kafka.connect.bigquery.write.batch.KCBQThreadPoolExecutor;
import com.wepay.kafka.connect.bigquery.write.batch.MergeBatches;
import org.apache.kafka.connect.sink.SinkRecord;
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.HashMap;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import static org.junit.Assert.assertEquals;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyObject;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
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 int BIGQUERY_RETRY = 3;
private static final int BIGQUERY_RETRY_WAIT = 1000;
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();
private static final SinkRecord TEST_SINK_RECORD = new SinkRecord("test", 0, null, null, null, null, 0);
@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, BIGQUERY_RETRY, BIGQUERY_RETRY_WAIT, mergeBatches, executor, bigQuery, schemaManager, context
);
}
private void initialiseMergeBatches() {
mergeBatches = new MergeBatches("_tmp_6_uuid_epoch");
mergeBatches.intermediateTableFor(DESTINATION_TABLE);
}
private static Schema constructIntermediateTable() {
List<Field> fields = new ArrayList<>();
List<Field> 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<Field> 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) + " dstTableAlias "
+ "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 dstTableAlias." + KEY + "=src.key "
+ "WHEN MATCHED "
+ "THEN UPDATE SET dstTableAlias.`f1`=src.value.f1, dstTableAlias.`f2`=src.value.f2, dstTableAlias.`f3`=src.value.f3, dstTableAlias.`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);
assertEquals(expectedQuery, actualQuery);
}
@Test
public void testUpsertQueryWithoutPartitionTime() {
String expectedQuery =
"MERGE " + table(DESTINATION_TABLE) + " dstTableAlias "
+ "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 dstTableAlias." + KEY + "=src.key "
+ "WHEN MATCHED "
+ "THEN UPDATE SET dstTableAlias.`f1`=src.value.f1, dstTableAlias.`f2`=src.value.f2, dstTableAlias.`f3`=src.value.f3, dstTableAlias.`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);
assertEquals(expectedQuery, actualQuery);
}
@Test
public void testDeleteQueryWithPartitionTime() {
String expectedQuery =
"MERGE " + table(DESTINATION_TABLE) + " "
+ "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);
assertEquals(expectedQuery, actualQuery);
}
@Test
public void testDeleteQueryWithoutPartitionTime() {
String expectedQuery =
"MERGE " + table(DESTINATION_TABLE) + " "
+ "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);
assertEquals(expectedQuery, actualQuery);
}
@Test
public void testUpsertDeleteQueryWithPartitionTime() {
String expectedQuery =
"MERGE " + table(DESTINATION_TABLE) + " dstTableAlias "
+ "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 dstTableAlias." + KEY + "=src.key "
+ "WHEN MATCHED AND src.value IS NOT NULL "
+ "THEN UPDATE SET dstTableAlias.`f1`=src.value.f1, dstTableAlias.`f2`=src.value.f2, dstTableAlias.`f3`=src.value.f3, dstTableAlias.`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);
assertEquals(expectedQuery, actualQuery);
}
@Test
public void testUpsertDeleteQueryWithoutPartitionTime() {
String expectedQuery =
"MERGE " + table(DESTINATION_TABLE) + " dstTableAlias "
+ "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 dstTableAlias." + KEY + "=src.key "
+ "WHEN MATCHED AND src.value IS NOT NULL "
+ "THEN UPDATE SET dstTableAlias.`f1`=src.value.f1, dstTableAlias.`f2`=src.value.f2, dstTableAlias.`f3`=src.value.f3, dstTableAlias.`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);
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.batchClearQuery(INTERMEDIATE_TABLE, BATCH_NUMBER);
assertEquals(expectedQuery, actualQuery);
}
@Test
public void testNoEmptyBatchCreation() {
initialiseMergeBatches();
mergeQueries(false, true, true).mergeFlush(INTERMEDIATE_TABLE);
assertEquals(0, mergeBatches.incrementBatch(INTERMEDIATE_TABLE));
}
@Test
public void testBatchCreation() {
initialiseMergeBatches();
mergeBatches.addToBatch(TEST_SINK_RECORD,INTERMEDIATE_TABLE, new HashMap<>());
mergeQueries(false, true, true).mergeFlush(INTERMEDIATE_TABLE);
assertEquals(1, mergeBatches.incrementBatch(INTERMEDIATE_TABLE));
}
@Test
public void testBigQueryJobInternalErrorRetry() throws InterruptedException {
// Arrange
mergeBatches.addToBatch(TEST_SINK_RECORD, INTERMEDIATE_TABLE, new HashMap<>());
TableResult tableResultReponse = mock(TableResult.class);
BigQueryError jobInternalError = new BigQueryError("jobInternalError", null, "The job encountered an internal error during execution and was unable to complete successfully.");
when(bigQuery.query(anyObject()))
.thenThrow(new BigQueryException(400, "mock job internal error", jobInternalError))
.thenReturn(tableResultReponse);
when(mergeBatches.destinationTableFor(INTERMEDIATE_TABLE)).thenReturn(DESTINATION_TABLE);
when(mergeBatches.incrementBatch(INTERMEDIATE_TABLE)).thenReturn(0);
when(mergeBatches.prepareToFlush(INTERMEDIATE_TABLE, 0)).thenReturn(true);
CountDownLatch latch = new CountDownLatch(1);
doAnswer(invocation -> {
Runnable runnable = invocation.getArgument(0);
runnable.run();
latch.countDown();
return null;
}).when(executor).execute(any());
MergeQueries mergeQueries = spy(mergeQueries(false, true, true));
// Act
mergeQueries.mergeFlush(INTERMEDIATE_TABLE);
// Assert
latch.await();
verify(bigQuery, times(3)).query(anyObject());
}
@Test
public void testBigQueryInvalidQueryErrorRetry() throws InterruptedException {
// Arrange
mergeBatches.addToBatch(TEST_SINK_RECORD, INTERMEDIATE_TABLE, new HashMap<>());
TableResult tableResultReponse = mock(TableResult.class);
BigQueryError jobInternalError = new BigQueryError("invalidQuery", null, "Could not serialize access to table my_table due to concurrent update");
when(bigQuery.query(anyObject()))
.thenThrow(new BigQueryException(400, "mock invalid query", jobInternalError))
.thenReturn(tableResultReponse);
when(mergeBatches.destinationTableFor(INTERMEDIATE_TABLE)).thenReturn(DESTINATION_TABLE);
when(mergeBatches.incrementBatch(INTERMEDIATE_TABLE)).thenReturn(0);
when(mergeBatches.prepareToFlush(INTERMEDIATE_TABLE, 0)).thenReturn(true);
CountDownLatch latch = new CountDownLatch(1);
doAnswer(invocation -> {
Runnable runnable = invocation.getArgument(0);
runnable.run();
latch.countDown();
return null;
}).when(executor).execute(any());
MergeQueries mergeQueries = mergeQueries(false, true, true);
// Act
mergeQueries.mergeFlush(INTERMEDIATE_TABLE);
// Assert
latch.await();
verify(bigQuery, times(3)).query(anyObject());
}
@Test(expected = BigQueryConnectException.class)
public void testBigQueryRetryExceeded() throws InterruptedException {
// Arrange
mergeBatches.addToBatch(TEST_SINK_RECORD, INTERMEDIATE_TABLE, new HashMap<>());
BigQueryError jobInternalError = new BigQueryError("invalidQuery", null, "Could not serialize access to table my_table due to concurrent update");
when(bigQuery.query(anyObject()))
.thenThrow(new BigQueryException(400, "mock invalid query", jobInternalError));
when(mergeBatches.destinationTableFor(INTERMEDIATE_TABLE)).thenReturn(DESTINATION_TABLE);
when(mergeBatches.incrementBatch(INTERMEDIATE_TABLE)).thenReturn(0);
when(mergeBatches.prepareToFlush(INTERMEDIATE_TABLE, 0)).thenReturn(true);
CountDownLatch latch = new CountDownLatch(1);
doAnswer(invocation -> {
Runnable runnable = invocation.getArgument(0);
runnable.run();
latch.countDown();
return null;
}).when(executor).execute(any());
MergeQueries mergeQueries = mergeQueries(false, true, true);
// Act
mergeQueries.mergeFlush(INTERMEDIATE_TABLE);
//Assert
latch.await();
}
private String table(TableId table) {
return String.format("`%s`.`%s`", table.getDataset(), table.getTable());
}
}