|
48 | 48 | import org.apache.nifi.proxy.ProxyConfiguration; |
49 | 49 | import org.apache.nifi.reporting.InitializationException; |
50 | 50 | import org.apache.nifi.schema.access.SchemaAccessUtils; |
| 51 | +import org.apache.nifi.util.LogMessage; |
| 52 | +import org.apache.nifi.util.MockComponentLog; |
51 | 53 | import org.apache.nifi.util.PropertyMigrationResult; |
52 | 54 | import org.apache.nifi.util.TestRunner; |
53 | 55 | import org.apache.nifi.util.TestRunners; |
|
80 | 82 | import static org.mockito.ArgumentMatchers.anyLong; |
81 | 83 | import static org.mockito.ArgumentMatchers.isA; |
82 | 84 | import static org.mockito.Mockito.mock; |
| 85 | +import static org.mockito.Mockito.never; |
83 | 86 | import static org.mockito.Mockito.reset; |
84 | 87 | import static org.mockito.Mockito.times; |
85 | 88 | import static org.mockito.Mockito.verify; |
@@ -367,6 +370,112 @@ void testUnknownColumnSkipped() { |
367 | 370 | runner.assertAllFlowFilesTransferred(PutBigQuery.REL_SUCCESS); |
368 | 371 | } |
369 | 372 |
|
| 373 | + @Test |
| 374 | + void testUnmatchedFieldWarnLogsAndWrites() { |
| 375 | + when(writeClient.createWriteStream(isA(CreateWriteStreamRequest.class))).thenReturn(writeStream); |
| 376 | + final TableSchema myTableSchema = mockTableSchema(FIELD_1_NAME, TableFieldSchema.Type.STRING, FIELD_2_NAME, TableFieldSchema.Type.STRING); |
| 377 | + when(writeStream.getTableSchema()).thenReturn(myTableSchema); |
| 378 | + when(streamWriter.append(isA(ProtoRows.class), isA(Long.class))) |
| 379 | + .thenReturn(ApiFutures.immediateFuture(AppendRowsResponse.newBuilder().setAppendResult(mock(AppendRowsResponse.AppendResult.class)).build())); |
| 380 | + |
| 381 | + runner.setProperty(PutBigQuery.UNMATCHED_FIELD_BEHAVIOR, UnmatchedFieldBehavior.WARN); |
| 382 | + |
| 383 | + final String unknownProperty = "myUnknownProperty"; |
| 384 | + runner.enqueue(CSV_HEADER + ",unknownField\nmyId,myValue," + unknownProperty); |
| 385 | + runner.run(); |
| 386 | + |
| 387 | + verify(streamWriter).append(protoRowsCaptor.capture(), offsetCaptor.capture()); |
| 388 | + final ProtoRows rows = protoRowsCaptor.getValue(); |
| 389 | + assertEquals(1, rows.getSerializedRowsCount()); |
| 390 | + assertFalse(rows.getSerializedRowsList().getFirst().toString().contains(unknownProperty)); |
| 391 | + |
| 392 | + runner.assertAllFlowFilesTransferred(PutBigQuery.REL_SUCCESS); |
| 393 | + assertUnmatchedFieldWarningLogged(runner); |
| 394 | + } |
| 395 | + |
| 396 | + @Test |
| 397 | + void testUnmatchedFieldFailRoutesToFailure() { |
| 398 | + when(writeClient.createWriteStream(isA(CreateWriteStreamRequest.class))).thenReturn(writeStream); |
| 399 | + final TableSchema myTableSchema = mockTableSchema(FIELD_1_NAME, TableFieldSchema.Type.STRING, FIELD_2_NAME, TableFieldSchema.Type.STRING); |
| 400 | + when(writeStream.getTableSchema()).thenReturn(myTableSchema); |
| 401 | + |
| 402 | + runner.setProperty(PutBigQuery.UNMATCHED_FIELD_BEHAVIOR, UnmatchedFieldBehavior.FAIL); |
| 403 | + runner.setProperty(PutBigQuery.SKIP_INVALID_ROWS, "false"); |
| 404 | + |
| 405 | + runner.enqueue(CSV_HEADER + ",unknownField\nmyId,myValue,extraField"); |
| 406 | + runner.run(); |
| 407 | + |
| 408 | + verify(streamWriter, never()).append(any(ProtoRows.class), anyLong()); |
| 409 | + runner.assertAllFlowFilesTransferred(PutBigQuery.REL_FAILURE); |
| 410 | + } |
| 411 | + |
| 412 | + @Test |
| 413 | + void testUnmatchedFieldFailIgnoresSkipInvalidRows() { |
| 414 | + when(writeClient.createWriteStream(isA(CreateWriteStreamRequest.class))).thenReturn(writeStream); |
| 415 | + final TableSchema myTableSchema = mockTableSchema(FIELD_1_NAME, TableFieldSchema.Type.STRING, FIELD_2_NAME, TableFieldSchema.Type.STRING); |
| 416 | + when(writeStream.getTableSchema()).thenReturn(myTableSchema); |
| 417 | + |
| 418 | + runner.setProperty(PutBigQuery.UNMATCHED_FIELD_BEHAVIOR, UnmatchedFieldBehavior.FAIL); |
| 419 | + runner.setProperty(PutBigQuery.SKIP_INVALID_ROWS, "true"); |
| 420 | + |
| 421 | + runner.enqueue(CSV_HEADER + ",unknownField\nidOne,valueOne,extraField"); |
| 422 | + runner.run(); |
| 423 | + |
| 424 | + verify(streamWriter, never()).append(any(ProtoRows.class), anyLong()); |
| 425 | + runner.assertAllFlowFilesTransferred(PutBigQuery.REL_FAILURE); |
| 426 | + } |
| 427 | + |
| 428 | + @Test |
| 429 | + void testUnmatchedFieldInNestedStructLogsDottedPath() throws InitializationException { |
| 430 | + when(writeClient.createWriteStream(isA(CreateWriteStreamRequest.class))).thenReturn(writeStream); |
| 431 | + |
| 432 | + final TableFieldSchema nestedKnown = mock(TableFieldSchema.class); |
| 433 | + when(nestedKnown.getName()).thenReturn("known"); |
| 434 | + when(nestedKnown.getType()).thenReturn(TableFieldSchema.Type.STRING); |
| 435 | + when(nestedKnown.getMode()).thenReturn(TableFieldSchema.Mode.NULLABLE); |
| 436 | + |
| 437 | + final TableFieldSchema parent = mock(TableFieldSchema.class); |
| 438 | + when(parent.getName()).thenReturn("parent"); |
| 439 | + when(parent.getType()).thenReturn(TableFieldSchema.Type.STRUCT); |
| 440 | + when(parent.getMode()).thenReturn(TableFieldSchema.Mode.NULLABLE); |
| 441 | + when(parent.getFieldsList()).thenReturn(List.of(nestedKnown)); |
| 442 | + |
| 443 | + final TableSchema schemaWithStruct = mock(TableSchema.class); |
| 444 | + when(schemaWithStruct.getFieldsList()).thenReturn(List.of(parent)); |
| 445 | + |
| 446 | + when(writeStream.getTableSchema()).thenReturn(schemaWithStruct); |
| 447 | + when(streamWriter.append(isA(ProtoRows.class), isA(Long.class))) |
| 448 | + .thenReturn(ApiFutures.immediateFuture(AppendRowsResponse.newBuilder().setAppendResult(mock(AppendRowsResponse.AppendResult.class)).build())); |
| 449 | + |
| 450 | + decorateWithNestedStructJsonReader(runner); |
| 451 | + |
| 452 | + runner.setProperty(PutBigQuery.UNMATCHED_FIELD_BEHAVIOR, UnmatchedFieldBehavior.WARN); |
| 453 | + |
| 454 | + runner.enqueue(""" |
| 455 | + {"parent":{"known":"ok","unexpected":"oops"}} |
| 456 | + """); |
| 457 | + runner.run(); |
| 458 | + |
| 459 | + runner.assertAllFlowFilesTransferred(PutBigQuery.REL_SUCCESS); |
| 460 | + assertUnmatchedFieldPathLogged(runner, "parent.unexpected"); |
| 461 | + } |
| 462 | + |
| 463 | + private static void assertUnmatchedFieldWarningLogged(final TestRunner runner) { |
| 464 | + final MockComponentLog logger = runner.getLogger(); |
| 465 | + final boolean found = logger.getWarnMessages().stream() |
| 466 | + .map(LogMessage::getMsg) |
| 467 | + .anyMatch(msg -> msg != null && msg.contains("not present in BigQuery table schema")); |
| 468 | + assertTrue(found, "Expected a warning about unmatched fields, got: " + logger.getWarnMessages()); |
| 469 | + } |
| 470 | + |
| 471 | + private static void assertUnmatchedFieldPathLogged(final TestRunner runner, final String expectedPath) { |
| 472 | + final MockComponentLog logger = runner.getLogger(); |
| 473 | + final boolean found = logger.getWarnMessages().stream() |
| 474 | + .map(LogMessage::getMsg) |
| 475 | + .anyMatch(msg -> msg != null && msg.contains(expectedPath)); |
| 476 | + assertTrue(found, "Expected warning to reference path '" + expectedPath + "', got: " + logger.getWarnMessages()); |
| 477 | + } |
| 478 | + |
370 | 479 | @Test |
371 | 480 | void testSchema() throws Exception { |
372 | 481 | when(writeClient.createWriteStream(isA(CreateWriteStreamRequest.class))).thenReturn(writeStream); |
@@ -561,6 +670,36 @@ private void decorateWithJsonRecordReaderWithSchema(TestRunner runner) throws In |
561 | 670 | runner.enableControllerService(jsonReader); |
562 | 671 | } |
563 | 672 |
|
| 673 | + private void decorateWithNestedStructJsonReader(TestRunner runner) throws InitializationException { |
| 674 | + String recordReaderSchema = """ |
| 675 | + { |
| 676 | + "name": "nested", |
| 677 | + "namespace": "nifi.examples", |
| 678 | + "type": "record", |
| 679 | + "fields": [ |
| 680 | + { |
| 681 | + "name": "parent", |
| 682 | + "type": { |
| 683 | + "type": "record", |
| 684 | + "name": "ParentRecord", |
| 685 | + "fields": [ |
| 686 | + {"name": "known", "type": ["null", "string"], "default": null}, |
| 687 | + {"name": "unexpected", "type": ["null", "string"], "default": null} |
| 688 | + ] |
| 689 | + } |
| 690 | + } |
| 691 | + ] |
| 692 | + }"""; |
| 693 | + |
| 694 | + JsonTreeReader jsonReader = new JsonTreeReader(); |
| 695 | + runner.addControllerService("nestedJsonReader", jsonReader); |
| 696 | + runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY); |
| 697 | + runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_TEXT, recordReaderSchema); |
| 698 | + runner.enableControllerService(jsonReader); |
| 699 | + |
| 700 | + runner.setProperty(PutBigQuery.RECORD_READER, "nestedJsonReader"); |
| 701 | + } |
| 702 | + |
564 | 703 | private TableSchema mockTableSchema(String name1, TableFieldSchema.Type type1, String name2, TableFieldSchema.Type type2) { |
565 | 704 | TableSchema myTableSchema = mock(TableSchema.class); |
566 | 705 |
|
|
0 commit comments