Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-5891: Proper handling of LogicalTypes in Cast #4633

Merged
merged 4 commits into from Aug 20, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -164,13 +164,17 @@ private Schema getOrBuildSchema(Schema valueSchema) {
} else {
builder = SchemaUtil.copySchemaBasics(valueSchema, SchemaBuilder.struct());
for (Field field : valueSchema.fields()) {
SchemaBuilder fieldBuilder =
convertFieldType(casts.containsKey(field.name()) ? casts.get(field.name()) : field.schema().type());
if (field.schema().isOptional())
fieldBuilder.optional();
if (field.schema().defaultValue() != null)
fieldBuilder.defaultValue(castValueToType(field.schema().defaultValue(), fieldBuilder.type()));
builder.field(field.name(), fieldBuilder.build());
if (casts.containsKey(field.name())) {
SchemaBuilder fieldBuilder = convertFieldType(casts.get(field.name()));
if (field.schema().isOptional())
fieldBuilder.optional();
if (field.schema().defaultValue() != null)
fieldBuilder.defaultValue(castValueToType(field.schema().defaultValue(), fieldBuilder.type()));
builder.field(field.name(), fieldBuilder.build());
} else {
builder.field(field.name(), field.schema());
}

}
}

Expand Down
Expand Up @@ -21,12 +21,14 @@
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.After;
import org.junit.Test;

import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.Map;

Expand Down Expand Up @@ -304,6 +306,7 @@ public void castFieldsWithSchema() {
builder.field("boolean", Schema.BOOLEAN_SCHEMA);
builder.field("string", Schema.STRING_SCHEMA);
builder.field("optional", Schema.OPTIONAL_FLOAT32_SCHEMA);
builder.field("timestamp", Timestamp.SCHEMA);
Schema supportedTypesSchema = builder.build();

Struct recordValue = new Struct(supportedTypesSchema);
Expand All @@ -315,6 +318,7 @@ public void castFieldsWithSchema() {
recordValue.put("float64", -64.);
recordValue.put("boolean", true);
recordValue.put("string", "42");
recordValue.put("timestamp", new Date(0));
// optional field intentionally omitted

SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0,
Expand All @@ -331,6 +335,7 @@ public void castFieldsWithSchema() {
assertEquals(true, ((Struct) transformed.value()).schema().field("float64").schema().defaultValue());
assertEquals((byte) 1, ((Struct) transformed.value()).get("boolean"));
assertEquals(42, ((Struct) transformed.value()).get("string"));
assertEquals(new Date(0), ((Struct) transformed.value()).get("timestamp"));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since this PR modifies the logic that builds the schema, it might be good to add a block that tests the schemas of each field in the resulting schema. Something like the following (which I don't know if it compiles):


        Schema transformedSchema = ((Struct) transformed.value()).schema();
        assertEquals(Schema.INT16_SCHEMA, transformedSchema.field("int8").schema());
        assertEquals(Schema.OPTIONAL_INT32_SCHEMA, transformedSchema.field("int16").schema());
        assertEquals(Schema.INT64_SCHEMA, transformedSchema.field("int32").schema());
        assertEquals(Schema.BOOLEAN_SCHEMA, transformedSchema.field("int64").schema());
        assertEquals(Schema.FLOAT64_SCHEMA, transformedSchema.field("float32").schema());
        assertEquals(Schema.BOOLEAN_SCHEMA, transformedSchema.field("float64").schema());
        assertEquals(Schema.INT8_SCHEMA, transformedSchema.field("boolean").schema());
        assertEquals(Schema.INT32_SCHEMA, transformedSchema.field("string").schema());
        assertEquals(Schema.OPTIONAL_INT32_SCHEMA, transformedSchema.field("optional").schema());
        // The following fields are not changed
        //assertEquals(Timestamp.SCHEMA, transformedSchema.field("timestamp").schema());

assertNull(((Struct) transformed.value()).get("optional"));
}

Expand Down