|
46 | 46 | import org.apache.flink.cdc.composer.flink.translator.SchemaOperatorTranslator; |
47 | 47 | import org.apache.flink.cdc.connectors.doris.sink.utils.DorisContainer; |
48 | 48 | import org.apache.flink.cdc.connectors.doris.sink.utils.DorisSinkTestBase; |
| 49 | +import org.apache.flink.cdc.connectors.doris.utils.DorisSchemaUtils; |
49 | 50 | import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; |
50 | 51 | import org.apache.flink.runtime.client.JobExecutionException; |
51 | 52 | import org.apache.flink.streaming.api.datastream.DataStream; |
@@ -527,6 +528,108 @@ void testDorisDropTable(boolean batchMode) throws Exception { |
527 | 528 | tableId.getTableName())); |
528 | 529 | } |
529 | 530 |
|
| 531 | + @ParameterizedTest(name = "batchMode: {0}") |
| 532 | + @ValueSource(booleans = {true, false}) |
| 533 | + void testMysqlDefaultTimestampValueConversionInCreateTable(boolean batchMode) throws Exception { |
| 534 | + TableId tableId = |
| 535 | + TableId.tableId( |
| 536 | + DorisContainer.DORIS_DATABASE_NAME, DorisContainer.DORIS_TABLE_NAME); |
| 537 | + |
| 538 | + Schema schema = |
| 539 | + Schema.newBuilder() |
| 540 | + .column(new PhysicalColumn("id", DataTypes.INT().notNull(), null)) |
| 541 | + .column(new PhysicalColumn("name", DataTypes.VARCHAR(50), null)) |
| 542 | + .column( |
| 543 | + new PhysicalColumn( |
| 544 | + "created_time", |
| 545 | + DataTypes.TIMESTAMP(), |
| 546 | + null, |
| 547 | + DorisSchemaUtils.INVALID_OR_MISSING_DATATIME)) |
| 548 | + .column( |
| 549 | + new PhysicalColumn( |
| 550 | + "updated_time", |
| 551 | + DataTypes.TIMESTAMP_LTZ(), |
| 552 | + null, |
| 553 | + DorisSchemaUtils.INVALID_OR_MISSING_DATATIME)) |
| 554 | + .primaryKey("id") |
| 555 | + .build(); |
| 556 | + |
| 557 | + runJobWithEvents( |
| 558 | + Collections.singletonList(new CreateTableEvent(tableId, schema)), batchMode); |
| 559 | + |
| 560 | + List<String> actual = inspectTableSchema(tableId); |
| 561 | + |
| 562 | + List<String> expected = |
| 563 | + Arrays.asList( |
| 564 | + "id | INT | Yes | true | null", |
| 565 | + "name | VARCHAR(150) | Yes | false | null", |
| 566 | + "created_time | DATETIME(6) | Yes | false | " |
| 567 | + + DorisSchemaUtils.DEFAULT_DATETIME, |
| 568 | + "updated_time | DATETIME(6) | Yes | false | " |
| 569 | + + DorisSchemaUtils.DEFAULT_DATETIME); |
| 570 | + |
| 571 | + assertEqualsInOrder(expected, actual); |
| 572 | + } |
| 573 | + |
| 574 | + @ParameterizedTest(name = "batchMode: {0}") |
| 575 | + @ValueSource(booleans = {true, false}) |
| 576 | + void testMysqlDefaultTimestampValueConversionInAddColumn(boolean batchMode) throws Exception { |
| 577 | + TableId tableId = |
| 578 | + TableId.tableId( |
| 579 | + DorisContainer.DORIS_DATABASE_NAME, DorisContainer.DORIS_TABLE_NAME); |
| 580 | + |
| 581 | + Schema initialSchema = |
| 582 | + Schema.newBuilder() |
| 583 | + .column(new PhysicalColumn("id", DataTypes.INT().notNull(), null)) |
| 584 | + .column(new PhysicalColumn("name", DataTypes.VARCHAR(50), null)) |
| 585 | + .primaryKey("id") |
| 586 | + .build(); |
| 587 | + |
| 588 | + List<Event> events = new ArrayList<>(); |
| 589 | + events.add(new CreateTableEvent(tableId, initialSchema)); |
| 590 | + |
| 591 | + PhysicalColumn createdTimeCol = |
| 592 | + new PhysicalColumn( |
| 593 | + "created_time", |
| 594 | + DataTypes.TIMESTAMP(), |
| 595 | + null, |
| 596 | + DorisSchemaUtils.INVALID_OR_MISSING_DATATIME); |
| 597 | + |
| 598 | + PhysicalColumn updatedTimeCol = |
| 599 | + new PhysicalColumn( |
| 600 | + "updated_time", |
| 601 | + DataTypes.TIMESTAMP_LTZ(), |
| 602 | + null, |
| 603 | + DorisSchemaUtils.INVALID_OR_MISSING_DATATIME); |
| 604 | + |
| 605 | + events.add( |
| 606 | + new AddColumnEvent( |
| 607 | + tableId, |
| 608 | + Collections.singletonList( |
| 609 | + new AddColumnEvent.ColumnWithPosition(createdTimeCol)))); |
| 610 | + |
| 611 | + events.add( |
| 612 | + new AddColumnEvent( |
| 613 | + tableId, |
| 614 | + Collections.singletonList( |
| 615 | + new AddColumnEvent.ColumnWithPosition(updatedTimeCol)))); |
| 616 | + |
| 617 | + runJobWithEvents(events, batchMode); |
| 618 | + |
| 619 | + List<String> actual = inspectTableSchema(tableId); |
| 620 | + |
| 621 | + List<String> expected = |
| 622 | + Arrays.asList( |
| 623 | + "id | INT | Yes | true | null", |
| 624 | + "name | VARCHAR(150) | Yes | false | null", |
| 625 | + "created_time | DATETIME(6) | Yes | false | " |
| 626 | + + DorisSchemaUtils.DEFAULT_DATETIME, |
| 627 | + "updated_time | DATETIME(6) | Yes | false | " |
| 628 | + + DorisSchemaUtils.DEFAULT_DATETIME); |
| 629 | + |
| 630 | + assertEqualsInOrder(expected, actual); |
| 631 | + } |
| 632 | + |
530 | 633 | private void runJobWithEvents(List<Event> events, boolean batchMode) throws Exception { |
531 | 634 | DataStream<Event> stream = |
532 | 635 | env.fromCollection(events, TypeInformation.of(Event.class)).setParallelism(1); |
|
0 commit comments