4444import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .base .Functions ;
4545import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .base .Preconditions ;
4646import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .collect .ImmutableMap ;
47- import org .joda .time .Days ;
48- import org .joda .time .Instant ;
49- import org .joda .time .ReadableInstant ;
47+ import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .primitives .Bytes ;
5048
5149/**
5250 * Utility methods for converting Avro {@link GenericRecord} objects to dynamic protocol message,
5351 * for use with the Storage write API.
5452 */
5553public class AvroGenericRecordToStorageApiProto {
54+
55+ private static final org .joda .time .LocalDate EPOCH_DATE = new org .joda .time .LocalDate (1970 , 1 , 1 );
56+
5657 static final Map <Schema .Type , TableFieldSchema .Type > PRIMITIVE_TYPES =
5758 ImmutableMap .<Schema .Type , TableFieldSchema .Type >builder ()
5859 .put (Schema .Type .INT , TableFieldSchema .Type .INT64 )
@@ -67,14 +68,37 @@ public class AvroGenericRecordToStorageApiProto {
6768 .build ();
6869
6970 // A map of supported logical types to the protobuf field type.
70- static final Map <String , TableFieldSchema .Type > LOGICAL_TYPES =
71- ImmutableMap .<String , TableFieldSchema .Type >builder ()
72- .put (LogicalTypes .date ().getName (), TableFieldSchema .Type .DATE )
73- .put (LogicalTypes .decimal (1 ).getName (), TableFieldSchema .Type .BIGNUMERIC )
74- .put (LogicalTypes .timestampMicros ().getName (), TableFieldSchema .Type .TIMESTAMP )
75- .put (LogicalTypes .timestampMillis ().getName (), TableFieldSchema .Type .TIMESTAMP )
76- .put (LogicalTypes .uuid ().getName (), TableFieldSchema .Type .STRING )
77- .build ();
71+ static Optional <TableFieldSchema .Type > logicalTypes (LogicalType logicalType ) {
72+ switch (logicalType .getName ()) {
73+ case "date" :
74+ return Optional .of (TableFieldSchema .Type .DATE );
75+ case "time-micros" :
76+ return Optional .of (TableFieldSchema .Type .TIME );
77+ case "time-millis" :
78+ return Optional .of (TableFieldSchema .Type .TIME );
79+ case "decimal" :
80+ LogicalTypes .Decimal decimal = (LogicalTypes .Decimal ) logicalType ;
81+ int scale = decimal .getScale ();
82+ int precision = decimal .getPrecision ();
83+ if (scale > 9 || precision - scale > 29 ) {
84+ return Optional .of (TableFieldSchema .Type .BIGNUMERIC );
85+ } else {
86+ return Optional .of (TableFieldSchema .Type .NUMERIC );
87+ }
88+ case "timestamp-micros" :
89+ return Optional .of (TableFieldSchema .Type .TIMESTAMP );
90+ case "timestamp-millis" :
91+ return Optional .of (TableFieldSchema .Type .TIMESTAMP );
92+ case "local-timestamp-micros" :
93+ return Optional .of (TableFieldSchema .Type .DATETIME );
94+ case "local-timestamp-millis" :
95+ return Optional .of (TableFieldSchema .Type .DATETIME );
96+ case "uuid" :
97+ return Optional .of (TableFieldSchema .Type .STRING );
98+ default :
99+ return Optional .empty ();
100+ }
101+ }
78102
79103 static final Map <Schema .Type , Function <Object , Object >> PRIMITIVE_ENCODERS =
80104 ImmutableMap .<Schema .Type , Function <Object , Object >>builder ()
@@ -92,16 +116,15 @@ public class AvroGenericRecordToStorageApiProto {
92116 // A map of supported logical types to their encoding functions.
93117 static final Map <String , BiFunction <LogicalType , Object , Object >> LOGICAL_TYPE_ENCODERS =
94118 ImmutableMap .<String , BiFunction <LogicalType , Object , Object >>builder ()
95- .put (LogicalTypes .date ().getName (), (logicalType , value ) -> convertDate (value ))
96- .put (
97- LogicalTypes .decimal (1 ).getName (), AvroGenericRecordToStorageApiProto ::convertDecimal )
98- .put (
99- LogicalTypes .timestampMicros ().getName (),
100- (logicalType , value ) -> convertTimestamp (value , true ))
101- .put (
102- LogicalTypes .timestampMillis ().getName (),
103- (logicalType , value ) -> convertTimestamp (value , false ))
104- .put (LogicalTypes .uuid ().getName (), (logicalType , value ) -> convertUUID (value ))
119+ .put ("date" , (logicalType , value ) -> convertDate (value ))
120+ .put ("time-micros" , (logicalType , value ) -> convertTime (value , true ))
121+ .put ("time-millis" , (logicalType , value ) -> convertTime (value , false ))
122+ .put ("decimal" , AvroGenericRecordToStorageApiProto ::convertDecimal )
123+ .put ("timestamp-micros" , (logicalType , value ) -> convertTimestamp (value , true ))
124+ .put ("timestamp-millis" , (logicalType , value ) -> convertTimestamp (value , false ))
125+ .put ("local-timestamp-micros" , (logicalType , value ) -> convertDateTime (value , true ))
126+ .put ("local-timestamp-millis" , (logicalType , value ) -> convertDateTime (value , false ))
127+ .put ("uuid" , (logicalType , value ) -> convertUUID (value ))
105128 .build ();
106129
107130 static String convertUUID (Object value ) {
@@ -115,34 +138,96 @@ static String convertUUID(Object value) {
115138 }
116139
117140 static Long convertTimestamp (Object value , boolean micros ) {
118- if (value instanceof ReadableInstant ) {
119- return ((ReadableInstant ) value ).getMillis () * (micros ? 1000 : 1 );
141+ if (value instanceof org .joda .time .ReadableInstant ) {
142+ return ((org .joda .time .ReadableInstant ) value ).getMillis () * 1_000L ;
143+ } else if (value instanceof java .time .Instant ) {
144+ java .time .Instant instant = (java .time .Instant ) value ;
145+ long seconds = instant .getEpochSecond ();
146+ int nanos = instant .getNano ();
147+
148+ if (seconds < 0 && nanos > 0 ) {
149+ long ms = Math .multiplyExact (seconds + 1 , 1_000_000L );
150+ long adjustment = (nanos / 1_000L ) - 1_000_000L ;
151+ return Math .addExact (ms , adjustment );
152+ } else {
153+ long ms = Math .multiplyExact (seconds , 1_000_000L );
154+ return Math .addExact (ms , nanos / 1_000L );
155+ }
120156 } else {
121157 Preconditions .checkArgument (
122- value instanceof Long , "Expecting a value as Long type (millis )." );
123- return (Long ) value ;
158+ value instanceof Long , "Expecting a value as Long type (timestamp )." );
159+ return (micros ? 1 : 1_000L ) * (( Long ) value ) ;
124160 }
125161 }
126162
127163 static Integer convertDate (Object value ) {
128- if (value instanceof ReadableInstant ) {
129- return Days .daysBetween (Instant .EPOCH , (ReadableInstant ) value ).getDays ();
164+ if (value instanceof org .joda .time .LocalDate ) {
165+ return org .joda .time .Days .daysBetween (EPOCH_DATE , (org .joda .time .LocalDate ) value ).getDays ();
166+ } else if (value instanceof java .time .LocalDate ) {
167+ return (int ) ((java .time .LocalDate ) value ).toEpochDay ();
130168 } else {
131169 Preconditions .checkArgument (
132170 value instanceof Integer , "Expecting a value as Integer type (days)." );
133171 return (Integer ) value ;
134172 }
135173 }
136174
175+ static Long convertTime (Object value , boolean micros ) {
176+ if (value instanceof org .joda .time .LocalTime ) {
177+ return 1_000L * (long ) ((org .joda .time .LocalTime ) value ).getMillisOfDay ();
178+ } else if (value instanceof java .time .LocalTime ) {
179+ return java .util .concurrent .TimeUnit .NANOSECONDS .toMicros (
180+ ((java .time .LocalTime ) value ).toNanoOfDay ());
181+ } else {
182+ if (micros ) {
183+ Preconditions .checkArgument (
184+ value instanceof Long , "Expecting a value as Long type (time)." );
185+ return (Long ) value ;
186+ } else {
187+ Preconditions .checkArgument (
188+ value instanceof Integer , "Expecting a value as Integer type (time)." );
189+ return 1_000L * (Integer ) value ;
190+ }
191+ }
192+ }
193+
194+ static Long convertDateTime (Object value , boolean micros ) {
195+ if (value instanceof org .joda .time .LocalDateTime ) {
196+ // we should never come here as local-timestamp has been added after joda deprecation
197+ // implement nonetheless for consistency
198+ org .joda .time .DateTime dateTime =
199+ ((org .joda .time .LocalDateTime ) value ).toDateTime (org .joda .time .DateTimeZone .UTC );
200+ return 1_000L * dateTime .getMillis ();
201+ } else if (value instanceof java .time .LocalDateTime ) {
202+ java .time .Instant instant =
203+ ((java .time .LocalDateTime ) value ).toInstant (java .time .ZoneOffset .UTC );
204+ return convertTimestamp (instant , micros );
205+ } else {
206+ Preconditions .checkArgument (
207+ value instanceof Long , "Expecting a value as Long type (local-timestamp)." );
208+ return (micros ? 1 : 1_000L ) * ((Long ) value );
209+ }
210+ }
211+
137212 static ByteString convertDecimal (LogicalType logicalType , Object value ) {
138- ByteBuffer byteBuffer = (ByteBuffer ) value ;
139- BigDecimal bigDecimal =
140- new Conversions .DecimalConversion ()
141- .fromBytes (
142- byteBuffer .duplicate (),
143- Schema .create (Schema .Type .NULL ), // dummy schema, not used
144- logicalType );
145- return BeamRowToStorageApiProto .serializeBigDecimalToNumeric (bigDecimal );
213+ ByteBuffer byteBuffer ;
214+ if (value instanceof BigDecimal ) {
215+ // BigDecimalByteStringEncoder does not support parametrized NUMERIC/BIGNUMERIC
216+ byteBuffer =
217+ new Conversions .DecimalConversion ()
218+ .toBytes (
219+ (BigDecimal ) value ,
220+ Schema .create (Schema .Type .NULL ), // dummy schema, not used
221+ logicalType );
222+ } else {
223+ Preconditions .checkArgument (
224+ value instanceof ByteBuffer , "Expecting a value as ByteBuffer type (decimal)." );
225+ byteBuffer = (ByteBuffer ) value ;
226+ }
227+ byte [] bytes = new byte [byteBuffer .remaining ()];
228+ byteBuffer .duplicate ().get (bytes );
229+ Bytes .reverse (bytes );
230+ return ByteString .copyFrom (bytes );
146231 }
147232
148233 static ByteString convertBytes (Object value ) {
@@ -223,7 +308,7 @@ public static DynamicMessage messageFromGenericRecord(
223308 return builder .build ();
224309 }
225310
226- private static TableFieldSchema fieldDescriptorFromAvroField (Schema .Field field ) {
311+ private static TableFieldSchema fieldDescriptorFromAvroField (org . apache . avro . Schema .Field field ) {
227312 @ Nullable Schema schema = field .schema ();
228313 Preconditions .checkNotNull (schema , "Unexpected null schema!" );
229314 if (StorageApiCDC .COLUMNS .contains (field .name ())) {
@@ -292,17 +377,34 @@ private static TableFieldSchema fieldDescriptorFromAvroField(Schema.Field field)
292377 break ;
293378 default :
294379 elementType = TypeWithNullability .create (schema ).getType ();
380+ Optional <LogicalType > logicalType =
381+ Optional .ofNullable (LogicalTypes .fromSchema (elementType ));
295382 @ Nullable
296383 TableFieldSchema .Type primitiveType =
297- Optional . ofNullable ( LogicalTypes . fromSchema ( elementType ))
298- .map ( logicalType -> LOGICAL_TYPES . get ( logicalType . getName ()) )
384+ logicalType
385+ .flatMap ( AvroGenericRecordToStorageApiProto :: logicalTypes )
299386 .orElse (PRIMITIVE_TYPES .get (elementType .getType ()));
300387 if (primitiveType == null ) {
301388 throw new RuntimeException ("Unsupported type " + elementType .getType ());
302389 }
303390 // a scalar will be required by default, if defined as part of union then
304391 // caller will set nullability requirements
305392 builder = builder .setType (primitiveType );
393+ // parametrized types
394+ if (logicalType .isPresent () && logicalType .get ().getName ().equals ("decimal" )) {
395+ LogicalTypes .Decimal decimal = (LogicalTypes .Decimal ) logicalType .get ();
396+ int precision = decimal .getPrecision ();
397+ int scale = decimal .getScale ();
398+ if (!(precision == 38 && scale == 9 ) // NUMERIC
399+ && !(precision == 77 && scale == 38 ) // BIGNUMERIC
400+ ) {
401+ // parametrized type
402+ builder = builder .setPrecision (precision );
403+ if (scale != 0 ) {
404+ builder = builder .setScale (scale );
405+ }
406+ }
407+ }
306408 }
307409 if (builder .getMode () != TableFieldSchema .Mode .REPEATED ) {
308410 if (TypeWithNullability .create (schema ).isNullable ()) {
0 commit comments