5050import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .annotations .VisibleForTesting ;
5151import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .io .BaseEncoding ;
5252import org .checkerframework .checker .nullness .qual .Nullable ;
53- import org .joda .time .format .DateTimeFormat ;
54- import org .joda .time .format .DateTimeFormatter ;
5553
5654/** A set of utilities for working with Avro files. */
5755class BigQueryAvroUtils {
@@ -60,7 +58,7 @@ class BigQueryAvroUtils {
6058 Optional .ofNullable (Schema .class .getPackage ())
6159 .map (Package ::getImplementationVersion )
6260 .orElse ("" );
63-
61+ private static final String TIMESTAMP_NANOS_LOGICAL_TYPE = "timestamp-nanos" ;
6462 // org.apache.avro.LogicalType
6563 static class DateTimeLogicalType extends LogicalType {
6664 public DateTimeLogicalType () {
@@ -161,36 +159,73 @@ static Schema getPrimitiveType(TableFieldSchema schema, Boolean useAvroLogicalTy
161159 * Formats BigQuery seconds-since-epoch into String matching JSON export. Thread-safe and
162160 * immutable.
163161 */
164- private static final DateTimeFormatter DATE_AND_SECONDS_FORMATTER =
165- DateTimeFormat .forPattern ("yyyy-MM-dd HH:mm:ss" ).withZoneUTC ();
166-
167- @ VisibleForTesting
168- static String formatTimestamp (Long timestampMicro ) {
169- String dateTime = formatDatetime (timestampMicro );
170- return dateTime + " UTC" ;
162+ private static final java .time .format .DateTimeFormatter DATE_TIME_FORMATTER =
163+ java .time .format .DateTimeFormatter .ofPattern ("yyyy-MM-dd HH:mm:ss" )
164+ .withZone (java .time .ZoneOffset .UTC );
165+
166+ /** Enum to define the precision of a timestamp since the epoch. */
167+ enum TimestampPrecision {
168+ MILLISECONDS ,
169+ MICROSECONDS ,
170+ NANOSECONDS ;
171+
172+ /** Converts an epoch value of this precision to an Instant. */
173+ java .time .Instant toInstant (long epochValue ) {
174+ switch (this ) {
175+ case MILLISECONDS :
176+ return java .time .Instant .ofEpochMilli (epochValue );
177+ case MICROSECONDS :
178+ {
179+ long seconds = Math .floorDiv (epochValue , 1_000_000L );
180+ long microsOfSecond = Math .floorMod (epochValue , 1_000_000L );
181+ return java .time .Instant .ofEpochSecond (seconds , microsOfSecond * 1_000L );
182+ }
183+ case NANOSECONDS :
184+ {
185+ long seconds = Math .floorDiv (epochValue , 1_000_000_000L );
186+ long nanosOfSecond = Math .floorMod (epochValue , 1_000_000_000L );
187+ return java .time .Instant .ofEpochSecond (seconds , nanosOfSecond );
188+ }
189+ default :
190+ throw new IllegalStateException ("Unknown precision: " + this );
191+ }
192+ }
171193 }
172194
195+ /**
196+ * Formats an Instant with minimal fractional second precision. Shows 0, 3, 6, or 9 decimal places
197+ * based on actual precision of the value.
198+ */
173199 @ VisibleForTesting
174- static String formatDatetime (Long timestampMicro ) {
175- // timestampMicro is in "microseconds since epoch" format,
176- // e.g., 1452062291123456L means "2016-01-06 06:38:11.123456 UTC".
177- // Separate into seconds and microseconds.
178- long timestampSec = timestampMicro / 1_000_000 ;
179- long micros = timestampMicro % 1_000_000 ;
180- if (micros < 0 ) {
181- micros += 1_000_000 ;
182- timestampSec -= 1 ;
183- }
184- String dayAndTime = DATE_AND_SECONDS_FORMATTER .print (timestampSec * 1000 );
185- if (micros == 0 ) {
186- return dayAndTime ;
187- } else if (micros % 1000 == 0 ) {
188- return String .format ("%s.%03d" , dayAndTime , micros / 1000 );
200+ @ SuppressWarnings ("JavaInstantGetSecondsGetNano" )
201+ static String formatDatetime (java .time .Instant instant ) {
202+ String dateTime = DATE_TIME_FORMATTER .format (instant );
203+ int nanos = instant .getNano ();
204+
205+ if (nanos == 0 ) {
206+ return dateTime ;
207+ } else if (nanos % 1_000_000 == 0 ) {
208+ return dateTime + String .format (".%03d" , nanos / 1_000_000 );
209+ } else if (nanos % 1_000 == 0 ) {
210+ return dateTime + String .format (".%06d" , nanos / 1_000 );
189211 } else {
190- return String .format ("%s.%06d " , dayAndTime , micros );
212+ return dateTime + String .format (".%09d " , nanos );
191213 }
192214 }
193215
216+ @ VisibleForTesting
217+ static String formatDatetime (long epochValue , TimestampPrecision precision ) {
218+ return formatDatetime (precision .toInstant (epochValue ));
219+ }
220+
221+ static String formatTimestamp (java .time .Instant instant ) {
222+ return formatDatetime (instant ) + " UTC" ;
223+ }
224+
225+ static String formatTimestamp (long epochValue , TimestampPrecision precision ) {
226+ return formatTimestamp (precision .toInstant (epochValue ));
227+ }
228+
194229 /**
195230 * This method formats a BigQuery DATE value into a String matching the format used by JSON
196231 * export. Date records are stored in "days since epoch" format, and BigQuery uses the proleptic
@@ -335,7 +370,6 @@ private static Object convertRequiredField(String name, Schema schema, Object v)
335370 // REQUIRED fields are represented as the corresponding Avro types. For example, a BigQuery
336371 // INTEGER type maps to an Avro LONG type.
337372 checkNotNull (v , "REQUIRED field %s should not be null" , name );
338-
339373 Type type = schema .getType ();
340374 LogicalType logicalType = schema .getLogicalType ();
341375 switch (type ) {
@@ -364,21 +398,26 @@ private static Object convertRequiredField(String name, Schema schema, Object v)
364398 } else if (logicalType instanceof LogicalTypes .TimestampMillis ) {
365399 // Write only: SQL type TIMESTAMP
366400 // ideally Instant but TableRowJsonCoder encodes as String
367- return formatTimestamp ((Long ) v * 1000L );
401+ return formatTimestamp ((Long ) v , TimestampPrecision . MILLISECONDS );
368402 } else if (logicalType instanceof LogicalTypes .TimestampMicros ) {
369403 // SQL type TIMESTAMP
370404 // ideally Instant but TableRowJsonCoder encodes as String
371- return formatTimestamp ((Long ) v );
405+ return formatTimestamp ((Long ) v , TimestampPrecision .MICROSECONDS );
406+ // TODO: Use LogicalTypes.TimestampNanos once avro version is updated.
407+ } else if (TIMESTAMP_NANOS_LOGICAL_TYPE .equals (schema .getProp ("logicalType" ))) {
408+ // SQL type TIMESTAMP
409+ // ideally Instant but TableRowJsonCoder encodes as String
410+ return formatTimestamp ((Long ) v , TimestampPrecision .NANOSECONDS );
372411 } else if (!(VERSION_AVRO .startsWith ("1.8" ) || VERSION_AVRO .startsWith ("1.9" ))
373412 && logicalType instanceof LogicalTypes .LocalTimestampMillis ) {
374413 // Write only: SQL type DATETIME
375414 // ideally LocalDateTime but TableRowJsonCoder encodes as String
376- return formatDatetime (((Long ) v ) * 1000 );
415+ return formatDatetime (((Long ) v ), TimestampPrecision . MILLISECONDS );
377416 } else if (!(VERSION_AVRO .startsWith ("1.8" ) || VERSION_AVRO .startsWith ("1.9" ))
378417 && logicalType instanceof LogicalTypes .LocalTimestampMicros ) {
379418 // Write only: SQL type DATETIME
380419 // ideally LocalDateTime but TableRowJsonCoder encodes as String
381- return formatDatetime ((Long ) v );
420+ return formatDatetime ((Long ) v , TimestampPrecision . MICROSECONDS );
382421 } else {
383422 // SQL type INT64 (INT, SMALLINT, INTEGER, BIGINT, TINYINT, BYTEINT)
384423 // ideally Long if in [2^53+1, 2^53-1] but keep consistency with BQ JSON export that uses
@@ -602,6 +641,11 @@ private static TableFieldSchema typedTableFieldSchema(Schema type, Boolean useAv
602641 return fieldSchema .setType ("INTEGER" );
603642 }
604643 case LONG :
644+ // TODO: Use LogicalTypes.TimestampNanos once avro version is updated.
645+ if (useAvroLogicalTypes
646+ && (TIMESTAMP_NANOS_LOGICAL_TYPE .equals (type .getProp ("logicalType" )))) {
647+ return fieldSchema .setType ("TIMESTAMP" );
648+ }
605649 if (logicalType instanceof LogicalTypes .TimeMicros ) {
606650 return fieldSchema .setType ("TIME" );
607651 } else if (!(VERSION_AVRO .startsWith ("1.8" ) || VERSION_AVRO .startsWith ("1.9" ))
0 commit comments