DBZ-91 Changed MicroTime to use INT64
There are more microseconds per day than can be represented with INT32, so this was changed to INT64.
This commit is contained in:
parent
19fc95fe08
commit
ba553c91e8
@ -11,7 +11,7 @@
|
||||
import org.apache.kafka.connect.data.SchemaBuilder;
|
||||
|
||||
/**
|
||||
* A utility for converting various Java time representations into the {@link SchemaBuilder#int32() INT32} number of
|
||||
* A utility for converting various Java time representations into the {@link SchemaBuilder#int64() INT64} number of
|
||||
* <em>microseconds</em> since midnight, and for defining a Kafka Connect {@link Schema} for time values with no date or timezone
|
||||
* information.
|
||||
*
|
||||
@ -25,8 +25,8 @@ public class MicroTime {
|
||||
|
||||
/**
|
||||
* Returns a {@link SchemaBuilder} for a {@link MicroTime}. The resulting schema will describe a field
|
||||
* with the {@value #SCHEMA_NAME} as the {@link Schema#name() name} and {@link SchemaBuilder#int32() INT32} for the literal
|
||||
* type storing the number of <em>milliseconds</em> past midnight.
|
||||
* with the {@value #SCHEMA_NAME} as the {@link Schema#name() name} and {@link SchemaBuilder#int64() INT64} for the literal
|
||||
* type storing the number of <em>microseconds</em> past midnight.
|
||||
* <p>
|
||||
* You can use the resulting SchemaBuilder to set or override additional schema settings such as required/optional, default
|
||||
* value, and documentation.
|
||||
@ -34,15 +34,15 @@ public class MicroTime {
|
||||
* @return the schema builder
|
||||
*/
|
||||
public static SchemaBuilder builder() {
|
||||
return SchemaBuilder.int32()
|
||||
return SchemaBuilder.int64()
|
||||
.name(SCHEMA_NAME)
|
||||
.version(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a Schema for a {@link MicroTime} but with all other default Schema settings. The schema describes a field
|
||||
* with the {@value #SCHEMA_NAME} as the {@link Schema#name() name} and {@link SchemaBuilder#int32() INT32} for the literal
|
||||
* type storing the number of <em>milliseconds</em> past midnight.
|
||||
* with the {@value #SCHEMA_NAME} as the {@link Schema#name() name} and {@link SchemaBuilder#int64() INT64} for the literal
|
||||
* type storing the number of <em>microseconds</em> past midnight.
|
||||
*
|
||||
* @return the schema
|
||||
* @see #builder()
|
||||
@ -60,11 +60,9 @@ public static Schema schema() {
|
||||
* @return the microseconds past midnight
|
||||
* @throws IllegalArgumentException if the value is not an instance of the acceptable types
|
||||
*/
|
||||
public static int toMicroOfDay(Object value) {
|
||||
public static long toMicroOfDay(Object value) {
|
||||
LocalTime time = Conversions.toLocalTime(value);
|
||||
long micros = Math.floorDiv(time.toNanoOfDay(), Conversions.NANOSECONDS_PER_MICROSECOND);
|
||||
assert Math.abs(micros) < Integer.MAX_VALUE;
|
||||
return (int)micros;
|
||||
return Math.floorDiv(time.toNanoOfDay(), Conversions.NANOSECONDS_PER_MICROSECOND);
|
||||
}
|
||||
|
||||
private MicroTime() {
|
||||
|
Loading…
Reference in New Issue
Block a user