Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ORC: Enable predicate pushdown and remove metrics workaround for Timestamps #1696

Merged
merged 1 commit into from
Nov 1, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
21 changes: 8 additions & 13 deletions core/src/test/java/org/apache/iceberg/TestMetrics.java
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,7 @@ public void testMetricsForTopLevelFields() throws IOException {
secondRecord.setField("timestampColAboveEpoch", DateTimeUtil.timestampFromMicros(900L));
secondRecord.setField("fixedCol", fixed);
secondRecord.setField("binaryCol", ByteBuffer.wrap("W".getBytes()));
secondRecord.setField("timestampColBelowEpoch", DateTimeUtil.timestampFromMicros(0L));
secondRecord.setField("timestampColBelowEpoch", DateTimeUtil.timestampFromMicros(-7_000L));

InputFile recordsFile = writeRecords(SIMPLE_SCHEMA, firstRecord, secondRecord);

Expand All @@ -221,26 +221,21 @@ public void testMetricsForTopLevelFields() throws IOException {
assertCounts(9, 2L, 0L, metrics);
assertBounds(9, TimeType.get(), 2000L, 3000L, metrics);
assertCounts(10, 2L, 0L, metrics);
if (fileFormat() == FileFormat.ORC) {
// ORC-611: ORC only supports millisecond precision, so we adjust by 1 millisecond
assertBounds(10, TimestampType.withoutZone(), 0L, 1000L, metrics);
} else {
assertBounds(10, TimestampType.withoutZone(), 0L, 900L, metrics);
}
assertBounds(10, TimestampType.withoutZone(), 0L, 900L, metrics);
assertCounts(11, 2L, 0L, metrics);
assertBounds(11, FixedType.ofLength(4),
ByteBuffer.wrap(fixed), ByteBuffer.wrap(fixed), metrics);
assertCounts(12, 2L, 0L, metrics);
assertBounds(12, BinaryType.get(),
ByteBuffer.wrap("S".getBytes()), ByteBuffer.wrap("W".getBytes()), metrics);
if (fileFormat() == FileFormat.ORC) {
// TODO: enable when ORC-342 is fixed - ORC-342: creates inaccurate timestamp/stats below epoch
// ORC-611: ORC only supports millisecond precision, so we adjust by 1 millisecond, e.g.
// assertBounds(13, TimestampType.withoutZone(), -1000L, 1000L, metrics); would fail for a value
// in the range `[1970-01-01 00:00:00.000,1970-01-01 00:00:00.999]`
assertBounds(13, TimestampType.withoutZone(), -1_901_000L, 1000L, metrics);
// TODO: The special condition for ORC can be removed when ORC-342 is fixed
// ORC-342: ORC writer creates inaccurate timestamp data and stats 1 sec below epoch
// Values in the range `[1969-12-31 23:59:59.000,1969-12-31 23:59:59.999]` will have 1 sec added to them
// So the upper bound value of -7_000 micros becomes 993_000 micros
assertBounds(13, TimestampType.withoutZone(), -1_900_300L, 993_000L, metrics);
} else {
assertBounds(13, TimestampType.withoutZone(), -1_900_300L, 0L, metrics);
assertBounds(13, TimestampType.withoutZone(), -1_900_300L, -7_000L, metrics);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -249,13 +249,8 @@ public static Object[][] parameters() {
{ "orc", "double", 2.11d, 1.97d },
{ "orc", "date", "2018-06-29", "2018-05-03" },
{ "orc", "time", "10:02:34.000000", "10:02:34.000001" },
// Temporarily disable filters on Timestamp columns due to ORC-611.
// ORC-611 is closed with fix versions of ORC 1.6.4 and 1.7.0, but
// testing on 1.6.4 still lead to failures and 1.7.0 is not published.
//
// See also HIVE-23036, which is still open.
// { "orc", "timestamp", "2018-06-29T10:02:34.000000", "2018-06-29T15:02:34.000000" },
// { "orc", "timestamptz", "2018-06-29T10:02:34.000000+00:00", "2018-06-29T10:02:34.000000-07:00" },
{ "orc", "timestamp", "2018-06-29T10:02:34.000000", "2018-06-29T15:02:34.000000" },
{ "orc", "timestamptz", "2018-06-29T10:02:34.000000+00:00", "2018-06-29T10:02:34.000000-07:00" },
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't we be using values that have microseconds?

{ "orc", "string", "tapir", "monthly" },
// uuid, fixed and binary types not supported yet
// { "orc", "uuid", uuid, UUID.randomUUID() },
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,6 @@ 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<TypeID> UNSUPPORTED_TYPES = ImmutableSet.of(
TypeID.TIMESTAMP, // Temporarily disable filters on Timestamp columns due to ORC-611
TypeID.BINARY,
TypeID.FIXED,
TypeID.UUID,
Expand Down
7 changes: 3 additions & 4 deletions orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.Metrics;
import org.apache.iceberg.MetricsConfig;
Expand All @@ -44,6 +43,7 @@
import org.apache.iceberg.types.Conversions;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.UnicodeUtil;
import org.apache.orc.BooleanColumnStatistics;
import org.apache.orc.ColumnStatistics;
Expand Down Expand Up @@ -198,7 +198,7 @@ private static Optional<ByteBuffer> fromOrcMin(Type type, ColumnStatistics colum
TimestampColumnStatistics tColStats = (TimestampColumnStatistics) columnStats;
Timestamp minValue = tColStats.getMinimumUTC();
min = Optional.ofNullable(minValue)
.map(v -> TimeUnit.MILLISECONDS.toMicros(v.getTime()))
.map(v -> DateTimeUtil.microsFromInstant(v.toInstant()))
.orElse(null);
} else if (columnStats instanceof BooleanColumnStatistics) {
BooleanColumnStatistics booleanStats = (BooleanColumnStatistics) columnStats;
Expand Down Expand Up @@ -235,8 +235,7 @@ private static Optional<ByteBuffer> fromOrcMax(Type type, ColumnStatistics colum
TimestampColumnStatistics tColStats = (TimestampColumnStatistics) columnStats;
Timestamp maxValue = tColStats.getMaximumUTC();
max = Optional.ofNullable(maxValue)
.map(v -> TimeUnit.MILLISECONDS.toMicros(v.getTime()))
.map(v -> v + 1_000) // Add 1 millisecond to handle precision issue due to ORC-611
.map(v -> DateTimeUtil.microsFromInstant(v.toInstant()))
.orElse(null);
} else if (columnStats instanceof BooleanColumnStatistics) {
BooleanColumnStatistics booleanStats = (BooleanColumnStatistics) columnStats;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
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;
Expand Down Expand Up @@ -129,8 +130,8 @@ public void testTimezoneSensitiveTypes() {
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(tsTzPredicate.toInstant()))
// .equals("`ts`", Type.TIMESTAMP, Timestamp.from(tsPredicate.toInstant()))
.equals("`tsTz`", Type.TIMESTAMP, Timestamp.from(tsTzPredicate.toInstant()))
.equals("`ts`", Type.TIMESTAMP, Timestamp.from(tsPredicate.toInstant()))
.end()
.build();

Expand Down