diff --git a/lang/java/avro/src/main/java/org/apache/avro/Conversions.java b/lang/java/avro/src/main/java/org/apache/avro/Conversions.java index 7d01fc62a37..1a1754226b4 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/Conversions.java +++ b/lang/java/avro/src/main/java/org/apache/avro/Conversions.java @@ -18,15 +18,18 @@ package org.apache.avro; -import java.math.RoundingMode; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericEnumSymbol; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.util.TimePeriod; import java.math.BigDecimal; import java.math.BigInteger; +import java.math.RoundingMode; import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.IntBuffer; import java.util.Arrays; import java.util.Collection; import java.util.Map; @@ -147,6 +150,43 @@ private static BigDecimal validate(final LogicalTypes.Decimal decimal, BigDecima } } + public static class DurationConversion extends Conversion { + @Override + public Class getConvertedType() { + return TimePeriod.class; + } + + @Override + public String getLogicalTypeName() { + return "duration"; + } + + @Override + public Schema getRecommendedSchema() { + return LogicalTypes.duration().addToSchema(Schema.createFixed("time.Duration", + "A 12-byte byte array encoding a duration in months, days and milliseconds.", null, 12)); + } + + @Override + public TimePeriod fromFixed(GenericFixed value, Schema schema, LogicalType type) { + IntBuffer buffer = ByteBuffer.wrap(value.bytes()).order(ByteOrder.LITTLE_ENDIAN).asIntBuffer(); + long months = Integer.toUnsignedLong(buffer.get()); + long days = Integer.toUnsignedLong(buffer.get()); + long millis = Integer.toUnsignedLong(buffer.get()); + return TimePeriod.of(months, days, millis); + } + + @Override + public GenericFixed toFixed(TimePeriod value, Schema schema, LogicalType type) { + ByteBuffer buffer = ByteBuffer.allocate(12).order(ByteOrder.LITTLE_ENDIAN); + IntBuffer intBuffer = buffer.asIntBuffer(); + intBuffer.put((int) value.getMonths()); + intBuffer.put((int) value.getDays()); + intBuffer.put((int) value.getMillis()); + return new GenericData.Fixed(schema, buffer.array()); + } + } + /** * Convert an underlying representation of a logical type (such as a ByteBuffer) * to a higher level object (such as a BigDecimal). diff --git a/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java b/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java index 086c5d266a2..4292756a2d2 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java +++ b/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java @@ -18,15 +18,15 @@ package org.apache.avro; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.Collections; import java.util.Map; import java.util.Objects; import java.util.ServiceLoader; import java.util.concurrent.ConcurrentHashMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - public class LogicalTypes { private static final Logger LOG = LoggerFactory.getLogger(LogicalTypes.class); @@ -182,6 +182,7 @@ private static LogicalType fromSchemaImpl(Schema schema, boolean throwErrors) { } private static final String DECIMAL = "decimal"; + private static final String DURATION = "duration"; private static final String UUID = "uuid"; private static final String DATE = "date"; private static final String TIME_MILLIS = "time-millis"; @@ -201,12 +202,18 @@ public static Decimal decimal(int precision, int scale) { return new Decimal(precision, scale); } - private static final LogicalType UUID_TYPE = new LogicalType("uuid"); + private static final LogicalType UUID_TYPE = new Uuid(); public static LogicalType uuid() { return UUID_TYPE; } + private static final LogicalType DURATION_TYPE = new Duration(); + + public static LogicalType duration() { + return DURATION_TYPE; + } + private static final Date DATE_TYPE = new Date(); public static Date date() { @@ -249,6 +256,38 @@ public static LocalTimestampMicros localTimestampMicros() { return LOCAL_TIMESTAMP_MICROS_TYPE; } + /** Uuid represents a uuid without a time */ + public static class Uuid extends LogicalType { + private Uuid() { + super(UUID); + } + + @Override + public void validate(Schema schema) { + super.validate(schema); + if (schema.getType() != Schema.Type.STRING) { + throw new IllegalArgumentException("Uuid can only be used with an underlying string type"); + } + } + } + + /** + * Duration represents a duration, consisting on months, days and milliseconds + */ + public static class Duration extends LogicalType { + private Duration() { + super(DURATION); + } + + @Override + public void validate(Schema schema) { + super.validate(schema); + if (schema.getType() != Schema.Type.FIXED || schema.getFixedSize() != 12) { + throw new IllegalArgumentException("Duration can only be used with an underlying fixed type of size 12."); + } + } + } + /** Decimal represents arbitrary-precision fixed-scale decimal numbers */ public static class Decimal extends LogicalType { private static final String PRECISION_PROP = "precision"; diff --git a/lang/java/avro/src/main/java/org/apache/avro/util/TimePeriod.java b/lang/java/avro/src/main/java/org/apache/avro/util/TimePeriod.java new file mode 100644 index 00000000000..a1f7fa4e8b5 --- /dev/null +++ b/lang/java/avro/src/main/java/org/apache/avro/util/TimePeriod.java @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.avro.util; + +import java.io.Serializable; +import java.time.DateTimeException; +import java.time.Duration; +import java.time.Period; +import java.time.chrono.ChronoPeriod; +import java.time.chrono.IsoChronology; +import java.time.temporal.ChronoUnit; +import java.time.temporal.Temporal; +import java.time.temporal.TemporalAmount; +import java.time.temporal.TemporalUnit; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +import static java.time.temporal.ChronoUnit.DAYS; +import static java.time.temporal.ChronoUnit.MILLIS; +import static java.time.temporal.ChronoUnit.MONTHS; +import static java.util.Collections.unmodifiableList; +import static java.util.Objects.requireNonNull; + +/** + *

+ * A temporal amount to model an {@link org.apache.avro.LogicalTypes.Duration + * Avro duration} (the logical type). + *

+ * + *

+ * It consists of a number of months, days and milliseconds, all modelled as an + * unsigned integer. + *

+ * + *

+ * Compared to {@link Period java.time.Period}, this class has a smaller range + * ('only' supporting a little less than 358 million years), and cannot support + * negative time periods. + *

+ * + *

+ * Compared to {@link Duration java.time.Duration}, this class has less + * precision (milliseconds compared to nanoseconds), cannot support negative + * durations, and has a much smaller range. Where {@code java.time.Duration} + * supports fixed ranges up to about 68 years, {@code TimePeriod} can only + * handle about 49 days. + *

+ * + *

+ * Comparison with the regular {@code java.time} classes: + *

+ * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
TimePeriod{@link Period}{@link Duration}
Precisionmillisecondsdaysnanoseconds
Time range (approx.)0 - 49 daysunsupported-68 - 68 years
Date range (approx.)0 to 370 million years-2.3 to 2.3 billion yearsunsupported
+ * + * @see Avro 1.11 + * specification on duration + */ +public final class TimePeriod implements TemporalAmount, Serializable { + private static final long MAX_UNSIGNED_INT = 0xffffffffL; + private static final long MONTHS_PER_YEAR = 12; + private static final long MONTHS_PER_DECADE = MONTHS_PER_YEAR * 10; + private static final long MONTHS_PER_CENTURY = MONTHS_PER_DECADE * 10; + private static final long MONTHS_PER_MILLENNIUM = MONTHS_PER_CENTURY * 10; + private static final long MILLIS_PER_SECOND = 1_000; + private static final long MILLIS_PER_MINUTE = MILLIS_PER_SECOND * 60; + private static final long MILLIS_PER_HOUR = MILLIS_PER_MINUTE * 60; + private static final long MILLIS_IN_HALF_DAY = MILLIS_PER_HOUR * 12; + private static final long MICROS_PER_MILLI = 1_000; + private static final long NANOS_PER_MILLI = 1_000_000; + + private final long months; + private final long days; + private final long millis; + + /** + * Create a TimePeriod from another TemporalAmount, such as a {@link Period} or + * a {@link Duration}. + * + * @param amount a temporal amount + * @return the corresponding TimePeriod + */ + public static TimePeriod from(TemporalAmount amount) { + if (requireNonNull(amount, "amount") instanceof TimePeriod) { + return (TimePeriod) amount; + } + if (amount instanceof ChronoPeriod) { + if (!IsoChronology.INSTANCE.equals(((ChronoPeriod) amount).getChronology())) { + throw new DateTimeException("TimePeriod requires ISO chronology: " + amount); + } + } + long months = 0; + long days = 0; + long millis = 0; + for (TemporalUnit unit : amount.getUnits()) { + if (unit instanceof ChronoUnit) { + long unitAmount = amount.get(unit); + switch ((ChronoUnit) unit) { + case MILLENNIA: + months = unsignedInt(months + unitAmount * MONTHS_PER_MILLENNIUM); + break; + case CENTURIES: + months = unsignedInt(months + unitAmount * MONTHS_PER_CENTURY); + break; + case DECADES: + months = unsignedInt(months + unitAmount * MONTHS_PER_DECADE); + break; + case YEARS: + months = unsignedInt(months + unitAmount * MONTHS_PER_YEAR); + break; + case MONTHS: + months = unsignedInt(months + unitAmount); + break; + case WEEKS: + days = unsignedInt(days + unitAmount * 7); + break; + case DAYS: + days = unsignedInt(days + unitAmount); + break; + case HALF_DAYS: + days = unsignedInt(days + (unitAmount / 2)); // Truncates halves + if (unitAmount % 2 != 0) { + millis = unsignedInt(millis + MILLIS_IN_HALF_DAY); + } + break; + case HOURS: + millis = unsignedInt(millis + unitAmount * MILLIS_PER_HOUR); + break; + case MINUTES: + millis = unsignedInt(millis + unitAmount * MILLIS_PER_MINUTE); + break; + case SECONDS: + millis = unsignedInt(millis + unitAmount * MILLIS_PER_SECOND); + break; + case MILLIS: + millis = unsignedInt(millis + unitAmount); + break; + case MICROS: + if (unitAmount % MICROS_PER_MILLI != 0) { + throw new DateTimeException( + "Cannot add " + unitAmount + " microseconds: not a whole number of milliseconds"); + } + millis = unsignedInt(millis + unitAmount / MICROS_PER_MILLI); + break; + case NANOS: + if (unitAmount % NANOS_PER_MILLI != 0) { + throw new DateTimeException( + "Cannot add " + unitAmount + " nanoseconds: not a whole number of milliseconds"); + } + millis = unsignedInt(millis + unitAmount / NANOS_PER_MILLI); + break; + default: + throw new UnsupportedTemporalTypeException("Unsupported unit: " + unit); + } + } else { + throw new UnsupportedTemporalTypeException("Unsupported unit: " + unit); + } + } + return new TimePeriod(months, days, millis); + } + + /** + * Create a TimePeriod from a number of months, days and milliseconds + * + * @param months a number of months + * @param days a number of days + * @param millis a number of milliseconds + * @return the corresponding TimePeriod + * @throws ArithmeticException if any of the parameters does not fit an unsigned + * long (0..4294967296) + */ + public static TimePeriod of(long months, long days, long millis) { + return new TimePeriod(unsignedInt(months), unsignedInt(days), unsignedInt(millis)); + } + + private static long unsignedInt(long number) { + if (number != (number & MAX_UNSIGNED_INT)) { + throw new ArithmeticException("Overflow/underflow of unsigned int"); + } + return number; + } + + private TimePeriod(long months, long days, long millis) { + this.months = months; + this.days = days; + this.millis = millis; + } + + public Duration toDuration() { + return Duration.from(this); + } + + public Period toPeriod() { + if (isDateBased()) { + // We use unsigned ints, which have double the range of a signed int that + // Period uses. We can split months to years and months to ensure there's no + // overflow. But we cannot split days, as both days and months have varying + // lengths. + int yearsAsInt = (int) (months / MONTHS_PER_YEAR); + int monthsAsInt = (int) (months % MONTHS_PER_YEAR); + int daysAsInt = (int) days; + if (days != daysAsInt) { + throw new DateTimeException("Too many days: a Period can contain at most " + Integer.MAX_VALUE + " days."); + } + return Period.ofYears(yearsAsInt).withMonths(monthsAsInt).withDays(daysAsInt); + } + throw new DateTimeException("Cannot convert this TimePeriod to a Period: is not date based"); + } + + /** + * Determines if the TimePeriod is date based (i.e., if its milliseconds + * component is 0). + * + * @return {@code true} iff the TimePeriod is date based + */ + public boolean isDateBased() { + return millis == 0; + } + + /** + * Determines if the TimePeriod is time based (i.e., if its months and days + * components are 0). + * + * @return {@code true} iff the TimePeriod is time based + */ + public boolean isTimeBased() { + return months == 0 && days == 0; + } + + public long getMonths() { + return months; + } + + public long getDays() { + return days; + } + + public long getMillis() { + return millis; + } + + @Override + public long get(TemporalUnit unit) { + if (unit == MONTHS) { + return months; + } else if (unit == DAYS) { + return days; + } else if (unit == MILLIS) { + return millis; + } else { + throw new UnsupportedTemporalTypeException("Unsupported unit: " + unit); + } + } + + @Override + public List getUnits() { + List units = new ArrayList<>(); + // The zero-checks ensure compatibility with the Java Time classes Period and + // Duration where possible. + if (months != 0) { + units.add(MONTHS); + } + if (days != 0) { + units.add(DAYS); + } + if (millis != 0) { + units.add(MILLIS); + } + return unmodifiableList(units); + } + + @Override + public Temporal addTo(Temporal temporal) { + return addTo(temporal, months, days, millis); + } + + @Override + public Temporal subtractFrom(Temporal temporal) { + return addTo(temporal, -months, -days, -millis); + } + + private Temporal addTo(Temporal temporal, long months, long days, long millis) { + // The zero-checks ensure we can add a TimePeriod to a Temporal even when it + // does not support all fields, as long as the unsupported fields are zero. + if (months != 0) { + temporal = temporal.plus(months, MONTHS); + } + if (days != 0) { + temporal = temporal.plus(days, DAYS); + } + if (millis != 0) { + temporal = temporal.plus(millis, MILLIS); + } + return temporal; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TimePeriod that = (TimePeriod) o; + return months == that.months && days == that.days && millis == that.millis; + } + + @Override + public int hashCode() { + return Objects.hash(months, days, millis); + } + + @Override + public String toString() { + StringBuilder buffer = new StringBuilder(); + buffer.append("P"); + if (months != 0) { + int years = (int) (months / MONTHS_PER_YEAR); + int monthsLeft = (int) (months % MONTHS_PER_YEAR); + if (years != 0) { + buffer.append(years).append("Y"); + } + if (monthsLeft != 0) { + buffer.append(monthsLeft).append("M"); + } + } + if (days != 0 || (months == 0 && millis == 0)) { + buffer.append(days); + } + if (millis != 0) { + long millisLeft = millis; + int hours = (int) (millisLeft / MILLIS_PER_HOUR); + millisLeft -= MILLIS_PER_HOUR * hours; + int minutes = (int) (millisLeft / MILLIS_PER_MINUTE); + millisLeft -= MILLIS_PER_MINUTE * minutes; + int seconds = (int) (millisLeft / MILLIS_PER_SECOND); + millisLeft -= MILLIS_PER_SECOND * seconds; + if (millisLeft != 0) { + buffer.append(String.format("T%02d:%02d:%02d.%03d", hours, minutes, seconds, millisLeft)); + } else if (seconds != 0) { + buffer.append(String.format("T%02d:%02d:%02d", hours, minutes, seconds)); + } else if (minutes != 0) { + buffer.append(String.format("T%02d:%02d", hours, minutes)); + } else { + buffer.append(String.format("T%02d", hours)); + } + } + return buffer.toString(); + } +} diff --git a/lang/java/avro/src/test/java/org/apache/avro/TestLogicalType.java b/lang/java/avro/src/test/java/org/apache/avro/TestLogicalType.java index acc8899b21c..4476ac7db2c 100644 --- a/lang/java/avro/src/test/java/org/apache/avro/TestLogicalType.java +++ b/lang/java/avro/src/test/java/org/apache/avro/TestLogicalType.java @@ -18,16 +18,21 @@ package org.apache.avro; -import java.util.Arrays; -import java.util.concurrent.Callable; - import org.hamcrest.collection.IsMapContaining; import org.junit.jupiter.api.Test; +import java.util.Arrays; +import java.util.concurrent.Callable; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; public class TestLogicalType { @@ -197,6 +202,30 @@ void bytesDecimalToFromJson() { assertEquals(schema, parsed, "Constructed and parsed schemas should match"); } + @Test + void uuidExtendsString() { + Schema uuidSchema = LogicalTypes.uuid().addToSchema(Schema.create(Schema.Type.STRING)); + assertEquals(LogicalTypes.uuid(), uuidSchema.getLogicalType()); + + assertThrows("UUID requires a string", IllegalArgumentException.class, + "Uuid can only be used with an underlying string type", + () -> LogicalTypes.uuid().addToSchema(Schema.create(Schema.Type.INT))); + } + + @Test + void durationExtendsFixed12() { + Schema durationSchema = LogicalTypes.duration().addToSchema(Schema.createFixed("f", null, null, 12)); + assertEquals(LogicalTypes.duration(), durationSchema.getLogicalType()); + + assertThrows("Duration requires a fixed(12)", IllegalArgumentException.class, + "Duration can only be used with an underlying fixed type of size 12.", + () -> LogicalTypes.duration().addToSchema(Schema.create(Schema.Type.INT))); + + assertThrows("Duration requires a fixed(12)", IllegalArgumentException.class, + "Duration can only be used with an underlying fixed type of size 12.", + () -> LogicalTypes.duration().addToSchema(Schema.createFixed("wrong", null, null, 42))); + } + @Test void logicalTypeEquals() { LogicalTypes.Decimal decimal90 = LogicalTypes.decimal(9); diff --git a/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericLogicalTypes.java b/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericLogicalTypes.java index 25a838db335..6df4a8af6a2 100644 --- a/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericLogicalTypes.java +++ b/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericLogicalTypes.java @@ -18,19 +18,6 @@ package org.apache.avro.generic; -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.UUID; - import org.apache.avro.Conversion; import org.apache.avro.Conversions; import org.apache.avro.CustomType; @@ -43,11 +30,26 @@ import org.apache.avro.file.FileReader; import org.apache.avro.io.DatumReader; import org.apache.avro.io.DatumWriter; - +import org.apache.avro.util.TimePeriod; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.UUID; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -64,6 +66,7 @@ public class TestGenericLogicalTypes { public static void addLogicalTypes() { GENERIC.addLogicalTypeConversion(new Conversions.DecimalConversion()); GENERIC.addLogicalTypeConversion(new Conversions.UUIDConversion()); + GENERIC.addLogicalTypeConversion(new Conversions.DurationConversion()); GENERIC.addLogicalTypeConversion(new TimeConversions.LocalTimestampMicrosConversion()); GENERIC.addLogicalTypeConversion(new TimeConversions.LocalTimestampMillisConversion()); } @@ -116,6 +119,37 @@ public void writeNullableUUID() throws IOException { "Should read UUIDs as Strings"); } + @Test + public void readWriteDuration() throws IOException { + Schema fixedSchema = Schema.createFixed("bare.Fixed", null, null, 12); + + Schema durationSchema = Schema.createFixed("time.Duration", null, null, 12); + LogicalTypes.duration().addToSchema(durationSchema); + + // These two are necessary for schema evolution! + fixedSchema.addAlias(durationSchema.getFullName()); + durationSchema.addAlias(fixedSchema.getFullName()); + + Random rng = new Random(); + TimePeriod d1 = TimePeriod.of(rng.nextInt(1000), rng.nextInt(1000), rng.nextInt(1000)); + ByteBuffer b1 = ByteBuffer.allocate(12).order(ByteOrder.LITTLE_ENDIAN).putInt((int) d1.getMonths()) + .putInt((int) d1.getDays()).putInt((int) d1.getMillis()); + GenericFixed f1 = new GenericData.Fixed(fixedSchema, b1.array()); + + TimePeriod d2 = TimePeriod.of(rng.nextInt(1000), rng.nextInt(1000), rng.nextInt(1000)); + ByteBuffer b2 = ByteBuffer.allocate(12).order(ByteOrder.LITTLE_ENDIAN).putInt((int) d2.getMonths()) + .putInt((int) d2.getDays()).putInt((int) d2.getMillis()); + GenericFixed f2 = new GenericData.Fixed(fixedSchema, b2.array()); + + File test = write(fixedSchema, f1, f2); + assertEquals(Arrays.asList(d1, d2), read(GENERIC.createDatumReader(durationSchema), test), + "Should convert fixed bytes to durations"); + + test = write(GENERIC, durationSchema, d2, d1); + assertEquals(Arrays.asList(f2, f1), read(GenericData.get().createDatumReader(fixedSchema), test), + "Should convert durations to fixed bytes"); + } + @Test public void readDecimalFixed() throws IOException { LogicalType decimal = LogicalTypes.decimal(9, 2); @@ -157,11 +191,11 @@ public void writeDecimalFixed() throws IOException { } @Test - public void decimalToFromBytes() throws IOException { + public void decimalToFromBytes() { LogicalType decimal = LogicalTypes.decimal(9, 2); Schema bytesSchema = Schema.create(Schema.Type.BYTES); - // Check that the round trip to and from bytes + // Check the round trip to and from bytes BigDecimal d1 = new BigDecimal("-34.34"); BigDecimal d2 = new BigDecimal("117230.00"); @@ -178,11 +212,11 @@ public void decimalToFromBytes() throws IOException { } @Test - public void decimalToFromFixed() throws IOException { + public void decimalToFromFixed() { LogicalType decimal = LogicalTypes.decimal(9, 2); Schema fixedSchema = Schema.createFixed("aFixed", null, null, 4); - // Check that the round trip to and from fixed data. + // Check the round trip to and from fixed data. BigDecimal d1 = new BigDecimal("-34.34"); BigDecimal d2 = new BigDecimal("117230.00"); @@ -247,7 +281,8 @@ private List read(DatumReader reader, File file) throws IOException { return data; } - private File write(Schema schema, D... data) throws IOException { + @SafeVarargs + private final File write(Schema schema, D... data) throws IOException { return write(GenericData.get(), schema, data); } @@ -308,12 +343,12 @@ private void testCopy(Schema schema, Object value, GenericData model) { // test nested in array Schema arraySchema = Schema.createArray(schema); - ArrayList array = new ArrayList(Collections.singletonList(value)); + ArrayList array = new ArrayList<>(Collections.singletonList(value)); checkCopy(array, model.deepCopy(arraySchema, array), true); // test record nested in array Schema recordArraySchema = Schema.createArray(recordSchema); - ArrayList recordArray = new ArrayList(Collections.singletonList(record)); + ArrayList recordArray = new ArrayList<>(Collections.singletonList(record)); checkCopy(recordArray, model.deepCopy(recordArraySchema, recordArray), true); } @@ -329,7 +364,7 @@ public void readLocalTimestampMillis() throws IOException { Schema longSchema = Schema.create(Schema.Type.LONG); Schema timestampSchema = timestamp.addToSchema(Schema.create(Schema.Type.LONG)); - LocalDateTime i1 = LocalDateTime.of(1986, 06, 26, 12, 07, 11, 42000000); + LocalDateTime i1 = LocalDateTime.of(1986, 6, 26, 12, 7, 11, 42000000); LocalDateTime i2 = LocalDateTime.ofInstant(Instant.ofEpochMilli(0), ZoneOffset.UTC); List expected = Arrays.asList(i1, i2); @@ -350,7 +385,7 @@ public void writeLocalTimestampMillis() throws IOException { Schema longSchema = Schema.create(Schema.Type.LONG); Schema timestampSchema = timestamp.addToSchema(Schema.create(Schema.Type.LONG)); - LocalDateTime i1 = LocalDateTime.of(1986, 06, 26, 12, 07, 11, 42000000); + LocalDateTime i1 = LocalDateTime.of(1986, 6, 26, 12, 7, 11, 42000000); LocalDateTime i2 = LocalDateTime.ofInstant(Instant.ofEpochMilli(0), ZoneOffset.UTC); Conversion conversion = new TimeConversions.LocalTimestampMillisConversion(); @@ -370,7 +405,7 @@ public void readLocalTimestampMicros() throws IOException { Schema longSchema = Schema.create(Schema.Type.LONG); Schema timestampSchema = timestamp.addToSchema(Schema.create(Schema.Type.LONG)); - LocalDateTime i1 = LocalDateTime.of(1986, 06, 26, 12, 07, 11, 420000); + LocalDateTime i1 = LocalDateTime.of(1986, 6, 26, 12, 7, 11, 420000); LocalDateTime i2 = LocalDateTime.ofInstant(Instant.ofEpochSecond(0, 4000), ZoneOffset.UTC); List expected = Arrays.asList(i1, i2); @@ -391,7 +426,7 @@ public void writeLocalTimestampMicros() throws IOException { Schema longSchema = Schema.create(Schema.Type.LONG); Schema timestampSchema = timestamp.addToSchema(Schema.create(Schema.Type.LONG)); - LocalDateTime i1 = LocalDateTime.of(1986, 06, 26, 12, 07, 11, 420000); + LocalDateTime i1 = LocalDateTime.of(1986, 6, 26, 12, 7, 11, 420000); LocalDateTime i2 = LocalDateTime.ofInstant(Instant.ofEpochSecond(0, 4000), ZoneOffset.UTC); Conversion conversion = new TimeConversions.LocalTimestampMicrosConversion(); diff --git a/lang/java/avro/src/test/java/org/apache/avro/util/TimePeriodTest.java b/lang/java/avro/src/test/java/org/apache/avro/util/TimePeriodTest.java new file mode 100644 index 00000000000..cd9809be414 --- /dev/null +++ b/lang/java/avro/src/test/java/org/apache/avro/util/TimePeriodTest.java @@ -0,0 +1,306 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.avro.util; + +import org.junit.jupiter.api.Test; + +import java.time.DateTimeException; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.Period; +import java.time.chrono.IsoChronology; +import java.time.chrono.JapaneseChronology; +import java.time.temporal.ChronoUnit; +import java.time.temporal.Temporal; +import java.time.temporal.TemporalAmount; +import java.time.temporal.TemporalUnit; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import static java.time.temporal.ChronoUnit.DAYS; +import static java.time.temporal.ChronoUnit.ERAS; +import static java.time.temporal.ChronoUnit.MICROS; +import static java.time.temporal.ChronoUnit.MILLIS; +import static java.time.temporal.ChronoUnit.MONTHS; +import static java.time.temporal.ChronoUnit.NANOS; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class TimePeriodTest { + // This Long is too large to fit into an unsigned int. + private static final long TOO_LARGE = Integer.MAX_VALUE * 3L; + + @Test + void validateConstruction() { + TimePeriod timePeriod = TimePeriod.of(12, 34, 56); + assertSame(timePeriod, TimePeriod.from(timePeriod)); + assertComponents(12, 34, 56, timePeriod); + + assertComponents(14, 3, 0, TimePeriod.from(IsoChronology.INSTANCE.period(1, 2, 3))); + + assertComponents(36_000, 0, 0, TimePeriod.from(TimeAmount.of(ChronoUnit.MILLENNIA, 3))); + assertComponents(3_600, 0, 0, TimePeriod.from(TimeAmount.of(ChronoUnit.CENTURIES, 3))); + assertComponents(360, 0, 0, TimePeriod.from(TimeAmount.of(ChronoUnit.DECADES, 3))); + assertComponents(36, 0, 0, TimePeriod.from(TimeAmount.of(ChronoUnit.YEARS, 3))); + assertComponents(3, 0, 0, TimePeriod.from(TimeAmount.of(MONTHS, 3))); + + assertComponents(0, 21, 0, TimePeriod.from(TimeAmount.of(ChronoUnit.WEEKS, 3))); + assertComponents(0, 3, 0, TimePeriod.from(TimeAmount.of(DAYS, 3))); + assertComponents(0, 2, 0, TimePeriod.from(TimeAmount.of(ChronoUnit.HALF_DAYS, 4))); + assertComponents(0, 2, 43_200_000, TimePeriod.from(TimeAmount.of(ChronoUnit.HALF_DAYS, 5))); + + assertComponents(0, 0, 10_800_000, TimePeriod.from(TimeAmount.of(ChronoUnit.HOURS, 3))); + assertComponents(0, 0, 180_000, TimePeriod.from(TimeAmount.of(ChronoUnit.MINUTES, 3))); + assertComponents(0, 0, 3_000, TimePeriod.from(TimeAmount.of(ChronoUnit.SECONDS, 3))); + assertComponents(0, 0, 3, TimePeriod.from(TimeAmount.of(MILLIS, 3))); + assertComponents(0, 0, 3, TimePeriod.from(TimeAmount.of(MICROS, 3_000))); + assertComponents(0, 0, 3, TimePeriod.from(TimeAmount.of(NANOS, 3_000_000))); + + // Micros and nanos must be a multiple of milliseconds + assertThrows(DateTimeException.class, () -> TimePeriod.from(TimeAmount.of(ChronoUnit.MICROS, 3))); + assertThrows(DateTimeException.class, () -> TimePeriod.from(TimeAmount.of(ChronoUnit.NANOS, 3))); + // Unsupported cases (null, non-ISO chronology, unknown temporal unit, + // non-ChronoUnit) + assertThrows(NullPointerException.class, () -> TimePeriod.from(null)); + assertThrows(DateTimeException.class, () -> TimePeriod.from(JapaneseChronology.INSTANCE.period(1, 2, 3))); + assertThrows(UnsupportedTemporalTypeException.class, () -> TimePeriod.from(TimeAmount.of(ChronoUnit.ERAS, 1))); + assertThrows(UnsupportedTemporalTypeException.class, () -> TimePeriod.from(TimeAmount.of(DummyUnit.INSTANCE, 3))); + // Arguments are long, but must fit an unsigned long + assertThrows(ArithmeticException.class, () -> TimePeriod.of(TOO_LARGE, 0, 0)); + assertThrows(ArithmeticException.class, () -> TimePeriod.of(0, TOO_LARGE, 0)); + assertThrows(ArithmeticException.class, () -> TimePeriod.of(0, 0, TOO_LARGE)); + + // Odd one out: querying an unsupported temporal unit + // (assertComponents handles all valid cases) + assertThrows(UnsupportedTemporalTypeException.class, () -> TimePeriod.of(1, 1, 1).get(ERAS)); + } + + @Test + void checkConversionsFromJavaTime() { + assertEquals(TimePeriod.of(12, 0, 0), TimePeriod.from(Period.ofYears(1))); + assertEquals(TimePeriod.of(2, 0, 0), TimePeriod.from(Period.ofMonths(2))); + assertEquals(TimePeriod.of(0, 21, 0), TimePeriod.from(Period.ofWeeks(3))); + assertEquals(TimePeriod.of(0, 4, 0), TimePeriod.from(Period.ofDays(4))); + + assertEquals(TimePeriod.of(0, 0, 1), TimePeriod.from(Duration.ofNanos(1_000_000))); + assertEquals(TimePeriod.of(0, 0, 2), TimePeriod.from(Duration.ofMillis(2))); + assertEquals(TimePeriod.of(0, 0, 3_000), TimePeriod.from(Duration.ofSeconds(3))); + assertEquals(TimePeriod.of(0, 0, 240000), TimePeriod.from(Duration.ofMinutes(4))); + assertEquals(TimePeriod.of(0, 0, 18000000), TimePeriod.from(Duration.ofHours(5))); + // Duration never takes into account things like daylight saving + assertEquals(TimePeriod.of(0, 0, 518400000), TimePeriod.from(Duration.ofDays(6))); + } + + @Test + void checkConversionsToJavaTime() { + TimePeriod months = TimePeriod.of(1, 0, 0); + TimePeriod days = TimePeriod.of(0, 2, 0); + TimePeriod time = TimePeriod.of(0, 0, 3); + TimePeriod all = TimePeriod.of(1, 2, 3); + + assertTrue(months.isDateBased()); + assertTrue(days.isDateBased()); + assertFalse(all.isDateBased()); + assertFalse(time.isDateBased()); + + assertEquals(Period.of(0, 1, 0), months.toPeriod()); + assertEquals(Period.of(0, 0, 2), days.toPeriod()); + assertThrows(DateTimeException.class, all::toPeriod); + assertThrows(DateTimeException.class, time::toPeriod); + + assertThrows(DateTimeException.class, () -> TimePeriod.of(0, Integer.MAX_VALUE * 2L, 0).toPeriod()); + + assertFalse(months.isTimeBased()); + assertFalse(days.isTimeBased()); + assertFalse(all.isTimeBased()); + assertTrue(time.isTimeBased()); + + assertThrows(DateTimeException.class, months::toDuration); + // Note: though Duration supports this, it uses a fixed 86400 seconds + assertEquals(Duration.ofSeconds(172800), days.toDuration()); + assertThrows(DateTimeException.class, all::toDuration); + assertEquals(Duration.ofMillis(3), time.toDuration()); + } + + @Test + void checkAddingToTemporalItems() { + TimePeriod monthAndTwoDays = TimePeriod.of(1, 2, 0); + TimePeriod threeMillis = TimePeriod.of(0, 0, 3); + TimePeriod complexTimePeriod = TimePeriod.of(1, 2, 3); + + LocalDateTime localDateTime = LocalDateTime.of(2001, 2, 3, 4, 5, 6, 7_000_000); + LocalDate localDate = LocalDate.of(2001, 2, 3); + LocalTime localTime = LocalTime.of(4, 5, 6, 7_000_000); + + assertEquals(localDateTime.plusMonths(1).plusDays(2), localDateTime.plus(monthAndTwoDays)); + assertEquals(localDateTime.plus(3, MILLIS), localDateTime.plus(threeMillis)); + assertEquals(localDateTime.plusMonths(1).plusDays(2).plus(3, MILLIS), localDateTime.plus(complexTimePeriod)); + + assertEquals(localDate.plusMonths(1).plusDays(2), localDate.plus(monthAndTwoDays)); + + assertEquals(localTime.plus(3, MILLIS), localTime.plus(threeMillis)); + + assertEquals(localDateTime.minusMonths(1).minusDays(2), localDateTime.minus(monthAndTwoDays)); + assertEquals(localDateTime.minus(3, MILLIS), localDateTime.minus(threeMillis)); + assertEquals(localDateTime.minusMonths(1).minusDays(2).minus(3, MILLIS), localDateTime.minus(complexTimePeriod)); + + assertEquals(localDate.minusMonths(1).minusDays(2), localDate.minus(monthAndTwoDays)); + + assertEquals(localTime.minus(3, MILLIS), localTime.minus(threeMillis)); + } + + @Test + void checkEqualityTests() { + TimePeriod timePeriod1a = TimePeriod.of(1, 2, 3); + TimePeriod timePeriod1b = TimePeriod.of(1, 2, 3); + TimePeriod timePeriod2 = TimePeriod.of(9, 9, 9); + TimePeriod timePeriod3 = TimePeriod.of(1, 9, 9); + TimePeriod timePeriod4 = TimePeriod.of(1, 2, 9); + + // noinspection EqualsWithItself + assertEquals(timePeriod1a, timePeriod1a); + assertEquals(timePeriod1a, timePeriod1b); + assertEquals(timePeriod1a.hashCode(), timePeriod1b.hashCode()); + + assertNotEquals(timePeriod1a, null); + // noinspection AssertBetweenInconvertibleTypes + assertNotEquals(timePeriod1a, "not equal"); + assertNotEquals(timePeriod1a, timePeriod2); + assertNotEquals(timePeriod1a.hashCode(), timePeriod2.hashCode()); + assertNotEquals(timePeriod1a, timePeriod3); + assertNotEquals(timePeriod1a.hashCode(), timePeriod3.hashCode()); + assertNotEquals(timePeriod1a, timePeriod4); + assertNotEquals(timePeriod1a.hashCode(), timePeriod4.hashCode()); + } + + @Test + void checkStringRepresentation() { + assertEquals("P0", TimePeriod.of(0, 0, 0).toString()); + assertEquals("P1Y", TimePeriod.of(12, 0, 0).toString()); + assertEquals("P2M", TimePeriod.of(2, 0, 0).toString()); + assertEquals("P3", TimePeriod.of(0, 3, 0).toString()); + assertEquals("P1Y2M3", TimePeriod.of(14, 3, 0).toString()); + assertEquals("PT04", TimePeriod.of(0, 0, 14400000).toString()); + assertEquals("PT00:05", TimePeriod.of(0, 0, 300000).toString()); + assertEquals("PT00:00:06", TimePeriod.of(0, 0, 6000).toString()); + assertEquals("PT00:00:00.007", TimePeriod.of(0, 0, 7).toString()); + assertEquals("P1Y2M3T04:05:06.007", TimePeriod.of(14, 3, 14706007).toString()); + + // Days and millis will never overflow to months/days, to respect differences + // in months and days (daylight saving). + assertEquals("P123T1193:02:47.295", TimePeriod.of(0, 123, 4294967295L).toString()); + } + + private void assertComponents(long months, long days, long millis, TimePeriod timePeriod) { + List expectedUnits = new ArrayList<>(Arrays.asList(MONTHS, DAYS, MILLIS)); + if (months == 0) { + expectedUnits.remove(MONTHS); + } + if (days == 0) { + expectedUnits.remove(DAYS); + } + if (millis == 0) { + expectedUnits.remove(MILLIS); + } + assertEquals(expectedUnits, timePeriod.getUnits()); + + assertEquals(months, timePeriod.getMonths()); + assertEquals(months, timePeriod.get(MONTHS)); + assertEquals(days, timePeriod.getDays()); + assertEquals(days, timePeriod.get(DAYS)); + assertEquals(millis, timePeriod.getMillis()); + assertEquals(millis, timePeriod.get(MILLIS)); + } + + private static class TimeAmount implements TemporalAmount { + private final Map amountsPerUnit = new LinkedHashMap<>(); + + static TimeAmount of(TemporalUnit unit, long amount) { + return new TimeAmount().with(unit, amount); + } + + TimeAmount with(TemporalUnit unit, long amount) { + amountsPerUnit.put(unit, amount); + return this; + } + + @Override + public long get(TemporalUnit unit) { + return amountsPerUnit.get(unit); + } + + @Override + public List getUnits() { + return new ArrayList<>(amountsPerUnit.keySet()); + } + + @Override + public Temporal addTo(Temporal temporal) { + throw new UnsupportedOperationException(); + } + + @Override + public Temporal subtractFrom(Temporal temporal) { + throw new UnsupportedOperationException(); + } + } + + private static class DummyUnit implements TemporalUnit { + private static final DummyUnit INSTANCE = new DummyUnit(); + + @Override + public Duration getDuration() { + return null; + } + + @Override + public boolean isDurationEstimated() { + return false; + } + + @Override + public boolean isDateBased() { + return false; + } + + @Override + public boolean isTimeBased() { + return false; + } + + @Override + public R addTo(R temporal, long amount) { + return null; + } + + @Override + public long between(Temporal temporal1Inclusive, Temporal temporal2Exclusive) { + return 0; + } + } +}