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

[BEAM-12385] Handle VARCHAR and other SQL specific logical types in AvroUtils #14858

Closed
wants to merge 7 commits into from
Original file line number Diff line number Diff line change
Expand Up @@ -906,6 +906,23 @@ private static org.apache.avro.Schema getFieldSchema(
.map(x -> getFieldSchema(x.getType(), x.getName(), namespace))
.collect(Collectors.toList()));
break;

case "NVARCHAR":
anantdamle marked this conversation as resolved.
Show resolved Hide resolved
case "VARCHAR":
case "LONGNVARCHAR":
case "LONGVARCHAR":
baseType = org.apache.avro.Schema.create(Type.STRING);
anantdamle marked this conversation as resolved.
Show resolved Hide resolved
break;

case "DATE":
baseType = LogicalTypes.date().addToSchema(org.apache.avro.Schema.create(Type.INT));
break;

case "TIME":
baseType =
LogicalTypes.timeMillis().addToSchema(org.apache.avro.Schema.create(Type.INT));
break;

default:
throw new RuntimeException(
"Unhandled logical type " + fieldType.getLogicalType().getIdentifier());
Expand Down Expand Up @@ -1017,6 +1034,19 @@ private static org.apache.avro.Schema getFieldSchema(
typeWithNullability.type.getTypes().get(oneOfValue.getCaseType().getValue()),
oneOfValue.getValue());
}

case "NVARCHAR":
anantdamle marked this conversation as resolved.
Show resolved Hide resolved
case "VARCHAR":
case "LONGNVARCHAR":
case "LONGVARCHAR":
return new Utf8((String) value);

case "DATE":
return Days.daysBetween(Instant.EPOCH, (Instant) value).getDays();

case "TIME":
return (int) ((Instant) value).getMillis();

default:
throw new RuntimeException(
"Unhandled logical type " + fieldType.getLogicalType().getIdentifier());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import com.pholser.junit.quickcheck.runner.JUnitQuickcheck;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.sql.JDBCType;
import java.util.List;
import java.util.Map;
import org.apache.avro.Conversions;
Expand Down Expand Up @@ -57,8 +58,13 @@
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.Days;
import org.joda.time.Instant;
import org.joda.time.LocalTime;
import org.junit.Test;
import org.junit.runner.RunWith;

Expand Down Expand Up @@ -550,6 +556,138 @@ public void testUnionFieldInBeamSchema() {
assertEquals(expectedGenericRecord, AvroUtils.toGenericRecord(row, avroSchema));
}

@Test
public void testJdbcLogicalVarCharRowDataToAvroSchema() {
String expectedAvroSchemaJson =
"{ "
+ " \"name\": \"topLevelRecord\", "
+ " \"type\": \"record\", "
+ " \"fields\": [{ "
+ " \"name\": \"my_varchar_field\", "
+ " \"type\": \"string\" "
+ " }, "
+ " { "
+ " \"name\": \"my_longvarchar_field\", "
+ " \"type\": \"string\" "
+ " }, "
+ " { "
+ " \"name\": \"my_nvarchar_field\", "
+ " \"type\": \"string\" "
+ " }, "
+ " { "
+ " \"name\": \"my_longnvarchar_field\", "
+ " \"type\": \"string\" "
+ " } "
+ " ] "
+ "}";

Schema beamSchema =
Schema.builder()
.addField(Field.of("my_varchar_field", FieldType.logicalType(JdbcType.VARCHAR)))
.addField(Field.of("my_longvarchar_field", FieldType.logicalType(JdbcType.LONGVARCHAR)))
.addField(Field.of("my_nvarchar_field", FieldType.logicalType(JdbcType.NVARCHAR)))
.addField(
Field.of("my_longnvarchar_field", FieldType.logicalType(JdbcType.LONGNVARCHAR)))
.build();

assertEquals(
new org.apache.avro.Schema.Parser().parse(expectedAvroSchemaJson),
AvroUtils.toAvroSchema(beamSchema));
}

@Test
public void testJdbcLogicalVarCharRowDataToGenericRecord() {
Schema beamSchema =
Schema.builder()
.addField(Field.of("my_varchar_field", FieldType.logicalType(JdbcType.VARCHAR)))
.addField(Field.of("my_longvarchar_field", FieldType.logicalType(JdbcType.LONGVARCHAR)))
.addField(Field.of("my_nvarchar_field", FieldType.logicalType(JdbcType.NVARCHAR)))
.addField(
Field.of("my_longnvarchar_field", FieldType.logicalType(JdbcType.LONGNVARCHAR)))
.build();

Row rowData =
Row.withSchema(beamSchema)
.addValue("varchar_value")
.addValue("longvarchar_value")
.addValue("nvarchar_value")
.addValue("longnvarchar_value")
.build();

org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(beamSchema);
GenericRecord expectedRecord =
new GenericRecordBuilder(avroSchema)
.set("my_varchar_field", "varchar_value")
.set("my_longvarchar_field", "longvarchar_value")
.set("my_nvarchar_field", "nvarchar_value")
.set("my_longnvarchar_field", "longnvarchar_value")
.build();

assertEquals(expectedRecord, AvroUtils.toGenericRecord(rowData, avroSchema));
}

@Test
public void testJdbcLogicalDateAndTimeRowDataToAvroSchema() {
String expectedAvroSchemaJson =
"{ "
+ " \"name\": \"topLevelRecord\", "
+ " \"type\": \"record\", "
+ " \"fields\": [{ "
+ " \"name\": \"my_date_field\", "
+ " \"type\": { \"type\": \"int\", \"logicalType\": \"date\" }"
+ " }, "
+ " { "
+ " \"name\": \"my_time_field\", "
+ " \"type\": { \"type\": \"int\", \"logicalType\": \"time-millis\" }"
+ " }"
+ " ] "
+ "}";

Schema beamSchema =
Schema.builder()
.addField(Field.of("my_date_field", FieldType.logicalType(JdbcType.DATE)))
.addField(Field.of("my_time_field", FieldType.logicalType(JdbcType.TIME)))
.build();

assertEquals(
new org.apache.avro.Schema.Parser().parse(expectedAvroSchemaJson),
AvroUtils.toAvroSchema(beamSchema));
}

@Test
public void testJdbcLogicalDateAndTimeRowDataToGenericRecord() {
// Test Fixed clock at
DateTime testDateTime = DateTime.parse("2021-05-29T11:15:16.234Z");

Schema beamSchema =
Schema.builder()
.addField(Field.of("my_date_field", FieldType.logicalType(JdbcType.DATE)))
.addField(Field.of("my_time_field", FieldType.logicalType(JdbcType.TIME)))
.build();

Row rowData =
Row.withSchema(beamSchema)
.addValue(testDateTime.toLocalDate().toDateTime(LocalTime.MIDNIGHT).toInstant())
.addValue(Instant.ofEpochMilli(testDateTime.toLocalTime().millisOfDay().get()))
.build();

int daysFromEpoch =
Days.daysBetween(
Instant.EPOCH,
testDateTime.toLocalDate().toDateTime(LocalTime.MIDNIGHT).toInstant())
.getDays();
int timeSinceMidNight = testDateTime.toLocalTime().getMillisOfDay();

org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(beamSchema);
GenericRecord expectedRecord =
new GenericRecordBuilder(avroSchema)
.set("my_date_field", daysFromEpoch)
.set("my_time_field", timeSinceMidNight)
.build();

assertEquals(expectedRecord, AvroUtils.toGenericRecord(rowData, avroSchema));
}

@Test
public void testBeamRowToGenericRecord() {
GenericRecord genericRecord = AvroUtils.toGenericRecord(getBeamRow(), null);
Expand Down Expand Up @@ -640,4 +778,56 @@ public void testNullSchemas() {
AvroUtils.getFromRowFunction(GenericRecord.class),
AvroUtils.getFromRowFunction(GenericRecord.class));
}

/** Helper class that simulates a JDBC Logical types. */
private static class JdbcType<T> implements Schema.LogicalType<T, T> {

private static final JdbcType<String> VARCHAR =
new JdbcType<>(JDBCType.VARCHAR, FieldType.INT32, FieldType.STRING);
private static final JdbcType<String> NVARCHAR =
new JdbcType<>(JDBCType.NVARCHAR, FieldType.INT32, FieldType.STRING);
private static final JdbcType<String> LONGVARCHAR =
new JdbcType<>(JDBCType.LONGVARCHAR, FieldType.INT32, FieldType.STRING);
private static final JdbcType<String> LONGNVARCHAR =
new JdbcType<>(JDBCType.LONGNVARCHAR, FieldType.INT32, FieldType.STRING);
private static final JdbcType<Instant> DATE =
new JdbcType<>(JDBCType.DATE, FieldType.STRING, FieldType.DATETIME);
private static final JdbcType<Instant> TIME =
new JdbcType<>(JDBCType.TIME, FieldType.STRING, FieldType.DATETIME);

private final String identifier;
private final FieldType argumentType;
private final FieldType baseType;

private JdbcType(JDBCType jdbcType, FieldType argumentType, FieldType baseType) {
this.identifier = jdbcType.getName();
this.argumentType = argumentType;
this.baseType = baseType;
}

@Override
public String getIdentifier() {
return identifier;
}

@Override
public @Nullable FieldType getArgumentType() {
return argumentType;
}

@Override
public FieldType getBaseType() {
return baseType;
}

@Override
public @NonNull T toBaseType(@NonNull T input) {
return input;
}

@Override
public @NonNull T toInputType(@NonNull T base) {
return base;
}
}
}