forked from apache/nifi
-
Notifications
You must be signed in to change notification settings - Fork 0
/
TestPutKudu.java
708 lines (572 loc) · 30.4 KB
/
TestPutKudu.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
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.nifi.processors.kudu;
import org.apache.kudu.ColumnSchema;
import org.apache.kudu.ColumnTypeAttributes;
import org.apache.kudu.Schema;
import org.apache.kudu.Type;
import org.apache.kudu.client.Insert;
import org.apache.kudu.client.KuduException;
import org.apache.kudu.client.KuduSession;
import org.apache.kudu.client.OperationResponse;
import org.apache.kudu.client.PartialRow;
import org.apache.kudu.client.RowError;
import org.apache.kudu.client.RowErrorsAndOverflowStatus;
import org.apache.kudu.client.SessionConfiguration.FlushMode;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.kerberos.KerberosCredentialsService;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.schema.access.SchemaNotFoundException;
import org.apache.nifi.serialization.MalformedRecordException;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.serialization.SimpleRecordSchema;
import org.apache.nifi.serialization.record.MapRecord;
import org.apache.nifi.serialization.record.MockRecordParser;
import org.apache.nifi.serialization.record.RecordField;
import org.apache.nifi.serialization.record.RecordFieldType;
import org.apache.nifi.serialization.record.RecordSchema;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.apache.nifi.util.Tuple;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.stubbing.OngoingStubbing;
import java.io.IOException;
import java.io.InputStream;
import java.math.BigDecimal;
import java.sql.Timestamp;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static org.apache.nifi.processors.kudu.TestPutKudu.ResultCode.EXCEPTION;
import static org.apache.nifi.processors.kudu.TestPutKudu.ResultCode.FAIL;
import static org.apache.nifi.processors.kudu.TestPutKudu.ResultCode.OK;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TestPutKudu {
public static final String DEFAULT_TABLE_NAME = "Nifi-Kudu-Table";
public static final String DEFAULT_MASTERS = "testLocalHost:7051";
public static final String SKIP_HEAD_LINE = "false";
public static final String TABLE_SCHEMA = "id,stringVal,num32Val,doubleVal,decimalVal";
private TestRunner testRunner;
private MockPutKudu processor;
private MockRecordParser readerFactory;
@Before
public void setUp() throws InitializationException {
processor = new MockPutKudu();
testRunner = TestRunners.newTestRunner(processor);
setUpTestRunner(testRunner);
}
private void setUpTestRunner(TestRunner testRunner) throws InitializationException {
testRunner.setProperty(PutKudu.TABLE_NAME, DEFAULT_TABLE_NAME);
testRunner.setProperty(PutKudu.KUDU_MASTERS, DEFAULT_MASTERS);
testRunner.setProperty(PutKudu.SKIP_HEAD_LINE, SKIP_HEAD_LINE);
testRunner.setProperty(PutKudu.IGNORE_NULL, "true");
testRunner.setProperty(PutKudu.LOWERCASE_FIELD_NAMES, "true");
testRunner.setProperty(PutKudu.RECORD_READER, "mock-reader-factory");
testRunner.setProperty(PutKudu.INSERT_OPERATION, OperationType.INSERT.toString());
}
@After
public void close() {
testRunner = null;
}
private void createRecordReader(int numOfRecord) throws InitializationException {
readerFactory = new MockRecordParser();
readerFactory.addSchemaField("id", RecordFieldType.INT);
readerFactory.addSchemaField("stringVal", RecordFieldType.STRING);
readerFactory.addSchemaField("num32Val", RecordFieldType.INT);
readerFactory.addSchemaField("doubleVal", RecordFieldType.DOUBLE);
readerFactory.addSchemaField(new RecordField("decimalVal", RecordFieldType.DECIMAL.getDecimalDataType(6, 3)));
for (int i=0; i < numOfRecord; i++) {
readerFactory.addRecord(i, "val_" + i, 1000 + i, 100.88 + i, new BigDecimal(111.111D).add(BigDecimal.valueOf(i)));
}
testRunner.addControllerService("mock-reader-factory", readerFactory);
testRunner.enableControllerService(readerFactory);
}
@Test
public void testCustomValidate() throws InitializationException {
createRecordReader(1);
testRunner.setProperty(PutKudu.KERBEROS_PRINCIPAL, "principal");
testRunner.assertNotValid();
testRunner.removeProperty(PutKudu.KERBEROS_PRINCIPAL);
testRunner.setProperty(PutKudu.KERBEROS_PASSWORD, "password");
testRunner.assertNotValid();
testRunner.setProperty(PutKudu.KERBEROS_PRINCIPAL, "principal");
testRunner.setProperty(PutKudu.KERBEROS_PASSWORD, "password");
testRunner.assertValid();
final KerberosCredentialsService kerberosCredentialsService = new MockKerberosCredentialsService("unit-test-principal", "unit-test-keytab");
testRunner.addControllerService("kerb", kerberosCredentialsService);
testRunner.enableControllerService(kerberosCredentialsService);
testRunner.setProperty(PutKudu.KERBEROS_CREDENTIALS_SERVICE, "kerb");
testRunner.assertNotValid();
testRunner.removeProperty(PutKudu.KERBEROS_PRINCIPAL);
testRunner.removeProperty(PutKudu.KERBEROS_PASSWORD);
testRunner.assertValid();
}
@Test
public void testWriteKuduWithDefaults() throws IOException, InitializationException {
createRecordReader(100);
final String filename = "testWriteKudu-" + System.currentTimeMillis();
final Map<String,String> flowFileAttributes = new HashMap<>();
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
testRunner.enqueue("trigger", flowFileAttributes);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_SUCCESS, 1);
// verify the successful flow file has the expected content & attributes
final MockFlowFile mockFlowFile = testRunner.getFlowFilesForRelationship(PutKudu.REL_SUCCESS).get(0);
mockFlowFile.assertAttributeEquals(CoreAttributes.FILENAME.key(), filename);
mockFlowFile.assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "100");
mockFlowFile.assertContentEquals("trigger");
// verify we generated a provenance event
final List<ProvenanceEventRecord> provEvents = testRunner.getProvenanceEvents();
Assert.assertEquals(1, provEvents.size());
// verify it was a SEND event with the correct URI
final ProvenanceEventRecord provEvent = provEvents.get(0);
Assert.assertEquals(ProvenanceEventType.SEND, provEvent.getEventType());
}
@Test
public void testKerberosEnabled() throws InitializationException {
createRecordReader(1);
final KerberosCredentialsService kerberosCredentialsService = new MockKerberosCredentialsService("unit-test-principal", "unit-test-keytab");
testRunner.addControllerService("kerb", kerberosCredentialsService);
testRunner.enableControllerService(kerberosCredentialsService);
testRunner.setProperty(PutKudu.KERBEROS_CREDENTIALS_SERVICE, "kerb");
testRunner.run(1, false);
final MockPutKudu proc = (MockPutKudu) testRunner.getProcessor();
assertTrue(proc.loggedIn());
assertFalse(proc.loggedOut());
testRunner.run(1, true, false);
assertTrue(proc.loggedOut());
}
@Test
public void testInsecureClient() throws InitializationException {
createRecordReader(1);
testRunner.run(1, false);
final MockPutKudu proc = (MockPutKudu) testRunner.getProcessor();
assertFalse(proc.loggedIn());
assertFalse(proc.loggedOut());
testRunner.run(1, true, false);
assertFalse(proc.loggedOut());
}
@Test
public void testInvalidReaderShouldRouteToFailure() throws InitializationException, SchemaNotFoundException, MalformedRecordException, IOException {
createRecordReader(0);
// simulate throwing an IOException when the factory creates a reader which is what would happen when
// invalid Avro is passed to the Avro reader factory
final RecordReaderFactory readerFactory = mock(RecordReaderFactory.class);
when(readerFactory.getIdentifier()).thenReturn("mock-reader-factory");
when(readerFactory.createRecordReader(any(FlowFile.class), any(InputStream.class), any(ComponentLog.class))).thenThrow(new IOException("NOT AVRO"));
testRunner.addControllerService("mock-reader-factory", readerFactory);
testRunner.enableControllerService(readerFactory);
testRunner.setProperty(PutKudu.RECORD_READER, "mock-reader-factory");
final String filename = "testInvalidAvroShouldRouteToFailure-" + System.currentTimeMillis();
final Map<String,String> flowFileAttributes = new HashMap<>();
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
testRunner.enqueue("trigger", flowFileAttributes);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_FAILURE, 1);
}
@Test
public void testValidSchemaShouldBeSuccessful() throws InitializationException, IOException {
createRecordReader(10);
final String filename = "testValidSchemaShouldBeSuccessful-" + System.currentTimeMillis();
// don't provide my.schema as an attribute
final Map<String,String> flowFileAttributes = new HashMap<>();
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
flowFileAttributes.put("my.schema", TABLE_SCHEMA);
testRunner.enqueue("trigger", flowFileAttributes);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_SUCCESS, 1);
}
@Test
public void testAddingMissingFieldsWhenHandleSchemaDriftIsAllowed() throws InitializationException, IOException {
// given
processor.setTableSchema(new Schema(Arrays.asList()));
createRecordReader(5);
final String filename = "testAddingMissingFieldsWhenHandleSchemaDriftIsAllowed-" + System.currentTimeMillis();
final Map<String,String> flowFileAttributes = new HashMap<>();
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
testRunner.setProperty(PutKudu.HANDLE_SCHEMA_DRIFT, "true");
testRunner.enqueue("trigger", flowFileAttributes);
// when
testRunner.run();
// then
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_SUCCESS, 1);
}
@Test
public void testMalformedRecordExceptionFromReaderShouldRouteToFailure() throws InitializationException, IOException, MalformedRecordException, SchemaNotFoundException {
createRecordReader(10);
final RecordReader recordReader = mock(RecordReader.class);
when(recordReader.nextRecord()).thenThrow(new MalformedRecordException("ERROR"));
final RecordReaderFactory readerFactory = mock(RecordReaderFactory.class);
when(readerFactory.getIdentifier()).thenReturn("mock-reader-factory");
when(readerFactory.createRecordReader(any(FlowFile.class), any(InputStream.class), any(ComponentLog.class))).thenReturn(recordReader);
testRunner.addControllerService("mock-reader-factory", readerFactory);
testRunner.enableControllerService(readerFactory);
testRunner.setProperty(PutKudu.RECORD_READER, "mock-reader-factory");
final String filename = "testMalformedRecordExceptionShouldRouteToFailure-" + System.currentTimeMillis();
final Map<String,String> flowFileAttributes = new HashMap<>();
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
testRunner.enqueue("trigger", flowFileAttributes);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_FAILURE, 1);
}
@Test
public void testReadAsStringAndWriteAsInt() throws InitializationException, IOException {
createRecordReader(0);
// add the favorite color as a string
readerFactory.addRecord(1, "name0", "0", "89.89", "111.111");
final String filename = "testReadAsStringAndWriteAsInt-" + System.currentTimeMillis();
final Map<String,String> flowFileAttributes = new HashMap<>();
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
testRunner.enqueue("trigger", flowFileAttributes);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_SUCCESS, 1);
}
@Test
public void testMissingColumInReader() throws InitializationException, IOException {
createRecordReader(0);
readerFactory.addRecord( "name0", "0", "89.89"); //missing id
final String filename = "testMissingColumInReader-" + System.currentTimeMillis();
final Map<String,String> flowFileAttributes = new HashMap<>();
flowFileAttributes.put(CoreAttributes.FILENAME.key(), filename);
testRunner.enqueue("trigger", flowFileAttributes);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_FAILURE, 1);
}
@Test
public void testInsertManyFlowFiles() throws Exception {
createRecordReader(50);
final String content1 = "{ \"field1\" : \"value1\", \"field2\" : \"valu11\" }";
final String content2 = "{ \"field1\" : \"value1\", \"field2\" : \"value11\" }";
final String content3 = "{ \"field1\" : \"value3\", \"field2\" : \"value33\" }";
testRunner.enqueue(content1.getBytes());
testRunner.enqueue(content2.getBytes());
testRunner.enqueue(content3.getBytes());
testRunner.run(3);
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_SUCCESS, 3);
List<MockFlowFile> flowFiles = testRunner.getFlowFilesForRelationship(PutKudu.REL_SUCCESS);
flowFiles.get(0).assertContentEquals(content1.getBytes());
flowFiles.get(0).assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "50");
flowFiles.get(1).assertContentEquals(content2.getBytes());
flowFiles.get(1).assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "50");
flowFiles.get(2).assertContentEquals(content3.getBytes());
flowFiles.get(2).assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "50");
}
@Test
public void testUpsertFlowFiles() throws Exception {
createRecordReader(50);
testRunner.setProperty(PutKudu.INSERT_OPERATION, OperationType.UPSERT.toString());
testRunner.enqueue("string".getBytes());
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_SUCCESS, 1);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(PutKudu.REL_SUCCESS).get(0);
flowFile.assertContentEquals("string".getBytes());
flowFile.assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "50");
}
@Test
public void testDeleteFlowFiles() throws Exception {
createRecordReader(50);
testRunner.setProperty(PutKudu.INSERT_OPERATION, "${kudu.record.delete}");
final Map<String,String> attributes = new HashMap<>();
attributes.put("kudu.record.delete", "DELETE");
testRunner.enqueue("string".getBytes(), attributes);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_SUCCESS, 1);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(PutKudu.REL_SUCCESS).get(0);
flowFile.assertContentEquals("string".getBytes());
flowFile.assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "50");
}
@Test
public void testUpdateFlowFiles() throws Exception {
createRecordReader(50);
testRunner.setProperty(PutKudu.INSERT_OPERATION, "${kudu.record.update}");
final Map<String,String> attributes = new HashMap<>();
attributes.put("kudu.record.update", "UPDATE");
testRunner.enqueue("string".getBytes(), attributes);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(PutKudu.REL_SUCCESS, 1);
MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(PutKudu.REL_SUCCESS).get(0);
flowFile.assertContentEquals("string".getBytes());
flowFile.assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "50");
}
@Test
public void testBuildRow() {
buildPartialRow((long) 1, "foo", (short) 10, "id", "id", "SFO",false);
}
@Test
public void testBuildPartialRowNullable() {
buildPartialRow((long) 1, null, (short) 10, "id", "id", null, false);
}
@Test(expected = IllegalArgumentException.class)
public void testBuildPartialRowNullPrimaryKey() {
buildPartialRow(null, "foo", (short) 10, "id", "id", "SFO", false);
}
@Test(expected = IllegalArgumentException.class)
public void testBuildPartialRowNotNullable() {
buildPartialRow((long) 1, "foo", null, "id", "id", "SFO",false);
}
@Test
public void testBuildPartialRowLowercaseFields() {
PartialRow row = buildPartialRow((long) 1, "foo", (short) 10, "id", "ID", "SFO",true);
row.getLong("id");
}
@Test(expected = IllegalArgumentException.class)
public void testBuildPartialRowLowercaseFieldsFalse() {
PartialRow row = buildPartialRow((long) 1, "foo", (short) 10, "id", "ID", "SFO",false);
row.getLong("id");
}
@Test
public void testBuildPartialRowLowercaseFieldsKuduUpper() {
PartialRow row = buildPartialRow((long) 1, "foo", (short) 10, "ID", "ID", "SFO", false);
row.getLong("ID");
}
@Test(expected = IllegalArgumentException.class)
public void testBuildPartialRowLowercaseFieldsKuduUpperFail() {
PartialRow row = buildPartialRow((long) 1, "foo", (short) 10, "ID", "ID", "SFO", true);
row.getLong("ID");
}
@Test
public void testBuildPartialRowVarCharTooLong() {
PartialRow row = buildPartialRow((long) 1, "foo", (short) 10, "id", "ID", "San Francisco", true);
Assert.assertEquals("Kudu client should truncate VARCHAR value to expected length", "San", row.getVarchar("airport_code"));
}
private PartialRow buildPartialRow(Long id, String name, Short age, String kuduIdName, String recordIdName, String airport_code, Boolean lowercaseFields) {
final Schema kuduSchema = new Schema(Arrays.asList(
new ColumnSchema.ColumnSchemaBuilder(kuduIdName, Type.INT64).key(true).build(),
new ColumnSchema.ColumnSchemaBuilder("name", Type.STRING).nullable(true).build(),
new ColumnSchema.ColumnSchemaBuilder("age", Type.INT16).nullable(false).build(),
new ColumnSchema.ColumnSchemaBuilder("updated_at", Type.UNIXTIME_MICROS).nullable(false).build(),
new ColumnSchema.ColumnSchemaBuilder("score", Type.DECIMAL).nullable(true).typeAttributes(
new ColumnTypeAttributes.ColumnTypeAttributesBuilder().precision(9).scale(0).build()
).build(),
new ColumnSchema.ColumnSchemaBuilder("airport_code", Type.VARCHAR).nullable(true).typeAttributes(
new ColumnTypeAttributes.ColumnTypeAttributesBuilder().length(3).build()
).build()));
final RecordSchema schema = new SimpleRecordSchema(Arrays.asList(
new RecordField(recordIdName, RecordFieldType.BIGINT.getDataType()),
new RecordField("name", RecordFieldType.STRING.getDataType()),
new RecordField("age", RecordFieldType.SHORT.getDataType()),
new RecordField("updated_at", RecordFieldType.TIMESTAMP.getDataType()),
new RecordField("score", RecordFieldType.LONG.getDataType()),
new RecordField("airport_code", RecordFieldType.STRING.getDataType())));
Map<String, Object> values = new HashMap<>();
PartialRow row = kuduSchema.newPartialRow();
values.put(recordIdName, id);
values.put("name", name);
values.put("age", age);
values.put("updated_at", new Timestamp(System.currentTimeMillis()));
values.put("score", 10000L);
values.put("airport_code", airport_code);
processor.buildPartialRow(
kuduSchema,
row,
new MapRecord(schema, values),
schema.getFieldNames(),
true,
lowercaseFields
);
return row;
}
private Tuple<Insert, OperationResponse> insert(boolean success) {
Insert insert = mock(Insert.class);
OperationResponse response = mock(OperationResponse.class, Mockito.RETURNS_DEEP_STUBS);
when(response.hasRowError()).thenReturn(!success);
if (!success) {
when(response.getRowError().getOperation()).thenReturn(insert);
}
return new Tuple<>(insert, response);
}
enum ResultCode {
OK,
FAIL,
EXCEPTION
}
private LinkedList<OperationResponse> queueInsert(MockPutKudu putKudu, KuduSession session, boolean sync, ResultCode... results) throws Exception {
LinkedList<OperationResponse> responses = new LinkedList<>();
for (ResultCode result : results) {
boolean ok = result == OK;
Tuple<Insert, OperationResponse> tuple = insert(ok);
putKudu.queue(tuple.getKey());
if (result == EXCEPTION) {
when(session.apply(tuple.getKey())).thenThrow(mock(KuduException.class));
// Stop processing the rest of the records on the first exception
break;
} else {
responses.add(tuple.getValue());
if (sync) {
when(session.apply(tuple.getKey())).thenReturn(ok ? null : tuple.getValue());
// In AUTO_FLUSH_SYNC mode, PutKudu immediately knows when an operation has failed.
// In that case, it does not process the rest of the records in the FlowFile.
if (result == FAIL) break;
}
}
}
return responses;
}
private static <T> void stubSlices(OngoingStubbing<T> stubbing, List<T> slices) {
for (T slice : slices) {
stubbing = stubbing.thenReturn(slice);
}
}
private void testKuduPartialFailure(FlushMode flushMode, int batchSize) throws Exception {
final int numFlowFiles = 4;
final int numRecordsPerFlowFile = 3;
final ResultCode[][] flowFileResults = new ResultCode[][] {
new ResultCode[]{OK, OK, FAIL},
// The last operation will not be submitted to Kudu if flush mode is AUTO_FLUSH_SYNC
new ResultCode[]{OK, FAIL, OK},
// Everything's okay
new ResultCode[]{OK, OK, OK},
// The last operation will not be submitted due to an exception from apply() call
new ResultCode[]{OK, EXCEPTION, OK},
};
KuduSession session = mock(KuduSession.class);
when(session.getFlushMode()).thenReturn(flushMode);
MockPutKudu putKudu = new MockPutKudu(session);
List<List<OperationResponse>> flowFileResponses = new ArrayList<>();
boolean sync = flushMode == FlushMode.AUTO_FLUSH_SYNC;
for (ResultCode[] results : flowFileResults) {
flowFileResponses.add(queueInsert(putKudu, session, sync, results));
}
switch (flushMode) {
case AUTO_FLUSH_SYNC:
// flush() or close() returns an empty list
when(session.close()).thenReturn(Collections.emptyList());
break;
case AUTO_FLUSH_BACKGROUND:
// close() will be called for each batch of FlowFiles, however we do not check
// the return value of it. Instead, we should check the pending errors of the session
// as buffered operations may have already been flushed.
when(session.close()).thenReturn(Collections.emptyList());
List<RowErrorsAndOverflowStatus> pendingErrorResponses = new ArrayList<>();
while (!flowFileResponses.isEmpty()) {
int sliceSize = Math.min(batchSize, flowFileResponses.size());
List<List<OperationResponse>> slice = flowFileResponses.subList(0, sliceSize);
RowErrorsAndOverflowStatus pendingErrorResponse = mock(RowErrorsAndOverflowStatus.class);
RowError[] rowErrors = slice.stream()
.flatMap(List::stream)
.filter(OperationResponse::hasRowError)
.map(OperationResponse::getRowError)
.toArray(RowError[]::new);
when(pendingErrorResponse.getRowErrors()).thenReturn(rowErrors);
pendingErrorResponses.add(pendingErrorResponse);
flowFileResponses = flowFileResponses.subList(sliceSize, flowFileResponses.size());
}
stubSlices(when(session.getPendingErrors()), pendingErrorResponses);
break;
case MANUAL_FLUSH:
// close() will be called at the end of a batch, but flush() will also be called
// whenever the mutation buffer of KuduSession becomes full. In PutKudu, we set
// the size of the mutation buffer to match batchSize, so flush() is called only
// when a FlowFile more than one record.
List<List<OperationResponse>> flushes = new ArrayList<>();
List<List<OperationResponse>> closes = new ArrayList<>();
while (!flowFileResponses.isEmpty()) {
int sliceSize = Math.min(batchSize, flowFileResponses.size());
List<List<OperationResponse>> slice = flowFileResponses.subList(0, sliceSize);
flowFileResponses = flowFileResponses.subList(sliceSize, flowFileResponses.size());
List<OperationResponse> batch = new ArrayList<>();
for (OperationResponse response : slice.stream().flatMap(List::stream).collect(Collectors.toList())) {
if (batch.size() == batchSize) {
flushes.add(batch);
batch = new ArrayList<>();
}
batch.add(response);
}
if (flowFileResponses.isEmpty() && batch.size() == batchSize) {
// To handle the case where PutKudu ends the batch with flush()
// instead of close() due to the exception from the subsequent apply call.
flushes.add(batch);
} else if (batch.size() > 0) {
closes.add(batch);
}
if (!flushes.isEmpty()) stubSlices(when(session.flush()), flushes);
if (!closes.isEmpty()) stubSlices(when(session.close()), closes);
}
break;
}
testRunner = TestRunners.newTestRunner(putKudu);
createRecordReader(numRecordsPerFlowFile);
setUpTestRunner(testRunner);
testRunner.setProperty(PutKudu.FLUSH_MODE, flushMode.name());
testRunner.setProperty(PutKudu.BATCH_SIZE, String.valueOf(batchSize));
testRunner.setProperty(PutKudu.FLOWFILE_BATCH_SIZE, String.valueOf(batchSize));
IntStream.range(0, numFlowFiles).forEach(i -> testRunner.enqueue(""));
testRunner.run(numFlowFiles);
testRunner.assertTransferCount(PutKudu.REL_FAILURE, 3);
List<MockFlowFile> failedFlowFiles = testRunner.getFlowFilesForRelationship(PutKudu.REL_FAILURE);
failedFlowFiles.get(0).assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "2");
failedFlowFiles.get(1).assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, sync ? "1" : "2");
failedFlowFiles.get(2).assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "1");
testRunner.assertTransferCount(PutKudu.REL_SUCCESS, 1);
testRunner.getFlowFilesForRelationship(PutKudu.REL_SUCCESS).get(0).assertAttributeEquals(PutKudu.RECORD_COUNT_ATTR, "3");
}
private void testKuduPartialFailure(FlushMode flushMode) throws Exception {
// Test against different batch sizes (up until the point where every record can be buffered at once)
for (int i = 1; i <= 11; i++) {
testKuduPartialFailure(flushMode, i);
}
}
@Test
public void testKuduPartialFailuresOnAutoFlushSync() throws Exception {
testKuduPartialFailure(FlushMode.AUTO_FLUSH_SYNC);
}
@Test
public void testKuduPartialFailuresOnAutoFlushBackground() throws Exception {
testKuduPartialFailure(FlushMode.AUTO_FLUSH_BACKGROUND);
}
@Test
public void testKuduPartialFailuresOnManualFlush() throws Exception {
testKuduPartialFailure(FlushMode.MANUAL_FLUSH);
}
public static class MockKerberosCredentialsService extends AbstractControllerService implements KerberosCredentialsService {
private final String keytab;
private final String principal;
public MockKerberosCredentialsService(final String keytab, final String principal) {
this.keytab = keytab;
this.principal = principal;
}
@Override
public String getKeytab() {
return keytab;
}
@Override
public String getPrincipal() {
return principal;
}
}
}