You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by bl...@apache.org on 2015/07/09 19:52:00 UTC

svn commit: r1690131 - in /avro/trunk: ./ lang/java/ lang/java/avro/ lang/java/avro/src/main/java/org/apache/avro/ lang/java/avro/src/main/java/org/apache/avro/data/ lang/java/avro/src/test/java/org/apache/avro/data/

Author: blue
Date: Thu Jul  9 17:51:59 2015
New Revision: 1690131

URL: http://svn.apache.org/r1690131
Log:
AVRO-1672: Add time logical types and conversions.

This uses Joda classes to represent the new logical types:
* date => LocalDate
* time-millis => LocalTime
* time-micros => LocalTime
* timestamp-millis => DateTime
* timestamp-micros => DateTime

Joda is an optional dependency and will only be loaded if the
conversions in org.apache.avro.data.TimeConversions are referenced. By
default, no logical type conversions are enabled so there is no
additional runtime dependency for existing applications.

Microsecond types use Joda objects with millisecond precision. Because
this representation loses the microseconds, there are two conversions
for both microsecond types: a read-only conversion that will throw an
exception if used to write, and a "lossy" conversion that will write a
millisecond value as microseconds by convertsion. This is called "lossy"
because reading a microsecond value and writing it back out will lose
the microseconds.

Added:
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/TimeConversions.java
    avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/TestTimeConversions.java
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/avro/pom.xml
    avro/trunk/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java
    avro/trunk/lang/java/pom.xml

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1690131&r1=1690130&r2=1690131&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Thu Jul  9 17:51:59 2015
@@ -55,6 +55,8 @@ Trunk (not yet released)
     AVRO-1683. Add microsecond time and timestamp logical types to the
     specification. (blue)
 
+    AVRO-1672. Java: Add date/time logical types and conversions. (blue)
+
   OPTIMIZATIONS
 
   IMPROVEMENTS

Modified: avro/trunk/lang/java/avro/pom.xml
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/pom.xml?rev=1690131&r1=1690130&r2=1690131&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/pom.xml (original)
+++ avro/trunk/lang/java/avro/pom.xml Thu Jul  9 17:51:59 2015
@@ -164,6 +164,11 @@
       <artifactId>xz</artifactId>
       <version>${tukaani.version}</version>
     </dependency>
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <optional>true</optional>
+    </dependency>
   </dependencies>
 
 </project>

Modified: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java?rev=1690131&r1=1690130&r2=1690131&view=diff
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java (original)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/LogicalTypes.java Thu Jul  9 17:51:59 2015
@@ -28,8 +28,6 @@ public class LogicalTypes {
 
   /**
    * Returns the {@link LogicalType} from the schema, if one is present.
-   * @param schema
-   * @return
    */
   public static LogicalType fromSchema(Schema schema) {
     return fromSchemaImpl(schema, true);
@@ -53,10 +51,20 @@ public class LogicalTypes {
 
     LogicalType logicalType;
     try {
-      if ("decimal".equals(typeName)) {
+      if (TIMESTAMP_MILLIS.equals(typeName)) {
+        logicalType = TIMESTAMP_MILLIS_TYPE;
+      } else if (DECIMAL.equals(typeName)) {
         logicalType = new Decimal(schema);
-      } else if ("uuid".equals(typeName)) {
+      } else if (UUID.equals(typeName)) {
         logicalType = UUID_TYPE;
+      } else if (DATE.equals(typeName)) {
+        logicalType = DATE_TYPE;
+      } else if (TIMESTAMP_MICROS.equals(typeName)) {
+        logicalType = TIMESTAMP_MICROS_TYPE;
+      } else if (TIME_MILLIS.equals(typeName)) {
+        logicalType = TIME_MILLIS_TYPE;
+      } else if (TIME_MICROS.equals(typeName)) {
+        logicalType = TIME_MICROS_TYPE;
       } else if (REGISTERED_TYPES.containsKey(typeName)) {
         logicalType = REGISTERED_TYPES.get(typeName).fromSchema(schema);
       } else {
@@ -78,6 +86,14 @@ public class LogicalTypes {
     return logicalType;
   }
 
+  private static final String DECIMAL = "decimal";
+  private static final String UUID = "uuid";
+  private static final String DATE = "date";
+  private static final String TIME_MILLIS = "time-millis";
+  private static final String TIME_MICROS = "time-micros";
+  private static final String TIMESTAMP_MILLIS = "timestamp-millis";
+  private static final String TIMESTAMP_MICROS = "timestamp-micros";
+
   /** Create a Decimal LogicalType with the given precision and scale 0 */
   public static Decimal decimal(int precision) {
     return decimal(precision, 0);
@@ -94,6 +110,38 @@ public class LogicalTypes {
     return UUID_TYPE;
   }
 
+  private static final Date DATE_TYPE = new Date();
+
+  public static Date date() {
+    return DATE_TYPE;
+  }
+
+  private static final TimeMillis TIME_MILLIS_TYPE = new TimeMillis();
+
+  public static TimeMillis timeMillis() {
+    return TIME_MILLIS_TYPE;
+  }
+
+  private static final TimeMicros TIME_MICROS_TYPE = new TimeMicros();
+
+  public static TimeMicros timeMicros() {
+    return TIME_MICROS_TYPE;
+  }
+
+  private static final TimestampMillis TIMESTAMP_MILLIS_TYPE =
+      new TimestampMillis();
+
+  public static TimestampMillis timestampMillis() {
+    return TIMESTAMP_MILLIS_TYPE;
+  }
+
+  private static final TimestampMicros TIMESTAMP_MICROS_TYPE =
+      new TimestampMicros();
+
+  public static TimestampMicros timestampMicros() {
+    return TIMESTAMP_MICROS_TYPE;
+  }
+
   /** Decimal represents arbitrary-precision fixed-scale decimal numbers  */
   public static class Decimal extends LogicalType {
     private static final String PRECISION_PROP = "precision";
@@ -103,7 +151,7 @@ public class LogicalTypes {
     private final int scale;
 
     private Decimal(int precision, int scale) {
-      super("decimal");
+      super(DECIMAL);
       this.precision = precision;
       this.scale = scale;
     }
@@ -215,4 +263,84 @@ public class LogicalTypes {
       return result;
     }
   }
+
+  /** Date represents a date without a time */
+  public static class Date extends LogicalType {
+    private Date() {
+      super(DATE);
+    }
+
+    @Override
+    public void validate(Schema schema) {
+      super.validate(schema);
+      if (schema.getType() != Schema.Type.INT) {
+        throw new IllegalArgumentException(
+            "Date can only be used with an underlying int type");
+      }
+    }
+  }
+
+  /** TimeMillis represents a time in milliseconds without a date */
+  public static class TimeMillis extends LogicalType {
+    private TimeMillis() {
+      super(TIME_MILLIS);
+    }
+
+    @Override
+    public void validate(Schema schema) {
+      super.validate(schema);
+      if (schema.getType() != Schema.Type.INT) {
+        throw new IllegalArgumentException(
+            "Time (millis) can only be used with an underlying int type");
+      }
+    }
+  }
+
+  /** TimeMicros represents a time in microseconds without a date */
+  public static class TimeMicros extends LogicalType {
+    private TimeMicros() {
+      super(TIME_MICROS);
+    }
+
+    @Override
+    public void validate(Schema schema) {
+      super.validate(schema);
+      if (schema.getType() != Schema.Type.LONG) {
+        throw new IllegalArgumentException(
+            "Time (micros) can only be used with an underlying long type");
+      }
+    }
+  }
+
+  /** TimestampMillis represents a date and time in milliseconds */
+  public static class TimestampMillis extends LogicalType {
+    private TimestampMillis() {
+      super(TIMESTAMP_MILLIS);
+    }
+
+    @Override
+    public void validate(Schema schema) {
+      super.validate(schema);
+      if (schema.getType() != Schema.Type.LONG) {
+        throw new IllegalArgumentException(
+            "Timestamp (millis) can only be used with an underlying long type");
+      }
+    }
+  }
+
+  /** TimestampMicros represents a date and time in microseconds */
+  public static class TimestampMicros extends LogicalType {
+    private TimestampMicros() {
+      super(TIMESTAMP_MICROS);
+    }
+
+    @Override
+    public void validate(Schema schema) {
+      super.validate(schema);
+      if (schema.getType() != Schema.Type.LONG) {
+        throw new IllegalArgumentException(
+            "Timestamp (micros) can only be used with an underlying long type");
+      }
+    }
+  }
 }

Added: avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/TimeConversions.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/TimeConversions.java?rev=1690131&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/TimeConversions.java (added)
+++ avro/trunk/lang/java/avro/src/main/java/org/apache/avro/data/TimeConversions.java Thu Jul  9 17:51:59 2015
@@ -0,0 +1,128 @@
+package org.apache.avro.data;
+
+import org.apache.avro.Conversion;
+import org.apache.avro.LogicalType;
+import org.apache.avro.Schema;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Days;
+import org.joda.time.LocalDate;
+import org.joda.time.LocalTime;
+
+public class TimeConversions {
+  public static class DateConversion extends Conversion<LocalDate> {
+    private static final LocalDate EPOCH_DATE = new LocalDate(1970, 1, 1);
+
+    @Override
+    public Class<LocalDate> getConvertedType() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public String getLogicalTypeName() {
+      return "date";
+    }
+
+    @Override
+    public LocalDate fromInt(Integer daysFromEpoch, Schema schema, LogicalType type) {
+      return EPOCH_DATE.plusDays(daysFromEpoch);
+    }
+
+    @Override
+    public Integer toInt(LocalDate date, Schema schema, LogicalType type) {
+      return Days.daysBetween(EPOCH_DATE, date).getDays();
+    }
+  }
+
+  public static class TimeConversion extends Conversion<LocalTime> {
+    @Override
+    public Class<LocalTime> getConvertedType() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public String getLogicalTypeName() {
+      return "time-millis";
+    }
+
+    @Override
+    public LocalTime fromInt(Integer millisFromMidnight, Schema schema, LogicalType type) {
+      return LocalTime.fromMillisOfDay(millisFromMidnight);
+    }
+
+    @Override
+    public Integer toInt(LocalTime time, Schema schema, LogicalType type) {
+      return time.millisOfDay().get();
+    }
+  }
+
+  public static class TimeMicrosConversion extends Conversion<LocalTime> {
+    @Override
+    public Class<LocalTime> getConvertedType() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public String getLogicalTypeName() {
+      return "time-micros";
+    }
+
+    @Override
+    public LocalTime fromLong(Long microsFromMidnight, Schema schema, LogicalType type) {
+      return LocalTime.fromMillisOfDay(microsFromMidnight / 1000);
+    }
+  }
+
+  public static class LossyTimeMicrosConversion extends TimeMicrosConversion {
+    @Override
+    public Long toLong(LocalTime time, Schema schema, LogicalType type) {
+      return 1000 * (long) time.millisOfDay().get();
+    }
+  }
+
+  public static class TimestampConversion extends Conversion<DateTime> {
+    @Override
+    public Class<DateTime> getConvertedType() {
+      return DateTime.class;
+    }
+
+    @Override
+    public String getLogicalTypeName() {
+      return "timestamp-millis";
+    }
+
+    @Override
+    public DateTime fromLong(Long millisFromEpoch, Schema schema, LogicalType type) {
+      return new DateTime(millisFromEpoch, DateTimeZone.UTC);
+    }
+
+    @Override
+    public Long toLong(DateTime timestamp, Schema schema, LogicalType type) {
+      return timestamp.getMillis();
+    }
+  }
+
+  public static class TimestampMicrosConversion extends Conversion<DateTime> {
+    @Override
+    public Class<DateTime> getConvertedType() {
+      return DateTime.class;
+    }
+
+    @Override
+    public String getLogicalTypeName() {
+      return "timestamp-micros";
+    }
+
+    @Override
+    public DateTime fromLong(Long microsFromEpoch, Schema schema, LogicalType type) {
+      return new DateTime(microsFromEpoch / 1000, DateTimeZone.UTC);
+    }
+  }
+
+  public static class LossyTimestampMicrosConversion extends TimestampMicrosConversion {
+    @Override
+    public Long toLong(DateTime timestamp, Schema schema, LogicalType type) {
+      return 1000 * timestamp.getMillis();
+    }
+  }
+}

Added: avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/TestTimeConversions.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/TestTimeConversions.java?rev=1690131&view=auto
==============================================================================
--- avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/TestTimeConversions.java (added)
+++ avro/trunk/lang/java/avro/src/test/java/org/apache/avro/data/TestTimeConversions.java Thu Jul  9 17:51:59 2015
@@ -0,0 +1,232 @@
+package org.apache.avro.data;
+
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.data.TimeConversions.DateConversion;
+import org.apache.avro.data.TimeConversions.LossyTimeMicrosConversion;
+import org.apache.avro.data.TimeConversions.LossyTimestampMicrosConversion;
+import org.apache.avro.data.TimeConversions.TimeMicrosConversion;
+import org.apache.avro.data.TimeConversions.TimestampMicrosConversion;
+import org.apache.avro.data.TimeConversions.TimeConversion;
+import org.apache.avro.data.TimeConversions.TimestampConversion;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.LocalDate;
+import org.joda.time.LocalTime;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import java.util.Date;
+
+public class TestTimeConversions {
+
+  public static Schema DATE_SCHEMA;
+  public static Schema TIME_MILLIS_SCHEMA;
+  public static Schema TIME_MICROS_SCHEMA;
+  public static Schema TIMESTAMP_MILLIS_SCHEMA;
+  public static Schema TIMESTAMP_MICROS_SCHEMA;
+
+  @BeforeClass
+  public static void createSchemas() {
+    TestTimeConversions.DATE_SCHEMA = LogicalTypes.date()
+        .addToSchema(Schema.create(Schema.Type.INT));
+    TestTimeConversions.TIME_MILLIS_SCHEMA = LogicalTypes.timeMillis()
+        .addToSchema(Schema.create(Schema.Type.INT));
+    TestTimeConversions.TIME_MICROS_SCHEMA = LogicalTypes.timeMicros()
+        .addToSchema(Schema.create(Schema.Type.LONG));
+    TestTimeConversions.TIMESTAMP_MILLIS_SCHEMA = LogicalTypes.timestampMillis()
+        .addToSchema(Schema.create(Schema.Type.LONG));
+    TestTimeConversions.TIMESTAMP_MICROS_SCHEMA = LogicalTypes.timestampMicros()
+        .addToSchema(Schema.create(Schema.Type.LONG));
+  }
+
+  @Test
+  public void testDateConversion() throws Exception {
+    DateConversion conversion = new DateConversion();
+    LocalDate Jan_6_1970 = new LocalDate(1970, 1, 6);    //  5
+    LocalDate Jan_1_1970 = new LocalDate(1970, 1, 1);    //  0
+    LocalDate Dec_27_1969 = new LocalDate(1969, 12, 27); // -5
+
+    Assert.assertEquals("6 Jan 1970 should be 5", 5,
+        (int) conversion.toInt(Jan_6_1970, DATE_SCHEMA, LogicalTypes.date()));
+    Assert.assertEquals("1 Jan 1970 should be 0", 0,
+        (int) conversion.toInt(Jan_1_1970, DATE_SCHEMA, LogicalTypes.date()));
+    Assert.assertEquals("27 Dec 1969 should be -5", -5,
+        (int) conversion.toInt(Dec_27_1969, DATE_SCHEMA, LogicalTypes.date()));
+
+    Assert.assertEquals("6 Jan 1970 should be 5",
+        conversion.fromInt(5, DATE_SCHEMA, LogicalTypes.date()), Jan_6_1970);
+    Assert.assertEquals("1 Jan 1970 should be 0",
+        conversion.fromInt(0, DATE_SCHEMA, LogicalTypes.date()), Jan_1_1970);
+    Assert.assertEquals("27 Dec 1969 should be -5",
+        conversion.fromInt(-5, DATE_SCHEMA, LogicalTypes.date()), Dec_27_1969);
+  }
+
+  @Test
+  public void testTimeMillisConversion() throws Exception {
+    TimeConversion conversion = new TimeConversion();
+    LocalTime oneAM = new LocalTime(1, 0);
+    LocalTime afternoon = new LocalTime(15, 14, 15, 926);
+    int afternoonMillis = ((15 * 60 + 14) * 60 + 15) * 1000 + 926;
+
+    Assert.assertEquals("Midnight should be 0", 0,
+        (int) conversion.toInt(
+            LocalTime.MIDNIGHT, TIME_MILLIS_SCHEMA, LogicalTypes.timeMillis()));
+    Assert.assertEquals("01:00 should be 3,600,000", 3600000,
+        (int) conversion.toInt(
+            oneAM, TIME_MILLIS_SCHEMA, LogicalTypes.timeMillis()));
+    Assert.assertEquals("15:14:15.926 should be " + afternoonMillis,
+        afternoonMillis,
+        (int) conversion.toInt(
+            afternoon, TIME_MILLIS_SCHEMA, LogicalTypes.timeMillis()));
+
+    Assert.assertEquals("Midnight should be 0",
+        LocalTime.MIDNIGHT,
+        conversion.fromInt(0, TIME_MILLIS_SCHEMA, LogicalTypes.timeMillis()));
+    Assert.assertEquals("01:00 should be 3,600,000",
+        oneAM,
+        conversion.fromInt(
+            3600000, TIME_MILLIS_SCHEMA, LogicalTypes.timeMillis()));
+    Assert.assertEquals("15:14:15.926 should be " + afternoonMillis,
+        afternoon,
+        conversion.fromInt(
+            afternoonMillis, TIME_MILLIS_SCHEMA, LogicalTypes.timeMillis()));
+  }
+
+  @Test
+  public void testTimeMicrosConversion() throws Exception {
+    TimeMicrosConversion conversion = new TimeMicrosConversion();
+    LocalTime oneAM = new LocalTime(1, 0);
+    LocalTime afternoon = new LocalTime(15, 14, 15, 926);
+    long afternoonMicros = ((long) (15 * 60 + 14) * 60 + 15) * 1000000 + 926551;
+
+    Assert.assertEquals("Midnight should be 0",
+        LocalTime.MIDNIGHT,
+        conversion.fromLong(0L, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+    Assert.assertEquals("01:00 should be 3,600,000,000",
+        oneAM,
+        conversion.fromLong(
+            3600000000L, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+    Assert.assertEquals("15:14:15.926000 should be " + afternoonMicros,
+        afternoon,
+        conversion.fromLong(
+            afternoonMicros, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+
+    try {
+      conversion.toLong(afternoon,
+          TIMESTAMP_MICROS_SCHEMA, LogicalTypes.timestampMicros());
+      Assert.fail("Should not convert LocalTime to long");
+    } catch (UnsupportedOperationException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testLossyTimeMicrosConversion() throws Exception {
+    TimeMicrosConversion conversion = new LossyTimeMicrosConversion();
+    LocalTime oneAM = new LocalTime(1, 0);
+    LocalTime afternoon = new LocalTime(15, 14, 15, 926);
+    long afternoonMicros = ((long) (15 * 60 + 14) * 60 + 15) * 1000000 + 926551;
+
+    Assert.assertEquals("Midnight should be 0", 0,
+        (long) conversion.toLong(
+            LocalTime.MIDNIGHT, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+    Assert.assertEquals("01:00 should be 3,600,000,000", 3600000000L,
+        (long) conversion.toLong(
+            oneAM, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+    Assert.assertEquals("15:14:15.926551 should be " + afternoonMicros,
+        dropMicros(afternoonMicros), // loses precision!
+        (long) conversion.toLong(
+            afternoon, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+
+    Assert.assertEquals("Midnight should be 0",
+        LocalTime.MIDNIGHT,
+        conversion.fromLong(0L, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+    Assert.assertEquals("01:00 should be 3,600,000,000",
+        oneAM,
+        conversion.fromLong(
+            3600000000L, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+    Assert.assertEquals("15:14:15.926000 should be " + afternoonMicros,
+        afternoon,
+        conversion.fromLong(
+            afternoonMicros, TIME_MICROS_SCHEMA, LogicalTypes.timeMicros()));
+  }
+
+  @Test
+  public void testTimestampMillisConversion() throws Exception {
+    TimestampConversion conversion = new TimestampConversion();
+    long nowInstant = new Date().getTime();
+
+    DateTime now = conversion.fromLong(
+        nowInstant, TIMESTAMP_MILLIS_SCHEMA, LogicalTypes.timestampMillis());
+    long roundTrip = conversion.toLong(
+        now, TIMESTAMP_MILLIS_SCHEMA, LogicalTypes.timestampMillis());
+    Assert.assertEquals("Round-trip conversion should work",
+        nowInstant, roundTrip);
+
+    long May_28_2015_21_46_53_221_instant = 1432849613221L;
+    DateTime May_28_2015_21_46_53_221 =
+        new DateTime(2015, 5, 28, 21, 46, 53, 221, DateTimeZone.UTC);
+
+    Assert.assertEquals("Known date should be correct",
+        May_28_2015_21_46_53_221,
+        conversion.fromLong(May_28_2015_21_46_53_221_instant,
+            TIMESTAMP_MILLIS_SCHEMA, LogicalTypes.timestampMillis()));
+    Assert.assertEquals("Known date should be correct",
+        May_28_2015_21_46_53_221_instant,
+        (long) conversion.toLong(May_28_2015_21_46_53_221,
+            TIMESTAMP_MILLIS_SCHEMA, LogicalTypes.timestampMillis()));
+  }
+
+  @Test
+  public void testTimestampMicrosConversion() throws Exception {
+    TimestampMicrosConversion conversion = new TimestampMicrosConversion();
+
+    long May_28_2015_21_46_53_221_843_instant = 1432849613221L * 1000 + 843;
+    DateTime May_28_2015_21_46_53_221 =
+        new DateTime(2015, 5, 28, 21, 46, 53, 221, DateTimeZone.UTC);
+
+    Assert.assertEquals("Known date should be correct",
+        May_28_2015_21_46_53_221,
+        conversion.fromLong(May_28_2015_21_46_53_221_843_instant,
+            TIMESTAMP_MICROS_SCHEMA, LogicalTypes.timestampMicros()));
+
+    try {
+      conversion.toLong(May_28_2015_21_46_53_221,
+          TIMESTAMP_MICROS_SCHEMA, LogicalTypes.timestampMicros());
+      Assert.fail("Should not convert DateTime to long");
+    } catch (UnsupportedOperationException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testLossyTimestampMicrosConversion() throws Exception {
+    TimestampMicrosConversion conversion = new LossyTimestampMicrosConversion();
+    long nowInstant = new Date().getTime() * 1000 + 674; // add fake micros
+
+    DateTime now = conversion.fromLong(
+        nowInstant, TIMESTAMP_MICROS_SCHEMA, LogicalTypes.timestampMicros());
+    long roundTrip = conversion.toLong(
+        now, TIMESTAMP_MICROS_SCHEMA, LogicalTypes.timestampMicros());
+    Assert.assertEquals("Round-trip conversion should lose microseconds",
+        dropMicros(nowInstant), roundTrip);
+
+    long May_28_2015_21_46_53_221_843_instant = 1432849613221L * 1000 + 843;
+    DateTime May_28_2015_21_46_53_221 =
+        new DateTime(2015, 5, 28, 21, 46, 53, 221, DateTimeZone.UTC);
+
+    Assert.assertEquals("Known date should be correct",
+        May_28_2015_21_46_53_221,
+        conversion.fromLong(May_28_2015_21_46_53_221_843_instant,
+            TIMESTAMP_MICROS_SCHEMA, LogicalTypes.timestampMicros()));
+    Assert.assertEquals("Known date should be correct",
+        dropMicros(May_28_2015_21_46_53_221_843_instant),
+        (long) conversion.toLong(May_28_2015_21_46_53_221,
+            TIMESTAMP_MICROS_SCHEMA, LogicalTypes.timestampMicros()));
+  }
+
+  private long dropMicros(long micros) {
+    return micros / 1000 * 1000;
+  }
+}

Modified: avro/trunk/lang/java/pom.xml
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/pom.xml?rev=1690131&r1=1690130&r2=1690131&view=diff
==============================================================================
--- avro/trunk/lang/java/pom.xml (original)
+++ avro/trunk/lang/java/pom.xml Thu Jul  9 17:51:59 2015
@@ -68,6 +68,7 @@
     <easymock.version>3.2</easymock.version>
     <hamcrest.version>1.3</hamcrest.version>
     <commons-httpclient.version>3.1</commons-httpclient.version>
+    <joda.version>2.7</joda.version>
 
     <!-- version properties for plugins -->
     <checkstyle-plugin.version>2.12.1</checkstyle-plugin.version>
@@ -467,6 +468,12 @@
         <artifactId>commons-compress</artifactId>
         <version>${commons-compress.version}</version>
       </dependency>
+      <dependency>
+        <groupId>joda-time</groupId>
+        <artifactId>joda-time</artifactId>
+        <version>${joda.version}</version>
+        <optional>true</optional>
+      </dependency>
     </dependencies>
   </dependencyManagement>