private static Object projectArray(Schema source, Object record, Schema target) throws SchemaProjectorException { List<?> array = (List<?>) record; List<Object> retArray = new ArrayList<>(); for (Object entry : array) { retArray.add(project(source.valueSchema(), entry, target.valueSchema())); } return retArray; }
private static Object projectRequiredSchema(Schema source, Object record, Schema target) throws SchemaProjectorException { switch (target.type()) { case INT8: case INT16: case INT32: case INT64: case FLOAT32: case FLOAT64: case BOOLEAN: case BYTES: case STRING: return projectPrimitive(source, record, target); case STRUCT: return projectStruct(source, (Struct) record, target); case ARRAY: return projectArray(source, record, target); case MAP: return projectMap(source, record, target); } return null; }
/** * This method project a value between compatible schemas and throw exceptions when non compatible schemas are provided * @param source the schema used to construct the record * @param record the value to project from source schema to target schema * @param target the schema to project the record to * @return the projected value with target schema * @throws SchemaProjectorException */ public static Object project(Schema source, Object record, Schema target) throws SchemaProjectorException { checkMaybeCompatible(source, target); if (source.isOptional() && !target.isOptional()) { if (target.defaultValue() != null) { if (record != null) { return projectRequiredSchema(source, record, target); } else { return target.defaultValue(); } } else { throw new SchemaProjectorException("Writer schema is optional, however, target schema does not provide a default value."); } } else { if (record != null) { return projectRequiredSchema(source, record, target); } else { return null; } } }
assert target.type().isPrimitive(); Object result; if (isPromotable(source.type(), target.type()) && record instanceof Number) { Number numberRecord = (Number) record; switch (target.type()) {
private static void checkMaybeCompatible(Schema source, Schema target) { if (source.type() != target.type() && !isPromotable(source.type(), target.type())) { throw new SchemaProjectorException("Schema type mismatch. source type: " + source.type() + " and target type: " + target.type()); } else if (!Objects.equals(source.name(), target.name())) { throw new SchemaProjectorException("Schema name mismatch. source name: " + source.name() + " and target name: " + target.name()); } else if (!Objects.equals(source.parameters(), target.parameters())) { throw new SchemaProjectorException("Schema parameters not equal. source parameters: " + source.parameters() + " and target parameters: " + target.parameters()); } }
private static Object projectMap(Schema source, Object record, Schema target) throws SchemaProjectorException { Map<?, ?> map = (Map<?, ?>) record; Map<Object, Object> retMap = new HashMap<>(); for (Map.Entry<?, ?> entry : map.entrySet()) { Object key = entry.getKey(); Object value = entry.getValue(); Object retKey = project(source.keySchema(), key, target.keySchema()); Object retValue = project(source.valueSchema(), value, target.valueSchema()); retMap.put(retKey, retValue); } return retMap; }
public static SinkRecord project(SinkRecord record, Schema currentSchema, Compatibility compatibility) { switch (compatibility) { case BACKWARD: case FULL: case FORWARD: Schema sourceSchema = record.valueSchema(); Object value = record.value(); if (sourceSchema == currentSchema || sourceSchema.equals(currentSchema)) { return record; } Object projected = SchemaProjector.project(sourceSchema, value, currentSchema); return new SinkRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), currentSchema, projected, record.kafkaOffset()); default: return record; } } }
private static Object projectStruct(Schema source, Struct sourceStruct, Schema target) throws SchemaProjectorException { Struct targetStruct = new Struct(target); for (Field targetField : target.fields()) { String fieldName = targetField.name(); Field sourceField = source.field(fieldName); if (sourceField != null) { Object sourceFieldValue = sourceStruct.get(fieldName); try { Object targetFieldValue = project(sourceField.schema(), sourceFieldValue, targetField.schema()); targetStruct.put(fieldName, targetFieldValue); } catch (SchemaProjectorException e) { throw new SchemaProjectorException("Error projecting " + sourceField.name(), e); } } else if (targetField.schema().isOptional()) { // Ignore missing field } else if (targetField.schema().defaultValue() != null) { targetStruct.put(fieldName, targetField.schema().defaultValue()); } else { throw new SchemaProjectorException("Required field `" + fieldName + "` is missing from source schema: " + source); } } return targetStruct; }