|
| 1 | +/* |
| 2 | + * Copyright 2023 Celonis SE |
| 3 | + * |
| 4 | + * Licensed under the Apache License, Version 2.0 (the "License"); |
| 5 | + * you may not use this file except in compliance with the License. |
| 6 | + * You may obtain a copy of the License at |
| 7 | + * |
| 8 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 9 | + * |
| 10 | + * Unless required by applicable law or agreed to in writing, software |
| 11 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 12 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 13 | + * See the License for the specific language governing permissions and |
| 14 | + * limitations under the License. |
| 15 | + */ |
| 16 | +package com.celonis.kafka.connect.schema; |
| 17 | + |
| 18 | +import org.apache.kafka.connect.data.Schema; |
| 19 | +import org.apache.kafka.connect.data.SchemaBuilder; |
| 20 | + |
| 21 | +import java.util.Objects; |
| 22 | + |
| 23 | +/** |
| 24 | + * StructSchemaEvolution is responsible for the recursively merging existing schema and schema from |
| 25 | + * new record. And thus evolving the schema with each new record received from connector. |
| 26 | + */ |
| 27 | +public class StructSchemaEvolution implements SchemaEvolution { |
| 28 | + |
| 29 | + /** |
| 30 | + * Merge top level Kafka Connect Structs |
| 31 | + * |
| 32 | + * @param currentSchema existing schema, must be of type Struct |
| 33 | + * @param recordSchema schema of new record, must be of type Struct |
| 34 | + * @return Schema after merging existing and new schema recursively |
| 35 | + */ |
| 36 | + @Override |
| 37 | + public Schema evolve(Schema currentSchema, Schema recordSchema) throws SchemaEvolutionException { |
| 38 | + if (currentSchema == recordSchema) return currentSchema; |
| 39 | + |
| 40 | + // RecordTransformer ensures that the top level schema are of type Struct. |
| 41 | + return mergeSchemas(null, currentSchema, recordSchema); |
| 42 | + } |
| 43 | + |
| 44 | + /** |
| 45 | + * Merge a Kafka Connect Schemas |
| 46 | + * |
| 47 | + * @param fieldName current field name, `null` when the recursion starts |
| 48 | + * @param currentSchema existing schema, (Accepted types MAP, ARRAY, STRUCT) |
| 49 | + * @param recordSchema schema of new record, (Accepted types MAP, ARRAY, STRUCT) |
| 50 | + * @return Schema after merging existing and new schemas recursively |
| 51 | + */ |
| 52 | + private Schema mergeSchemas(String fieldName, Schema currentSchema, Schema recordSchema) { |
| 53 | + // validationsFirst |
| 54 | + validateSchemasTypes(fieldName, currentSchema, recordSchema); |
| 55 | + |
| 56 | + switch (currentSchema.type()) { |
| 57 | + case STRUCT: |
| 58 | + return mergeStructs(currentSchema, recordSchema); |
| 59 | + case ARRAY: |
| 60 | + return SchemaBuilder.array( |
| 61 | + mergeSchemas(fieldName, currentSchema.valueSchema(), recordSchema.valueSchema())) |
| 62 | + .build(); |
| 63 | + case MAP: |
| 64 | + var keySchema = |
| 65 | + mergeSchemas(fieldName, currentSchema.keySchema(), recordSchema.keySchema()); |
| 66 | + var valueSchema = |
| 67 | + mergeSchemas(fieldName, currentSchema.valueSchema(), recordSchema.valueSchema()); |
| 68 | + return SchemaBuilder.map(keySchema, valueSchema).build(); |
| 69 | + default: |
| 70 | + return currentSchema; |
| 71 | + } |
| 72 | + } |
| 73 | + |
| 74 | + private Schema mergeStructs(Schema currentSchema, Schema recordSchema) |
| 75 | + throws SchemaEvolutionException { |
| 76 | + SchemaBuilder result = SchemaUtils.withMetadata(SchemaBuilder.struct(), currentSchema); |
| 77 | + |
| 78 | + // First currentSchemaFields |
| 79 | + currentSchema.fields().stream() |
| 80 | + .forEach( |
| 81 | + currentSchemaField -> { |
| 82 | + final var recordSchemaField = recordSchema.field(currentSchemaField.name()); |
| 83 | + if (recordSchemaField == null) { |
| 84 | + // If not present in recordSchema, just add it |
| 85 | + result.field(currentSchemaField.name(), currentSchemaField.schema()); |
| 86 | + } else { |
| 87 | + // Recursively evolve otherwise |
| 88 | + result.field( |
| 89 | + currentSchemaField.name(), |
| 90 | + mergeSchemas( |
| 91 | + currentSchemaField.name(), |
| 92 | + currentSchemaField.schema(), |
| 93 | + recordSchemaField.schema())); |
| 94 | + } |
| 95 | + }); |
| 96 | + |
| 97 | + // Just add remaining record schema fields as they are |
| 98 | + recordSchema.fields().stream() |
| 99 | + .filter(rf -> currentSchema.field(rf.name()) == null) |
| 100 | + .forEach(rf -> result.field(rf.name(), rf.schema())); |
| 101 | + |
| 102 | + return result.build(); |
| 103 | + } |
| 104 | + |
| 105 | + private void validateSchemasTypes(String fieldName, Schema currentSchema, Schema recordSchema) { |
| 106 | + if (bothPrimitives(currentSchema, recordSchema) && !sameLogicalType(currentSchema, recordSchema) |
| 107 | + || !currentSchema.type().equals(recordSchema.type())) { |
| 108 | + |
| 109 | + throw new SchemaEvolutionException( |
| 110 | + String.format( |
| 111 | + "New schema has field '%s' with a different type! " |
| 112 | + + "previous type: %s, current type: %s", |
| 113 | + fieldName, currentSchema, recordSchema)); |
| 114 | + } |
| 115 | + } |
| 116 | + |
| 117 | + private boolean bothPrimitives(Schema s1, Schema s2) { |
| 118 | + return s1.type().isPrimitive() && s2.type().isPrimitive(); |
| 119 | + } |
| 120 | + |
| 121 | + private boolean sameLogicalType(Schema s1, Schema s2) { |
| 122 | + return Objects.equals(s1.type(), s2.type()) |
| 123 | + && Objects.equals(s1.name(), s2.name()) |
| 124 | + && Objects.equals(s1.version(), s2.version()) |
| 125 | + && Objects.equals(s1.parameters(), s2.parameters()); |
| 126 | + } |
| 127 | +} |
0 commit comments