diff --git a/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java index f51fb5e3d7c0..cd80f237546a 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java +++ b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java @@ -56,7 +56,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.junit.Assert; -import org.junit.Assume; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -457,7 +456,6 @@ private DataFile writeFile(String location, String filename, Schema schema, List @Test public void testFilterWithDateAndTimestamp() throws IOException { - Assume.assumeFalse(format == FileFormat.ORC); Schema schema = new Schema( required(1, "timestamp_with_zone", Types.TimestampType.withZone()), required(2, "timestamp_without_zone", Types.TimestampType.withoutZone()), diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java index 8647a9c7dee0..17d3b992f138 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java @@ -253,12 +253,13 @@ public static Object[][] parameters() { new Object[] { "orc", "double", 2.11d, 1.97d }, new Object[] { "orc", "date", "2018-06-29", "2018-05-03" }, new Object[] { "orc", "time", "10:02:34.000000", "10:02:34.000001" }, - new Object[] { "orc", "timestamp", - "2018-06-29T10:02:34.000000", - "2018-06-29T15:02:34.000000" }, - new Object[] { "orc", "timestamptz", - "2018-06-29T10:02:34.000000+00:00", - "2018-06-29T10:02:34.000000-07:00" }, + // Temporarily disable filters on Timestamp columns due to ORC-611 + // new Object[] { "orc", "timestamp", + // "2018-06-29T10:02:34.000000", + // "2018-06-29T15:02:34.000000" }, + // new Object[] { "orc", "timestamptz", + // "2018-06-29T10:02:34.000000+00:00", + // "2018-06-29T10:02:34.000000-07:00" }, new Object[] { "orc", "string", "tapir", "monthly" }, // uuid, fixed and binary types not supported yet // new Object[] { "orc", "uuid", uuid, UUID.randomUUID() }, diff --git a/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java b/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java index 8f25e1fee6d2..716e92e613be 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java @@ -53,6 +53,7 @@ static SearchArgument convert(Expression expr, TypeDescription readSchema) { // Currently every predicate in ORC requires a PredicateLeaf.Type field which is not available for these Iceberg types private static final Set UNSUPPORTED_TYPES = ImmutableSet.of( + TypeID.TIMESTAMP, // Temporarily disable filters on Timestamp columns due to ORC-611 TypeID.BINARY, TypeID.FIXED, TypeID.UUID, @@ -254,8 +255,10 @@ private Object literal(Type icebergType, T icebergLiteral) { return Date.valueOf(LocalDate.ofEpochDay((Integer) icebergLiteral)); case TIMESTAMP: long microsFromEpoch = (Long) icebergLiteral; - return Timestamp.from(Instant.ofEpochSecond(Math.floorDiv(microsFromEpoch, 1_000_000), - (microsFromEpoch % 1_000_000) * 1_000)); + return Timestamp.from(Instant.ofEpochSecond( + Math.floorDiv(microsFromEpoch, 1_000_000), + Math.floorMod(microsFromEpoch, 1_000_000) * 1_000 + )); case DECIMAL: return new HiveDecimalWritable(HiveDecimal.create((BigDecimal) icebergLiteral, false)); default: diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java b/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java index f09d811b9a2f..9ddf8d02368b 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java @@ -22,10 +22,12 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.sql.Date; -import java.sql.Timestamp; import java.time.Instant; import java.time.LocalDate; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; +import java.time.temporal.ChronoUnit; import java.util.TimeZone; import java.util.UUID; import org.apache.iceberg.Schema; @@ -105,6 +107,10 @@ public void testPrimitiveTypes() { public void testTimezoneSensitiveTypes() { for (String timezone : new String[]{"America/New_York", "Asia/Kolkata", "UTC/Greenwich"}) { TimeZone.setDefault(TimeZone.getTimeZone(timezone)); + OffsetDateTime tsTzPredicate = OffsetDateTime.parse("2019-10-02T00:47:28.207366Z"); + OffsetDateTime tsPredicate = OffsetDateTime.parse("1968-01-16T13:07:59.048625Z"); + OffsetDateTime epoch = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + Schema schema = new Schema( required(1, "date", Types.DateType.get()), required(2, "tsTz", Types.TimestampType.withZone()), @@ -112,15 +118,16 @@ public void testTimezoneSensitiveTypes() { ); Expression expr = and( - and(equal("date", 10L), equal("tsTz", 10 * 3600 * 1000000L)), - equal("ts", 20 * 3600 * 1000000L) + and(equal("date", 10L), equal("tsTz", ChronoUnit.MICROS.between(epoch, tsTzPredicate))), + equal("ts", ChronoUnit.MICROS.between(epoch, tsPredicate)) ); Expression boundFilter = Binder.bind(schema.asStruct(), expr, true); SearchArgument expected = SearchArgumentFactory.newBuilder() .startAnd() .equals("`date`", Type.DATE, Date.valueOf(LocalDate.parse("1970-01-11", DateTimeFormatter.ISO_LOCAL_DATE))) - .equals("`tsTz`", Type.TIMESTAMP, Timestamp.from(Instant.ofEpochSecond(10 * 3600))) - .equals("`ts`", Type.TIMESTAMP, Timestamp.from(Instant.ofEpochSecond(20 * 3600))) + // Temporarily disable filters on Timestamp columns due to ORC-611 + // .equals("`tsTz`", Type.TIMESTAMP, Timestamp.from(tsTzPredicate.toInstant())) + // .equals("`ts`", Type.TIMESTAMP, Timestamp.from(tsPredicate.toInstant())) .end() .build();