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

API: implement types timestamp_ns and timestamptz_ns #9008

Merged
merged 42 commits into from
Sep 3, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
42 commits
Select commit Hold shift + click to select a range
0e098f0
API: implement types timestamp_ns and timestamptz_ns
jacobmarble Feb 20, 2024
b283a5a
Redo as separate type
epgif Mar 26, 2024
a114044
Try to hew closer to the original satisfiesOrderOf logic.
epgif Mar 26, 2024
0ebdf9f
Test that Avro produces a value within 1 micro
epgif Mar 26, 2024
bea3487
address most comments
epgif Apr 11, 2024
753aed5
Bucket timestamp and timestamp_ns the same.
epgif Apr 12, 2024
19605d6
Fix bug caught by TestPartitionSpecParser#testTransforms .
epgif Apr 25, 2024
d0b4627
address review comments
epgif Jun 5, 2024
2f71da9
address style improvements
epgif Jun 6, 2024
c3c1288
test DateTimeUtil.convertNanos on negative input
epgif Jun 6, 2024
ec01d68
use Math.toIntExact in Timestamps NANOS conversion
epgif Jun 6, 2024
08e123c
Merge branch 'main' into jgm-timestamp-nanos-api
epgif Jun 18, 2024
eb5e382
address review comments
epgif Jul 4, 2024
421ed86
Adjust tests as requested.
epgif Jul 8, 2024
d7a1326
add nanosecond tests to TestTimestamps
jacobmarble Jul 8, 2024
068b18b
add timestamptz and timestampns_tz to fromPrimitiveString test
jacobmarble Jul 8, 2024
269b3e9
correct TestYears - was testing days
jacobmarble Jul 8, 2024
8462374
test Avro timestamp conversion precisely
jacobmarble Jul 8, 2024
16319d8
Add requested additional tests.
epgif Jul 8, 2024
e38f68a
Remove redundant and erroneous tests.
epgif Jul 8, 2024
654fc61
Merge remote-tracking branch 'jacobmarble/jgm-timestamp-nanos-api' in…
epgif Jul 11, 2024
d34daad
Update TestBucketing as requested.
epgif Jul 11, 2024
42ca8a4
Convert long to TimestampLiteral and then that to TimestampNanoLiteral.
epgif Jul 31, 2024
0cbdeb8
Use DateTimeUtil conversion instead of /.
rdblue Aug 23, 2024
48626e5
Update Literals to use DateTimeUtil, add new tests.
rdblue Aug 23, 2024
743e872
Fix test for DateTimeUtil.isoTimestampToNanos.
rdblue Aug 23, 2024
dedeb19
Fix TestDateTimeUtil and add test cases.
rdblue Aug 23, 2024
c9f4273
Simplify Timestamps transform get.
rdblue Aug 23, 2024
97489ab
Remove ChronoUnit wrapper enum.
rdblue Aug 23, 2024
8c3cc67
Restore Timestamps as enum and simplify boilerplate.
rdblue Aug 25, 2024
6f25c99
Minor fix to bucket transform.
rdblue Aug 25, 2024
f006cb2
Fix style
rdblue Aug 25, 2024
066c955
Fix typos in TestTimestamps.
rdblue Aug 25, 2024
4d77202
Add a comment to clarify conversion test.
rdblue Aug 25, 2024
9a3d16f
Split timestamp and timestamp_ns comparator test cases.
rdblue Aug 25, 2024
274de56
Fix spec update to specify microsecond hashing.
rdblue Aug 25, 2024
20e7085
Merge pull request #1 from rdblue/jgm-timestamp-nanos-api
jacobmarble Aug 26, 2024
0bbd3d6
Run :iceberg-api:spotlessApply
epgif Aug 26, 2024
58f11a3
fix testTimestampWithZoneHumanString
epgif Aug 30, 2024
8ea5777
Prevent creating table metadata with nanosecond timestamps before v3.
rdblue Sep 2, 2024
1321952
Merge pull request #2 from rdblue/jgm-timestamp-nanos-api
Sep 3, 2024
47d4b64
fix merge conflict
jacobmarble Sep 3, 2024
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
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.iceberg.transforms.Transforms;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.DateTimeUtil;

/** Expression utility methods. */
public class ExpressionUtil {
Expand All @@ -48,10 +49,15 @@ public class ExpressionUtil {
private static final Pattern DATE = Pattern.compile("\\d{4}-\\d{2}-\\d{2}");
private static final Pattern TIME = Pattern.compile("\\d{2}:\\d{2}(:\\d{2}(.\\d{1,9})?)?");
private static final Pattern TIMESTAMP =
Pattern.compile("\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{1,9})?)?");
Pattern.compile("\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{1,6})?)?");
private static final Pattern TIMESTAMPNS =
Pattern.compile("\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?");
private static final Pattern TIMESTAMPTZ =
Pattern.compile(
"\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{1,9})?)?([-+]\\d{2}:\\d{2}|Z)");
"\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{1,6})?)?([-+]\\d{2}:\\d{2}|Z)");
private static final Pattern TIMESTAMPTZNS =
Pattern.compile(
"\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?([-+]\\d{2}:\\d{2}|Z)");
static final int LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD = 10;
private static final int LONG_IN_PREDICATE_ABBREVIATION_MIN_GAIN = 5;

Expand Down Expand Up @@ -514,7 +520,7 @@ private static String sanitize(Type type, Object value, long now, int today) {
case TIME:
return "(time)";
case TIMESTAMP:
return sanitizeTimestamp((long) value, now);
return sanitizeTimestamp(((Types.TimestampType) type).unit(), (long) value, now);
case STRING:
return sanitizeString((CharSequence) value, now, today);
case BOOLEAN:
Expand All @@ -535,7 +541,8 @@ private static String sanitize(Literal<?> literal, long now, int today) {
} else if (literal instanceof Literals.DateLiteral) {
return sanitizeDate(((Literals.DateLiteral) literal).value(), today);
} else if (literal instanceof Literals.TimestampLiteral) {
return sanitizeTimestamp(((Literals.TimestampLiteral) literal).value(), now);
Literals.TimestampLiteral tsLiteral = ((Literals.TimestampLiteral) literal);
return sanitizeTimestamp(tsLiteral.unit(), tsLiteral.value(), now);
} else if (literal instanceof Literals.TimeLiteral) {
return "(time)";
} else if (literal instanceof Literals.IntegerLiteral) {
Expand Down Expand Up @@ -564,7 +571,18 @@ private static String sanitizeDate(int days, int today) {
return "(date)";
}

private static String sanitizeTimestamp(long micros, long now) {
private static String sanitizeTimestamp(Types.TimestampType.Unit unit, long timeUnits, long now) {
final long micros;
switch (unit) {
case MICROS:
micros = timeUnits;
break;
case NANOS:
micros = DateTimeUtil.nanosToMicros(timeUnits);
break;
default:
throw new UnsupportedOperationException("Unsupported timestamp unit: " + unit);
}
String isPast = now > micros ? "ago" : "from-now";
long diff = Math.abs(now - micros);
if (diff < FIVE_MINUTES_IN_MICROS) {
Expand Down Expand Up @@ -595,11 +613,17 @@ private static String sanitizeString(CharSequence value, long now, int today) {
Literal<Integer> date = Literal.of(value).to(Types.DateType.get());
return sanitizeDate(date.value(), today);
} else if (TIMESTAMP.matcher(value).matches()) {
Literal<Long> ts = Literal.of(value).to(Types.TimestampType.withoutZone());
return sanitizeTimestamp(ts.value(), now);
Literal<Long> ts = Literal.of(value).to(Types.TimestampType.microsWithoutZone());
return sanitizeTimestamp(Types.TimestampType.Unit.MICROS, ts.value(), now);
} else if (TIMESTAMPNS.matcher(value).matches()) {
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
Literal<Long> ts = Literal.of(value).to(Types.TimestampType.nanosWithoutZone());
return sanitizeTimestamp(Types.TimestampType.Unit.NANOS, ts.value(), now);
} else if (TIMESTAMPTZ.matcher(value).matches()) {
Literal<Long> ts = Literal.of(value).to(Types.TimestampType.withZone());
return sanitizeTimestamp(ts.value(), now);
Literal<Long> ts = Literal.of(value).to(Types.TimestampType.microsWithZone());
return sanitizeTimestamp(Types.TimestampType.Unit.MICROS, ts.value(), now);
} else if (TIMESTAMPTZNS.matcher(value).matches()) {
Literal<Long> ts = Literal.of(value).to(Types.TimestampType.nanosWithZone());
return sanitizeTimestamp(Types.TimestampType.Unit.NANOS, ts.value(), now);
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
} else if (TIME.matcher(value).matches()) {
return "(time)";
} else {
Expand Down
77 changes: 56 additions & 21 deletions api/src/main/java/org/apache/iceberg/expressions/Literals.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import java.nio.ByteBuffer;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
Expand All @@ -39,7 +38,9 @@
import org.apache.iceberg.types.Conversions;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.types.Types.TimestampType;
import org.apache.iceberg.util.ByteBuffers;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.NaNUtil;

class Literals {
Expand Down Expand Up @@ -298,7 +299,7 @@ public <T> Literal<T> to(Type type) {
case TIME:
return (Literal<T>) new TimeLiteral(value());
case TIMESTAMP:
return (Literal<T>) new TimestampLiteral(value());
return (Literal<T>) new TimestampLiteral(((TimestampType) type).unit(), value());
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
case DATE:
if ((long) Integer.MAX_VALUE < value()) {
return aboveMax();
Expand Down Expand Up @@ -426,23 +427,55 @@ protected Type.TypeID typeId() {
}

static class TimestampLiteral extends ComparableLiteral<Long> {
TimestampLiteral(Long value) {
private final TimestampType.Unit unit;

TimestampLiteral(TimestampType.Unit unit, Long value) {
super(value);
this.unit = unit;
}

@Override
@SuppressWarnings("unchecked")
public <T> Literal<T> to(Type type) {
switch (type.typeId()) {
case TIMESTAMP:
return (Literal<T>) this;
TimestampType.Unit toUnit = ((TimestampType) type).unit();
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
switch (unit) {
case MICROS:
switch (toUnit) {
case MICROS:
return (Literal<T>) this;
case NANOS:
return (Literal<T>)
new TimestampLiteral(unit, DateTimeUtil.microsToNanos(value()));
}
break;
case NANOS:
switch (toUnit) {
case MICROS:
return (Literal<T>)
new TimestampLiteral(unit, DateTimeUtil.nanosToMicros(value()));
case NANOS:
return (Literal<T>) this;
}
break;
}
break;
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
case DATE:
return (Literal<T>)
new DateLiteral(
(int)
ChronoUnit.DAYS.between(
EPOCH_DAY, EPOCH.plus(value(), ChronoUnit.MICROS).toLocalDate()));
default:
switch (unit) {
case MICROS:
return (Literal<T>)
new DateLiteral(
(int)
ChronoUnit.DAYS.between(
EPOCH_DAY, EPOCH.plus(value(), ChronoUnit.MICROS).toLocalDate()));
case NANOS:
return (Literal<T>)
new DateLiteral(
(int)
ChronoUnit.DAYS.between(
EPOCH_DAY, EPOCH.plusNanos(value()).toLocalDate()));
}
}
return null;
}
Expand All @@ -451,6 +484,10 @@ public <T> Literal<T> to(Type type) {
protected Type.TypeID typeId() {
return Type.TypeID.TIMESTAMP;
}

protected TimestampType.Unit unit() {
return unit;
}
}

static class DecimalLiteral extends ComparableLiteral<BigDecimal> {
Expand Down Expand Up @@ -501,19 +538,17 @@ public <T> Literal<T> to(Type type) {
return (Literal<T>) new TimeLiteral(timeMicros);

case TIMESTAMP:
if (((Types.TimestampType) type).shouldAdjustToUTC()) {
long timestampMicros =
ChronoUnit.MICROS.between(
EPOCH, OffsetDateTime.parse(value(), DateTimeFormatter.ISO_DATE_TIME));
return (Literal<T>) new TimestampLiteral(timestampMicros);
final TimestampType tsType = (TimestampType) type;
final String value = value().toString();
final java.time.temporal.Temporal valueAsTemporal;
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
if (tsType.shouldAdjustToUTC()) {
valueAsTemporal = DateTimeUtil.isoTimestamptzToOffsetDateTime(value);
} else {
long timestampMicros =
ChronoUnit.MICROS.between(
EPOCH,
LocalDateTime.parse(value(), DateTimeFormatter.ISO_LOCAL_DATE_TIME)
.atOffset(ZoneOffset.UTC));
return (Literal<T>) new TimestampLiteral(timestampMicros);
valueAsTemporal =
DateTimeUtil.isoTimestampToLocalDateTime(value).atOffset(ZoneOffset.UTC);
}
final long timestampUnits = tsType.unit().between(EPOCH, valueAsTemporal);
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
return (Literal<T>) new TimestampLiteral(tsType.unit(), timestampUnits);

case STRING:
return (Literal<T>) this;
Expand Down
14 changes: 8 additions & 6 deletions api/src/main/java/org/apache/iceberg/transforms/Days.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.iceberg.transforms;

import java.io.ObjectStreamException;
import java.time.temporal.ChronoUnit;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;

Expand All @@ -37,7 +38,7 @@ protected Transform<T, Integer> toEnum(Type type) {
case DATE:
return (Transform<T, Integer>) Dates.DAY;
case TIMESTAMP:
return (Transform<T, Integer>) Timestamps.DAY;
return (Transform<T, Integer>) Timestamps.get((Types.TimestampType) type, ChronoUnit.DAYS);
default:
throw new IllegalArgumentException("Unsupported type: " + type);
}
Expand All @@ -55,14 +56,15 @@ public boolean satisfiesOrderOf(Transform<?, ?> other) {
}

if (other instanceof Timestamps) {
return Timestamps.DAY.satisfiesOrderOf(other);
ChronoUnit otherResultTypeUnit = ((Timestamps) other).getResultTypeUnit();
return otherResultTypeUnit == ChronoUnit.DAYS
|| otherResultTypeUnit == ChronoUnit.MONTHS
|| otherResultTypeUnit == ChronoUnit.YEARS;
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
} else if (other instanceof Dates) {
return Dates.DAY.satisfiesOrderOf(other);
} else if (other instanceof Days || other instanceof Months || other instanceof Years) {
return true;
} else {
return other instanceof Days || other instanceof Months || other instanceof Years;
}

return false;
}

@Override
Expand Down
21 changes: 12 additions & 9 deletions api/src/main/java/org/apache/iceberg/transforms/Hours.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.iceberg.transforms;

import java.io.ObjectStreamException;
import java.time.temporal.ChronoUnit;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;

Expand All @@ -34,7 +35,7 @@ static <T> Hours<T> get() {
@SuppressWarnings("unchecked")
protected Transform<T, Integer> toEnum(Type type) {
if (type.typeId() == Type.TypeID.TIMESTAMP) {
return (Transform<T, Integer>) Timestamps.HOUR;
return (Transform<T, Integer>) Timestamps.get((Types.TimestampType) type, ChronoUnit.HOURS);
}

throw new IllegalArgumentException("Unsupported type: " + type);
Expand All @@ -57,15 +58,17 @@ public boolean satisfiesOrderOf(Transform<?, ?> other) {
}

if (other instanceof Timestamps) {
return other == Timestamps.HOUR;
} else if (other instanceof Hours
|| other instanceof Days
|| other instanceof Months
|| other instanceof Years) {
return true;
ChronoUnit otherResultTypeUnit = ((Timestamps) other).getResultTypeUnit();
return otherResultTypeUnit == ChronoUnit.HOURS
|| otherResultTypeUnit == ChronoUnit.DAYS
|| otherResultTypeUnit == ChronoUnit.MONTHS
|| otherResultTypeUnit == ChronoUnit.YEARS;
} else {
return other instanceof Hours
|| other instanceof Days
|| other instanceof Months
|| other instanceof Years;
}

return false;
}

@Override
Expand Down
13 changes: 7 additions & 6 deletions api/src/main/java/org/apache/iceberg/transforms/Months.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.iceberg.transforms;

import java.io.ObjectStreamException;
import java.time.temporal.ChronoUnit;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;

Expand All @@ -37,7 +38,8 @@ protected Transform<T, Integer> toEnum(Type type) {
case DATE:
return (Transform<T, Integer>) Dates.MONTH;
case TIMESTAMP:
return (Transform<T, Integer>) Timestamps.MONTH;
return (Transform<T, Integer>)
Timestamps.get((Types.TimestampType) type, ChronoUnit.MONTHS);
default:
throw new IllegalArgumentException("Unsupported type: " + type);
}
Expand All @@ -55,14 +57,13 @@ public boolean satisfiesOrderOf(Transform<?, ?> other) {
}

if (other instanceof Timestamps) {
return Timestamps.MONTH.satisfiesOrderOf(other);
ChronoUnit otherResultTypeUnit = ((Timestamps) other).getResultTypeUnit();
return otherResultTypeUnit == ChronoUnit.MONTHS || otherResultTypeUnit == ChronoUnit.YEARS;
} else if (other instanceof Dates) {
return Dates.MONTH.satisfiesOrderOf(other);
} else if (other instanceof Months || other instanceof Years) {
return true;
} else {
return other instanceof Months || other instanceof Years;
}

return false;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,16 +122,23 @@ static <R> R visit(Schema schema, PartitionField field, PartitionSpecVisitor<R>
int width = ((Truncate<?>) transform).width();
return visitor.truncate(field.fieldId(), sourceName, field.sourceId(), width);
} else if (transform == Dates.YEAR
|| transform == Timestamps.YEAR
|| transform == Timestamps.YEAR_FROM_MICROS
|| transform == Timestamps.YEAR_FROM_NANOS
|| transform instanceof Years) {
return visitor.year(field.fieldId(), sourceName, field.sourceId());
} else if (transform == Dates.MONTH
|| transform == Timestamps.MONTH
|| transform == Timestamps.MONTH_FROM_MICROS
|| transform == Timestamps.MONTH_FROM_NANOS
|| transform instanceof Months) {
return visitor.month(field.fieldId(), sourceName, field.sourceId());
} else if (transform == Dates.DAY || transform == Timestamps.DAY || transform instanceof Days) {
} else if (transform == Dates.DAY
|| transform == Timestamps.DAY_FROM_MICROS
|| transform == Timestamps.DAY_FROM_NANOS
|| transform instanceof Days) {
return visitor.day(field.fieldId(), sourceName, field.sourceId());
} else if (transform == Timestamps.HOUR || transform instanceof Hours) {
} else if (transform == Timestamps.HOUR_FROM_MICROS
|| transform == Timestamps.HOUR_FROM_NANOS
|| transform instanceof Hours) {
return visitor.hour(field.fieldId(), sourceName, field.sourceId());
} else if (transform instanceof VoidTransform) {
return visitor.alwaysNull(field.fieldId(), sourceName, field.sourceId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,22 +84,16 @@ static <R> List<R> visit(SortOrder sortOrder, SortOrderVisitor<R> visitor) {
results.add(
visitor.truncate(
sourceName, field.sourceId(), width, field.direction(), field.nullOrder()));
} else if (transform == Dates.YEAR
|| transform == Timestamps.YEAR
|| transform instanceof Years) {
} else if ("year".equalsIgnoreCase(transform.toString())) {
Fokko marked this conversation as resolved.
Show resolved Hide resolved
results.add(
visitor.year(sourceName, field.sourceId(), field.direction(), field.nullOrder()));
} else if (transform == Dates.MONTH
|| transform == Timestamps.MONTH
|| transform instanceof Months) {
} else if ("month".equalsIgnoreCase(transform.toString())) {
jacobmarble marked this conversation as resolved.
Show resolved Hide resolved
results.add(
visitor.month(sourceName, field.sourceId(), field.direction(), field.nullOrder()));
} else if (transform == Dates.DAY
|| transform == Timestamps.DAY
|| transform instanceof Days) {
} else if ("day".equalsIgnoreCase(transform.toString())) {
results.add(
visitor.day(sourceName, field.sourceId(), field.direction(), field.nullOrder()));
} else if (transform == Timestamps.HOUR || transform instanceof Hours) {
} else if ("hour".equalsIgnoreCase(transform.toString())) {
results.add(
visitor.hour(sourceName, field.sourceId(), field.direction(), field.nullOrder()));
} else if (transform instanceof UnknownTransform) {
Expand Down
Loading