|
19 | 19 |
|
20 | 20 | import org.apache.flink.api.java.tuple.Tuple2;
|
21 | 21 | import org.apache.flink.cdc.common.annotation.Internal;
|
| 22 | +import org.apache.flink.cdc.common.event.AddColumnEvent; |
| 23 | +import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; |
22 | 24 | import org.apache.flink.cdc.common.event.CreateTableEvent;
|
| 25 | +import org.apache.flink.cdc.common.event.DropColumnEvent; |
| 26 | +import org.apache.flink.cdc.common.event.RenameColumnEvent; |
23 | 27 | import org.apache.flink.cdc.common.event.SchemaChangeEvent;
|
24 | 28 | import org.apache.flink.cdc.common.event.SchemaChangeEventType;
|
25 | 29 | import org.apache.flink.cdc.common.event.TableId;
|
26 | 30 | import org.apache.flink.cdc.common.exceptions.SchemaEvolveException;
|
27 | 31 | import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior;
|
| 32 | +import org.apache.flink.cdc.common.schema.Column; |
| 33 | +import org.apache.flink.cdc.common.schema.Schema; |
28 | 34 | import org.apache.flink.cdc.common.sink.MetadataApplier;
|
| 35 | +import org.apache.flink.cdc.common.types.DataType; |
29 | 36 | import org.apache.flink.cdc.runtime.operators.schema.event.RefreshPendingListsResponse;
|
30 | 37 | import org.apache.flink.cdc.runtime.operators.schema.event.ReleaseUpstreamRequest;
|
31 | 38 | import org.apache.flink.cdc.runtime.operators.schema.event.ReleaseUpstreamResponse;
|
|
41 | 48 |
|
42 | 49 | import java.io.Closeable;
|
43 | 50 | import java.io.IOException;
|
| 51 | +import java.util.ArrayList; |
44 | 52 | import java.util.Collections;
|
| 53 | +import java.util.HashMap; |
45 | 54 | import java.util.HashSet;
|
46 | 55 | import java.util.LinkedList;
|
47 | 56 | import java.util.List;
|
| 57 | +import java.util.Map; |
48 | 58 | import java.util.Set;
|
49 | 59 | import java.util.concurrent.CompletableFuture;
|
50 | 60 | import java.util.concurrent.ExecutorService;
|
51 | 61 | import java.util.concurrent.Executors;
|
| 62 | +import java.util.stream.Collectors; |
| 63 | +import java.util.stream.Stream; |
52 | 64 |
|
53 | 65 | import static org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistryRequestHandler.RequestStatus.RECEIVED_RELEASE_REQUEST;
|
54 | 66 | import static org.apache.flink.cdc.runtime.operators.schema.event.CoordinationResponseUtils.wrap;
|
@@ -170,7 +182,7 @@ public CompletableFuture<CoordinationResponse> handleSchemaChangeRequest(
|
170 | 182 | }
|
171 | 183 | schemaManager.applyUpstreamSchemaChange(request.getSchemaChangeEvent());
|
172 | 184 | List<SchemaChangeEvent> derivedSchemaChangeEvents =
|
173 |
| - schemaDerivation.applySchemaChange(request.getSchemaChangeEvent()); |
| 185 | + calculateDerivedSchemaChangeEvents(request.getSchemaChangeEvent()); |
174 | 186 | CompletableFuture<CoordinationResponse> response =
|
175 | 187 | CompletableFuture.completedFuture(
|
176 | 188 | wrap(new SchemaChangeResponse(derivedSchemaChangeEvents)));
|
@@ -261,7 +273,7 @@ private void startNextSchemaChangeRequest() {
|
261 | 273 | } else {
|
262 | 274 | schemaManager.applyUpstreamSchemaChange(request.getSchemaChangeEvent());
|
263 | 275 | List<SchemaChangeEvent> derivedSchemaChangeEvents =
|
264 |
| - schemaDerivation.applySchemaChange(request.getSchemaChangeEvent()); |
| 276 | + calculateDerivedSchemaChangeEvents(request.getSchemaChangeEvent()); |
265 | 277 | pendingSchemaChange
|
266 | 278 | .getResponseFuture()
|
267 | 279 | .complete(wrap(new SchemaChangeResponse(derivedSchemaChangeEvents)));
|
@@ -301,6 +313,114 @@ public void close() throws IOException {
|
301 | 313 | }
|
302 | 314 | }
|
303 | 315 |
|
| 316 | + private List<SchemaChangeEvent> calculateDerivedSchemaChangeEvents(SchemaChangeEvent event) { |
| 317 | + if (SchemaChangeBehavior.LENIENT.equals(schemaChangeBehavior)) { |
| 318 | + return lenientizeSchemaChangeEvent(event).stream() |
| 319 | + .flatMap(evt -> schemaDerivation.applySchemaChange(evt).stream()) |
| 320 | + .collect(Collectors.toList()); |
| 321 | + } else { |
| 322 | + return schemaDerivation.applySchemaChange(event); |
| 323 | + } |
| 324 | + } |
| 325 | + |
| 326 | + private List<SchemaChangeEvent> lenientizeSchemaChangeEvent(SchemaChangeEvent event) { |
| 327 | + if (event instanceof CreateTableEvent) { |
| 328 | + return Collections.singletonList(event); |
| 329 | + } |
| 330 | + TableId tableId = event.tableId(); |
| 331 | + Schema evolvedSchema = |
| 332 | + schemaManager |
| 333 | + .getLatestEvolvedSchema(tableId) |
| 334 | + .orElseThrow( |
| 335 | + () -> |
| 336 | + new IllegalStateException( |
| 337 | + "Evolved schema does not exist, not ready for schema change event " |
| 338 | + + event)); |
| 339 | + switch (event.getType()) { |
| 340 | + case ADD_COLUMN: |
| 341 | + { |
| 342 | + AddColumnEvent addColumnEvent = (AddColumnEvent) event; |
| 343 | + return Collections.singletonList( |
| 344 | + new AddColumnEvent( |
| 345 | + tableId, |
| 346 | + addColumnEvent.getAddedColumns().stream() |
| 347 | + .map( |
| 348 | + col -> |
| 349 | + new AddColumnEvent.ColumnWithPosition( |
| 350 | + Column.physicalColumn( |
| 351 | + col.getAddColumn() |
| 352 | + .getName(), |
| 353 | + col.getAddColumn() |
| 354 | + .getType() |
| 355 | + .nullable(), |
| 356 | + col.getAddColumn() |
| 357 | + .getComment()))) |
| 358 | + .collect(Collectors.toList()))); |
| 359 | + } |
| 360 | + case DROP_COLUMN: |
| 361 | + { |
| 362 | + DropColumnEvent dropColumnEvent = (DropColumnEvent) event; |
| 363 | + Map<String, DataType> convertNullableColumns = |
| 364 | + dropColumnEvent.getDroppedColumnNames().stream() |
| 365 | + .map(evolvedSchema::getColumn) |
| 366 | + .flatMap(e -> e.map(Stream::of).orElse(Stream.empty())) |
| 367 | + .filter(col -> !col.getType().isNullable()) |
| 368 | + .collect( |
| 369 | + Collectors.toMap( |
| 370 | + Column::getName, |
| 371 | + column -> column.getType().nullable())); |
| 372 | + |
| 373 | + if (convertNullableColumns.isEmpty()) { |
| 374 | + return Collections.emptyList(); |
| 375 | + } else { |
| 376 | + return Collections.singletonList( |
| 377 | + new AlterColumnTypeEvent(tableId, convertNullableColumns)); |
| 378 | + } |
| 379 | + } |
| 380 | + case RENAME_COLUMN: |
| 381 | + { |
| 382 | + RenameColumnEvent renameColumnEvent = (RenameColumnEvent) event; |
| 383 | + List<AddColumnEvent.ColumnWithPosition> appendColumns = new ArrayList<>(); |
| 384 | + Map<String, DataType> convertNullableColumns = new HashMap<>(); |
| 385 | + renameColumnEvent |
| 386 | + .getNameMapping() |
| 387 | + .forEach( |
| 388 | + (key, value) -> { |
| 389 | + Column column = |
| 390 | + evolvedSchema |
| 391 | + .getColumn(key) |
| 392 | + .orElseThrow( |
| 393 | + () -> |
| 394 | + new IllegalArgumentException( |
| 395 | + "Non-existed column " |
| 396 | + + key |
| 397 | + + " in evolved schema.")); |
| 398 | + if (!column.getType().isNullable()) { |
| 399 | + // It's a not-nullable column, we need to cast it to |
| 400 | + // nullable first |
| 401 | + convertNullableColumns.put( |
| 402 | + key, column.getType().nullable()); |
| 403 | + } |
| 404 | + appendColumns.add( |
| 405 | + new AddColumnEvent.ColumnWithPosition( |
| 406 | + Column.physicalColumn( |
| 407 | + value, |
| 408 | + column.getType().nullable(), |
| 409 | + column.getComment()))); |
| 410 | + }); |
| 411 | + |
| 412 | + List<SchemaChangeEvent> events = new ArrayList<>(); |
| 413 | + events.add(new AddColumnEvent(tableId, appendColumns)); |
| 414 | + if (!convertNullableColumns.isEmpty()) { |
| 415 | + events.add(new AlterColumnTypeEvent(tableId, convertNullableColumns)); |
| 416 | + } |
| 417 | + return events; |
| 418 | + } |
| 419 | + default: |
| 420 | + return Collections.singletonList(event); |
| 421 | + } |
| 422 | + } |
| 423 | + |
304 | 424 | private static class PendingSchemaChange {
|
305 | 425 | private final SchemaChangeRequest changeRequest;
|
306 | 426 | private List<SchemaChangeEvent> derivedSchemaChangeEvents;
|
|
0 commit comments