-
Notifications
You must be signed in to change notification settings - Fork 4.2k
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
Support SqlTypes Date and Timestamp (MicrosInstant) in AvroUtils #23969
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -34,6 +34,7 @@ | |
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Objects; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.stream.Collectors; | ||
import javax.annotation.Nonnull; | ||
import net.bytebuddy.description.type.TypeDescription.ForLoadedType; | ||
|
@@ -76,6 +77,7 @@ | |
import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes; | ||
import org.apache.beam.sdk.schemas.logicaltypes.FixedString; | ||
import org.apache.beam.sdk.schemas.logicaltypes.OneOfType; | ||
import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; | ||
import org.apache.beam.sdk.schemas.logicaltypes.VariableBytes; | ||
import org.apache.beam.sdk.schemas.logicaltypes.VariableString; | ||
import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertType; | ||
|
@@ -87,6 +89,7 @@ | |
import org.apache.beam.sdk.transforms.SimpleFunction; | ||
import org.apache.beam.sdk.values.Row; | ||
import org.apache.beam.sdk.values.TypeDescriptor; | ||
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; | ||
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.CaseFormat; | ||
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; | ||
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; | ||
|
@@ -98,7 +101,42 @@ | |
import org.joda.time.Instant; | ||
import org.joda.time.ReadableInstant; | ||
|
||
/** Utils to convert AVRO records to Beam rows. */ | ||
/** | ||
* Utils to convert AVRO records to Beam rows. Imposes a mapping between common avro types and Beam | ||
* portable schemas (https://s.apache.org/beam-schemas): | ||
* | ||
* <pre> | ||
* Avro Beam Field Type | ||
* INT <-----> INT32 | ||
* LONG <-----> INT64 | ||
* FLOAT <-----> FLOAT | ||
* DOUBLE <-----> DOUBLE | ||
* BOOLEAN <-----> BOOLEAN | ||
* STRING <-----> STRING | ||
* BYTES <-----> BYTES | ||
* <------ LogicalType(urn="beam:logical_type:var_bytes:v1") | ||
* FIXED <-----> LogicalType(urn="beam:logical_type:fixed_bytes:v1") | ||
* ARRAY <-----> ARRAY | ||
* ENUM <-----> LogicalType(EnumerationType) | ||
* MAP <-----> MAP | ||
* RECORD <-----> ROW | ||
* UNION <-----> LogicalType(OneOfType) | ||
* LogicalTypes.Date <-----> LogicalType(DATE) | ||
* <------ LogicalType(urn="beam:logical_type:date:v1") | ||
* LogicalTypes.TimestampMillis <-----> DATETIME | ||
* LogicalTypes.TimestampMicros <-----> LogicalType(urn="beam:logical_type:micros_instant:v1") | ||
* LogicalTypes.Decimal <-----> DECIMAL | ||
* </pre> | ||
* | ||
* For SQL CHAR/VARCHAR types, an Avro schema | ||
* | ||
* <pre> | ||
* LogicalType({"type":"string","logicalType":"char","maxLength":MAX_LENGTH}) or | ||
* LogicalType({"type":"string","logicalType":"varchar","maxLength":MAX_LENGTH}) | ||
* </pre> | ||
* | ||
* is used. | ||
*/ | ||
@Experimental(Kind.SCHEMAS) | ||
@SuppressWarnings({ | ||
"nullness", // TODO(https://github.com/apache/beam/issues/20497) | ||
|
@@ -772,9 +810,11 @@ private static Schema.FieldType toFieldType(TypeWithNullability type) { | |
if (logicalType instanceof LogicalTypes.Decimal) { | ||
fieldType = FieldType.DECIMAL; | ||
} else if (logicalType instanceof LogicalTypes.TimestampMillis) { | ||
// TODO: There is a desire to move Beam schema DATETIME to a micros representation. When | ||
// this is done, this logical type needs to be changed. | ||
// TODO(https://github.com/apache/beam/issues/19215) DATETIME primitive will be removed when | ||
// fully migrates to java.time lib from joda-time | ||
fieldType = FieldType.DATETIME; | ||
} else if (logicalType instanceof LogicalTypes.TimestampMicros) { | ||
fieldType = FieldType.logicalType(SqlTypes.TIMESTAMP); | ||
} else if (logicalType instanceof LogicalTypes.Date) { | ||
fieldType = FieldType.DATETIME; | ||
} | ||
|
@@ -887,8 +927,8 @@ private static org.apache.avro.Schema getFieldSchema( | |
break; | ||
|
||
case DATETIME: | ||
// TODO: There is a desire to move Beam schema DATETIME to a micros representation. When | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this TODO actually resolved? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||
// this is done, this logical type needs to be changed. | ||
// TODO(https://github.com/apache/beam/issues/19215) DATETIME primitive will be removed when | ||
// fully migrates to java.time lib from joda-time | ||
baseType = | ||
LogicalTypes.timestampMillis().addToSchema(org.apache.avro.Schema.create(Type.LONG)); | ||
break; | ||
|
@@ -933,10 +973,13 @@ private static org.apache.avro.Schema getFieldSchema( | |
oneOfType.getOneOfSchema().getFields().stream() | ||
.map(x -> getFieldSchema(x.getType(), x.getName(), namespace)) | ||
.collect(Collectors.toList())); | ||
} else if ("DATE".equals(identifier)) { | ||
} else if ("DATE".equals(identifier) || SqlTypes.DATE.getIdentifier().equals(identifier)) { | ||
baseType = LogicalTypes.date().addToSchema(org.apache.avro.Schema.create(Type.INT)); | ||
} else if ("TIME".equals(identifier)) { | ||
baseType = LogicalTypes.timeMillis().addToSchema(org.apache.avro.Schema.create(Type.INT)); | ||
} else if (SqlTypes.TIMESTAMP.getIdentifier().equals(identifier)) { | ||
baseType = | ||
LogicalTypes.timestampMicros().addToSchema(org.apache.avro.Schema.create(Type.LONG)); | ||
} else { | ||
throw new RuntimeException( | ||
"Unhandled logical type " + fieldType.getLogicalType().getIdentifier()); | ||
|
@@ -1061,9 +1104,17 @@ private static org.apache.avro.Schema getFieldSchema( | |
oneOfValue.getValue()); | ||
} | ||
} else if ("DATE".equals(identifier)) { | ||
// "Date" is backed by joda.time.Instant | ||
return Days.daysBetween(Instant.EPOCH, (Instant) value).getDays(); | ||
} else if (SqlTypes.DATE.getIdentifier().equals(identifier)) { | ||
// portable SqlTypes.DATE is backed by java.time.LocalDate | ||
return ((java.time.LocalDate) value).toEpochDay(); | ||
} else if ("TIME".equals(identifier)) { | ||
// "TIME" is backed by joda.time.Instant | ||
return (int) ((Instant) value).getMillis(); | ||
} else if (SqlTypes.TIMESTAMP.getIdentifier().equals(identifier)) { | ||
// portable SqlTypes.TIMESTAMP is backed by java.time.Instant | ||
return getMicrosFromJavaInstant((java.time.Instant) value); | ||
} else { | ||
throw new RuntimeException("Unhandled logical type " + identifier); | ||
} | ||
|
@@ -1137,10 +1188,19 @@ private static org.apache.avro.Schema getFieldSchema( | |
} else { | ||
return convertDateTimeStrict((Long) value, fieldType); | ||
} | ||
} else if (logicalType instanceof LogicalTypes.TimestampMicros) { | ||
if (value instanceof java.time.Instant) { | ||
return convertMicroMillisStrict( | ||
getMicrosFromJavaInstant((java.time.Instant) value), fieldType); | ||
} else { | ||
return convertMicroMillisStrict((Long) value, fieldType); | ||
} | ||
} else if (logicalType instanceof LogicalTypes.Date) { | ||
if (value instanceof ReadableInstant) { | ||
int epochDays = Days.daysBetween(Instant.EPOCH, (ReadableInstant) value).getDays(); | ||
return convertDateStrict(epochDays, fieldType); | ||
} else if (value instanceof java.time.LocalDate) { | ||
return convertDateStrict((int) ((java.time.LocalDate) value).toEpochDay(), fieldType); | ||
} else { | ||
return convertDateStrict((Integer) value, fieldType); | ||
} | ||
|
@@ -1198,6 +1258,14 @@ private static org.apache.avro.Schema getFieldSchema( | |
} | ||
} | ||
|
||
/** Helper method to get epoch micros required by Avro TimeStampMicros logical type. */ | ||
@SuppressWarnings("JavaInstantGetSecondsGetNano") | ||
@VisibleForTesting | ||
static long getMicrosFromJavaInstant(java.time.Instant value) { | ||
return TimeUnit.SECONDS.toMicros(value.getEpochSecond()) | ||
+ TimeUnit.NANOSECONDS.toMicros(value.getNano()); | ||
} | ||
|
||
private static Object convertRecordStrict(GenericRecord record, Schema.FieldType fieldType) { | ||
checkTypeName(fieldType.getTypeName(), Schema.TypeName.ROW, "record"); | ||
return toBeamRowStrict(record, fieldType.getRowSchema()); | ||
|
@@ -1247,6 +1315,17 @@ private static Object convertDateTimeStrict(Long value, Schema.FieldType fieldTy | |
return new Instant(value); | ||
} | ||
|
||
private static Object convertMicroMillisStrict(Long value, Schema.FieldType fieldType) { | ||
checkTypeName( | ||
fieldType.getTypeName(), TypeName.LOGICAL_TYPE, SqlTypes.TIMESTAMP.getIdentifier()); | ||
checkArgument( | ||
fieldType.getLogicalType().getIdentifier().equals(SqlTypes.TIMESTAMP.getIdentifier())); | ||
|
||
return java.time.Instant.ofEpochSecond( | ||
TimeUnit.MICROSECONDS.toSeconds(value), | ||
TimeUnit.MICROSECONDS.toNanos(Math.floorMod(value, TimeUnit.SECONDS.toMicros(1)))); | ||
} | ||
|
||
private static Object convertFloatStrict(Float value, Schema.FieldType fieldType) { | ||
checkTypeName(fieldType.getTypeName(), Schema.TypeName.FLOAT, "float"); | ||
return value; | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This piece apparently can introduce breaking change. Previously TimestampMicros Avro Logical Type will return
FieldType.INT64
and now it returnsFieldType.logicalType(SqlTypes.TIMESTAMP)
. For modules not yet support this Beam logical type this will fail existing user code.