diff --git a/core/trino-spi/src/main/java/io/trino/spi/type/Int128Math.java b/core/trino-spi/src/main/java/io/trino/spi/type/Int128Math.java index b4a2e07aec00..f7fb0a3ed7d2 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/type/Int128Math.java +++ b/core/trino-spi/src/main/java/io/trino/spi/type/Int128Math.java @@ -189,6 +189,13 @@ public static void add(long leftHigh, long leftLow, long rightHigh, long rightLo } } + public static Int128 add(Int128 left, Int128 right) + { + long[] result = new long[2]; + add(left.getHigh(), left.getLow(), right.getHigh(), right.getLow(), result, 0); + return Int128.valueOf(result); + } + public static long addWithOverflow(long leftHigh, long leftLow, long rightHigh, long rightLow, long[] decimal, int offset) { long low = leftLow + rightLow; diff --git a/docs/src/main/sphinx/connector/faker.md b/docs/src/main/sphinx/connector/faker.md index d5fb56791659..21260339de23 100644 --- a/docs/src/main/sphinx/connector/faker.md +++ b/docs/src/main/sphinx/connector/faker.md @@ -111,6 +111,9 @@ The following table details all supported column properties. * - `allowed_values` - List of allowed values. Cannot be set together with the `min`, or `max` properties. +* - `step` + - If set, generate sequential values with this step. For date and time columns + set this to a duration. Cannot be set for character-based type columns. ::: ### Character types diff --git a/plugin/trino-faker/pom.xml b/plugin/trino-faker/pom.xml index 8a87b91f122e..c0866b9f3445 100644 --- a/plugin/trino-faker/pom.xml +++ b/plugin/trino-faker/pom.xml @@ -40,6 +40,11 @@ configuration + + io.airlift + units + + io.trino trino-main @@ -119,12 +124,6 @@ runtime - - io.airlift - units - runtime - - io.trino trino-client diff --git a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/ColumnInfo.java b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/ColumnInfo.java index bd52ae4fa776..9bd28af238b4 100644 --- a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/ColumnInfo.java +++ b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/ColumnInfo.java @@ -27,6 +27,7 @@ public record ColumnInfo(FakerColumnHandle handle, ColumnMetadata metadata) public static final String MIN_PROPERTY = "min"; public static final String MAX_PROPERTY = "max"; public static final String ALLOWED_VALUES_PROPERTY = "allowed_values"; + public static final String STEP_PROPERTY = "step"; public ColumnInfo { diff --git a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerColumnHandle.java b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerColumnHandle.java index 96d870eafb8e..770ee83d20d4 100644 --- a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerColumnHandle.java +++ b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerColumnHandle.java @@ -15,6 +15,7 @@ package io.trino.plugin.faker; import com.google.common.collect.ImmutableList; +import io.airlift.units.Duration; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; @@ -22,20 +23,30 @@ import io.trino.spi.predicate.Range; import io.trino.spi.predicate.ValueSet; import io.trino.spi.type.CharType; +import io.trino.spi.type.TimeType; +import io.trino.spi.type.TimeWithTimeZoneType; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TimestampWithTimeZoneType; import io.trino.spi.type.Type; import io.trino.spi.type.VarbinaryType; import io.trino.spi.type.VarcharType; import java.util.Collection; import java.util.List; +import java.util.concurrent.TimeUnit; +import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.plugin.faker.ColumnInfo.ALLOWED_VALUES_PROPERTY; import static io.trino.plugin.faker.ColumnInfo.GENERATOR_PROPERTY; import static io.trino.plugin.faker.ColumnInfo.MAX_PROPERTY; import static io.trino.plugin.faker.ColumnInfo.MIN_PROPERTY; import static io.trino.plugin.faker.ColumnInfo.NULL_PROBABILITY_PROPERTY; +import static io.trino.plugin.faker.ColumnInfo.STEP_PROPERTY; import static io.trino.spi.StandardErrorCode.INVALID_COLUMN_PROPERTY; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.IntegerType.INTEGER; import static java.util.Objects.requireNonNull; public record FakerColumnHandle( @@ -44,13 +55,17 @@ public record FakerColumnHandle( Type type, double nullProbability, String generator, - Domain domain) + Domain domain, + ValueSet step) implements ColumnHandle { public FakerColumnHandle { requireNonNull(name, "name is null"); requireNonNull(type, "type is null"); + requireNonNull(domain, "domain is null"); + requireNonNull(step, "step is null"); + checkState(step.isNone() || step.isSingleValue(), "step must be a single value"); } public static FakerColumnHandle of(int columnId, ColumnMetadata column, double defaultNullProbability) @@ -63,20 +78,8 @@ public static FakerColumnHandle of(int columnId, ColumnMetadata column, double d if (generator != null && !isCharacterColumn(column)) { throw new TrinoException(INVALID_COLUMN_PROPERTY, "The `%s` property can only be set for CHAR, VARCHAR or VARBINARY columns".formatted(GENERATOR_PROPERTY)); } - Object min; - try { - min = Literal.parse((String) column.getProperties().get(MIN_PROPERTY), column.getType()); - } - catch (IllegalArgumentException e) { - throw new TrinoException(INVALID_COLUMN_PROPERTY, "The `%s` property must be a valid %s literal".formatted(MIN_PROPERTY, column.getType().getDisplayName()), e); - } - Object max; - try { - max = Literal.parse((String) column.getProperties().get(MAX_PROPERTY), column.getType()); - } - catch (IllegalArgumentException e) { - throw new TrinoException(INVALID_COLUMN_PROPERTY, "The `%s` property must be a valid %s literal".formatted(MAX_PROPERTY, column.getType().getDisplayName()), e); - } + Object min = propertyValue(column, MIN_PROPERTY); + Object max = propertyValue(column, MAX_PROPERTY); Domain domain = Domain.all(column.getType()); if (min != null || max != null) { if (isCharacterColumn(column)) { @@ -106,7 +109,8 @@ public static FakerColumnHandle of(int columnId, ColumnMetadata column, double d column.getType(), nullProbability, generator, - domain); + domain, + stepValue(column)); } private static boolean isCharacterColumn(ColumnMetadata column) @@ -114,6 +118,42 @@ private static boolean isCharacterColumn(ColumnMetadata column) return column.getType() instanceof CharType || column.getType() instanceof VarcharType || column.getType() instanceof VarbinaryType; } + private static Object propertyValue(ColumnMetadata column, String property) + { + try { + return Literal.parse((String) column.getProperties().get(property), column.getType()); + } + catch (IllegalArgumentException e) { + throw new TrinoException(INVALID_COLUMN_PROPERTY, "The `%s` property must be a valid %s literal".formatted(property, column.getType().getDisplayName()), e); + } + } + + private static ValueSet stepValue(ColumnMetadata column) + { + Type type = column.getType(); + String rawStep = (String) column.getProperties().get(STEP_PROPERTY); + if (rawStep == null) { + return ValueSet.none(type); + } + if (isCharacterColumn(column)) { + throw new TrinoException(INVALID_COLUMN_PROPERTY, "The `%s` property cannot be set for CHAR, VARCHAR or VARBINARY columns".formatted(STEP_PROPERTY)); + } + if (DATE.equals(column.getType()) || type instanceof TimestampType || type instanceof TimestampWithTimeZoneType || type instanceof TimeType || type instanceof TimeWithTimeZoneType) { + try { + return ValueSet.of(BIGINT, Duration.valueOf(rawStep).roundTo(TimeUnit.NANOSECONDS)); + } + catch (IllegalArgumentException e) { + throw new TrinoException(INVALID_COLUMN_PROPERTY, "The `%s` property for a %s column must be a valid duration literal".formatted(STEP_PROPERTY, column.getType().getDisplayName()), e); + } + } + try { + return ValueSet.of(type, Literal.parse(rawStep, type)); + } + catch (IllegalArgumentException e) { + throw new TrinoException(INVALID_COLUMN_PROPERTY, "The `%s` property for a %s column must be a valid %s literal".formatted(STEP_PROPERTY, column.getType().getDisplayName(), type.getDisplayName()), e); + } + } + private static Range range(Type type, Object min, Object max) { requireNonNull(type, "type is null"); diff --git a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerConnector.java b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerConnector.java index 2a5819c04b47..734fa513b355 100644 --- a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerConnector.java +++ b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerConnector.java @@ -39,6 +39,7 @@ import static io.trino.plugin.faker.ColumnInfo.ALLOWED_VALUES_PROPERTY; import static io.trino.plugin.faker.ColumnInfo.MAX_PROPERTY; import static io.trino.plugin.faker.ColumnInfo.MIN_PROPERTY; +import static io.trino.plugin.faker.ColumnInfo.STEP_PROPERTY; import static io.trino.spi.StandardErrorCode.INVALID_COLUMN_PROPERTY; import static io.trino.spi.StandardErrorCode.INVALID_SCHEMA_PROPERTY; import static io.trino.spi.StandardErrorCode.INVALID_TABLE_PROPERTY; @@ -188,7 +189,12 @@ public List> getColumnProperties() value -> ((List) value).stream() .map(String.class::cast) .collect(toImmutableList()), - value -> value)); + value -> value), + stringProperty( + STEP_PROPERTY, + "If set, generate sequential values with this step. For date and time columns set this to a duration", + null, + false)); } private static void checkProperty(boolean expression, ErrorCodeSupplier errorCode, String errorMessage) diff --git a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerMetadata.java b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerMetadata.java index 275d04d62004..46d246e9d8d0 100644 --- a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerMetadata.java +++ b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerMetadata.java @@ -43,6 +43,8 @@ import io.trino.spi.function.FunctionMetadata; import io.trino.spi.function.SchemaFunctionName; import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.ValueSet; import io.trino.spi.security.TrinoPrincipal; import io.trino.spi.statistics.ComputedStatistics; import io.trino.spi.type.BigintType; @@ -336,7 +338,8 @@ public synchronized FakerOutputTableHandle beginCreateTable(ConnectorSession ses BigintType.BIGINT, 0, "", - Domain.all(BigintType.BIGINT)), + Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BigintType.BIGINT, 0L)), false), + ValueSet.of(BigintType.BIGINT, 1L)), ColumnMetadata.builder() .setName(ROW_ID_COLUMN_NAME) .setType(BigintType.BIGINT) diff --git a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerPageSource.java b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerPageSource.java index aa750f147208..5b27a72557bd 100644 --- a/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerPageSource.java +++ b/plugin/trino-faker/src/main/java/io/trino/plugin/faker/FakerPageSource.java @@ -50,7 +50,6 @@ import java.util.Random; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.faker.FakerMetadata.ROW_ID_COLUMN_NAME; import static io.trino.spi.StandardErrorCode.INVALID_ROW_FILTER; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; @@ -66,10 +65,17 @@ import static io.trino.spi.type.LongTimestampWithTimeZone.fromEpochMillisAndFraction; import static io.trino.spi.type.RealType.REAL; import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_DAY; +import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND; import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_DAY; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MICROSECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_SECOND; import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_DAY; import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_MICROSECOND; import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_NANOSECOND; import static io.trino.spi.type.Timestamps.roundDiv; import static io.trino.spi.type.TinyintType.TINYINT; import static io.trino.spi.type.UuidType.UUID; @@ -126,7 +132,7 @@ class FakerPageSource Faker faker, Random random, List columns, - long offset, + long rowOffset, long limit) { this.faker = requireNonNull(faker, "faker is null"); @@ -141,29 +147,38 @@ class FakerPageSource this.generators = columns .stream() - .map(column -> getGenerator(column, offset)) + .map(column -> getGenerator(column, rowOffset)) .collect(toImmutableList()); this.pageBuilder = new PageBuilder(types); } private Generator getGenerator( FakerColumnHandle column, - long offset) + long rowOffset) { - if (ROW_ID_COLUMN_NAME.equals(column.name())) { - return new Generator() - { - long currentRowId = offset; - - @Override - public void accept(BlockBuilder blockBuilder) - { - BIGINT.writeLong(blockBuilder, currentRowId++); - } - }; + if (column.domain().getValues().isDiscreteSet()) { + List values = column.domain().getValues().getDiscreteSet(); + ObjectWriter singleValueWriter = objectWriter(column.type()); + return (blockBuilder) -> singleValueWriter.accept(blockBuilder, values.get(random.nextInt(values.size()))); } - - return constraintedValueGenerator(column); + Generator generator; + if (!column.step().isNone()) { + generator = sequenceGenerator(column, rowOffset); + } + else { + generator = randomValueGenerator(column); + } + if (column.nullProbability() == 0) { + return generator; + } + return (blockBuilder) -> { + if (random.nextDouble() <= column.nullProbability()) { + blockBuilder.appendNull(); + } + else { + generator.accept(blockBuilder); + } + }; } @Override @@ -227,27 +242,143 @@ public void close() closed = true; } - private Generator constraintedValueGenerator(FakerColumnHandle handle) + private Generator sequenceGenerator(FakerColumnHandle handle, long rowOffset) { - if (handle.domain().getValues().isDiscreteSet()) { - List values = handle.domain().getValues().getDiscreteSet(); - ObjectWriter singleValueWriter = objectWriter(handle.type()); - return (blockBuilder) -> singleValueWriter.accept(blockBuilder, values.get(random.nextInt(values.size()))); - } - Generator generator = randomValueGenerator(handle); - if (handle.nullProbability() == 0) { - return generator; - } - return (blockBuilder) -> { - if (random.nextDouble() <= handle.nullProbability()) { - blockBuilder.appendNull(); - } - else { - generator.accept(blockBuilder); + SequenceWriter writer = sequenceWriter(handle); + + return new Generator() + { + long currentRowId = rowOffset; + + @Override + public void accept(BlockBuilder blockBuilder) + { + writer.accept(blockBuilder, currentRowId++); } }; } + private SequenceWriter sequenceWriter(FakerColumnHandle handle) + { + Range genericRange = handle.domain().getValues().getRanges().getSpan(); + Type type = handle.type(); + // check every type in order defined in StandardTypes + if (BIGINT.equals(type)) { + LongRange range = LongRange.of(genericRange); + long step = (long) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> BIGINT.writeLong(blockBuilder, Math.min(range.low + rowId * step, range.high - 1)); + } + if (INTEGER.equals(type)) { + IntRange range = IntRange.of(genericRange); + long step = (long) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> INTEGER.writeLong(blockBuilder, Math.min(range.low + rowId * step, range.high - 1)); + } + if (SMALLINT.equals(type)) { + IntRange range = IntRange.of(genericRange, Short.MIN_VALUE, Short.MAX_VALUE); + long step = (long) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> SMALLINT.writeLong(blockBuilder, Math.min(range.low + rowId * step, range.high - 1)); + } + if (TINYINT.equals(type)) { + IntRange range = IntRange.of(genericRange, Byte.MIN_VALUE, Byte.MAX_VALUE); + long step = (long) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> TINYINT.writeLong(blockBuilder, Math.min(range.low + rowId * step, range.high - 1)); + } + // boolean sequences are not supported + if (DATE.equals(type)) { + IntRange range = IntRange.of(genericRange); + long step = (long) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> DATE.writeLong(blockBuilder, Math.min(range.low + roundDiv(rowId * step, NANOSECONDS_PER_DAY), range.high - 1)); + } + if (type instanceof DecimalType decimalType) { + if (decimalType.isShort()) { + ShortDecimalRange range = ShortDecimalRange.of(genericRange, decimalType.getPrecision()); + long step = (long) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> decimalType.writeLong(blockBuilder, Math.min(range.low + rowId * step, range.high - 1)); + } + Int128Range range = Int128Range.of(genericRange); + Int128 step = (Int128) handle.step().getSingleValue(); + Int128 high = Int128Math.subtract(range.high, Int128.valueOf(1)); + return (blockBuilder, rowId) -> { + Int128 nextValue = Int128Math.add(range.low, Int128Math.multiply(Int128.valueOf(rowId), step)); + decimalType.writeObject(blockBuilder, high.compareTo(nextValue) < 0 ? high : nextValue); + }; + } + if (REAL.equals(type)) { + FloatRange range = FloatRange.of(genericRange); + float step = intBitsToFloat(toIntExact((long) handle.step().getSingleValue())); + return (blockBuilder, rowId) -> REAL.writeLong(blockBuilder, floatToRawIntBits(Math.min(range.low + rowId * step, Math.nextDown(range.high)))); + } + if (DOUBLE.equals(type)) { + DoubleRange range = DoubleRange.of(genericRange); + double step = (double) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> DOUBLE.writeDouble(blockBuilder, Math.min(range.low + rowId * step, Math.nextDown(range.high))); + } + // not supported: HYPER_LOG_LOG, QDIGEST, TDIGEST, P4_HYPER_LOG_LOG + if (INTERVAL_DAY_TIME.equals(type) || INTERVAL_YEAR_MONTH.equals(type)) { + IntRange range = IntRange.of(genericRange); + // seconds or months + long step = (long) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> type.writeLong(blockBuilder, Math.min(range.low + rowId * step, range.high - 1)); + } + if (type instanceof TimestampType timestampType) { + long step = (long) handle.step().getSingleValue(); + if (timestampType.isShort()) { + long factor = POWERS_OF_TEN[6 - timestampType.getPrecision()]; + LongRange range = LongRange.of(genericRange, factor); + return (blockBuilder, rowId) -> timestampType.writeLong(blockBuilder, Math.min(range.low + roundDiv(rowId * step, factor * NANOSECONDS_PER_MICROSECOND), range.high - 1) * factor); + } + LongTimestampRange range = LongTimestampRange.of(genericRange, timestampType.getPrecision()); + return (blockBuilder, rowId) -> { + long epochMicros = Math.min(range.low.getEpochMicros() + roundDiv(rowId * step, NANOSECONDS_PER_MICROSECOND), range.high.getEpochMicros()); + // TODO support nanosecond increments + timestampType.writeObject(blockBuilder, new LongTimestamp(epochMicros, 0)); + }; + } + if (type instanceof TimestampWithTimeZoneType tzType) { + long step = (long) handle.step().getSingleValue(); + if (tzType.isShort()) { + ShortTimestampWithTimeZoneRange range = ShortTimestampWithTimeZoneRange.of(genericRange, tzType.getPrecision()); + return (blockBuilder, rowId) -> { + long millis = Math.min(range.low + roundDiv(rowId * step, range.factor * NANOSECONDS_PER_MILLISECOND), range.high - 1) * range.factor; + tzType.writeLong(blockBuilder, packDateTimeWithZone(millis, range.defaultTZ)); + }; + } + LongTimestampWithTimeZoneRange range = LongTimestampWithTimeZoneRange.of(genericRange, tzType.getPrecision()); + return (blockBuilder, rowId) -> { + long millis = Math.min(range.low.getEpochMillis() + roundDiv(rowId * step, NANOSECONDS_PER_MILLISECOND), range.high.getEpochMillis()); + // TODO support microsecond increments + tzType.writeObject(blockBuilder, fromEpochMillisAndFraction(millis, 0, range.defaultTZ)); + }; + } + if (type instanceof TimeType timeType) { + long factor = POWERS_OF_TEN[12 - timeType.getPrecision()]; + LongRange range = LongRange.of(genericRange, factor, 0, PICOSECONDS_PER_DAY); + long step = (long) handle.step().getSingleValue() * PICOSECONDS_PER_NANOSECOND; + return (blockBuilder, rowId) -> timeType.writeLong(blockBuilder, Math.min(range.low + roundDiv(rowId * step, factor), range.high - 1) * factor); + } + if (type instanceof TimeWithTimeZoneType timeType) { + if (timeType.isShort()) { + ShortTimeWithTimeZoneRange range = ShortTimeWithTimeZoneRange.of(genericRange, timeType.getPrecision()); + long step = (long) handle.step().getSingleValue(); + return (blockBuilder, rowId) -> { + long nanos = Math.min(range.low + roundDiv(rowId * step, range.factor), range.high - 1) * range.factor; + timeType.writeLong(blockBuilder, packTimeWithTimeZone(nanos, range.offsetMinutes)); + }; + } + LongTimeWithTimeZoneRange range = LongTimeWithTimeZoneRange.of(genericRange, timeType.getPrecision()); + long step = (long) handle.step().getSingleValue() * PICOSECONDS_PER_NANOSECOND; + return (blockBuilder, rowId) -> { + long picoseconds = Math.min(range.low + roundDiv(rowId * step, range.factor), range.high - 1) * range.factor; + timeType.writeObject(blockBuilder, new LongTimeWithTimeZone(picoseconds, range.offsetMinutes)); + }; + } + // not supported: VARBINARY, VARCHAR, CHAR + // not supported: ROW, ARRAY, MAP, JSON + // not supported: IPADDRESS, GEOMETRY, UUID + + throw new IllegalArgumentException("Unsupported type " + type); + } + private Generator randomValueGenerator(FakerColumnHandle handle) { Range range = handle.domain().getValues().getRanges().getSpan(); @@ -290,11 +421,11 @@ private Generator randomValueGenerator(FakerColumnHandle handle) } if (REAL.equals(type)) { FloatRange floatRange = FloatRange.of(range); - return (blockBuilder) -> REAL.writeLong(blockBuilder, floatToRawIntBits(floatRange.low + (floatRange.high - floatRange.low) * random.nextFloat())); + return (blockBuilder) -> REAL.writeLong(blockBuilder, floatToRawIntBits(floatRange.low == floatRange.high ? floatRange.low : random.nextFloat(floatRange.low, floatRange.high))); } if (DOUBLE.equals(type)) { DoubleRange doubleRange = DoubleRange.of(range); - return (blockBuilder) -> DOUBLE.writeDouble(blockBuilder, doubleRange.low + (doubleRange.high - doubleRange.low) * random.nextDouble()); + return (blockBuilder) -> DOUBLE.writeDouble(blockBuilder, doubleRange.low == doubleRange.high ? doubleRange.low : random.nextDouble(doubleRange.low, doubleRange.high)); } // not supported: HYPER_LOG_LOG, QDIGEST, TDIGEST, P4_HYPER_LOG_LOG if (INTERVAL_DAY_TIME.equals(type)) { @@ -305,19 +436,19 @@ private Generator randomValueGenerator(FakerColumnHandle handle) IntRange intRange = IntRange.of(range); return (blockBuilder) -> INTERVAL_YEAR_MONTH.writeLong(blockBuilder, numberBetween(intRange.low, intRange.high)); } - if (type instanceof TimestampType) { - return timestampGenerator(range, (TimestampType) type); + if (type instanceof TimestampType timestampType) { + return timestampGenerator(range, timestampType); } - if (type instanceof TimestampWithTimeZoneType) { - return timestampWithTimeZoneGenerator(range, (TimestampWithTimeZoneType) type); + if (type instanceof TimestampWithTimeZoneType timestampWithTimeZoneType) { + return timestampWithTimeZoneGenerator(range, timestampWithTimeZoneType); } if (type instanceof TimeType timeType) { long factor = POWERS_OF_TEN[12 - timeType.getPrecision()]; LongRange longRange = LongRange.of(range, factor, 0, PICOSECONDS_PER_DAY); return (blockBuilder) -> timeType.writeLong(blockBuilder, numberBetween(longRange.low, longRange.high) * factor); } - if (type instanceof TimeWithTimeZoneType) { - return timeWithTimeZoneGenerator(range, (TimeWithTimeZoneType) type); + if (type instanceof TimeWithTimeZoneType timeWithTimeZoneType) { + return timeWithTimeZoneGenerator(range, timeWithTimeZoneType); } if (type instanceof VarbinaryType varType) { if (!range.isAll()) { @@ -446,9 +577,7 @@ private ObjectWriter objectWriter(Type type) private Generator decimalGenerator(Range genericRange, DecimalType decimalType) { if (decimalType.isShort()) { - long min = -999999999999999999L / POWERS_OF_TEN[18 - decimalType.getPrecision()]; - long max = 999999999999999999L / POWERS_OF_TEN[18 - decimalType.getPrecision()]; - LongRange range = LongRange.of(genericRange, min, max); + ShortDecimalRange range = ShortDecimalRange.of(genericRange, decimalType.getPrecision()); return (blockBuilder) -> decimalType.writeLong(blockBuilder, numberBetween(range.low, range.high)); } Int128Range range = Int128Range.of(genericRange); @@ -495,15 +624,10 @@ else if (epochMicros == range.high.getEpochMicros()) { private Generator timestampWithTimeZoneGenerator(Range genericRange, TimestampWithTimeZoneType tzType) { if (tzType.isShort()) { - TimeZoneKey defaultTZ = genericRange.getLowValue() - .map(v -> unpackZoneKey((long) v)) - .orElse(genericRange.getHighValue() - .map(v -> unpackZoneKey((long) v)) - .orElse(TimeZoneKey.UTC_KEY)); ShortTimestampWithTimeZoneRange range = ShortTimestampWithTimeZoneRange.of(genericRange, tzType.getPrecision()); return (blockBuilder) -> { long millis = numberBetween(range.low, range.high) * range.factor; - tzType.writeLong(blockBuilder, packDateTimeWithZone(millis, defaultTZ)); + tzType.writeLong(blockBuilder, packDateTimeWithZone(millis, range.defaultTZ)); }; } LongTimestampWithTimeZoneRange range = LongTimestampWithTimeZoneRange.of(genericRange, tzType.getPrecision()); @@ -593,8 +717,8 @@ static FloatRange of(Range range) low = Math.nextUp(low); } float high = range.getHighValue().map(v -> intBitsToFloat(toIntExact((long) v))).orElse(Float.MAX_VALUE); - if (!range.isHighUnbounded() && !range.isHighInclusive()) { - high = Math.nextDown(high); + if (!range.isHighUnbounded() && range.isHighInclusive()) { + high = Math.nextUp(high); } return new FloatRange(low, high); } @@ -609,13 +733,25 @@ static DoubleRange of(Range range) low = Math.nextUp(low); } double high = (double) range.getHighValue().orElse(Double.MAX_VALUE); - if (!range.isHighUnbounded() && !range.isHighInclusive()) { - high = Math.nextDown(high); + if (!range.isHighUnbounded() && range.isHighInclusive()) { + high = Math.nextUp(high); } return new DoubleRange(low, high); } } + private record ShortDecimalRange(long low, long high) + { + static ShortDecimalRange of(Range range, int precision) + { + long defaultMin = -999999999999999999L / POWERS_OF_TEN[18 - precision]; + long defaultMax = 999999999999999999L / POWERS_OF_TEN[18 - precision]; + long low = (long) range.getLowValue().orElse(defaultMin) + (!range.isLowUnbounded() && !range.isLowInclusive() ? 1 : 0); + long high = (long) range.getHighValue().orElse(defaultMax) + (!range.isHighUnbounded() && range.isHighInclusive() ? 1 : 0); + return new ShortDecimalRange(low, high); + } + } + private record Int128Range(Int128 low, Int128 high) { static Int128Range of(Range range) @@ -669,14 +805,19 @@ static LongTimestampRange of(Range range, int precision) } } - private record ShortTimestampWithTimeZoneRange(long low, long high, long factor) + private record ShortTimestampWithTimeZoneRange(long low, long high, long factor, TimeZoneKey defaultTZ) { static ShortTimestampWithTimeZoneRange of(Range range, int precision) { + TimeZoneKey defaultTZ = range.getLowValue() + .map(v -> unpackZoneKey((long) v)) + .orElse(range.getHighValue() + .map(v -> unpackZoneKey((long) v)) + .orElse(TimeZoneKey.UTC_KEY)); long factor = POWERS_OF_TEN[3 - precision]; long low = roundDiv(unpackMillisUtc((long) range.getLowValue().orElse(Long.MIN_VALUE)), factor) + (!range.isLowUnbounded() && !range.isLowInclusive() ? 1 : 0); long high = roundDiv(unpackMillisUtc((long) range.getHighValue().orElse(Long.MAX_VALUE)), factor) + (!range.isHighUnbounded() && range.isHighInclusive() ? 1 : 0); - return new ShortTimestampWithTimeZoneRange(low, high, factor); + return new ShortTimestampWithTimeZoneRange(low, high, factor, defaultTZ); } } @@ -834,6 +975,12 @@ private Generator generateUUID(Range range) }; } + @FunctionalInterface + private interface SequenceWriter + { + void accept(BlockBuilder blockBuilder, long rowId); + } + @FunctionalInterface private interface ObjectWriter { diff --git a/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerQueries.java b/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerQueries.java index f5b537bcb36b..526cf2ce0705 100644 --- a/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerQueries.java +++ b/plugin/trino-faker/src/test/java/io/trino/plugin/faker/TestFakerQueries.java @@ -311,8 +311,8 @@ void testSelectRangeProperties() .add(new TestDataType("rnd_decimal3", "decimal(38,0)", Map.of("min", "99999999999999999999999999999999999999"), "count(distinct rnd_decimal3)", "1")) .add(new TestDataType("rnd_decimal4", "decimal(38,38)", Map.of("min", "0.99999999999999999999999999999999999999"), "count(distinct rnd_decimal4)", "1")) .add(new TestDataType("rnd_decimal5", "decimal(5,2)", Map.of("min", "999.99"), "count(distinct rnd_decimal5)", "1")) - .add(new TestDataType("rnd_real", "real", Map.of("min", "1.4E45"), "count(distinct rnd_real)", "1")) - .add(new TestDataType("rnd_double", "double", Map.of("min", "4.9E324"), "count(distinct rnd_double)", "1")) + .add(new TestDataType("rnd_real", "real", Map.of("min", "3.4028235E38"), "count(distinct rnd_real)", "1")) + .add(new TestDataType("rnd_double", "double", Map.of("min", "1.7976931348623157E308"), "count(distinct rnd_double)", "1")) // interval literals can't represent smallest possible values allowed by the engine, so they're not included here // can't test timestamps because their extreme values cannot be expressed as literals .add(new TestDataType("rnd_time", "time", Map.of("min", "23:59:59.999"), "count(distinct rnd_time)", "1")) @@ -385,6 +385,51 @@ void testSelectValuesProperty() } } + @Test + void testSelectStepProperties() + { + // small step in small ranges that produce only 10 unique values for 1000 rows + List testCases = ImmutableList.builder() + .add(new TestDataType("rnd_bigint", "bigint", Map.of("min", "0", "max", "9", "step", "1"), "count(distinct rnd_bigint)", "10")) + .add(new TestDataType("rnd_integer", "integer", Map.of("min", "0", "max", "9", "step", "1"), "count(distinct rnd_integer)", "10")) + .add(new TestDataType("rnd_smallint", "smallint", Map.of("min", "0", "max", "9", "step", "1"), "count(distinct rnd_smallint)", "10")) + .add(new TestDataType("rnd_tinyint", "tinyint", Map.of("min", "0", "max", "9", "step", "1"), "count(distinct rnd_tinyint)", "10")) + .add(new TestDataType("rnd_date", "date", Map.of("min", "2022-03-01", "max", "2022-03-10", "step", "1d"), "count(distinct rnd_date)", "10")) + .add(new TestDataType("rnd_decimal1", "decimal", Map.of("min", "0", "max", "9", "step", "1"), "count(distinct rnd_decimal1)", "10")) + .add(new TestDataType("rnd_decimal2", "decimal(18,5)", Map.of("min", "0.00000", "max", "0.00009", "step", "0.00001"), "count(distinct rnd_decimal2)", "10")) + .add(new TestDataType("rnd_decimal3", "decimal(38,0)", Map.of("min", "0", "max", "9", "step", "1"), "count(distinct rnd_decimal3)", "10")) + .add(new TestDataType("rnd_decimal4", "decimal(38,38)", Map.of("min", "0.00000000000000000000000000000000000000", "max", "0.00000000000000000000000000000000000009", "step", "0.00000000000000000000000000000000000001"), "count(distinct rnd_decimal4)", "10")) + .add(new TestDataType("rnd_decimal5", "decimal(5,2)", Map.of("min", "0.00", "max", "1.09", "step", "0.01"), "count(distinct rnd_decimal5)", "110")) + .add(new TestDataType("rnd_real", "real", Map.of("min", "0.0", "max", "1.3E-44", "step", "1.4E-45"), "count(distinct rnd_real)", "10")) + .add(new TestDataType("rnd_double", "double", Map.of("min", "0.0", "max", "4.4E-323", "step", "4.9E-324"), "count(distinct rnd_double)", "10")) + .add(new TestDataType("rnd_interval1", "interval day to second", Map.of("min", "0.000", "max", "0.009", "step", "0.001"), "count(distinct rnd_interval1)", "10")) + .add(new TestDataType("rnd_interval2", "interval year to month", Map.of("min", "0", "max", "9", "step", "1"), "count(distinct rnd_interval2)", "10")) + .add(new TestDataType("rnd_timestamp", "timestamp", Map.of("min", "2022-03-21 00:00:00.000", "max", "2022-03-21 00:00:00.009", "step", "1ms"), "count(distinct rnd_timestamp)", "10")) + .add(new TestDataType("rnd_timestamp0", "timestamp(0)", Map.of("min", "2022-03-21 00:00:00", "max", "2022-03-21 00:00:09", "step", "1s"), "count(distinct rnd_timestamp0)", "10")) + .add(new TestDataType("rnd_timestamp6", "timestamp(6)", Map.of("min", "2022-03-21 00:00:00.000000", "max", "2022-03-21 00:00:00.000009", "step", "1us"), "count(distinct rnd_timestamp6)", "10")) + .add(new TestDataType("rnd_timestamp9", "timestamp(9)", Map.of("min", "2022-03-21 00:00:00.000000000", "max", "2022-03-21 00:00:00.000009000", "step", "1us"), "count(distinct rnd_timestamp9)", "10")) + .add(new TestDataType("rnd_timestamptz", "timestamp with time zone", Map.of("min", "2022-03-21 00:00:00.000 +01:00", "max", "2022-03-21 00:00:00.009 +01:00", "step", "1ms"), "count(distinct rnd_timestamptz)", "10")) + .add(new TestDataType("rnd_timestamptz0", "timestamp(0) with time zone", Map.of("min", "2022-03-21 00:00:00 +01:00", "max", "2022-03-21 00:00:09 +01:00", "step", "1s"), "count(distinct rnd_timestamptz0)", "10")) + .add(new TestDataType("rnd_timestamptz6", "timestamp(6) with time zone", Map.of("min", "2022-03-21 00:00:00.000000 +01:00", "max", "2022-03-21 00:00:00.009000 +01:00", "step", "1ms"), "count(distinct rnd_timestamptz6)", "10")) + .add(new TestDataType("rnd_timestamptz9", "timestamp(9) with time zone", Map.of("min", "2022-03-21 00:00:00.000000000 +01:00", "max", "2022-03-21 00:00:00.009000000 +01:00", "step", "1ms"), "count(distinct rnd_timestamptz9)", "10")) + .add(new TestDataType("rnd_time", "time", Map.of("min", "01:02:03.456", "max", "01:02:03.465", "step", "1ms"), "count(distinct rnd_time)", "10")) + .add(new TestDataType("rnd_time0", "time(0)", Map.of("min", "01:02:03", "max", "01:02:12", "step", "1s"), "count(distinct rnd_time0)", "10")) + .add(new TestDataType("rnd_time6", "time(6)", Map.of("min", "01:02:03.000456", "max", "01:02:03.000465", "step", "1us"), "count(distinct rnd_time6)", "10")) + .add(new TestDataType("rnd_time9", "time(9)", Map.of("min", "01:02:03.000000456", "max", "01:02:03.000000465", "step", "1ns"), "count(distinct rnd_time9)", "10")) + .add(new TestDataType("rnd_timetz", "time with time zone", Map.of("min", "01:02:03.456 +01:00", "max", "01:02:03.465 +01:00", "step", "1ms"), "count(distinct rnd_timetz)", "10")) + .add(new TestDataType("rnd_timetz0", "time(0) with time zone", Map.of("min", "01:02:03 +01:00", "max", "01:02:12 +01:00", "step", "1s"), "count(distinct rnd_timetz0)", "10")) + .add(new TestDataType("rnd_timetz6", "time(6) with time zone", Map.of("min", "01:02:03.000456 +01:00", "max", "01:02:03.000465 +01:00", "step", "1us"), "count(distinct rnd_timetz6)", "10")) + .add(new TestDataType("rnd_timetz9", "time(9) with time zone", Map.of("min", "01:02:03.000000456 +01:00", "max", "01:02:03.000000465 +01:00", "step", "1ns"), "count(distinct rnd_timetz9)", "10")) + .add(new TestDataType("rnd_timetz12", "time(12) with time zone", Map.of("min", "01:02:03.000000000456 +01:00", "max", "01:02:03.000000009456 +01:00", "step", "1ns"), "count(distinct rnd_timetz12)", "10")) + .build(); + + for (TestDataType testCase : testCases) { + try (TestTable table = new TestTable(getQueryRunner()::execute, "step_small_" + testCase.name(), "(%s)".formatted(testCase.columnSchema()))) { + assertQuery("SELECT %s FROM %s".formatted(testCase.queryExpression(), table.getName()), "VALUES (%s)".formatted(testCase.expectedValue())); + } + } + } + private record TestDataType(String name, String type, Map properties, String queryExpression, String expectedValue) { public TestDataType(String name, String type, String queryExpression, String expectedValue)