From 532c89d3f4f38bb2d3980578c54ea8c0793236e9 Mon Sep 17 00:00:00 2001 From: Rob Marrowstone Date: Sun, 11 Aug 2024 09:50:54 -0700 Subject: [PATCH 01/18] Implement Native Trino Ion PageSource and FileWriter This is the initial implementation of a Native Trino Ion Hive format. It does not handle the full range of Trino or Ion values, nor the set of coercions that we will likely need to implement. But I think it's a pretty good start. I implemented enough of the scalar types to make the tests pass but there are definitely some edges that need cleaning. I also added the ion-hive-serde as a test dependency so that we can add Ion to TestHiveFileFormats. I chose to avoid building off of the "Line" abstractions. While that could be done, and more closely mimics how the Ion Hive SerDe works, doing so is suboptimal, and for reading is actually more complicated. This is because while Ion is unschema'ed, each value stream has an evolving "encoding context." In Ion 1.0 that is the "Symbol Table" which is a form of dictionary encoding for the field names and other common text values. The other high-level choice was _not_ to use the ion-java-path-extraction library that is used in the Hive SerDe. The main value in that was to allow users to do path navigation in the SerDe config. But using it to go into the Block abstractions without a middle layer (as it uses in Hive SerDe) is complex, and I'm concerned that some data quality issues are effectively silenced. Given Trino's current capability set with nested and complex structures I doubt the value proposition. In fact, using SerDe config for what can be expressed in SQL is actually an anti-pattern. --- lib/trino-hive-formats/pom.xml | 7 + .../io/trino/hive/formats/HiveClassNames.java | 4 + .../io/trino/hive/formats/ion/IonDecoder.java | 30 ++ .../hive/formats/ion/IonDecoderFactory.java | 306 ++++++++++++++++++ .../io/trino/hive/formats/ion/IonEncoder.java | 32 ++ .../hive/formats/ion/IonEncoderFactory.java | 246 ++++++++++++++ .../trino/hive/formats/FormatTestUtils.java | 12 + .../trino/hive/formats/ion/TestIonFormat.java | 263 +++++++++++++++ plugin/trino-hive/pom.xml | 14 + .../java/io/trino/plugin/hive/HiveModule.java | 4 + .../trino/plugin/hive/HiveStorageFormat.java | 10 +- .../trino/plugin/hive/HiveWriterFactory.java | 13 +- .../trino/plugin/hive/ion/IonFileWriter.java | 132 ++++++++ .../plugin/hive/ion/IonFileWriterFactory.java | 110 +++++++ .../trino/plugin/hive/ion/IonPageSource.java | 110 +++++++ .../plugin/hive/ion/IonPageSourceFactory.java | 140 ++++++++ .../plugin/hive/BaseHiveConnectorTest.java | 1 + .../io/trino/plugin/hive/HiveTestUtils.java | 61 ++++ .../plugin/hive/TestHiveFileFormats.java | 20 ++ .../hive/ion/IonPageSourceSmokeTest.java | 199 ++++++++++++ .../product/hive/TestHiveStorageFormats.java | 2 + 21 files changed, 1710 insertions(+), 6 deletions(-) create mode 100644 lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoder.java create mode 100644 lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java create mode 100644 lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoder.java create mode 100644 lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java create mode 100644 lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java create mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java diff --git a/lib/trino-hive-formats/pom.xml b/lib/trino-hive-formats/pom.xml index 44bf90e72f63..c8d6fa745f58 100644 --- a/lib/trino-hive-formats/pom.xml +++ b/lib/trino-hive-formats/pom.xml @@ -17,6 +17,13 @@ + + + com.amazon.ion + ion-java + 1.11.9 + + com.fasterxml.jackson.core jackson-core diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java index 43a357b59bd9..01ccab194f98 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java @@ -27,6 +27,10 @@ public final class HiveClassNames public static final String HUDI_PARQUET_REALTIME_INPUT_FORMAT = "org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat"; public static final String HUDI_INPUT_FORMAT = "com.uber.hoodie.hadoop.HoodieInputFormat"; public static final String HUDI_REALTIME_INPUT_FORMAT = "com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat"; + public static final String ION_SERDE_CLASS = "com.amazon.ionhiveserde.IonHiveSerDe"; + public static final String ION_INPUT_FORMAT = "com.amazon.ionhiveserde.formats.IonInputFormat"; + public static final String ION_OUTPUT_FORMAT = "com.amazon.ionhiveserde.formats.IonOutputFormat"; + public static final String JSON_SERDE_CLASS = "org.apache.hive.hcatalog.data.JsonSerDe"; public static final String LEGACY_JSON_SERDE_CLASS = "org.apache.hadoop.hive.serde2.JsonSerDe"; public static final String OPENX_JSON_SERDE_CLASS = "org.openx.data.jsonserde.JsonSerDe"; diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoder.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoder.java new file mode 100644 index 000000000000..422ba16fb712 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoder.java @@ -0,0 +1,30 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.hive.formats.ion; + +import com.amazon.ion.IonException; +import com.amazon.ion.IonReader; +import io.trino.spi.PageBuilder; + +public interface IonDecoder +{ + /** + * Reads the _current_ top-level-value from the IonReader. + *

+ * Expects that the calling code has called IonReader.next() + * to position the reader at the value to be decoded. + */ + void decode(IonReader reader, PageBuilder builder) + throws IonException; +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java new file mode 100644 index 000000000000..d63bab565e2f --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java @@ -0,0 +1,306 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.hive.formats.ion; + +import com.amazon.ion.IonException; +import com.amazon.ion.IonReader; +import com.amazon.ion.IonType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.slice.Slices; +import io.trino.hive.formats.line.Column; +import io.trino.spi.PageBuilder; +import io.trino.spi.block.ArrayBlockBuilder; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.RowBlockBuilder; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.BooleanType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DateType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.DoubleType; +import io.trino.spi.type.Int128; +import io.trino.spi.type.IntegerType; +import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.RealType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.SmallintType; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TinyintType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarbinaryType; +import io.trino.spi.type.VarcharType; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.time.ZoneId; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.IntFunction; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class IonDecoderFactory +{ + private IonDecoderFactory() {} + + /** + * Builds a decoder for the given columns. + *

+ * The decoder expects to decode the _current_ Ion Value. + * It also expects that the calling code will manage the PageBuilder. + *

+ */ + public static IonDecoder buildDecoder(List columns) + { + return RowDecoder.forFields( + columns.stream() + .map(c -> new RowType.Field(Optional.of(c.name()), c.type())) + .toList()); + } + + private interface BlockDecoder + { + void decode(IonReader reader, BlockBuilder builder); + } + + private static BlockDecoder decoderForType(Type type) + { + return switch (type) { + case TinyintType _ -> wrapDecoder(byteDecoder, IonType.INT); + case SmallintType _ -> wrapDecoder(shortDecoder, IonType.INT); + case IntegerType _ -> wrapDecoder(intDecoder, IonType.INT); + case BigintType _ -> wrapDecoder(longDecoder, IonType.INT); + case RealType _ -> wrapDecoder(realDecoder, IonType.FLOAT); + case DoubleType _ -> wrapDecoder(floatDecoder, IonType.FLOAT); + case BooleanType _ -> wrapDecoder(boolDecoder, IonType.BOOL); + case DateType _ -> wrapDecoder(dateDecoder, IonType.TIMESTAMP); + case TimestampType t -> wrapDecoder(timestampDecoder(t), IonType.TIMESTAMP); + case DecimalType t -> wrapDecoder(decimalDecoder(t), IonType.DECIMAL); + case VarcharType _, CharType _ -> wrapDecoder(stringDecoder, IonType.STRING, IonType.SYMBOL); + case VarbinaryType _ -> wrapDecoder(binaryDecoder, IonType.BLOB, IonType.CLOB); + case RowType r -> wrapDecoder(RowDecoder.forFields(r.getFields()), IonType.STRUCT); + case ArrayType a -> wrapDecoder(new ArrayDecoder(decoderForType(a.getElementType())), IonType.LIST, IonType.SEXP); + default -> throw new IllegalArgumentException(String.format("Unsupported type: %s", type)); + }; + } + + /** + * Wraps decoders for common handling logic. + *

+ * Handles un-typed and correctly typed null values. + * Throws for mistyped values, whether null or not. + * Delegates to Decoder for correctly-typed, non-null values. + *

+ * This code treats all values as nullable. + */ + private static BlockDecoder wrapDecoder(BlockDecoder decoder, IonType... allowedTypes) + { + final Set allowedWithNull = new HashSet<>(Arrays.asList(allowedTypes)); + allowedWithNull.add(IonType.NULL); + + return (reader, builder) -> { + final IonType type = reader.getType(); + if (!allowedWithNull.contains(type)) { + final String expected = allowedWithNull.stream().map(IonType::name).collect(Collectors.joining(", ")); + throw new IonException(String.format("Encountered value with IonType: %s, required one of %s ", type, expected)); + } + if (reader.isNullValue()) { + builder.appendNull(); + } + else { + decoder.decode(reader, builder); + } + }; + } + + /** + * Class is both the Top-Level-Value Decoder and the Row Decoder for nested + * structs. + */ + private record RowDecoder(Map fieldPositions, List fieldDecoders) + implements IonDecoder, BlockDecoder + { + private static RowDecoder forFields(List fields) + { + ImmutableList.Builder decoderBuilder = ImmutableList.builder(); + ImmutableMap.Builder fieldPositionBuilder = ImmutableMap.builder(); + IntStream.range(0, fields.size()) + .forEach(position -> { + RowType.Field field = fields.get(position); + decoderBuilder.add(decoderForType(field.getType())); + fieldPositionBuilder.put(field.getName().get(), position); + }); + + return new RowDecoder(fieldPositionBuilder.buildOrThrow(), decoderBuilder.build()); + } + + @Override + public void decode(IonReader ionReader, PageBuilder pageBuilder) + { + // todo: we could also map an Ion List to a Struct + if (ionReader.getType() != IonType.STRUCT) { + throw new IonException("RowType must be Structs! Encountered: " + ionReader.getType()); + } + if (ionReader.isNullValue()) { + // todo: is this an error or just a null value? + // i think in the hive serde it's a null record. + throw new IonException("Top Level Values must not be null!"); + } + decode(ionReader, pageBuilder::getBlockBuilder); + } + + @Override + public void decode(IonReader ionReader, BlockBuilder blockBuilder) + { + ((RowBlockBuilder) blockBuilder) + .buildEntry(fieldBuilders -> decode(ionReader, fieldBuilders::get)); + } + + // assumes that the reader is positioned on a non-null struct value + private void decode(IonReader ionReader, IntFunction blockSelector) + { + boolean[] encountered = new boolean[fieldDecoders.size()]; + ionReader.stepIn(); + + while (ionReader.next() != null) { + // todo: case insensitivity + final Integer fieldIndex = fieldPositions.get(ionReader.getFieldName()); + if (fieldIndex == null) { + continue; + } + final BlockBuilder blockBuilder = blockSelector.apply(fieldIndex); + if (encountered[fieldIndex]) { + blockBuilder.resetTo(blockBuilder.getPositionCount() - 1); + } + else { + encountered[fieldIndex] = true; + } + fieldDecoders.get(fieldIndex).decode(ionReader, blockBuilder); + } + + for (int i = 0; i < encountered.length; i++) { + if (!encountered[i]) { + blockSelector.apply(i).appendNull(); + } + } + + ionReader.stepOut(); + } + } + + private record ArrayDecoder(BlockDecoder elementDecoder) + implements BlockDecoder + { + @Override + public void decode(IonReader ionReader, BlockBuilder blockBuilder) + { + ((ArrayBlockBuilder) blockBuilder) + .buildEntry(elementBuilder -> { + ionReader.stepIn(); + while (ionReader.next() != null) { + elementDecoder.decode(ionReader, elementBuilder); + } + ionReader.stepOut(); + }); + } + } + + private static BlockDecoder timestampDecoder(TimestampType type) + { + // todo: no attempt is made at handling offsets or lack thereof + if (type.isShort()) { + return (reader, builder) -> { + long micros = reader.timestampValue().getDecimalMillis() + .setScale(type.getPrecision() - 3, RoundingMode.HALF_EVEN) + .movePointRight(3) + .longValue(); + type.writeLong(builder, micros); + }; + } + else { + return (reader, builder) -> { + BigDecimal decimalMicros = reader.timestampValue().getDecimalMillis() + .movePointRight(3); + BigDecimal subMicrosFrac = decimalMicros.remainder(BigDecimal.ONE) + .movePointRight(6); + type.writeObject(builder, new LongTimestamp(decimalMicros.longValue(), subMicrosFrac.intValue())); + }; + } + } + + private static BlockDecoder decimalDecoder(DecimalType type) + { + if (type.isShort()) { + return (reader, builder) -> { + long unscaled = reader.bigDecimalValue() + .setScale(type.getScale(), RoundingMode.UNNECESSARY) + .unscaledValue() + .longValue(); + type.writeLong(builder, unscaled); + }; + } + else { + return (reader, builder) -> { + Int128 unscaled = Int128.valueOf(reader.bigDecimalValue() + .setScale(type.getScale(), RoundingMode.UNNECESSARY) + .unscaledValue()); + type.writeObject(builder, unscaled); + }; + } + } + + private static final BlockDecoder byteDecoder = (ionReader, blockBuilder) -> + TinyintType.TINYINT.writeLong(blockBuilder, ionReader.longValue()); + + private static final BlockDecoder shortDecoder = (ionReader, blockBuilder) -> + SmallintType.SMALLINT.writeLong(blockBuilder, ionReader.longValue()); + + private static final BlockDecoder intDecoder = (ionReader, blockBuilder) -> + IntegerType.INTEGER.writeLong(blockBuilder, ionReader.longValue()); + + private static final BlockDecoder longDecoder = (ionReader, blockBuilder) -> + BigintType.BIGINT.writeLong(blockBuilder, ionReader.longValue()); + + private static final BlockDecoder realDecoder = (ionReader, blockBuilder) -> { + double readValue = ionReader.doubleValue(); + if (readValue == (float) readValue) { + RealType.REAL.writeFloat(blockBuilder, (float) ionReader.doubleValue()); + } + else { + // todo: some kind of "permissive truncate" flag + throw new IllegalArgumentException("Won't truncate double precise float to real!"); + } + }; + + private static final BlockDecoder floatDecoder = (ionReader, blockBuilder) -> + DoubleType.DOUBLE.writeDouble(blockBuilder, ionReader.doubleValue()); + + private static final BlockDecoder stringDecoder = (ionReader, blockBuilder) -> + VarcharType.VARCHAR.writeSlice(blockBuilder, Slices.utf8Slice(ionReader.stringValue())); + + private static final BlockDecoder boolDecoder = (ionReader, blockBuilder) -> + BooleanType.BOOLEAN.writeBoolean(blockBuilder, ionReader.booleanValue()); + + private static final BlockDecoder dateDecoder = (ionReader, blockBuilder) -> + DateType.DATE.writeLong(blockBuilder, ionReader.timestampValue().dateValue().toInstant().atZone(ZoneId.of("UTC")).toLocalDate().toEpochDay()); + + private static final BlockDecoder binaryDecoder = (ionReader, blockBuilder) -> + VarbinaryType.VARBINARY.writeSlice(blockBuilder, Slices.wrappedBuffer(ionReader.newBytes())); +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoder.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoder.java new file mode 100644 index 000000000000..7e7fd899a8f3 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoder.java @@ -0,0 +1,32 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.hive.formats.ion; + +import com.amazon.ion.IonWriter; +import io.trino.spi.Page; + +import java.io.IOException; + +public interface IonEncoder +{ + /** + * Encodes the Page into the IonWriter provided. + *

+ * Will flush() the writer after encoding the page. + * Expects that the calling code is responsible for closing + * the writer after all pages are written. + */ + void encode(IonWriter writer, Page page) + throws IOException; +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java new file mode 100644 index 000000000000..44174d85a6bf --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java @@ -0,0 +1,246 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.hive.formats.ion; + +import com.amazon.ion.IonType; +import com.amazon.ion.IonWriter; +import com.amazon.ion.Timestamp; +import com.google.common.collect.ImmutableList; +import io.trino.hive.formats.line.Column; +import io.trino.spi.Page; +import io.trino.spi.block.ArrayBlock; +import io.trino.spi.block.Block; +import io.trino.spi.block.RowBlock; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.BooleanType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DateType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.DoubleType; +import io.trino.spi.type.Int128; +import io.trino.spi.type.IntegerType; +import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.RealType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.SmallintType; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TinyintType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarbinaryType; +import io.trino.spi.type.VarcharType; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.ZoneId; +import java.util.Date; +import java.util.List; +import java.util.Optional; +import java.util.function.IntFunction; + +public class IonEncoderFactory +{ + private IonEncoderFactory() {} + + public static IonEncoder buildEncoder(List columns) + { + return RowEncoder.forFields(columns.stream() + .map(c -> new RowType.Field(Optional.of(c.name()), c.type())) + .toList()); + } + + private interface BlockEncoder + { + void encode(IonWriter writer, Block block, int position) + throws IOException; + } + + private static BlockEncoder encoderForType(Type type) + { + return switch (type) { + case TinyintType _ -> byteEncoder; + case SmallintType _ -> shortEncoder; + case IntegerType _ -> intEncoder; + case BigintType _ -> longEncoder; + case BooleanType _ -> boolEncoder; + case VarbinaryType _ -> binaryEncoder; + case RealType _ -> realEncoder; + case DoubleType _ -> doubleEncoder; + case VarcharType _, CharType _ -> stringEncoder; + case DecimalType t -> decimalEncoder(t); + case DateType _ -> dateEncoder; + case TimestampType t -> timestampEncoder(t); + case RowType t -> RowEncoder.forFields(t.getFields()); + case ArrayType t -> new ArrayEncoder(wrapEncoder(encoderForType(t.getElementType()))); + default -> throw new IllegalArgumentException(String.format("Unsupported type: %s", type)); + }; + } + + private static BlockEncoder wrapEncoder(BlockEncoder encoder) + { + return (writer, block, position) -> + { + if (block.isNull(position)) { + writer.writeNull(); + } + else { + encoder.encode(writer, block, position); + } + }; + } + + private record RowEncoder(List fieldNames, List fieldEncoders) + implements BlockEncoder, IonEncoder + { + private static RowEncoder forFields(List fields) + { + ImmutableList.Builder fieldNamesBuilder = ImmutableList.builder(); + ImmutableList.Builder fieldEncodersBuilder = ImmutableList.builder(); + + for (RowType.Field field : fields) { + fieldNamesBuilder.add(field.getName().get()); + fieldEncodersBuilder.add(wrapEncoder(encoderForType(field.getType()))); + } + + return new RowEncoder(fieldNamesBuilder.build(), fieldEncodersBuilder.build()); + } + + @Override + public void encode(IonWriter writer, Block block, int position) + throws IOException + { + encodeStruct(writer, ((RowBlock) block)::getFieldBlock, position); + } + + @Override + public void encode(IonWriter writer, Page page) + throws IOException + { + for (int i = 0; i < page.getPositionCount(); i++) { + encodeStruct(writer, page::getBlock, i); + } + // todo: it's probably preferable to decouple ion writer flushes + // from page sizes, but it's convenient for now + writer.flush(); + } + + private void encodeStruct(IonWriter writer, IntFunction blockSelector, int position) + throws IOException + { + writer.stepIn(IonType.STRUCT); + for (int i = 0; i < fieldEncoders.size(); i++) { + // todo: the Hive SerDe omits fields when null by default + writer.setFieldName(fieldNames.get(i)); + fieldEncoders.get(i) + .encode(writer, blockSelector.apply(i), position); + } + writer.stepOut(); + } + } + + private record ArrayEncoder(BlockEncoder elementEncoder) + implements BlockEncoder + { + @Override + public void encode(IonWriter writer, Block block, int position) + throws IOException + { + writer.stepIn(IonType.LIST); + Block elementBlock = ((ArrayBlock) block).getArray(position); + for (int i = 0; i < elementBlock.getPositionCount(); i++) { + elementEncoder.encode(writer, elementBlock, i); + } + writer.stepOut(); + } + } + + private static BlockEncoder timestampEncoder(TimestampType type) + { + if (type.isShort()) { + return (writer, block, position) -> { + long epochMicros = type.getLong(block, position); + BigDecimal decimalMillis = BigDecimal.valueOf(epochMicros) + .movePointLeft(3) + .setScale(type.getPrecision() - 3, RoundingMode.UNNECESSARY); + + writer.writeTimestamp(Timestamp.forMillis(decimalMillis, 0)); + }; + } + else { + return (writer, block, position) -> { + LongTimestamp longTimestamp = (LongTimestamp) type.getObject(block, position); + BigDecimal picosOfMicros = BigDecimal.valueOf(longTimestamp.getPicosOfMicro()) + .movePointLeft(9); + BigDecimal decimalMillis = BigDecimal.valueOf(longTimestamp.getEpochMicros()) + .movePointLeft(3) + .add(picosOfMicros) + .setScale(type.getPrecision() - 3, RoundingMode.UNNECESSARY); + + writer.writeTimestamp(Timestamp.forMillis(decimalMillis, 0)); + }; + } + } + + private static BlockEncoder decimalEncoder(DecimalType type) + { + if (type.isShort()) { + return (writer, block, position) -> { + writer.writeDecimal(BigDecimal.valueOf(type.getLong(block, position), type.getScale())); + }; + } + else { + return (writer, block, position) -> { + writer.writeDecimal(new BigDecimal(((Int128) type.getObject(block, position)).toBigInteger(), type.getScale())); + }; + } + } + + private static final BlockEncoder byteEncoder = (writer, block, position) -> + writer.writeInt(TinyintType.TINYINT.getLong(block, position)); + + private static final BlockEncoder shortEncoder = (writer, block, position) -> + writer.writeInt(SmallintType.SMALLINT.getLong(block, position)); + + private static final BlockEncoder intEncoder = (writer, block, position) -> + writer.writeInt(IntegerType.INTEGER.getInt(block, position)); + + private static final BlockEncoder stringEncoder = (writer, block, position) -> + writer.writeString(VarcharType.VARCHAR.getSlice(block, position).toString(StandardCharsets.UTF_8)); + + private static final BlockEncoder boolEncoder = (writer, block, position) -> + writer.writeBool(BooleanType.BOOLEAN.getBoolean(block, position)); + + private static final BlockEncoder binaryEncoder = (writer, block, position) -> + writer.writeBlob(VarbinaryType.VARBINARY.getSlice(block, position).getBytes()); + + private static final BlockEncoder longEncoder = (writer, block, position) -> + writer.writeInt(BigintType.BIGINT.getLong(block, position)); + + private static final BlockEncoder realEncoder = (writer, block, position) -> + writer.writeFloat(RealType.REAL.getFloat(block, position)); + + private static final BlockEncoder doubleEncoder = (writer, block, position) -> + writer.writeFloat(DoubleType.DOUBLE.getDouble(block, position)); + + private static final BlockEncoder dateEncoder = (writer, block, position) -> + writer.writeTimestamp( + Timestamp.forDateZ( + Date.from( + LocalDate.ofEpochDay(DateType.DATE.getLong(block, position)) + .atStartOfDay(ZoneId.of("UTC")) + .toInstant()))); +} diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java index 8efdf1e816e0..cc1e912d9bb4 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java @@ -483,6 +483,18 @@ public static Page toSingleRowPage(List columns, List expectedValues) return page; } + public static Page toPage(List columns, List... expectedValues) + { + PageBuilder pageBuilder = new PageBuilder(columns.stream().map(Column::type).collect(toImmutableList())); + for (List expectedValue : expectedValues) { + pageBuilder.declarePosition(); + for (int col = 0; col < columns.size(); col++) { + writeTrinoValue(columns.get(col).type(), pageBuilder.getBlockBuilder(col), expectedValue.get(col)); + } + } + return pageBuilder.build(); + } + public static void writeTrinoValue(Type type, BlockBuilder blockBuilder, Object value) { if (value == null) { diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java new file mode 100644 index 000000000000..839e839f145a --- /dev/null +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -0,0 +1,263 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.hive.formats.ion; + +import com.amazon.ion.IonDatagram; +import com.amazon.ion.IonException; +import com.amazon.ion.IonReader; +import com.amazon.ion.IonSystem; +import com.amazon.ion.IonWriter; +import com.amazon.ion.Timestamp; +import com.amazon.ion.system.IonReaderBuilder; +import com.amazon.ion.system.IonSystemBuilder; +import io.trino.hive.formats.line.Column; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.BooleanType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.SqlDecimal; +import io.trino.spi.type.SqlTimestamp; +import io.trino.spi.type.SqlVarbinary; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.VarbinaryType; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static io.trino.hive.formats.FormatTestUtils.assertColumnValuesEquals; +import static io.trino.hive.formats.FormatTestUtils.readTrinoValues; +import static io.trino.hive.formats.FormatTestUtils.toPage; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RowType.field; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestIonFormat +{ + @Test + public void testSuperBasicStruct() + throws IOException + { + assertValues( + RowType.rowType( + field("foo", INTEGER), + field("bar", VARCHAR)), + "{ bar: baz, foo: 31, ignored: true }", + List.of(31, "baz")); + } + + @Test + public void testStructWithNullAndMissingValues() + throws IOException + { + final List listWithNulls = new ArrayList<>(); + listWithNulls.add(null); + listWithNulls.add(null); + + assertValues( + RowType.rowType( + field("foo", INTEGER), + field("bar", VARCHAR)), + "{ bar: null.symbol }", + listWithNulls); + } + + @Test + public void testStructWithDuplicateKeys() + throws IOException + { + // this test is not making a value judgement; capturing the last + // is not necessarily the "right" behavior. the test just + // documents what the behavior is, which is based on the behavior + // of the hive serde, and is consistent with the trino json parser. + assertValues( + RowType.rowType(field("foo", INTEGER)), + "{ foo: 17, foo: 31, foo: 53 } { foo: 67 }", + List.of(53), List.of(67)); + } + + // todo: test for mistyped null and non-null values + + @Test + public void testNestedList() + throws IOException + { + assertValues( + RowType.rowType( + field("primes", new ArrayType(INTEGER))), + "{ primes: [ 17, 31, 51 ] }", + List.of(List.of(17, 31, 51))); + } + + @Test + public void testNestedStruct() + throws IOException + { + assertValues( + RowType.rowType( + field("name", RowType.rowType( + field("first", VARCHAR), + field("last", VARCHAR)))), + "{ name: { first: Woody, last: Guthrie } }", + List.of(List.of("Woody", "Guthrie"))); + } + + @Test + public void testStructInList() + throws IOException + { + assertValues( + RowType.rowType( + field("elements", new ArrayType( + RowType.rowType( + field("foo", INTEGER))))), + "{ elements: [ { foo: 13 }, { foo: 17 } ] }", + // yes, there are three layers of list here: + // top-level struct (row), list of elements (array), then inner struct (row) + List.of( + List.of(List.of(13), List.of(17)))); + } + + @Test + public void testPicoPreciseTimestamp() + throws IOException + { + Timestamp ionTimestamp = Timestamp.forSecond(2067, 8, 9, 11, 22, new BigDecimal("33.445566"), 0); + long epochMicros = ionTimestamp.getDecimalMillis().movePointRight(3).longValue(); + assertValues( + RowType.rowType(field("my_ts", TimestampType.TIMESTAMP_PICOS)), + "{ my_ts: 2067-08-09T11:22:33.445566778899Z }", + List.of(SqlTimestamp.newInstance(12, epochMicros, 778899))); + } + + @Test + public void testOverPreciseTimestamps() + throws IonException + { + // todo: implement + } + + @Test + public void testDecimalPrecisionAndScale() + throws IOException + { + assertValues( + RowType.rowType( + field("amount", DecimalType.createDecimalType(10, 2)), + field("big_amount", DecimalType.createDecimalType(25, 5))), + "{ amount: 1234.00, big_amount: 1234.00000 }" + + "{ amount: 1234d0, big_amount: 1234d0 }" + + "{ amount: 12d2, big_amount: 12d2 }" + + "{ amount: 1234.000, big_amount: 1234.000000 }", + List.of(new SqlDecimal(BigInteger.valueOf(123400), 10, 2), new SqlDecimal(BigInteger.valueOf(123400000), 25, 5)), + List.of(new SqlDecimal(BigInteger.valueOf(123400), 10, 2), new SqlDecimal(BigInteger.valueOf(123400000), 25, 5)), + List.of(new SqlDecimal(BigInteger.valueOf(120000), 10, 2), new SqlDecimal(BigInteger.valueOf(120000000), 25, 5)), + List.of(new SqlDecimal(BigInteger.valueOf(123400), 10, 2), new SqlDecimal(BigInteger.valueOf(123400000), 25, 5))); + } + + @Test + public void testOversizeOrOverpreciseDecimals() + { + // todo: implement + } + + @Test + public void testEncode() + throws IOException + { + List columns = List.of( + new Column("magic_num", INTEGER, 0), + new Column("some_text", VARCHAR, 1), + new Column("is_summer", BooleanType.BOOLEAN, 2), + new Column("byte_clob", VarbinaryType.VARBINARY, 3), + new Column("sequencer", new ArrayType(INTEGER), 4), + new Column("struction", RowType.rowType( + field("foo", INTEGER), + field("bar", VARCHAR)), 5)); + + List row1 = List.of(17, "something", true, new SqlVarbinary(new byte[] {(byte) 0xff}), List.of(1, 2, 3), List.of(51, "baz")); + List row2 = List.of(31, "somebody", false, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), List.of(67, "qux")); + String ionText = """ + { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{ foo:51, bar:"baz"}} + { magic_num:31, some_text:"somebody", is_summer:false, byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{ foo:67, bar:"qux"}} + """; + + Page page = toPage(columns, row1, row2); + assertIonEquivalence(columns, page, ionText); + } + + private void assertValues(RowType rowType, String ionText, List... expected) + throws IOException + { + List fields = rowType.getFields(); + List columns = IntStream.range(0, fields.size()) + .boxed() + .map(i -> { + final RowType.Field field = fields.get(i); + return new Column(field.getName().get(), field.getType(), i); + }) + .toList(); + IonDecoder decoder = IonDecoderFactory.buildDecoder(columns); + PageBuilder pageBuilder = new PageBuilder(expected.length, rowType.getFields().stream().map(RowType.Field::getType).toList()); + + try (IonReader ionReader = IonReaderBuilder.standard().build(ionText)) { + for (int i = 0; i < expected.length; i++) { + assertThat(ionReader.next()).isNotNull(); + pageBuilder.declarePosition(); + decoder.decode(ionReader, pageBuilder); + } + assertThat(ionReader.next()).isNull(); + } + + for (int i = 0; i < expected.length; i++) { + List actual = readTrinoValues(columns, pageBuilder.build(), i); + assertColumnValuesEquals(columns, actual, expected[i]); + } + } + + /** + * Encodes the page as Ion and asserts its equivalence to ionText, per the Ion datamodel. + *
+ * This allows us to make assertions about how the data is encoded that may be equivalent + * in the trino datamodel but distinct per the Ion datamodel. Some examples: + * - absent fields vs null field values + * - Symbol vs String for text values + * - Timestamps with UTC vs unknown offset + */ + private void assertIonEquivalence(List columns, Page page, String ionText) + throws IOException + { + IonSystem system = IonSystemBuilder.standard().build(); + IonDatagram datagram = system.newDatagram(); + IonEncoder encoder = IonEncoderFactory.buildEncoder(columns); + IonWriter ionWriter = system.newWriter(datagram); + encoder.encode(ionWriter, page); + ionWriter.close(); + + IonDatagram expected = system.getLoader().load(ionText); + Assertions.assertEquals(datagram.size(), expected.size()); + for (int i = 0; i < expected.size(); i++) { + // IonValue.equals() is Ion model equivalence. + Assertions.assertEquals(expected.get(i), datagram.get(i)); + } + } +} diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index fdda2afcc447..86d38eb7e9c9 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -14,6 +14,13 @@ Trino - Hive connector + + + com.amazon.ion + ion-java + 1.11.9 + + com.amazonaws aws-java-sdk-core @@ -386,6 +393,13 @@ runtime + + com.amazon.ion + ion-hive3-serde + 1.2.0 + test + + io.airlift http-server diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java index 253cd16e01b7..ce8d8feae5da 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java @@ -26,6 +26,8 @@ import io.trino.plugin.hive.fs.CachingDirectoryLister; import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryListerFactory; +import io.trino.plugin.hive.ion.IonFileWriterFactory; +import io.trino.plugin.hive.ion.IonPageSourceFactory; import io.trino.plugin.hive.line.CsvFileWriterFactory; import io.trino.plugin.hive.line.CsvPageSourceFactory; import io.trino.plugin.hive.line.JsonFileWriterFactory; @@ -121,6 +123,7 @@ public void configure(Binder binder) pageSourceFactoryBinder.addBinding().to(ParquetPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(RcFilePageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(AvroPageSourceFactory.class).in(Scopes.SINGLETON); + pageSourceFactoryBinder.addBinding().to(IonPageSourceFactory.class).in(Scopes.SINGLETON); Multibinder fileWriterFactoryBinder = newSetBinder(binder, HiveFileWriterFactory.class); binder.bind(OrcFileWriterFactory.class).in(Scopes.SINGLETON); @@ -136,6 +139,7 @@ public void configure(Binder binder) fileWriterFactoryBinder.addBinding().to(OrcFileWriterFactory.class).in(Scopes.SINGLETON); fileWriterFactoryBinder.addBinding().to(RcFileFileWriterFactory.class).in(Scopes.SINGLETON); fileWriterFactoryBinder.addBinding().to(AvroFileWriterFactory.class).in(Scopes.SINGLETON); + fileWriterFactoryBinder.addBinding().to(IonFileWriterFactory.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(ParquetReaderConfig.class); configBinder(binder).bindConfig(ParquetWriterConfig.class); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java index b9036479cff0..db3c4a2b5ffa 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java @@ -35,6 +35,9 @@ import static io.trino.hive.formats.HiveClassNames.COLUMNAR_SERDE_CLASS; import static io.trino.hive.formats.HiveClassNames.HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS; import static io.trino.hive.formats.HiveClassNames.HIVE_SEQUENCEFILE_OUTPUT_FORMAT_CLASS; +import static io.trino.hive.formats.HiveClassNames.ION_INPUT_FORMAT; +import static io.trino.hive.formats.HiveClassNames.ION_OUTPUT_FORMAT; +import static io.trino.hive.formats.HiveClassNames.ION_SERDE_CLASS; import static io.trino.hive.formats.HiveClassNames.JSON_SERDE_CLASS; import static io.trino.hive.formats.HiveClassNames.LAZY_BINARY_COLUMNAR_SERDE_CLASS; import static io.trino.hive.formats.HiveClassNames.LAZY_SIMPLE_SERDE_CLASS; @@ -101,7 +104,11 @@ public enum HiveStorageFormat REGEX( REGEX_SERDE_CLASS, TEXT_INPUT_FORMAT_CLASS, - HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS); + HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS), + ION( + ION_SERDE_CLASS, + ION_INPUT_FORMAT, + ION_OUTPUT_FORMAT); private final String serde; private final String inputFormat; @@ -135,6 +142,7 @@ public boolean isSplittable(String path) return switch (this) { case ORC, PARQUET, AVRO, RCBINARY, RCTEXT, SEQUENCEFILE -> true; case JSON, OPENX_JSON, TEXTFILE, CSV, REGEX -> CompressionKind.forFile(path).isEmpty(); + case ION -> false; }; } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java index 1cdc3935d84d..15e722e71425 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java @@ -64,6 +64,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.MoreCollectors.onlyElement; import static io.trino.hive.formats.HiveClassNames.HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS; +import static io.trino.hive.formats.HiveClassNames.ION_OUTPUT_FORMAT; import static io.trino.metastore.AcidOperation.CREATE_TABLE; import static io.trino.metastore.Partitions.makePartName; import static io.trino.plugin.hive.HiveCompressionCodecs.selectCompressionCodec; @@ -674,11 +675,13 @@ public static int getBucketFromFileName(String fileName) public static String getFileExtension(HiveCompressionCodec compression, StorageFormat format) { - // text format files must have the correct extension when compressed - return compression.getHiveCompressionKind() - .filter(_ -> format.getOutputFormat().equals(HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS)) - .map(CompressionKind::getFileExtension) - .orElse(""); + // text format files and ion format files must have the correct extension when compressed + return switch (format.getOutputFormat()) { + case ION_OUTPUT_FORMAT, HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS -> compression.getHiveCompressionKind() + .map(CompressionKind::getFileExtension) + .orElse(""); + default -> ""; + }; } @VisibleForTesting diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java new file mode 100644 index 000000000000..7191f6cbe5f9 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java @@ -0,0 +1,132 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.ion; + +import com.amazon.ion.IonWriter; +import com.amazon.ion.system.IonTextWriterBuilder; +import com.google.common.io.CountingOutputStream; +import io.trino.hive.formats.compression.CompressionKind; +import io.trino.hive.formats.ion.IonEncoder; +import io.trino.hive.formats.ion.IonEncoderFactory; +import io.trino.hive.formats.line.Column; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.plugin.hive.FileWriter; +import io.trino.spi.Page; +import io.trino.spi.TrinoException; +import io.trino.spi.type.TypeManager; + +import java.io.Closeable; +import java.io.IOException; +import java.io.OutputStream; +import java.util.List; +import java.util.Optional; +import java.util.function.LongSupplier; + +import static io.trino.plugin.hive.HiveErrorCode.HIVE_WRITER_CLOSE_ERROR; + +public class IonFileWriter + implements FileWriter +{ + private final AggregatedMemoryContext outputStreamMemoryContext; + private final Closeable rollbackAction; + private final IonEncoder pageEncoder; + private final IonWriter writer; + private final OutputStream outputStream; + private final LongSupplier bytesWritten; + + public IonFileWriter( + OutputStream outputStream, + AggregatedMemoryContext outputStreamMemoryContext, + Closeable rollbackAction, + TypeManager typeManager, + Optional compressionKind, + List columns) + throws IOException + { + this.outputStreamMemoryContext = outputStreamMemoryContext; + this.rollbackAction = rollbackAction; + this.pageEncoder = IonEncoderFactory.buildEncoder(columns); + CountingOutputStream countingOutputStream = new CountingOutputStream(outputStream); + this.bytesWritten = countingOutputStream::getCount; + if (compressionKind.isPresent()) { + this.outputStream = compressionKind.get().createCodec() + .createStreamCompressor(countingOutputStream); + } + else { + this.outputStream = countingOutputStream; + } + + this.writer = IonTextWriterBuilder + .minimal() + .build(this.outputStream); + } + + @Override + public long getWrittenBytes() + { + return bytesWritten.getAsLong(); + } + + @Override + public long getMemoryUsage() + { + return outputStreamMemoryContext.getBytes(); + } + + @Override + public Closeable commit() + { + try { + writer.close(); + } + catch (Exception e) { + try { + rollbackAction.close(); + } + catch (Exception _) { + // ignore + } + throw new TrinoException(HIVE_WRITER_CLOSE_ERROR, "Error committing write to Hive", e); + } + return rollbackAction; + } + + @Override + public void rollback() + { + try (rollbackAction) { + writer.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public long getValidationCpuNanos() + { + return 0; + } + + @Override + public void appendRows(Page page) + { + try { + pageEncoder.encode(writer, page); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java new file mode 100644 index 000000000000..afec4d3eda49 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java @@ -0,0 +1,110 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.ion; + +import com.google.inject.Inject; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoOutputFile; +import io.trino.hive.formats.line.Column; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.metastore.StorageFormat; +import io.trino.plugin.hive.FileWriter; +import io.trino.plugin.hive.HiveCompressionCodec; +import io.trino.plugin.hive.HiveFileWriterFactory; +import io.trino.plugin.hive.WriterKind; +import io.trino.plugin.hive.acid.AcidTransaction; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; + +import java.io.Closeable; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.stream.IntStream; + +import static io.trino.hive.formats.HiveClassNames.ION_OUTPUT_FORMAT; +import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_WRITER_OPEN_ERROR; +import static io.trino.plugin.hive.HiveSessionProperties.getTimestampPrecision; +import static io.trino.plugin.hive.util.HiveTypeUtil.getType; +import static io.trino.plugin.hive.util.HiveUtil.getColumnNames; +import static io.trino.plugin.hive.util.HiveUtil.getColumnTypes; + +public class IonFileWriterFactory + implements HiveFileWriterFactory +{ + private final TrinoFileSystemFactory fileSystemFactory; + private final TypeManager typeManager; + + @Inject + public IonFileWriterFactory( + TrinoFileSystemFactory fileSystemFactory, + TypeManager typeManager) + { + this.fileSystemFactory = fileSystemFactory; + this.typeManager = typeManager; + } + + @Override + public Optional createFileWriter( + Location location, + List inputColumnNames, + StorageFormat storageFormat, + HiveCompressionCodec compressionCodec, + Map schema, + ConnectorSession session, + OptionalInt bucketNumber, + AcidTransaction transaction, + boolean useAcidSchema, + WriterKind writerKind) + { + if (!ION_OUTPUT_FORMAT.equals(storageFormat.getOutputFormat())) { + return Optional.empty(); + } + try { + TrinoFileSystem fileSystem = fileSystemFactory.create(session); + TrinoOutputFile outputFile = fileSystem.newOutputFile(location); + AggregatedMemoryContext outputStreamMemoryContext = newSimpleAggregatedMemoryContext(); + + Closeable rollbackAction = () -> fileSystem.deleteFile(location); + + // we take the column names from the schema, not what was input + // this is what the LineWriterFactory does, I don't understand why + List fileColumnNames = getColumnNames(schema); + List fileColumnTypes = getColumnTypes(schema).stream() + .map(hiveType -> getType(hiveType, typeManager, getTimestampPrecision(session))) + .toList(); + + List columns = IntStream.range(0, fileColumnNames.size()) + .mapToObj(ordinal -> new Column(fileColumnNames.get(ordinal), fileColumnTypes.get(ordinal), ordinal)) + .toList(); + + return Optional.of(new IonFileWriter( + outputFile.create(outputStreamMemoryContext), + outputStreamMemoryContext, + rollbackAction, + typeManager, + compressionCodec.getHiveCompressionKind(), + columns)); + } + catch (Exception e) { + throw new TrinoException(HIVE_WRITER_OPEN_ERROR, "Error creating Ion Output", e); + } + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java new file mode 100644 index 000000000000..d20a5cc887ff --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java @@ -0,0 +1,110 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.ion; + +import com.amazon.ion.IonReader; +import com.amazon.ion.IonType; +import io.trino.hive.formats.ion.IonDecoder; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.connector.ConnectorPageSource; + +import java.io.IOException; +import java.util.OptionalLong; +import java.util.function.LongSupplier; + +public class IonPageSource + implements ConnectorPageSource +{ + private final IonReader ionReader; + private final PageBuilder pageBuilder; + private final IonDecoder decoder; + private final LongSupplier counter; + private int completedPositions; + private boolean finished; + + public IonPageSource(IonReader ionReader, LongSupplier counter, IonDecoder decoder, PageBuilder pageBuilder) + { + this.ionReader = ionReader; + this.decoder = decoder; + this.pageBuilder = pageBuilder; + this.counter = counter; + this.completedPositions = 0; + } + + @Override + public long getCompletedBytes() + { + return counter.getAsLong(); + } + + @Override + public OptionalLong getCompletedPositions() + { + return OptionalLong.of(completedPositions); + } + + @Override + public long getReadTimeNanos() + { + return 0; + } + + @Override + public boolean isFinished() + { + return finished; + } + + @Override + public Page getNextPage() + { + while (!pageBuilder.isFull()) { + if (!readNextValue()) { + finished = true; + break; + } + } + + Page page = pageBuilder.build(); + completedPositions += page.getPositionCount(); + pageBuilder.reset(); + return page; + } + + @Override + public long getMemoryUsage() + { + return 4096; + } + + @Override + public void close() + throws IOException + { + ionReader.close(); + } + + private boolean readNextValue() + { + final IonType type = ionReader.next(); + if (type == null) { + return false; + } + + pageBuilder.declarePosition(); + decoder.decode(ionReader, pageBuilder); + return true; + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java new file mode 100644 index 000000000000..79d4d8fb6fb1 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java @@ -0,0 +1,140 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.ion; + +import com.amazon.ion.IonReader; +import com.amazon.ion.system.IonReaderBuilder; +import com.google.common.io.CountingInputStream; +import com.google.inject.Inject; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoInputFile; +import io.trino.hive.formats.compression.Codec; +import io.trino.hive.formats.compression.CompressionKind; +import io.trino.hive.formats.ion.IonDecoder; +import io.trino.hive.formats.ion.IonDecoderFactory; +import io.trino.hive.formats.line.Column; +import io.trino.plugin.hive.AcidInfo; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePageSourceFactory; +import io.trino.plugin.hive.ReaderColumns; +import io.trino.plugin.hive.ReaderPageSource; +import io.trino.plugin.hive.Schema; +import io.trino.plugin.hive.acid.AcidTransaction; +import io.trino.spi.PageBuilder; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.EmptyPageSource; +import io.trino.spi.predicate.TupleDomain; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.Optional; +import java.util.OptionalInt; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.hive.formats.HiveClassNames.ION_SERDE_CLASS; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT; +import static io.trino.plugin.hive.HivePageSourceProvider.projectBaseColumns; +import static io.trino.plugin.hive.ReaderPageSource.noProjectionAdaptation; +import static io.trino.plugin.hive.util.HiveUtil.splitError; + +public class IonPageSourceFactory + implements HivePageSourceFactory +{ + private final TrinoFileSystemFactory trinoFileSystemFactory; + + @Inject + public IonPageSourceFactory(TrinoFileSystemFactory trinoFileSystemFactory) + { + this.trinoFileSystemFactory = trinoFileSystemFactory; + } + + @Override + public Optional createPageSource( + ConnectorSession session, + Location path, + long start, + long length, + long estimatedFileSize, + long lastModifiedTime, + Schema schema, + List columns, + TupleDomain effectivePredicate, + Optional acidInfo, + OptionalInt bucketNumber, + boolean originalFile, + AcidTransaction transaction) + { + if (!ION_SERDE_CLASS.equals(schema.serializationLibraryName())) { + return Optional.empty(); + } + checkArgument(acidInfo.isEmpty(), "Acid is not supported for Ion files"); + + // Skip empty inputs + if (length == 0) { + return Optional.of(noProjectionAdaptation(new EmptyPageSource())); + } + + if (start != 0) { + throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, "Split start must be 0 for Ion files"); + } + + List projectedReaderColumns = columns; + Optional readerProjections = projectBaseColumns(columns); + + if (readerProjections.isPresent()) { + projectedReaderColumns = readerProjections.get().get().stream() + .map(HiveColumnHandle.class::cast) + .collect(toImmutableList()); + } + + TrinoFileSystem trinoFileSystem = trinoFileSystemFactory.create(session); + TrinoInputFile inputFile = trinoFileSystem.newInputFile(path, estimatedFileSize); + + // todo: optimization for small files that should just be read into memory + try { + Optional codec = CompressionKind.forFile(inputFile.location().fileName()) + .map(CompressionKind::createCodec); + CountingInputStream countingInputStream = new CountingInputStream(inputFile.newStream()); + InputStream inputStream; + if (codec.isPresent()) { + inputStream = codec.get().createStreamDecompressor(countingInputStream); + } + else { + inputStream = countingInputStream; + } + + IonReader ionReader = IonReaderBuilder + .standard() + .build(inputStream); + PageBuilder pageBuilder = new PageBuilder(projectedReaderColumns.stream() + .map(HiveColumnHandle::getType) + .toList()); + List decoderColumns = projectedReaderColumns.stream() + .map(hc -> new Column(hc.getName(), hc.getType(), hc.getBaseHiveColumnIndex())) + .toList(); + IonDecoder decoder = IonDecoderFactory.buildDecoder(decoderColumns); + IonPageSource pageSource = new IonPageSource(ionReader, countingInputStream::getCount, decoder, pageBuilder); + + return Optional.of(new ReaderPageSource(pageSource, readerProjections)); + } + catch (IOException e) { + throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, splitError(e, path, start, length), e); + } + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 765e7625827c..bb4f51595cb6 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -5685,6 +5685,7 @@ private boolean isMappingByName(HiveStorageFormat format) case PARQUET -> true; case AVRO -> true; case JSON -> true; + case ION -> true; case ORC -> false; case RCBINARY -> false; case RCTEXT -> false; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java index 1d74a78334ed..70aa296cc498 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java @@ -36,6 +36,8 @@ import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.hive.avro.AvroFileWriterFactory; import io.trino.plugin.hive.avro.AvroPageSourceFactory; +import io.trino.plugin.hive.ion.IonFileWriterFactory; +import io.trino.plugin.hive.ion.IonPageSourceFactory; import io.trino.plugin.hive.line.CsvFileWriterFactory; import io.trino.plugin.hive.line.CsvPageSourceFactory; import io.trino.plugin.hive.line.JsonFileWriterFactory; @@ -57,6 +59,7 @@ import io.trino.plugin.hive.parquet.ParquetReaderConfig; import io.trino.plugin.hive.parquet.ParquetWriterConfig; import io.trino.plugin.hive.rcfile.RcFilePageSourceFactory; +import io.trino.plugin.hive.util.HiveTypeTranslator; import io.trino.spi.PageSorter; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; @@ -189,6 +192,7 @@ public static Set getDefaultHivePageSourceFactories(Trino .add(new RcFilePageSourceFactory(fileSystemFactory, hiveConfig)) .add(new OrcPageSourceFactory(new OrcReaderConfig(), fileSystemFactory, stats, hiveConfig)) .add(new ParquetPageSourceFactory(fileSystemFactory, stats, new ParquetReaderConfig(), hiveConfig)) + .add(new IonPageSourceFactory(fileSystemFactory)) .build(); } @@ -211,6 +215,7 @@ public static Set getDefaultHiveFileWriterFactories(HiveC .add(new RcFileFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, nodeVersion, hiveConfig)) .add(new OrcFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, nodeVersion, new FileFormatDataSourceStats(), new OrcWriterConfig())) .add(new ParquetFileWriterFactory(fileSystemFactory, nodeVersion, TESTING_TYPE_MANAGER, hiveConfig, new FileFormatDataSourceStats())) + .add(new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) .build(); } @@ -318,6 +323,62 @@ public static Object toNativeContainerValue(Type type, Object hiveValue) throw new IllegalArgumentException("Unsupported type: " + type); } + public static HiveColumnHandle toHiveBaseColumnHandle(String name, Type type, int ordinal) + { + return new HiveColumnHandle( + name, + ordinal, + HiveTypeTranslator.toHiveType(type), + type, + Optional.empty(), + HiveColumnHandle.ColumnType.REGULAR, + Optional.empty()); + } + + public static HiveColumnHandle projectedColumn(HiveColumnHandle baseColumn, String... path) + { + if (path.length == 0) { + throw new IllegalArgumentException("path must have at least one element"); + } + + final Type baseType = baseColumn.getBaseType(); + Type type = baseType; + ImmutableList.Builder derefBuilder = ImmutableList.builder(); + + for (String fieldName : path) { + if (type instanceof RowType rowType) { + List fields = rowType.getFields(); + type = null; + for (int pos = 0; pos < fields.size(); pos++) { + if (fields.get(pos).getName().get().equals(fieldName)) { + derefBuilder.add(pos); + type = fields.get(pos).getType(); + break; + } + } + if (type == null) { + throw new IllegalArgumentException(String.format("could not find field named: %s!", fieldName)); + } + } + else { + throw new IllegalArgumentException("cannot step into non-RowType!"); + } + } + + return new HiveColumnHandle( + baseColumn.getBaseColumnName(), + baseColumn.getBaseHiveColumnIndex(), + HiveTypeTranslator.toHiveType(baseType), + baseType, + Optional.of(new HiveColumnProjectionInfo( + derefBuilder.build(), + ImmutableList.copyOf(path), + HiveTypeTranslator.toHiveType(type), + type)), + baseColumn.getColumnType(), + baseColumn.getComment()); + } + private static UUID uuidFromBytes(byte[] bytes) { ByteBuffer buffer = ByteBuffer.wrap(bytes); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index d4efe5fe6fc3..2a07fc5b8885 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -32,6 +32,8 @@ import io.trino.plugin.base.type.DecodedTimestamp; import io.trino.plugin.hive.avro.AvroFileWriterFactory; import io.trino.plugin.hive.avro.AvroPageSourceFactory; +import io.trino.plugin.hive.ion.IonFileWriterFactory; +import io.trino.plugin.hive.ion.IonPageSourceFactory; import io.trino.plugin.hive.line.CsvFileWriterFactory; import io.trino.plugin.hive.line.CsvPageSourceFactory; import io.trino.plugin.hive.line.JsonFileWriterFactory; @@ -141,6 +143,7 @@ import static io.trino.plugin.hive.HivePageSourceProvider.ColumnMapping.buildColumnMappings; import static io.trino.plugin.hive.HiveStorageFormat.AVRO; import static io.trino.plugin.hive.HiveStorageFormat.CSV; +import static io.trino.plugin.hive.HiveStorageFormat.ION; import static io.trino.plugin.hive.HiveStorageFormat.JSON; import static io.trino.plugin.hive.HiveStorageFormat.OPENX_JSON; import static io.trino.plugin.hive.HiveStorageFormat.ORC; @@ -365,6 +368,23 @@ public void testOpenXJson(int rowCount, long fileSizePadding) .isReadableByPageSource(fileSystemFactory -> new OpenXJsonPageSourceFactory(fileSystemFactory, new HiveConfig())); } + @Test(dataProvider = "validRowAndFileSizePadding") + public void testIon(int rowCount, long fileSizePadding) + throws Exception + { + List testColumns = TEST_COLUMNS.stream() + // todo: add support for maps to trino impl + .filter(tc -> !(tc.type instanceof MapType)) + .collect(toList()); + + assertThatFileFormat(ION) + .withColumns(testColumns) + .withRowsCount(rowCount) + .withFileSizePadding(fileSizePadding) + .withFileWriterFactory(fileSystemFactory -> new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) + .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory)); + } + @Test(dataProvider = "validRowAndFileSizePadding") public void testRcTextPageSource(int rowCount, long fileSizePadding) throws Exception diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java new file mode 100644 index 000000000000..70bbc44ef3e5 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -0,0 +1,199 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.ion; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoOutputFile; +import io.trino.filesystem.memory.MemoryFileSystemFactory; +import io.trino.metastore.HiveType; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HiveConfig; +import io.trino.plugin.hive.HivePageSourceProvider; +import io.trino.plugin.hive.Schema; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.RowType; +import io.trino.testing.MaterializedResult; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.stream.Collectors; + +import static io.trino.plugin.hive.HivePageSourceProvider.ColumnMapping.buildColumnMappings; +import static io.trino.plugin.hive.HiveStorageFormat.ION; +import static io.trino.plugin.hive.HiveTestUtils.getHiveSession; +import static io.trino.plugin.hive.HiveTestUtils.projectedColumn; +import static io.trino.plugin.hive.HiveTestUtils.toHiveBaseColumnHandle; +import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; +import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; +import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RowType.field; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; + +/** + * Most basic test to reflect PageSource-fu is wired up correctly. + */ +public class IonPageSourceSmokeTest +{ + @Test + public void testReadTwoValues() + throws IOException + { + List tableColumns = List.of( + toHiveBaseColumnHandle("foo", INTEGER, 0), + toHiveBaseColumnHandle("bar", VARCHAR, 1)); + + assertRowCount( + tableColumns, + tableColumns, + "{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }", + 2); + } + + @Test + public void testReadArray() + throws IOException + { + List tablesColumns = List.of( + toHiveBaseColumnHandle("my_seq", new ArrayType(BOOLEAN), 0)); + + assertRowCount( + tablesColumns, + tablesColumns, + "{ my_seq: ( true false ) } { my_seq: [false, false, true] }", + 2); + } + + @Test + public void testProjectedColumn() + throws IOException + { + final RowType spamType = RowType.rowType(field("nested_to_prune", INTEGER), field("eggs", INTEGER)); + List tableColumns = List.of( + toHiveBaseColumnHandle("spam", spamType, 0), + toHiveBaseColumnHandle("ham", BOOLEAN, 1)); + List projectedColumns = List.of( + projectedColumn(tableColumns.get(0), "eggs")); + + assertRowCount( + tableColumns, + projectedColumns, + // the data below reflects that "ham" is not decoded, that column is pruned + // "nested_to_prune" is decoded, however, because nested fields are not pruned, yet. + // so this test will fail if you change that to something other than an int + "{ spam: { nested_to_prune: 31, eggs: 12 }, ham: exploding }", + 1); + } + + private void assertRowCount(List tableColumns, List projectedColumns, String ionText, int rowCount) + throws IOException + { + TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); + Location location = Location.of("memory:///test.ion"); + + final ConnectorSession session = getHiveSession(new HiveConfig()); + + writeIonTextFile(ionText, location, fileSystemFactory.create(session)); + + try (ConnectorPageSource pageSource = createPageSource(fileSystemFactory, location, tableColumns, projectedColumns, session)) { + final MaterializedResult result = MaterializedResult.materializeSourceDataStream(session, pageSource, projectedColumns.stream().map(HiveColumnHandle::getType).toList()); + Assertions.assertEquals(rowCount, result.getRowCount()); + } + } + + private int writeIonTextFile(String ionText, Location location, TrinoFileSystem fileSystem) + throws IOException + { + final TrinoOutputFile outputFile = fileSystem.newOutputFile(location); + int written = 0; + try (OutputStream outputStream = outputFile.create()) { + byte[] bytes = ionText.getBytes(StandardCharsets.UTF_8); + outputStream.write(bytes); + outputStream.flush(); + written = bytes.length; + } + return written; + } + + /** + * todo: this is very similar to what's in TestOrcPredicates, factor out. + */ + private static ConnectorPageSource createPageSource( + TrinoFileSystemFactory fileSystemFactory, + Location location, + List tableColumns, + List projectedColumns, + ConnectorSession session) + throws IOException + { + IonPageSourceFactory factory = new IonPageSourceFactory(fileSystemFactory); + + long length = fileSystemFactory.create(session).newInputFile(location).length(); + long nowMillis = Instant.now().toEpochMilli(); + + List columnMappings = buildColumnMappings( + "", + ImmutableList.of(), + projectedColumns, + ImmutableList.of(), + ImmutableMap.of(), + location.toString(), + OptionalInt.empty(), + length, + nowMillis); + + final Map tableProperties = ImmutableMap.builder() + .put(LIST_COLUMNS, tableColumns.stream().map(HiveColumnHandle::getName).collect(Collectors.joining(","))) + .put(LIST_COLUMN_TYPES, tableColumns.stream().map(HiveColumnHandle::getHiveType).map(HiveType::toString).collect(Collectors.joining(","))) + .buildOrThrow(); + + return HivePageSourceProvider.createHivePageSource( + ImmutableSet.of(factory), + session, + location, + OptionalInt.empty(), + 0, + length, + length, + nowMillis, + new Schema(ION.getSerde(), false, tableProperties), + TupleDomain.all(), + TESTING_TYPE_MANAGER, + Optional.empty(), + Optional.empty(), + Optional.empty(), + false, + NO_ACID_TRANSACTION, + columnMappings) + .orElseThrow(); + } +} diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java index fdb07b5d4007..6d1ced716263 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java @@ -307,6 +307,8 @@ public void verifyDataProviderCompleteness() .filter(format -> !"JSON".equals(format)) // OPENX is not supported in Hive by default .filter(format -> !"OPENX_JSON".equals(format)) + // Ion is not supported in Hive by default + .filter(format -> !"ION".equals(format)) .collect(toImmutableSet()); assertThat(ImmutableSet.copyOf(storageFormats())) From f8d61841c25603ccdc1b9f3f03456dfe58eb49c9 Mon Sep 17 00:00:00 2001 From: Khushboo Desai Date: Mon, 11 Nov 2024 18:28:58 -0800 Subject: [PATCH 02/18] Adds support for case-insensitivity for field names Converts the field name into lower case for both Decoder and Encoder to ensure case-insensitivity for field names/keys --- .../hive/formats/ion/IonDecoderFactory.java | 5 ++-- .../hive/formats/ion/IonEncoderFactory.java | 5 ++-- .../trino/hive/formats/ion/TestIonFormat.java | 26 +++++++++++++++++++ 3 files changed, 32 insertions(+), 4 deletions(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java index d63bab565e2f..0d03efb0d0e3 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java @@ -49,6 +49,7 @@ import java.util.Arrays; import java.util.HashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -145,7 +146,7 @@ private static RowDecoder forFields(List fields) .forEach(position -> { RowType.Field field = fields.get(position); decoderBuilder.add(decoderForType(field.getType())); - fieldPositionBuilder.put(field.getName().get(), position); + fieldPositionBuilder.put(field.getName().get().toLowerCase(Locale.ROOT), position); }); return new RowDecoder(fieldPositionBuilder.buildOrThrow(), decoderBuilder.build()); @@ -181,7 +182,7 @@ private void decode(IonReader ionReader, IntFunction blockSelector while (ionReader.next() != null) { // todo: case insensitivity - final Integer fieldIndex = fieldPositions.get(ionReader.getFieldName()); + final Integer fieldIndex = fieldPositions.get(ionReader.getFieldName().toLowerCase(Locale.ROOT)); if (fieldIndex == null) { continue; } diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java index 44174d85a6bf..f62257a5c1e2 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java @@ -49,6 +49,7 @@ import java.time.ZoneId; import java.util.Date; import java.util.List; +import java.util.Locale; import java.util.Optional; import java.util.function.IntFunction; @@ -59,7 +60,7 @@ private IonEncoderFactory() {} public static IonEncoder buildEncoder(List columns) { return RowEncoder.forFields(columns.stream() - .map(c -> new RowType.Field(Optional.of(c.name()), c.type())) + .map(c -> new RowType.Field(Optional.of(c.name().toLowerCase(Locale.ROOT)), c.type())) .toList()); } @@ -112,7 +113,7 @@ private static RowEncoder forFields(List fields) ImmutableList.Builder fieldEncodersBuilder = ImmutableList.builder(); for (RowType.Field field : fields) { - fieldNamesBuilder.add(field.getName().get()); + fieldNamesBuilder.add(field.getName().get().toLowerCase(Locale.ROOT)); fieldEncodersBuilder.add(wrapEncoder(encoderForType(field.getType()))); } diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index 839e839f145a..455c9de8241e 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -65,6 +65,32 @@ public void testSuperBasicStruct() List.of(31, "baz")); } + @Test + public void testCaseInsensitivityOfKeys() + throws IOException + { + assertValues( + RowType.rowType( + field("Foo", INTEGER), + field("BAR", VARCHAR)), + "{ Bar: baz, foo: 31 }", + List.of(31, "baz")); + } + + @Test + public void testCaseInsensitivityOfDuplicateKeys() + throws IOException + { + // this test asserts that duplicate key behavior works as expected(i.e. capturing the last value), + // for duplicate keys with different casing. + assertValues( + RowType.rowType( + field("Foo", INTEGER), + field("BAR", VARCHAR)), + "{ bar: baz, Foo: 31, foo: 5 }", + List.of(5, "baz")); + } + @Test public void testStructWithNullAndMissingValues() throws IOException From 98f9cc904ce7f87ec4c6448f63021a631ba4c8a2 Mon Sep 17 00:00:00 2001 From: Khushboo <68757952+desaikd@users.noreply.github.com> Date: Mon, 18 Nov 2024 10:15:02 -0800 Subject: [PATCH 03/18] Adds `HiveConfig` fetaure flag for Ion native trino integration * Adds `ion.nativetrino` hvie config feature flag * Adds changes for Ion page source to use the feature flag * Adds test changes for feature flag --- .../java/io/trino/plugin/hive/HiveConfig.java | 19 +++ .../plugin/hive/ion/IonPageSourceFactory.java | 11 +- .../plugin/hive/BaseHiveConnectorTest.java | 5 + .../io/trino/plugin/hive/HiveTestUtils.java | 5 +- .../io/trino/plugin/hive/TestHiveConfig.java | 3 + .../plugin/hive/TestHiveFileFormats.java | 7 +- .../hive/ion/IonPageSourceSmokeTest.java | 124 ++++++++++++++---- 7 files changed, 146 insertions(+), 28 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java index 5b392f4bf14f..5ea6dedd8b02 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java @@ -111,6 +111,12 @@ public class HiveConfig private String orcLegacyTimeZone = TimeZone.getDefault().getID(); + // This is a feature flag for Ion native trino integration. + // Default value is false and requires explicitly setting it to true to enable Ion native trino integration. + // TODO: This should change to default as true in future once we have a complete implementation of Ion native + // trino integration supported. + private boolean ionNativeTrinoEnabled; + private String parquetTimeZone = TimeZone.getDefault().getID(); private boolean useParquetColumnNames = true; @@ -714,6 +720,19 @@ public HiveConfig setOrcLegacyTimeZone(String orcLegacyTimeZone) return this; } + public boolean getIonNativeTrinoEnabled() + { + return ionNativeTrinoEnabled; + } + + @Config("hive.ion.nativetrino") + @ConfigDescription("Feature flag to enable Ion native trino integration") + public HiveConfig setIonNativeTrinoEnabled(boolean ionNativeTrinoEnabled) + { + this.ionNativeTrinoEnabled = ionNativeTrinoEnabled; + return this; + } + public DateTimeZone getParquetDateTimeZone() { TimeZone timeZone = TimeZone.getTimeZone(ZoneId.of(parquetTimeZone)); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java index 79d4d8fb6fb1..f8ddd9d7ee9b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java @@ -28,6 +28,7 @@ import io.trino.hive.formats.line.Column; import io.trino.plugin.hive.AcidInfo; import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HivePageSourceFactory; import io.trino.plugin.hive.ReaderColumns; import io.trino.plugin.hive.ReaderPageSource; @@ -57,11 +58,14 @@ public class IonPageSourceFactory implements HivePageSourceFactory { private final TrinoFileSystemFactory trinoFileSystemFactory; + // this is used as a feature flag to enable Ion native trino integration + private final boolean nativeTrinoEnabled; @Inject - public IonPageSourceFactory(TrinoFileSystemFactory trinoFileSystemFactory) + public IonPageSourceFactory(TrinoFileSystemFactory trinoFileSystemFactory, HiveConfig hiveConfig) { this.trinoFileSystemFactory = trinoFileSystemFactory; + this.nativeTrinoEnabled = hiveConfig.getIonNativeTrinoEnabled(); } @Override @@ -80,6 +84,11 @@ public Optional createPageSource( boolean originalFile, AcidTransaction transaction) { + if (!this.nativeTrinoEnabled) { + // this allows user to defer to a legacy hive implementation(like ion-hive-serde) or throw an error based + // on their use case + return Optional.empty(); + } if (!ION_SERDE_CLASS.equals(schema.serializationLibraryName())) { return Optional.empty(); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index bb4f51595cb6..49e7b4a5aecf 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -225,6 +225,11 @@ protected static QueryRunner createHiveQueryRunner(HiveQueryRunner.Builder bu // Make weighted split scheduling more conservative to avoid OOMs in test .addHiveProperty("hive.minimum-assigned-split-weight", "0.5") .addHiveProperty("hive.partition-projection-enabled", "true") + // This is needed for Ion native trino support as the implementation is in progress and can be + // backwards incompatible. It requires explicitly opting to use this feature. + // TODO: In future this property should change to `true` as default and then the following statement can + // be removed. + .addHiveProperty("hive.ion.nativetrino", "true") // This is needed for e2e scale writers test otherwise 50% threshold of // bufferSize won't get exceeded for scaling to happen. .addExtraProperty("task.max-local-exchange-buffer-size", "32MB") diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java index 70aa296cc498..b883576a8177 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java @@ -181,6 +181,9 @@ public static Set getDefaultHivePageSourceFactories(HdfsE public static Set getDefaultHivePageSourceFactories(TrinoFileSystemFactory fileSystemFactory, HiveConfig hiveConfig) { FileFormatDataSourceStats stats = new FileFormatDataSourceStats(); + // set IonNativeTrino as true in hiveConfig for testing + // TODO: In future this flag should change to `true` as default and then the following statement can be removed. + hiveConfig.setIonNativeTrinoEnabled(true); return ImmutableSet.builder() .add(new CsvPageSourceFactory(fileSystemFactory, hiveConfig)) .add(new JsonPageSourceFactory(fileSystemFactory, hiveConfig)) @@ -192,7 +195,7 @@ public static Set getDefaultHivePageSourceFactories(Trino .add(new RcFilePageSourceFactory(fileSystemFactory, hiveConfig)) .add(new OrcPageSourceFactory(new OrcReaderConfig(), fileSystemFactory, stats, hiveConfig)) .add(new ParquetPageSourceFactory(fileSystemFactory, stats, new ParquetReaderConfig(), hiveConfig)) - .add(new IonPageSourceFactory(fileSystemFactory)) + .add(new IonPageSourceFactory(fileSystemFactory, hiveConfig)) .build(); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java index d07f3a58b821..c02fe9725d5e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java @@ -78,6 +78,7 @@ public void testDefaults() .setParallelPartitionedBucketedWrites(true) .setTextMaxLineLength(DataSize.of(100, Unit.MEGABYTE)) .setOrcLegacyTimeZone(TimeZone.getDefault().getID()) + .setIonNativeTrinoEnabled(false) .setParquetTimeZone(TimeZone.getDefault().getID()) .setUseParquetColumnNames(true) .setRcfileTimeZone(TimeZone.getDefault().getID()) @@ -162,6 +163,7 @@ public void testExplicitPropertyMappings() .put("hive.max-partition-drops-per-query", "1000") .put("hive.text.max-line-length", "13MB") .put("hive.orc.time-zone", nonDefaultTimeZone().getID()) + .put("hive.ion.nativetrino", "true") .put("hive.parquet.time-zone", nonDefaultTimeZone().getID()) .put("hive.parquet.use-column-names", "false") .put("hive.rcfile.time-zone", nonDefaultTimeZone().getID()) @@ -245,6 +247,7 @@ public void testExplicitPropertyMappings() .setParallelPartitionedBucketedWrites(false) .setTextMaxLineLength(DataSize.of(13, Unit.MEGABYTE)) .setOrcLegacyTimeZone(nonDefaultTimeZone().getID()) + .setIonNativeTrinoEnabled(true) .setParquetTimeZone(nonDefaultTimeZone().getID()) .setUseParquetColumnNames(false) .setRcfileTimeZone(nonDefaultTimeZone().getID()) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index 2a07fc5b8885..971a2b6179f9 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -377,12 +377,17 @@ public void testIon(int rowCount, long fileSizePadding) .filter(tc -> !(tc.type instanceof MapType)) .collect(toList()); + HiveConfig hiveConfig = new HiveConfig(); + // enable Ion native trino integration for testing while the implementation is in progress + // TODO: In future this flag should change to `true` as default and then the following statement can be removed. + hiveConfig.setIonNativeTrinoEnabled(true); + assertThatFileFormat(ION) .withColumns(testColumns) .withRowsCount(rowCount) .withFileSizePadding(fileSizePadding) .withFileWriterFactory(fileSystemFactory -> new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) - .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory)); + .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory, hiveConfig)); } @Test(dataProvider = "validRowAndFileSizePadding") diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index 70bbc44ef3e5..b7bc5a519ccf 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -25,6 +25,7 @@ import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HivePageSourceProvider; +import io.trino.plugin.hive.ReaderPageSource; import io.trino.plugin.hive.Schema; import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.connector.ConnectorSession; @@ -64,6 +65,8 @@ */ public class IonPageSourceSmokeTest { + public static final String TEST_ION_LOCATION = "memory:///test.ion"; + @Test public void testReadTwoValues() throws IOException @@ -114,17 +117,47 @@ public void testProjectedColumn() 1); } + @Test + public void testPageSourceWithNativeTrinoDisabled() + throws IOException + { + List tableColumns = List.of( + toHiveBaseColumnHandle("foo", INTEGER, 0), + toHiveBaseColumnHandle("bar", VARCHAR, 1)); + String ionText = "{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }"; + + TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); + Location location = Location.of(TEST_ION_LOCATION); + + // by default Ion native Trino integration is disabled + HiveConfig hiveConfig = new HiveConfig(); + + final ConnectorSession session = getHiveSession(hiveConfig); + + writeIonTextFile(ionText, location, fileSystemFactory.create(session)); + + final Optional pageSource = createReaderPageSource(fileSystemFactory, + hiveConfig, location, tableColumns, session); + + Assertions.assertTrue(pageSource.isEmpty(), "Expected empty page source when native Trino is disabled"); + } + private void assertRowCount(List tableColumns, List projectedColumns, String ionText, int rowCount) throws IOException { TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); - Location location = Location.of("memory:///test.ion"); + Location location = Location.of(TEST_ION_LOCATION); - final ConnectorSession session = getHiveSession(new HiveConfig()); + HiveConfig hiveConfig = new HiveConfig(); + // enable Ion native trino integration for testing while the implementation is in progress + hiveConfig.setIonNativeTrinoEnabled(true); + + final ConnectorSession session = getHiveSession(hiveConfig); writeIonTextFile(ionText, location, fileSystemFactory.create(session)); - try (ConnectorPageSource pageSource = createPageSource(fileSystemFactory, location, tableColumns, projectedColumns, session)) { + try (ConnectorPageSource pageSource = createConnectorPageSource(fileSystemFactory, hiveConfig, location, tableColumns, + projectedColumns, session)) { final MaterializedResult result = MaterializedResult.materializeSourceDataStream(session, pageSource, projectedColumns.stream().map(HiveColumnHandle::getType).toList()); Assertions.assertEquals(rowCount, result.getRowCount()); } @@ -147,15 +180,72 @@ private int writeIonTextFile(String ionText, Location location, TrinoFileSystem /** * todo: this is very similar to what's in TestOrcPredicates, factor out. */ - private static ConnectorPageSource createPageSource( + private static ConnectorPageSource createConnectorPageSource( TrinoFileSystemFactory fileSystemFactory, + HiveConfig hiveConfig, Location location, List tableColumns, List projectedColumns, ConnectorSession session) throws IOException { - IonPageSourceFactory factory = new IonPageSourceFactory(fileSystemFactory); + final PageSourceParameters pageSourceParameters = preparePageSourceParameters( + fileSystemFactory, hiveConfig, location, tableColumns, projectedColumns, session); + + return HivePageSourceProvider.createHivePageSource( + ImmutableSet.of(pageSourceParameters.factory()), + session, + location, + OptionalInt.empty(), + 0, + pageSourceParameters.length(), + pageSourceParameters.length(), + pageSourceParameters.nowMillis(), + new Schema(ION.getSerde(), false, pageSourceParameters.tableProperties()), + TupleDomain.all(), + TESTING_TYPE_MANAGER, + Optional.empty(), + Optional.empty(), + Optional.empty(), + false, + NO_ACID_TRANSACTION, + pageSourceParameters.columnMappings()) + .orElseThrow(); + } + + private static Optional createReaderPageSource(TrinoFileSystemFactory fileSystemFactory, + HiveConfig hiveConfig, Location location, + List tableColumns, + ConnectorSession session) + throws IOException + { + final PageSourceParameters pageSourceParameters = preparePageSourceParameters( + fileSystemFactory, hiveConfig, location, tableColumns, ImmutableList.of(), session); + + return pageSourceParameters.factory().createPageSource( + session, + location, + 0, + pageSourceParameters.length(), + pageSourceParameters.length(), + pageSourceParameters.nowMillis(), + new Schema(ION.getSerde(), false, pageSourceParameters.tableProperties()), + tableColumns, + TupleDomain.all(), + Optional.empty(), + OptionalInt.empty(), + false, + NO_ACID_TRANSACTION); + } + + private static PageSourceParameters preparePageSourceParameters(TrinoFileSystemFactory fileSystemFactory, + HiveConfig hiveConfig, Location location, + List tableColumns, + List projectedColumns, + ConnectorSession session) + throws IOException + { + IonPageSourceFactory factory = new IonPageSourceFactory(fileSystemFactory, hiveConfig); long length = fileSystemFactory.create(session).newInputFile(location).length(); long nowMillis = Instant.now().toEpochMilli(); @@ -175,25 +265,9 @@ private static ConnectorPageSource createPageSource( .put(LIST_COLUMNS, tableColumns.stream().map(HiveColumnHandle::getName).collect(Collectors.joining(","))) .put(LIST_COLUMN_TYPES, tableColumns.stream().map(HiveColumnHandle::getHiveType).map(HiveType::toString).collect(Collectors.joining(","))) .buildOrThrow(); - - return HivePageSourceProvider.createHivePageSource( - ImmutableSet.of(factory), - session, - location, - OptionalInt.empty(), - 0, - length, - length, - nowMillis, - new Schema(ION.getSerde(), false, tableProperties), - TupleDomain.all(), - TESTING_TYPE_MANAGER, - Optional.empty(), - Optional.empty(), - Optional.empty(), - false, - NO_ACID_TRANSACTION, - columnMappings) - .orElseThrow(); + return new PageSourceParameters(factory, length, nowMillis, columnMappings, tableProperties); } + + private record PageSourceParameters(IonPageSourceFactory factory, long length, long nowMillis, List columnMappings, Map tableProperties) + { } } From 4ee0ba3d872b1c0960ad6a7e03b30f73d01dd0a1 Mon Sep 17 00:00:00 2001 From: linlin-s Date: Wed, 20 Nov 2024 15:34:40 -0800 Subject: [PATCH 04/18] Add support of configurable Ion encoding format --- .../trino/plugin/hive/ion/IonFileWriter.java | 7 +- .../plugin/hive/ion/IonFileWriterFactory.java | 2 + .../plugin/hive/ion/IonWriterOptions.java | 68 ++++++++++++ .../plugin/hive/TestHiveFileFormats.java | 67 +++++++++++- .../hive/ion/IonPageSourceSmokeTest.java | 103 +++++++++++++++++- 5 files changed, 234 insertions(+), 13 deletions(-) create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java index 7191f6cbe5f9..89d0bd408762 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java @@ -14,7 +14,6 @@ package io.trino.plugin.hive.ion; import com.amazon.ion.IonWriter; -import com.amazon.ion.system.IonTextWriterBuilder; import com.google.common.io.CountingOutputStream; import io.trino.hive.formats.compression.CompressionKind; import io.trino.hive.formats.ion.IonEncoder; @@ -51,6 +50,7 @@ public IonFileWriter( Closeable rollbackAction, TypeManager typeManager, Optional compressionKind, + IonWriterOptions.IonEncoding ionEncoding, List columns) throws IOException { @@ -66,10 +66,7 @@ public IonFileWriter( else { this.outputStream = countingOutputStream; } - - this.writer = IonTextWriterBuilder - .minimal() - .build(this.outputStream); + this.writer = ionEncoding.createWriter(this.outputStream); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java index afec4d3eda49..5a4f82354aaa 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java @@ -42,6 +42,7 @@ import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static io.trino.plugin.hive.HiveErrorCode.HIVE_WRITER_OPEN_ERROR; import static io.trino.plugin.hive.HiveSessionProperties.getTimestampPrecision; +import static io.trino.plugin.hive.ion.IonWriterOptions.getIonEncoding; import static io.trino.plugin.hive.util.HiveTypeUtil.getType; import static io.trino.plugin.hive.util.HiveUtil.getColumnNames; import static io.trino.plugin.hive.util.HiveUtil.getColumnTypes; @@ -101,6 +102,7 @@ public Optional createFileWriter( rollbackAction, typeManager, compressionCodec.getHiveCompressionKind(), + getIonEncoding(schema), columns)); } catch (Exception e) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java new file mode 100644 index 000000000000..41a20df51cb2 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java @@ -0,0 +1,68 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.ion; + +import com.amazon.ion.IonWriter; +import com.amazon.ion.system.IonBinaryWriterBuilder; +import com.amazon.ion.system.IonTextWriterBuilder; +import io.trino.spi.TrinoException; + +import java.io.OutputStream; +import java.util.Locale; +import java.util.Map; + +import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNSUPPORTED_FORMAT; + +public final class IonWriterOptions +{ + public static final String ION_ENCODING_PROPERTY = "ion.encoding"; + public static final String TEXT_ENCODING = "text"; + public static final String BINARY_ENCODING = "binary"; + + public enum IonEncoding + { + BINARY + { + @Override + public IonWriter createWriter(OutputStream outputStream) + { + return IonBinaryWriterBuilder.standard().build(outputStream); + } + }, + + TEXT + { + @Override + public IonWriter createWriter(OutputStream outputStream) + { + return IonTextWriterBuilder.minimal().build(outputStream); + } + }; + + public abstract IonWriter createWriter(OutputStream outputStream); + } + + public static IonEncoding getIonEncoding(Map schema) + { + String encodingStr = schema.getOrDefault(ION_ENCODING_PROPERTY, BINARY_ENCODING); + return switch (encodingStr.toLowerCase(Locale.ROOT)) { + case TEXT_ENCODING -> IonEncoding.TEXT; + case BINARY_ENCODING -> IonEncoding.BINARY; + default -> throw new TrinoException(HIVE_UNSUPPORTED_FORMAT, + "Unsupported Ion encoding format: " + encodingStr); + }; + } + + private IonWriterOptions() {} +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index 971a2b6179f9..b9198faf6cf6 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -140,6 +140,7 @@ import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_WRITER_OPEN_ERROR; import static io.trino.plugin.hive.HivePageSourceProvider.ColumnMapping.buildColumnMappings; import static io.trino.plugin.hive.HiveStorageFormat.AVRO; import static io.trino.plugin.hive.HiveStorageFormat.CSV; @@ -156,6 +157,8 @@ import static io.trino.plugin.hive.HiveTestUtils.getHiveSession; import static io.trino.plugin.hive.HiveTestUtils.mapType; import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_ENCODING_PROPERTY; +import static io.trino.plugin.hive.ion.IonWriterOptions.TEXT_ENCODING; import static io.trino.plugin.hive.util.HiveTypeTranslator.toHiveType; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; @@ -231,6 +234,7 @@ public final class TestHiveFileFormats private static final FileFormatDataSourceStats STATS = new FileFormatDataSourceStats(); private static final ConnectorSession PARQUET_SESSION = getHiveSession(createParquetHiveConfig(false)); private static final ConnectorSession PARQUET_SESSION_USE_NAME = getHiveSession(createParquetHiveConfig(true)); + private static final String ERROR_ENCODING = "error_encoding"; @DataProvider(name = "rowCount") public static Object[][] rowCountProvider() @@ -369,7 +373,7 @@ public void testOpenXJson(int rowCount, long fileSizePadding) } @Test(dataProvider = "validRowAndFileSizePadding") - public void testIon(int rowCount, long fileSizePadding) + public void testIonWithBinaryEncoding(int rowCount, long fileSizePadding) throws Exception { List testColumns = TEST_COLUMNS.stream() @@ -390,6 +394,54 @@ public void testIon(int rowCount, long fileSizePadding) .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory, hiveConfig)); } + @Test(dataProvider = "validRowAndFileSizePadding") + public void testIonWithTextEncoding(int rowCount, long fileSizePadding) + throws Exception + { + List testColumns = TEST_COLUMNS.stream() + // todo: add support for maps to trino impl + .filter(tc -> !(tc.type instanceof MapType)) + .collect(toList()); + + HiveConfig hiveConfig = new HiveConfig(); + // enable Ion native trino integration for testing while the implementation is in progress + // TODO: In future this flag should change to `true` as default and then the following statement can be removed. + hiveConfig.setIonNativeTrinoEnabled(true); + + assertThatFileFormat(ION) + .withColumns(testColumns) + .withRowsCount(rowCount) + .withFileSizePadding(fileSizePadding) + .withTableProperties(ImmutableMap.of(ION_ENCODING_PROPERTY, TEXT_ENCODING)) + .withFileWriterFactory(fileSystemFactory -> new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) + .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory, hiveConfig)); + } + + @Test(dataProvider = "validRowAndFileSizePadding") + public void testInvalidIonEncoding(int rowCount, long fileSizePadding) + throws Exception + { + List testColumns = TEST_COLUMNS.stream() + // todo: add support for maps to trino impl + .filter(tc -> !(tc.type instanceof MapType)) + .collect(toList()); + + HiveConfig hiveConfig = new HiveConfig(); + // enable Ion native trino integration for testing while the implementation is in progress + // TODO: In future this flag should change to `true` as default and then the following statement can be removed. + hiveConfig.setIonNativeTrinoEnabled(true); + + assertTrinoExceptionThrownBy(() -> assertThatFileFormat(ION) + .withColumns(testColumns) + .withRowsCount(rowCount) + .withFileSizePadding(fileSizePadding) + .withTableProperties(ImmutableMap.of(ION_ENCODING_PROPERTY, ERROR_ENCODING)) + .withFileWriterFactory(fileSystemFactory -> new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) + .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory, hiveConfig))) + .hasErrorCode(HIVE_WRITER_OPEN_ERROR) + .hasMessage("Error creating Ion Output"); + } + @Test(dataProvider = "validRowAndFileSizePadding") public void testRcTextPageSource(int rowCount, long fileSizePadding) throws Exception @@ -1223,6 +1275,7 @@ private static class FileFormatAssertion private boolean skipGenericWrite; private HiveFileWriterFactory fileWriterFactory; private long fileSizePadding; + private Map customTableProperties = ImmutableMap.of(); private final TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); @@ -1280,6 +1333,12 @@ public FileFormatAssertion withRowsCount(int rowsCount) return this; } + public FileFormatAssertion withTableProperties(Map tableProperties) + { + this.customTableProperties = requireNonNull(tableProperties, "customTableProperties is null"); + return this; + } + public FileFormatAssertion withSession(ConnectorSession session) { this.session = requireNonNull(session, "session is null"); @@ -1338,7 +1397,7 @@ private void assertRead(HivePageSourceFactory pageSourceFactory) if (fileWriterFactory == null) { continue; } - createTestFileTrino(location, storageFormat, compressionCodec, writeColumns, session, rowsCount, fileWriterFactory); + createTestFileTrino(location, storageFormat, compressionCodec, writeColumns, session, rowsCount, fileWriterFactory, customTableProperties); } else { if (skipGenericWrite) { @@ -1368,7 +1427,8 @@ private static void createTestFileTrino( List testColumns, ConnectorSession session, int numRows, - HiveFileWriterFactory fileWriterFactory) + HiveFileWriterFactory fileWriterFactory, + Map customTableProperties) { // filter out partition keys, which are not written to the file testColumns = testColumns.stream() @@ -1393,6 +1453,7 @@ private static void createTestFileTrino( Map tableProperties = ImmutableMap.builder() .put(LIST_COLUMNS, testColumns.stream().map(TestColumn::name).collect(Collectors.joining(","))) .put(LIST_COLUMN_TYPES, testColumns.stream().map(TestColumn::type).map(HiveTypeTranslator::toHiveType).map(HiveType::toString).collect(Collectors.joining(","))) + .putAll(customTableProperties) .buildOrThrow(); Optional fileWriter = fileWriterFactory.createFileWriter( diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index b7bc5a519ccf..7f3153724ac4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -22,11 +22,17 @@ import io.trino.filesystem.TrinoOutputFile; import io.trino.filesystem.memory.MemoryFileSystemFactory; import io.trino.metastore.HiveType; +import io.trino.plugin.hive.FileWriter; import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HiveCompressionCodec; import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HivePageSourceProvider; import io.trino.plugin.hive.ReaderPageSource; import io.trino.plugin.hive.Schema; +import io.trino.plugin.hive.WriterKind; +import io.trino.spi.Page; +import io.trino.spi.block.IntArrayBlock; +import io.trino.spi.block.RunLengthEncodedBlock; import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.predicate.TupleDomain; @@ -40,6 +46,7 @@ import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.time.Instant; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; @@ -52,6 +59,9 @@ import static io.trino.plugin.hive.HiveTestUtils.projectedColumn; import static io.trino.plugin.hive.HiveTestUtils.toHiveBaseColumnHandle; import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; +import static io.trino.plugin.hive.ion.IonWriterOptions.BINARY_ENCODING; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_ENCODING_PROPERTY; +import static io.trino.plugin.hive.ion.IonWriterOptions.TEXT_ENCODING; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; import static io.trino.spi.type.BooleanType.BOOLEAN; @@ -59,12 +69,18 @@ import static io.trino.spi.type.RowType.field; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; +import static java.util.stream.Collectors.toList; /** * Most basic test to reflect PageSource-fu is wired up correctly. */ public class IonPageSourceSmokeTest { + // In the Ion binary format, a value stream is always start with binary version marker. This help distinguish Ion binary + // data from other formats, including Ion text format. + private static final byte[] BINARY_VERSION_MARKER = {(byte) 0xE0, (byte) 0x01, (byte) 0x00, (byte) 0XEA}; + private static final String EXPECTED_TEXT = "{foo:3,bar:6}"; + public static final String TEST_ION_LOCATION = "memory:///test.ion"; @Test @@ -117,6 +133,50 @@ public void testProjectedColumn() 1); } + @Test + public void testTextEncoding() + throws IOException + { + List tableColumns = List.of( + toHiveBaseColumnHandle("foo", INTEGER, 0), + toHiveBaseColumnHandle("bar", INTEGER, 0)); + + assertEncoding(tableColumns, TEXT_ENCODING); + } + + @Test + public void testBinaryEncoding() + throws IOException + { + List tableColumns = List.of( + toHiveBaseColumnHandle("foo", INTEGER, 0), + toHiveBaseColumnHandle("bar", INTEGER, 0)); + + assertEncoding(tableColumns, BINARY_ENCODING); + } + + private void assertEncoding(List tableColumns, + String encoding) + throws IOException + { + TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); + Location location = Location.of(TEST_ION_LOCATION); + ConnectorSession session = getHiveSession(new HiveConfig()); + writeTestData(session, fileSystemFactory, location, encoding, tableColumns); + byte[] inputStreamBytes = fileSystemFactory.create(session) + .newInputFile(location) + .newStream() + .readAllBytes(); + + if (encoding.equals(BINARY_ENCODING)) { + // Check if the first 4 bytes is binary version marker + Assertions.assertArrayEquals(Arrays.copyOfRange(inputStreamBytes, 0, 4), BINARY_VERSION_MARKER); + } + else { + Assertions.assertEquals(new String(inputStreamBytes, StandardCharsets.UTF_8), EXPECTED_TEXT); + } + } + @Test public void testPageSourceWithNativeTrinoDisabled() throws IOException @@ -163,6 +223,31 @@ private void assertRowCount(List tableColumns, List tableColumns) + throws IOException + { + FileWriter ionFileWriter = new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER) + .createFileWriter( + location, + tableColumns.stream().map(HiveColumnHandle::getName).collect(toList()), + ION.toStorageFormat(), + HiveCompressionCodec.NONE, + getTablePropertiesWithEncoding(tableColumns, encoding), + session, + OptionalInt.empty(), + NO_ACID_TRANSACTION, + false, + WriterKind.INSERT) + .orElseThrow(); + ionFileWriter.appendRows(new Page( + RunLengthEncodedBlock.create(new IntArrayBlock(1, Optional.empty(), new int[] {3}), 1), + RunLengthEncodedBlock.create(new IntArrayBlock(1, Optional.empty(), new int[] {6}), 1))); + ionFileWriter.commit(); + } + private int writeIonTextFile(String ionText, Location location, TrinoFileSystem fileSystem) throws IOException { @@ -261,13 +346,21 @@ private static PageSourceParameters preparePageSourceParameters(TrinoFileSystemF length, nowMillis); - final Map tableProperties = ImmutableMap.builder() - .put(LIST_COLUMNS, tableColumns.stream().map(HiveColumnHandle::getName).collect(Collectors.joining(","))) - .put(LIST_COLUMN_TYPES, tableColumns.stream().map(HiveColumnHandle::getHiveType).map(HiveType::toString).collect(Collectors.joining(","))) - .buildOrThrow(); - return new PageSourceParameters(factory, length, nowMillis, columnMappings, tableProperties); + return new PageSourceParameters(factory, length, nowMillis, columnMappings, getTablePropertiesWithEncoding(tableColumns, BINARY_ENCODING)); } private record PageSourceParameters(IonPageSourceFactory factory, long length, long nowMillis, List columnMappings, Map tableProperties) { } + + /** + * Creates table properties for IonFileWriter with encoding flag. + */ + private static Map getTablePropertiesWithEncoding(List tableColumns, String encoding) + { + return ImmutableMap.builder() + .put(LIST_COLUMNS, tableColumns.stream().map(HiveColumnHandle::getName).collect(Collectors.joining(","))) + .put(LIST_COLUMN_TYPES, tableColumns.stream().map(HiveColumnHandle::getHiveType).map(HiveType::toString).collect(Collectors.joining(","))) + .put(ION_ENCODING_PROPERTY, encoding) + .buildOrThrow(); + } } From 02813f03b8bdbd9d43e972fe89e936e1470a2458 Mon Sep 17 00:00:00 2001 From: Rob Marrowstone Date: Fri, 6 Dec 2024 11:37:32 -0800 Subject: [PATCH 05/18] Refactor IonPageSoureSmokeTest This change factors all of the setup involved in creating PageSources and FileWriters into a single TestFixture inner class. This reduces some existing duplication and should make it simpler to add/change table properties and hive config in the test. --- .../hive/ion/IonPageSourceSmokeTest.java | 326 +++++++++--------- 1 file changed, 155 insertions(+), 171 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index 7f3153724ac4..b4660c47e305 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -17,8 +17,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; import io.trino.filesystem.memory.MemoryFileSystemFactory; import io.trino.metastore.HiveType; @@ -27,7 +27,6 @@ import io.trino.plugin.hive.HiveCompressionCodec; import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HivePageSourceProvider; -import io.trino.plugin.hive.ReaderPageSource; import io.trino.plugin.hive.Schema; import io.trino.plugin.hive.WriterKind; import io.trino.spi.Page; @@ -47,6 +46,7 @@ import java.nio.charset.StandardCharsets; import java.time.Instant; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -133,6 +133,22 @@ public void testProjectedColumn() 1); } + @Test + public void testPageSourceWithNativeTrinoDisabled() + throws IOException + { + List tableColumns = List.of( + toHiveBaseColumnHandle("foo", INTEGER, 0), + toHiveBaseColumnHandle("bar", VARCHAR, 1)); + + TestFixture fixture = new TestFixture(tableColumns) + .withNativeIonDisabled(); + fixture.writeIonTextFile("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }"); + + Optional connectorPageSource = fixture.getOptionalPageSource(); + Assertions.assertTrue(connectorPageSource.isEmpty(), "Expected empty page source when native Trino is disabled"); + } + @Test public void testTextEncoding() throws IOException @@ -156,15 +172,14 @@ public void testBinaryEncoding() } private void assertEncoding(List tableColumns, - String encoding) + String encoding) throws IOException { - TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); - Location location = Location.of(TEST_ION_LOCATION); - ConnectorSession session = getHiveSession(new HiveConfig()); - writeTestData(session, fileSystemFactory, location, encoding, tableColumns); - byte[] inputStreamBytes = fileSystemFactory.create(session) - .newInputFile(location) + TestFixture fixture = new TestFixture(tableColumns) + .withEncoding(encoding); + + writeTestData(fixture.getFileWriter()); + byte[] inputStreamBytes = fixture.getTrinoInputFile() .newStream() .readAllBytes(); @@ -177,190 +192,159 @@ private void assertEncoding(List tableColumns, } } - @Test - public void testPageSourceWithNativeTrinoDisabled() - throws IOException - { - List tableColumns = List.of( - toHiveBaseColumnHandle("foo", INTEGER, 0), - toHiveBaseColumnHandle("bar", VARCHAR, 1)); - String ionText = "{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }"; - - TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); - Location location = Location.of(TEST_ION_LOCATION); - - // by default Ion native Trino integration is disabled - HiveConfig hiveConfig = new HiveConfig(); - - final ConnectorSession session = getHiveSession(hiveConfig); - - writeIonTextFile(ionText, location, fileSystemFactory.create(session)); - - final Optional pageSource = createReaderPageSource(fileSystemFactory, - hiveConfig, location, tableColumns, session); - - Assertions.assertTrue(pageSource.isEmpty(), "Expected empty page source when native Trino is disabled"); - } - private void assertRowCount(List tableColumns, List projectedColumns, String ionText, int rowCount) throws IOException { - TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); - Location location = Location.of(TEST_ION_LOCATION); - - HiveConfig hiveConfig = new HiveConfig(); - // enable Ion native trino integration for testing while the implementation is in progress - hiveConfig.setIonNativeTrinoEnabled(true); - - final ConnectorSession session = getHiveSession(hiveConfig); - - writeIonTextFile(ionText, location, fileSystemFactory.create(session)); - - try (ConnectorPageSource pageSource = createConnectorPageSource(fileSystemFactory, hiveConfig, location, tableColumns, - projectedColumns, session)) { - final MaterializedResult result = MaterializedResult.materializeSourceDataStream(session, pageSource, projectedColumns.stream().map(HiveColumnHandle::getType).toList()); + TestFixture fixture = new TestFixture(tableColumns, projectedColumns); + fixture.writeIonTextFile(ionText); + + try (ConnectorPageSource pageSource = fixture.getPageSource()) { + final MaterializedResult result = MaterializedResult.materializeSourceDataStream( + fixture.getSession(), + pageSource, + projectedColumns.stream().map(HiveColumnHandle::getType).toList()); Assertions.assertEquals(rowCount, result.getRowCount()); } } - /** - * todo: At some point, we might need to combine writeIonTextFile with this method and add logic to write iontext to Page. - */ - private static void writeTestData(ConnectorSession session, TrinoFileSystemFactory fileSystemFactory, Location location, String encoding, List tableColumns) - throws IOException + private static void writeTestData(FileWriter ionFileWriter) { - FileWriter ionFileWriter = new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER) - .createFileWriter( - location, - tableColumns.stream().map(HiveColumnHandle::getName).collect(toList()), - ION.toStorageFormat(), - HiveCompressionCodec.NONE, - getTablePropertiesWithEncoding(tableColumns, encoding), - session, - OptionalInt.empty(), - NO_ACID_TRANSACTION, - false, - WriterKind.INSERT) - .orElseThrow(); ionFileWriter.appendRows(new Page( RunLengthEncodedBlock.create(new IntArrayBlock(1, Optional.empty(), new int[] {3}), 1), RunLengthEncodedBlock.create(new IntArrayBlock(1, Optional.empty(), new int[] {6}), 1))); ionFileWriter.commit(); } - private int writeIonTextFile(String ionText, Location location, TrinoFileSystem fileSystem) - throws IOException + private static class TestFixture { - final TrinoOutputFile outputFile = fileSystem.newOutputFile(location); - int written = 0; - try (OutputStream outputStream = outputFile.create()) { - byte[] bytes = ionText.getBytes(StandardCharsets.UTF_8); - outputStream.write(bytes); - outputStream.flush(); - written = bytes.length; + private TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); + private Location fileLocation = Location.of(TEST_ION_LOCATION); + private HiveConfig hiveConfig = new HiveConfig(); + private Map tableProperties = new HashMap<>(); + private List columns; + private List projections; + + private ConnectorSession session; + + TestFixture(List columns) + { + this(columns, columns); } - return written; - } - /** - * todo: this is very similar to what's in TestOrcPredicates, factor out. - */ - private static ConnectorPageSource createConnectorPageSource( - TrinoFileSystemFactory fileSystemFactory, - HiveConfig hiveConfig, - Location location, - List tableColumns, - List projectedColumns, - ConnectorSession session) - throws IOException - { - final PageSourceParameters pageSourceParameters = preparePageSourceParameters( - fileSystemFactory, hiveConfig, location, tableColumns, projectedColumns, session); - - return HivePageSourceProvider.createHivePageSource( - ImmutableSet.of(pageSourceParameters.factory()), - session, - location, - OptionalInt.empty(), - 0, - pageSourceParameters.length(), - pageSourceParameters.length(), - pageSourceParameters.nowMillis(), - new Schema(ION.getSerde(), false, pageSourceParameters.tableProperties()), - TupleDomain.all(), - TESTING_TYPE_MANAGER, - Optional.empty(), - Optional.empty(), - Optional.empty(), - false, - NO_ACID_TRANSACTION, - pageSourceParameters.columnMappings()) - .orElseThrow(); - } + TestFixture(List columns, List projections) + { + this.columns = columns; + this.projections = projections; + tableProperties.put(LIST_COLUMNS, columns.stream() + .map(HiveColumnHandle::getName) + .collect(Collectors.joining(","))); + tableProperties.put(LIST_COLUMN_TYPES, columns.stream().map(HiveColumnHandle::getHiveType) + .map(HiveType::toString) + .collect(Collectors.joining(","))); + // the default at runtime is false, but most of our testing obviously assumes it is enabled. + hiveConfig.setIonNativeTrinoEnabled(true); + } - private static Optional createReaderPageSource(TrinoFileSystemFactory fileSystemFactory, - HiveConfig hiveConfig, Location location, - List tableColumns, - ConnectorSession session) - throws IOException - { - final PageSourceParameters pageSourceParameters = preparePageSourceParameters( - fileSystemFactory, hiveConfig, location, tableColumns, ImmutableList.of(), session); - - return pageSourceParameters.factory().createPageSource( - session, - location, - 0, - pageSourceParameters.length(), - pageSourceParameters.length(), - pageSourceParameters.nowMillis(), - new Schema(ION.getSerde(), false, pageSourceParameters.tableProperties()), - tableColumns, - TupleDomain.all(), - Optional.empty(), - OptionalInt.empty(), - false, - NO_ACID_TRANSACTION); - } + TestFixture withEncoding(String encoding) + { + tableProperties.put(ION_ENCODING_PROPERTY, encoding); + return this; + } - private static PageSourceParameters preparePageSourceParameters(TrinoFileSystemFactory fileSystemFactory, - HiveConfig hiveConfig, Location location, - List tableColumns, - List projectedColumns, - ConnectorSession session) - throws IOException - { - IonPageSourceFactory factory = new IonPageSourceFactory(fileSystemFactory, hiveConfig); + TestFixture withNativeIonDisabled() + { + hiveConfig.setIonNativeTrinoEnabled(false); + return this; + } - long length = fileSystemFactory.create(session).newInputFile(location).length(); - long nowMillis = Instant.now().toEpochMilli(); + Optional getOptionalPageSource() + throws IOException + { + IonPageSourceFactory pageSourceFactory = new IonPageSourceFactory(fileSystemFactory, hiveConfig); + + long length = fileSystemFactory.create(getSession()).newInputFile(fileLocation).length(); + long nowMillis = Instant.now().toEpochMilli(); + + List columnMappings = buildColumnMappings( + "", + ImmutableList.of(), + projections, + ImmutableList.of(), + ImmutableMap.of(), + fileLocation.toString(), + OptionalInt.empty(), + length, + nowMillis); + + return HivePageSourceProvider.createHivePageSource( + ImmutableSet.of(pageSourceFactory), + getSession(), + fileLocation, + OptionalInt.empty(), + 0, + length, + length, + nowMillis, + new Schema(ION.getSerde(), false, tableProperties), + TupleDomain.all(), + TESTING_TYPE_MANAGER, + Optional.empty(), + Optional.empty(), + Optional.empty(), + false, + NO_ACID_TRANSACTION, + columnMappings); + } - List columnMappings = buildColumnMappings( - "", - ImmutableList.of(), - projectedColumns, - ImmutableList.of(), - ImmutableMap.of(), - location.toString(), - OptionalInt.empty(), - length, - nowMillis); - - return new PageSourceParameters(factory, length, nowMillis, columnMappings, getTablePropertiesWithEncoding(tableColumns, BINARY_ENCODING)); - } + ConnectorPageSource getPageSource() + throws IOException + { + return getOptionalPageSource().orElseThrow(); + } - private record PageSourceParameters(IonPageSourceFactory factory, long length, long nowMillis, List columnMappings, Map tableProperties) - { } + ConnectorSession getSession() + { + if (session == null) { + session = getHiveSession(hiveConfig); + } + return session; + } - /** - * Creates table properties for IonFileWriter with encoding flag. - */ - private static Map getTablePropertiesWithEncoding(List tableColumns, String encoding) - { - return ImmutableMap.builder() - .put(LIST_COLUMNS, tableColumns.stream().map(HiveColumnHandle::getName).collect(Collectors.joining(","))) - .put(LIST_COLUMN_TYPES, tableColumns.stream().map(HiveColumnHandle::getHiveType).map(HiveType::toString).collect(Collectors.joining(","))) - .put(ION_ENCODING_PROPERTY, encoding) - .buildOrThrow(); + int writeIonTextFile(String ionText) + throws IOException + { + final TrinoOutputFile outputFile = fileSystemFactory.create(getSession()).newOutputFile(fileLocation); + int written = 0; + try (OutputStream outputStream = outputFile.create()) { + byte[] bytes = ionText.getBytes(StandardCharsets.UTF_8); + outputStream.write(bytes); + outputStream.flush(); + written = bytes.length; + } + return written; + } + + FileWriter getFileWriter() + { + return new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER) + .createFileWriter( + fileLocation, + columns.stream().map(HiveColumnHandle::getName).collect(toList()), + ION.toStorageFormat(), + HiveCompressionCodec.NONE, + tableProperties, + getSession(), + OptionalInt.empty(), + NO_ACID_TRANSACTION, + false, + WriterKind.INSERT) + .orElseThrow(); + } + + TrinoInputFile getTrinoInputFile() + { + return fileSystemFactory.create(getSession()) + .newInputFile(fileLocation); + } } } From 66d6b1747684728c9761acb757b531b0c948cb9c Mon Sep 17 00:00:00 2001 From: linlin-s Date: Wed, 11 Dec 2024 09:28:38 -0800 Subject: [PATCH 06/18] Omit null fields instead of encoding them as null --- .../hive/formats/ion/IonEncoderFactory.java | 8 ++- .../trino/hive/formats/ion/TestIonFormat.java | 53 ++++++++++++++----- 2 files changed, 47 insertions(+), 14 deletions(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java index f62257a5c1e2..619e7086c5be 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java @@ -144,10 +144,14 @@ private void encodeStruct(IonWriter writer, IntFunction blockSelector, in { writer.stepIn(IonType.STRUCT); for (int i = 0; i < fieldEncoders.size(); i++) { - // todo: the Hive SerDe omits fields when null by default + // Omit the filed when the field is null + Block block = blockSelector.apply(i); + if (block.isNull(position)) { + continue; + } writer.setFieldName(fieldNames.get(i)); fieldEncoders.get(i) - .encode(writer, blockSelector.apply(i), position); + .encode(writer, block, position); } writer.stepOut(); } diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index 455c9de8241e..63d1b179fc72 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -40,6 +40,7 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.stream.IntStream; @@ -53,6 +54,16 @@ public class TestIonFormat { + private static final List TEST_COLUMNS = List.of( + new Column("magic_num", INTEGER, 0), + new Column("some_text", VARCHAR, 1), + new Column("is_summer", BooleanType.BOOLEAN, 2), + new Column("byte_clob", VarbinaryType.VARBINARY, 3), + new Column("sequencer", new ArrayType(INTEGER), 4), + new Column("struction", RowType.rowType( + field("foo", INTEGER), + field("bar", VARCHAR)), 5)); + @Test public void testSuperBasicStruct() throws IOException @@ -210,16 +221,6 @@ public void testOversizeOrOverpreciseDecimals() public void testEncode() throws IOException { - List columns = List.of( - new Column("magic_num", INTEGER, 0), - new Column("some_text", VARCHAR, 1), - new Column("is_summer", BooleanType.BOOLEAN, 2), - new Column("byte_clob", VarbinaryType.VARBINARY, 3), - new Column("sequencer", new ArrayType(INTEGER), 4), - new Column("struction", RowType.rowType( - field("foo", INTEGER), - field("bar", VARCHAR)), 5)); - List row1 = List.of(17, "something", true, new SqlVarbinary(new byte[] {(byte) 0xff}), List.of(1, 2, 3), List.of(51, "baz")); List row2 = List.of(31, "somebody", false, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), List.of(67, "qux")); String ionText = """ @@ -227,8 +228,36 @@ public void testEncode() { magic_num:31, some_text:"somebody", is_summer:false, byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{ foo:67, bar:"qux"}} """; - Page page = toPage(columns, row1, row2); - assertIonEquivalence(columns, page, ionText); + Page page = toPage(TEST_COLUMNS, row1, row2); + assertIonEquivalence(TEST_COLUMNS, page, ionText); + } + + @Test + public void testEncodeWithNullField() + throws IOException + { + List row1 = Arrays.asList(null, null, null, null, null, null); + String ionText = """ + {} + """; + + Page page = toPage(TEST_COLUMNS, row1); + assertIonEquivalence(TEST_COLUMNS, page, ionText); + } + + @Test + public void testEncodeWithNullNestedField() + throws IOException + { + List row1 = Arrays.asList(17, "something", true, new SqlVarbinary(new byte[] {(byte) 0xff}), List.of(1, 2, 3), Arrays.asList(null, "baz")); + List row2 = Arrays.asList(31, "somebody", null, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), Arrays.asList(null, "qux")); + String ionText = """ + { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{bar:"baz"}} + { magic_num:31, some_text:"somebody", byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{bar:"qux"}} + """; + + Page page = toPage(TEST_COLUMNS, row1, row2); + assertIonEquivalence(TEST_COLUMNS, page, ionText); } private void assertValues(RowType rowType, String ionText, List... expected) From 4c02014a2652649a54861501932665edd219c5c4 Mon Sep 17 00:00:00 2001 From: Khushboo <68757952+desaikd@users.noreply.github.com> Date: Mon, 16 Dec 2024 10:57:10 -0800 Subject: [PATCH 07/18] Adds support for map type with string key --- .../hive/formats/ion/IonDecoderFactory.java | 59 +++++++++++++++++++ .../hive/formats/ion/IonEncoderFactory.java | 40 +++++++++++++ .../trino/hive/formats/ion/TestIonFormat.java | 55 +++++++++++++---- 3 files changed, 144 insertions(+), 10 deletions(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java index 0d03efb0d0e3..76087632273b 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java @@ -19,11 +19,14 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.slice.Slices; +import io.trino.hive.formats.DistinctMapKeys; import io.trino.hive.formats.line.Column; import io.trino.spi.PageBuilder; import io.trino.spi.block.ArrayBlockBuilder; import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.MapBlockBuilder; import io.trino.spi.block.RowBlockBuilder; +import io.trino.spi.block.ValueBlock; import io.trino.spi.type.ArrayType; import io.trino.spi.type.BigintType; import io.trino.spi.type.BooleanType; @@ -34,6 +37,7 @@ import io.trino.spi.type.Int128; import io.trino.spi.type.IntegerType; import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.MapType; import io.trino.spi.type.RealType; import io.trino.spi.type.RowType; import io.trino.spi.type.SmallintType; @@ -96,6 +100,8 @@ private static BlockDecoder decoderForType(Type type) case DecimalType t -> wrapDecoder(decimalDecoder(t), IonType.DECIMAL); case VarcharType _, CharType _ -> wrapDecoder(stringDecoder, IonType.STRING, IonType.SYMBOL); case VarbinaryType _ -> wrapDecoder(binaryDecoder, IonType.BLOB, IonType.CLOB); + case MapType mapType -> wrapDecoder(new MapDecoder(mapType, decoderForType(mapType.getValueType())), + IonType.STRUCT); case RowType r -> wrapDecoder(RowDecoder.forFields(r.getFields()), IonType.STRUCT); case ArrayType a -> wrapDecoder(new ArrayDecoder(decoderForType(a.getElementType())), IonType.LIST, IonType.SEXP); default -> throw new IllegalArgumentException(String.format("Unsupported type: %s", type)); @@ -206,6 +212,59 @@ private void decode(IonReader ionReader, IntFunction blockSelector } } + private static class MapDecoder + implements BlockDecoder + { + private final BlockDecoder valueDecoder; + private final Type keyType; + private final Type valueType; + private final DistinctMapKeys distinctMapKeys; + private BlockBuilder keyBlockBuilder; + private BlockBuilder valueBlockBuilder; + + public MapDecoder(MapType mapType, BlockDecoder valueDecoder) + { + this.keyType = mapType.getKeyType(); + if (!(keyType instanceof VarcharType _ || keyType instanceof CharType _)) { + throw new UnsupportedOperationException("Unsupported map key type: " + keyType); + } + this.valueType = mapType.getValueType(); + this.valueDecoder = valueDecoder; + this.distinctMapKeys = new DistinctMapKeys(mapType, true); + this.keyBlockBuilder = mapType.getKeyType().createBlockBuilder(null, 128); + this.valueBlockBuilder = mapType.getValueType().createBlockBuilder(null, 128); + } + + @Override + public void decode(IonReader ionReader, BlockBuilder builder) + { + ionReader.stepIn(); + // buffer the keys and values + while (ionReader.next() != null) { + VarcharType.VARCHAR.writeSlice(keyBlockBuilder, Slices.utf8Slice(ionReader.getFieldName())); + valueDecoder.decode(ionReader, valueBlockBuilder); + } + ValueBlock keys = keyBlockBuilder.buildValueBlock(); + ValueBlock values = valueBlockBuilder.buildValueBlock(); + keyBlockBuilder = keyType.createBlockBuilder(null, keys.getPositionCount()); + valueBlockBuilder = valueType.createBlockBuilder(null, values.getPositionCount()); + + // copy the distinct key entries to the output + boolean[] distinctKeys = distinctMapKeys.selectDistinctKeys(keys); + + ((MapBlockBuilder) builder).buildEntry((keyBuilder, valueBuilder) -> { + for (int index = 0; index < distinctKeys.length; index++) { + boolean distinctKey = distinctKeys[index]; + if (distinctKey) { + keyBuilder.append(keys, index); + valueBuilder.append(values, index); + } + } + }); + ionReader.stepOut(); + } + } + private record ArrayDecoder(BlockDecoder elementDecoder) implements BlockDecoder { diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java index 619e7086c5be..bcb8e469399d 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java @@ -22,6 +22,7 @@ import io.trino.spi.block.ArrayBlock; import io.trino.spi.block.Block; import io.trino.spi.block.RowBlock; +import io.trino.spi.block.SqlMap; import io.trino.spi.type.ArrayType; import io.trino.spi.type.BigintType; import io.trino.spi.type.BooleanType; @@ -32,6 +33,7 @@ import io.trino.spi.type.Int128; import io.trino.spi.type.IntegerType; import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.MapType; import io.trino.spi.type.RealType; import io.trino.spi.type.RowType; import io.trino.spi.type.SmallintType; @@ -53,6 +55,8 @@ import java.util.Optional; import java.util.function.IntFunction; +import static com.google.common.base.Preconditions.checkArgument; + public class IonEncoderFactory { private IonEncoderFactory() {} @@ -85,6 +89,8 @@ private static BlockEncoder encoderForType(Type type) case DecimalType t -> decimalEncoder(t); case DateType _ -> dateEncoder; case TimestampType t -> timestampEncoder(t); + case MapType t -> new MapEncoder(t, t.getKeyType(), + encoderForType(t.getValueType())); case RowType t -> RowEncoder.forFields(t.getFields()); case ArrayType t -> new ArrayEncoder(wrapEncoder(encoderForType(t.getElementType()))); default -> throw new IllegalArgumentException(String.format("Unsupported type: %s", type)); @@ -157,6 +163,40 @@ private void encodeStruct(IonWriter writer, IntFunction blockSelector, in } } + private record MapEncoder(MapType mapType, Type keyType, + BlockEncoder encoder) + implements BlockEncoder + { + public MapEncoder(MapType mapType, Type keyType, BlockEncoder encoder) + + { + this.mapType = mapType; + if (!(keyType instanceof VarcharType _ || keyType instanceof CharType _)) { + throw new UnsupportedOperationException("Unsupported map key type: " + keyType); + } + this.keyType = keyType; + this.encoder = encoder; + } + + @Override + public void encode(IonWriter writer, Block block, int position) + throws IOException + { + SqlMap sqlMap = mapType.getObject(block, position); + int rawOffset = sqlMap.getRawOffset(); + Block rawKeyBlock = sqlMap.getRawKeyBlock(); + Block rawValueBlock = sqlMap.getRawValueBlock(); + + writer.stepIn(IonType.STRUCT); + for (int i = 0; i < sqlMap.getSize(); i++) { + checkArgument(!rawKeyBlock.isNull(rawOffset + i), "map key is null"); + writer.setFieldName(VarcharType.VARCHAR.getSlice(rawKeyBlock, rawOffset + i).toString(StandardCharsets.UTF_8)); + encoder.encode(writer, rawValueBlock, rawOffset + i); + } + writer.stepOut(); + } + } + private record ArrayEncoder(BlockEncoder elementEncoder) implements BlockEncoder { diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index 63d1b179fc72..8dfcc6353067 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -21,17 +21,20 @@ import com.amazon.ion.Timestamp; import com.amazon.ion.system.IonReaderBuilder; import com.amazon.ion.system.IonSystemBuilder; +import com.google.common.collect.ImmutableMap; import io.trino.hive.formats.line.Column; import io.trino.spi.Page; import io.trino.spi.PageBuilder; import io.trino.spi.type.ArrayType; import io.trino.spi.type.BooleanType; import io.trino.spi.type.DecimalType; +import io.trino.spi.type.MapType; import io.trino.spi.type.RowType; import io.trino.spi.type.SqlDecimal; import io.trino.spi.type.SqlTimestamp; import io.trino.spi.type.SqlVarbinary; import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TypeOperators; import io.trino.spi.type.VarbinaryType; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -54,6 +57,8 @@ public class TestIonFormat { + private static final TypeOperators TYPE_OPERATORS = new TypeOperators(); + private static final List TEST_COLUMNS = List.of( new Column("magic_num", INTEGER, 0), new Column("some_text", VARCHAR, 1), @@ -62,7 +67,8 @@ public class TestIonFormat new Column("sequencer", new ArrayType(INTEGER), 4), new Column("struction", RowType.rowType( field("foo", INTEGER), - field("bar", VARCHAR)), 5)); + field("bar", VARCHAR)), 5), + new Column("map", new MapType(VARCHAR, INTEGER, TYPE_OPERATORS), 6)); @Test public void testSuperBasicStruct() @@ -76,6 +82,20 @@ public void testSuperBasicStruct() List.of(31, "baz")); } + @Test + public void testMap() + throws IOException + { + MapType mapType = new MapType(VARCHAR, INTEGER, TYPE_OPERATORS); + assertValues( + RowType.rowType(field("foo", mapType)), + "{ foo: { a: 1, a: 2, b: 5 } }", + List.of(ImmutableMap.builder() + .put("a", 2) + .put("b", 5) + .buildOrThrow())); + } + @Test public void testCaseInsensitivityOfKeys() throws IOException @@ -221,11 +241,19 @@ public void testOversizeOrOverpreciseDecimals() public void testEncode() throws IOException { - List row1 = List.of(17, "something", true, new SqlVarbinary(new byte[] {(byte) 0xff}), List.of(1, 2, 3), List.of(51, "baz")); - List row2 = List.of(31, "somebody", false, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), List.of(67, "qux")); + List row1 = List.of(17, "something", true, new SqlVarbinary(new byte[] {(byte) 0xff}), List.of(1, 2, + 3), List.of(51, "baz"), ImmutableMap.builder() + .put("a", 2) + .put("b", 5) + .buildOrThrow()); + List row2 = List.of(31, "somebody", false, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), + List.of(7, 8, 9), List.of(67, "qux"), ImmutableMap.builder() + .put("foo", 12) + .put("bar", 50) + .buildOrThrow()); String ionText = """ - { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{ foo:51, bar:"baz"}} - { magic_num:31, some_text:"somebody", is_summer:false, byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{ foo:67, bar:"qux"}} + { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{ foo:51, bar:"baz"}, map: {a: 2, b: 5}} + { magic_num:31, some_text:"somebody", is_summer:false, byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{ foo:67, bar:"qux"}, map: {foo: 12, bar: 50}} """; Page page = toPage(TEST_COLUMNS, row1, row2); @@ -236,7 +264,7 @@ public void testEncode() public void testEncodeWithNullField() throws IOException { - List row1 = Arrays.asList(null, null, null, null, null, null); + List row1 = Arrays.asList(null, null, null, null, null, null, null); String ionText = """ {} """; @@ -249,11 +277,18 @@ public void testEncodeWithNullField() public void testEncodeWithNullNestedField() throws IOException { - List row1 = Arrays.asList(17, "something", true, new SqlVarbinary(new byte[] {(byte) 0xff}), List.of(1, 2, 3), Arrays.asList(null, "baz")); - List row2 = Arrays.asList(31, "somebody", null, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), Arrays.asList(null, "qux")); + List row1 = Arrays.asList(17, "something", true, new SqlVarbinary(new byte[] {(byte) 0xff}), + List.of(1, 2, 3), Arrays.asList(null, "baz"), ImmutableMap.builder() + .put("a", 2) + .put("b", 5) + .buildOrThrow()); + List row2 = Arrays.asList(31, "somebody", null, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), Arrays.asList(null, "qux"), ImmutableMap.builder() + .put("foo", 12) + .put("bar", 50) + .buildOrThrow()); String ionText = """ - { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{bar:"baz"}} - { magic_num:31, some_text:"somebody", byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{bar:"qux"}} + { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{bar:"baz"}, map: {a: 2, b: 5}} + { magic_num:31, some_text:"somebody", byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{bar:"qux"}, map: {foo: 12, bar: 50}} """; Page page = toPage(TEST_COLUMNS, row1, row2); From e045a5d4b058759a658120007bc2f14b882885ea Mon Sep 17 00:00:00 2001 From: Rob Marrowstone <99764876+rmarrowstone@users.noreply.github.com> Date: Mon, 16 Dec 2024 12:22:04 -0800 Subject: [PATCH 08/18] Change Handling of Some Mistypings and Oversized Values There are a few changes related to how mistyped or oversize Ion Values are Handled: * Enable Strict or Lax Path Typing This change adds a SerDe Property to support non-strict path typing. It defaults to false to mimic the behavior of the ion-hive-serde which used path extraction for the top-level-values, whether the user had defined any extractions or not. Without support for pathing this is effectively a type-check (or not) for TLVs. With support for extraction the behavior is a little more subtle than that, so I named the property for how it will behave. The name is also consistent with other properties. I also added some tests for nested mistypings and changed the code to consistently throw TrinoExceptions. * IonInts are coerced to Decimals (as Ion Hive does) * Decimals with more whole digits than fit is now an error (oddly this is not an error in Ion Hive) Fairly exhaustive test cases were added for each of the changes. * Ensure Timestamps are Rounded Consistently This change makes it so that Ion Timestamps are rounded using Timestamps.round(). This makes it so that Timestamp down casting works consistently with the other formats. I considered using DecodedTimestamp and the TrinoTimestampEncoders but those didn't cover picos. So this code uses the same pattern they do and the Timestamps.round(). This code simply ignores anything after picos. * Truncate Char and Varchar Columns With this change we truncate text that is longer than the length of the Char or Varchar column. Before this change that caused an error. Note that this is an error with the Hive Serde, but per the public Athena docs and the behavior of the other Hive formats, truncation is preferred. --- .../hive/formats/ion/IonDecoderFactory.java | 260 +++++++++++------- .../trino/hive/formats/FormatTestUtils.java | 7 +- .../trino/hive/formats/ion/TestIonFormat.java | 228 +++++++++++++-- .../plugin/hive/ion/IonPageSourceFactory.java | 3 +- .../plugin/hive/ion/IonReaderOptions.java | 30 ++ .../hive/ion/IonPageSourceSmokeTest.java | 80 ++++-- 6 files changed, 451 insertions(+), 157 deletions(-) create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java index 76087632273b..6ddfc58a64fd 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java @@ -16,12 +16,14 @@ import com.amazon.ion.IonException; import com.amazon.ion.IonReader; import com.amazon.ion.IonType; +import com.amazon.ion.Timestamp; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.slice.Slices; import io.trino.hive.formats.DistinctMapKeys; import io.trino.hive.formats.line.Column; -import io.trino.spi.PageBuilder; +import io.trino.spi.StandardErrorCode; +import io.trino.spi.TrinoException; import io.trino.spi.block.ArrayBlockBuilder; import io.trino.spi.block.BlockBuilder; import io.trino.spi.block.MapBlockBuilder; @@ -31,8 +33,10 @@ import io.trino.spi.type.BigintType; import io.trino.spi.type.BooleanType; import io.trino.spi.type.CharType; +import io.trino.spi.type.Chars; import io.trino.spi.type.DateType; import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Decimals; import io.trino.spi.type.DoubleType; import io.trino.spi.type.Int128; import io.trino.spi.type.IntegerType; @@ -42,14 +46,17 @@ import io.trino.spi.type.RowType; import io.trino.spi.type.SmallintType; import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Timestamps; import io.trino.spi.type.TinyintType; import io.trino.spi.type.Type; import io.trino.spi.type.VarbinaryType; import io.trino.spi.type.VarcharType; +import io.trino.spi.type.Varchars; import java.math.BigDecimal; +import java.math.BigInteger; import java.math.RoundingMode; -import java.time.ZoneId; +import java.time.LocalDate; import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -58,8 +65,6 @@ import java.util.Optional; import java.util.Set; import java.util.function.IntFunction; -import java.util.stream.Collectors; -import java.util.stream.IntStream; public class IonDecoderFactory { @@ -71,13 +76,37 @@ private IonDecoderFactory() {} * The decoder expects to decode the _current_ Ion Value. * It also expects that the calling code will manage the PageBuilder. *

+ * + * @param strictPathing controls behavior when encountering mistyped + * values during path extraction. That is outside (before), the trino + * type model. The ion-hive-serde used path extraction for navigating + * the top-level-values even if no path extractions were configured. + * So, in absence of support for path extraction configurations this + * still affects the handling of mistyped top-level-values. + * todo: revisit the above once path extraction config is supported. */ - public static IonDecoder buildDecoder(List columns) + public static IonDecoder buildDecoder(List columns, boolean strictPathing) { - return RowDecoder.forFields( + RowDecoder rowDecoder = RowDecoder.forFields( columns.stream() .map(c -> new RowType.Field(Optional.of(c.name()), c.type())) .toList()); + + return (ionReader, pageBuilder) -> { + IonType ionType = ionReader.getType(); + IntFunction blockSelector = pageBuilder::getBlockBuilder; + + if (ionType == IonType.STRUCT && !ionReader.isNullValue()) { + rowDecoder.decode(ionReader, blockSelector); + } + else if (ionType == IonType.STRUCT || ionType == IonType.NULL || !strictPathing) { + rowDecoder.appendNulls(blockSelector); + } + else { + throw new TrinoException(StandardErrorCode.GENERIC_USER_ERROR, + "Top-level-value of IonType %s is not valid with strict typing.".formatted(ionType)); + } + }; } private interface BlockDecoder @@ -88,22 +117,22 @@ private interface BlockDecoder private static BlockDecoder decoderForType(Type type) { return switch (type) { - case TinyintType _ -> wrapDecoder(byteDecoder, IonType.INT); - case SmallintType _ -> wrapDecoder(shortDecoder, IonType.INT); - case IntegerType _ -> wrapDecoder(intDecoder, IonType.INT); - case BigintType _ -> wrapDecoder(longDecoder, IonType.INT); - case RealType _ -> wrapDecoder(realDecoder, IonType.FLOAT); - case DoubleType _ -> wrapDecoder(floatDecoder, IonType.FLOAT); - case BooleanType _ -> wrapDecoder(boolDecoder, IonType.BOOL); - case DateType _ -> wrapDecoder(dateDecoder, IonType.TIMESTAMP); - case TimestampType t -> wrapDecoder(timestampDecoder(t), IonType.TIMESTAMP); - case DecimalType t -> wrapDecoder(decimalDecoder(t), IonType.DECIMAL); - case VarcharType _, CharType _ -> wrapDecoder(stringDecoder, IonType.STRING, IonType.SYMBOL); - case VarbinaryType _ -> wrapDecoder(binaryDecoder, IonType.BLOB, IonType.CLOB); - case MapType mapType -> wrapDecoder(new MapDecoder(mapType, decoderForType(mapType.getValueType())), - IonType.STRUCT); - case RowType r -> wrapDecoder(RowDecoder.forFields(r.getFields()), IonType.STRUCT); - case ArrayType a -> wrapDecoder(new ArrayDecoder(decoderForType(a.getElementType())), IonType.LIST, IonType.SEXP); + case TinyintType t -> wrapDecoder(byteDecoder, t, IonType.INT); + case SmallintType t -> wrapDecoder(shortDecoder, t, IonType.INT); + case IntegerType t -> wrapDecoder(intDecoder, t, IonType.INT); + case BigintType t -> wrapDecoder(longDecoder, t, IonType.INT); + case RealType t -> wrapDecoder(realDecoder, t, IonType.FLOAT); + case DoubleType t -> wrapDecoder(floatDecoder, t, IonType.FLOAT); + case DecimalType t -> wrapDecoder(decimalDecoder(t), t, IonType.DECIMAL, IonType.INT); + case BooleanType t -> wrapDecoder(boolDecoder, t, IonType.BOOL); + case DateType t -> wrapDecoder(dateDecoder, t, IonType.TIMESTAMP); + case TimestampType t -> wrapDecoder(timestampDecoder(t), t, IonType.TIMESTAMP); + case VarcharType t -> wrapDecoder(varcharDecoder(t), t, IonType.STRING, IonType.SYMBOL); + case CharType t -> wrapDecoder(charDecoder(t), t, IonType.STRING, IonType.SYMBOL); + case VarbinaryType t -> wrapDecoder(binaryDecoder, t, IonType.BLOB, IonType.CLOB); + case RowType t -> wrapDecoder(RowDecoder.forFields(t.getFields()), t, IonType.STRUCT); + case ArrayType t -> wrapDecoder(new ArrayDecoder(decoderForType(t.getElementType())), t, IonType.LIST, IonType.SEXP); + case MapType t -> wrapDecoder(new MapDecoder(t, decoderForType(t.getValueType())), t, IonType.STRUCT); default -> throw new IllegalArgumentException(String.format("Unsupported type: %s", type)); }; } @@ -117,16 +146,16 @@ private static BlockDecoder decoderForType(Type type) *

* This code treats all values as nullable. */ - private static BlockDecoder wrapDecoder(BlockDecoder decoder, IonType... allowedTypes) + private static BlockDecoder wrapDecoder(BlockDecoder decoder, Type trinoType, IonType... allowedTypes) { - final Set allowedWithNull = new HashSet<>(Arrays.asList(allowedTypes)); + Set allowedWithNull = new HashSet<>(Arrays.asList(allowedTypes)); allowedWithNull.add(IonType.NULL); return (reader, builder) -> { - final IonType type = reader.getType(); - if (!allowedWithNull.contains(type)) { - final String expected = allowedWithNull.stream().map(IonType::name).collect(Collectors.joining(", ")); - throw new IonException(String.format("Encountered value with IonType: %s, required one of %s ", type, expected)); + final IonType ionType = reader.getType(); + if (!allowedWithNull.contains(ionType)) { + throw new TrinoException(StandardErrorCode.GENERIC_USER_ERROR, + "Cannot coerce IonType %s to Trino type %s".formatted(ionType, trinoType)); } if (reader.isNullValue()) { builder.appendNull(); @@ -138,39 +167,22 @@ private static BlockDecoder wrapDecoder(BlockDecoder decoder, IonType... allowed } /** - * Class is both the Top-Level-Value Decoder and the Row Decoder for nested - * structs. + * The RowDecoder is used as the BlockDecoder for nested RowTypes and is used for decoding + * top-level structs into pages. */ private record RowDecoder(Map fieldPositions, List fieldDecoders) - implements IonDecoder, BlockDecoder + implements BlockDecoder { private static RowDecoder forFields(List fields) { ImmutableList.Builder decoderBuilder = ImmutableList.builder(); ImmutableMap.Builder fieldPositionBuilder = ImmutableMap.builder(); - IntStream.range(0, fields.size()) - .forEach(position -> { - RowType.Field field = fields.get(position); - decoderBuilder.add(decoderForType(field.getType())); - fieldPositionBuilder.put(field.getName().get().toLowerCase(Locale.ROOT), position); - }); - - return new RowDecoder(fieldPositionBuilder.buildOrThrow(), decoderBuilder.build()); - } - - @Override - public void decode(IonReader ionReader, PageBuilder pageBuilder) - { - // todo: we could also map an Ion List to a Struct - if (ionReader.getType() != IonType.STRUCT) { - throw new IonException("RowType must be Structs! Encountered: " + ionReader.getType()); - } - if (ionReader.isNullValue()) { - // todo: is this an error or just a null value? - // i think in the hive serde it's a null record. - throw new IonException("Top Level Values must not be null!"); + for (int pos = 0; pos < fields.size(); pos++) { + RowType.Field field = fields.get(pos); + decoderBuilder.add(decoderForType(field.getType())); + fieldPositionBuilder.put(field.getName().get().toLowerCase(Locale.ROOT), pos); } - decode(ionReader, pageBuilder::getBlockBuilder); + return new RowDecoder(fieldPositionBuilder.buildOrThrow(), decoderBuilder.build()); } @Override @@ -187,7 +199,6 @@ private void decode(IonReader ionReader, IntFunction blockSelector ionReader.stepIn(); while (ionReader.next() != null) { - // todo: case insensitivity final Integer fieldIndex = fieldPositions.get(ionReader.getFieldName().toLowerCase(Locale.ROOT)); if (fieldIndex == null) { continue; @@ -210,6 +221,13 @@ private void decode(IonReader ionReader, IntFunction blockSelector ionReader.stepOut(); } + + private void appendNulls(IntFunction blockSelector) + { + for (int i = 0; i < fieldDecoders.size(); i++) { + blockSelector.apply(i).appendNull(); + } + } } private static class MapDecoder @@ -284,82 +302,120 @@ public void decode(IonReader ionReader, BlockBuilder blockBuilder) private static BlockDecoder timestampDecoder(TimestampType type) { - // todo: no attempt is made at handling offsets or lack thereof - if (type.isShort()) { - return (reader, builder) -> { - long micros = reader.timestampValue().getDecimalMillis() - .setScale(type.getPrecision() - 3, RoundingMode.HALF_EVEN) - .movePointRight(3) - .longValue(); - type.writeLong(builder, micros); - }; - } - else { - return (reader, builder) -> { - BigDecimal decimalMicros = reader.timestampValue().getDecimalMillis() - .movePointRight(3); - BigDecimal subMicrosFrac = decimalMicros.remainder(BigDecimal.ONE) - .movePointRight(6); - type.writeObject(builder, new LongTimestamp(decimalMicros.longValue(), subMicrosFrac.intValue())); - }; - } + // Ion supports arbitrarily precise Timestamps. + // Other Hive formats are using the DecodedTimestamp and TrinoTimestampEncoders in + // io.trino.plugin.base.type but those don't cover picos. + // This code uses same pattern of splitting the parsed timestamp into (seconds, fraction) + // then rounding the fraction using Timestamps.round() ensures consistency with the others + // while capturing picos if present. Fractional precision beyond picos is ignored. + return (reader, builder) -> { + BigDecimal decimalSeconds = reader.timestampValue() + .getDecimalMillis() + .movePointLeft(3); + BigDecimal decimalPicos = decimalSeconds.remainder(BigDecimal.ONE) + .movePointRight(12); + + long fractionalPicos = Timestamps.round(decimalPicos.longValue(), 12 - type.getPrecision()); + long epochMicros = decimalSeconds.longValue() * Timestamps.MICROSECONDS_PER_SECOND + + fractionalPicos / Timestamps.PICOSECONDS_PER_MICROSECOND; + + if (type.isShort()) { + type.writeLong(builder, epochMicros); + } + else { + type.writeObject(builder, + new LongTimestamp(epochMicros, (int) (fractionalPicos % Timestamps.PICOSECONDS_PER_MICROSECOND))); + } + }; } private static BlockDecoder decimalDecoder(DecimalType type) { - if (type.isShort()) { - return (reader, builder) -> { - long unscaled = reader.bigDecimalValue() - .setScale(type.getScale(), RoundingMode.UNNECESSARY) - .unscaledValue() - .longValue(); - type.writeLong(builder, unscaled); - }; - } - else { - return (reader, builder) -> { - Int128 unscaled = Int128.valueOf(reader.bigDecimalValue() - .setScale(type.getScale(), RoundingMode.UNNECESSARY) - .unscaledValue()); - type.writeObject(builder, unscaled); - }; - } + int precision = type.getPrecision(); + int scale = type.getScale(); + + return (reader, builder) -> { + try { + BigDecimal decimal = reader.bigDecimalValue(); + BigInteger unscaled = decimal + .setScale(scale, RoundingMode.UNNECESSARY) + .unscaledValue(); + + if (Decimals.overflows(unscaled, precision)) { + throw new TrinoException(StandardErrorCode.NUMERIC_VALUE_OUT_OF_RANGE, + "Decimal value %s does not fit %d digits of precision and %d of scale!" + .formatted(decimal, precision, scale)); + } + if (type.isShort()) { + type.writeLong(builder, unscaled.longValue()); + } + else { + type.writeObject(builder, Int128.valueOf(unscaled)); + } + } + catch (ArithmeticException e) { + throw new TrinoException(StandardErrorCode.NUMERIC_VALUE_OUT_OF_RANGE, + "Decimal value %s does not fit %d digits of scale!".formatted(reader.bigDecimalValue(), scale)); + } + }; + } + + private static BlockDecoder varcharDecoder(VarcharType type) + { + return (ionReader, blockBuilder) -> + type.writeSlice(blockBuilder, Varchars.truncateToLength(Slices.utf8Slice(ionReader.stringValue()), type)); + } + + private static BlockDecoder charDecoder(CharType type) + { + return (ionReader, blockBuilder) -> + type.writeSlice(blockBuilder, Chars.truncateToLengthAndTrimSpaces(Slices.utf8Slice(ionReader.stringValue()), type)); } private static final BlockDecoder byteDecoder = (ionReader, blockBuilder) -> - TinyintType.TINYINT.writeLong(blockBuilder, ionReader.longValue()); + TinyintType.TINYINT.writeLong(blockBuilder, readLong(ionReader)); private static final BlockDecoder shortDecoder = (ionReader, blockBuilder) -> - SmallintType.SMALLINT.writeLong(blockBuilder, ionReader.longValue()); + SmallintType.SMALLINT.writeLong(blockBuilder, readLong(ionReader)); private static final BlockDecoder intDecoder = (ionReader, blockBuilder) -> - IntegerType.INTEGER.writeLong(blockBuilder, ionReader.longValue()); + IntegerType.INTEGER.writeLong(blockBuilder, readLong(ionReader)); private static final BlockDecoder longDecoder = (ionReader, blockBuilder) -> - BigintType.BIGINT.writeLong(blockBuilder, ionReader.longValue()); + BigintType.BIGINT.writeLong(blockBuilder, readLong(ionReader)); + + private static long readLong(IonReader ionReader) + { + try { + return ionReader.longValue(); + } + catch (IonException e) { + throw new TrinoException(StandardErrorCode.GENERIC_USER_ERROR, e.getMessage()); + } + } private static final BlockDecoder realDecoder = (ionReader, blockBuilder) -> { double readValue = ionReader.doubleValue(); if (readValue == (float) readValue) { - RealType.REAL.writeFloat(blockBuilder, (float) ionReader.doubleValue()); + RealType.REAL.writeFloat(blockBuilder, (float) readValue); } else { - // todo: some kind of "permissive truncate" flag - throw new IllegalArgumentException("Won't truncate double precise float to real!"); + throw new TrinoException(StandardErrorCode.GENERIC_USER_ERROR, + "Won't truncate double precise float to real!"); } }; private static final BlockDecoder floatDecoder = (ionReader, blockBuilder) -> DoubleType.DOUBLE.writeDouble(blockBuilder, ionReader.doubleValue()); - private static final BlockDecoder stringDecoder = (ionReader, blockBuilder) -> - VarcharType.VARCHAR.writeSlice(blockBuilder, Slices.utf8Slice(ionReader.stringValue())); - private static final BlockDecoder boolDecoder = (ionReader, blockBuilder) -> BooleanType.BOOLEAN.writeBoolean(blockBuilder, ionReader.booleanValue()); - private static final BlockDecoder dateDecoder = (ionReader, blockBuilder) -> - DateType.DATE.writeLong(blockBuilder, ionReader.timestampValue().dateValue().toInstant().atZone(ZoneId.of("UTC")).toLocalDate().toEpochDay()); + private static final BlockDecoder dateDecoder = (ionReader, blockBuilder) -> { + Timestamp ionTs = ionReader.timestampValue(); + LocalDate localDate = LocalDate.of(ionTs.getZYear(), ionTs.getZMonth(), ionTs.getZDay()); + DateType.DATE.writeLong(blockBuilder, localDate.toEpochDay()); + }; private static final BlockDecoder binaryDecoder = (ionReader, blockBuilder) -> VarbinaryType.VARBINARY.writeSlice(blockBuilder, Slices.wrappedBuffer(ionReader.newBytes())); diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java index cc1e912d9bb4..fa8454e04f87 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/FormatTestUtils.java @@ -665,6 +665,11 @@ public static Object toHiveWriteValue(Type type, Object value, Optional TEST_COLUMNS = List.of( new Column("magic_num", INTEGER, 0), new Column("some_text", VARCHAR, 1), - new Column("is_summer", BooleanType.BOOLEAN, 2), - new Column("byte_clob", VarbinaryType.VARBINARY, 3), + new Column("is_summer", BOOLEAN, 2), + new Column("byte_clob", VARBINARY, 3), new Column("sequencer", new ArrayType(INTEGER), 4), new Column("struction", RowType.rowType( field("foo", INTEGER), @@ -96,6 +103,77 @@ public void testMap() .buildOrThrow())); } + @Test + public void testVariousTlvsStrict() + throws IOException + { + RowType rowType = RowType.rowType(field("foo", INTEGER), field("bar", VARCHAR)); + List expected = new ArrayList<>(2); + expected.add(null); + expected.add(null); + + assertValues(rowType, + // empty struct, untyped null, struct null, and explicitly typed null null, phew. + "{} null null.struct null.null", + expected, expected, expected, expected); + + Assertions.assertThrows(TrinoException.class, () -> { + assertValues(rowType, "null.int", expected); + assertValues(rowType, "[]", expected); + }); + } + + @Test + public void testVariousTlvsLax() + throws IOException + { + RowType rowType = RowType.rowType(field("foo", INTEGER), field("bar", VARCHAR)); + List expected = new ArrayList<>(2); + expected.add(null); + expected.add(null); + + assertValues(rowType, + false, + "{} 37 null.list null.struct null spam false", + expected, expected, expected, expected, expected, expected, expected); + } + + @Test + public void testColumnMistypings() + { + RowType rowType = RowType.rowType(field("foo", INTEGER), field("bar", BOOLEAN)); + + List ions = List.of( + "{ foo: blarg, bar: false }", + "{ foo: 12345, bar: blarg }", + "{ foo: null.list, bar: false }", + "{ foo: 12345, bar: null.int }"); + + for (String ion : ions) { + Assertions.assertThrows(TrinoException.class, () -> { + assertValues(rowType, ion, List.of()); + }); + } + } + + @Test + public void testTextTruncation() + throws IOException + { + String ionText = """ + { my_text: 'abcdefghijk' } + { my_text: 'abcd ' } + { my_text: 'abcd ijk' }"""; + + assertValues(RowType.rowType(field("my_text", VarcharType.createVarcharType(8))), + ionText, + List.of("abcdefgh"), List.of("abcd "), List.of("abcd ")); + + assertValues(RowType.rowType(field("my_text", CharType.createCharType(8))), + ionText, + List.of("abcdefgh"), List.of("abcd "), List.of("abcd ")); + } + @Test public void testCaseInsensitivityOfKeys() throws IOException @@ -152,8 +230,6 @@ public void testStructWithDuplicateKeys() List.of(53), List.of(67)); } - // todo: test for mistyped null and non-null values - @Test public void testNestedList() throws IOException @@ -195,22 +271,86 @@ public void testStructInList() } @Test - public void testPicoPreciseTimestamp() + public void testIonIntTooLargeForLong() + throws IOException + { + Assertions.assertThrows(TrinoException.class, () -> { + assertValues(RowType.rowType(field("my_bigint", BIGINT)), + "{ my_bigint: 18446744073709551786 }", + List.of()); + }); + } + + @Test + public void testDoubleAsFloat() throws IOException { - Timestamp ionTimestamp = Timestamp.forSecond(2067, 8, 9, 11, 22, new BigDecimal("33.445566"), 0); - long epochMicros = ionTimestamp.getDecimalMillis().movePointRight(3).longValue(); + RowType rowType = RowType.rowType(field("my_float", RealType.REAL)); assertValues( - RowType.rowType(field("my_ts", TimestampType.TIMESTAMP_PICOS)), - "{ my_ts: 2067-08-09T11:22:33.445566778899Z }", - List.of(SqlTimestamp.newInstance(12, epochMicros, 778899))); + rowType, + "{ my_float: 625e-3 }", + List.of(.625f)); + + Assertions.assertThrows(TrinoException.class, () -> { + assertValues( + rowType, + "{ my_float: 9e+99 }", + List.of()); + }); + } + + @Test + public void testDateDecoding() + throws IOException + { + RowType rowType = RowType.rowType(field("my_date", DateType.DATE)); + SqlDate expected = new SqlDate((int) LocalDate.of(2022, 2, 22).toEpochDay()); + + List ions = List.of( + "{ my_date: 2022-02-22T }", + "{ my_date: 2022-02-21T12:00-12:00 } ", + "{ my_date: 2022-02-22T22:22:22Z }", + "{ my_date: 2022-02-23T00:00+01:00 }", + "{ my_date: 2022-02-22T00:01Z }", + "{ my_date: 2022-02-22T00:00:01Z }", + "{ my_date: 2022-02-22T00:00:00.001Z }", + "{ my_date: 2022-02-22T23:59:59.999999999Z }"); + + for (String ion : ions) { + assertValues(rowType, ion, List.of(expected)); + } } @Test - public void testOverPreciseTimestamps() - throws IonException + public void testTimestampDecoding() + throws IOException { - // todo: implement + List ions = List.of( + "{ my_ts: 2067-08-09T11:22:33Z }", + "{ my_ts: 2067-08-09T11:22:33.111Z }", + "{ my_ts: 2067-08-09T11:22:33.111222Z }", + "{ my_ts: 2067-08-09T11:22:33.111222333Z }", + "{ my_ts: 2067-08-09T11:22:33.111222333444Z }", + // fraction beyond picos is truncated + "{ my_ts: 2067-08-09T11:22:33.111222333444555Z }"); + + LocalDateTime dateTimeToSeconds = LocalDateTime.of(2067, 8, 9, 11, 22, 33); + List sqlTimestamps = List.of( + toSqlTimestamp(TimestampType.TIMESTAMP_SECONDS, dateTimeToSeconds), + toSqlTimestamp(TimestampType.TIMESTAMP_MILLIS, dateTimeToSeconds.plusNanos(111000000)), + toSqlTimestamp(TimestampType.TIMESTAMP_MICROS, dateTimeToSeconds.plusNanos(111222000)), + toSqlTimestamp(TimestampType.TIMESTAMP_NANOS, dateTimeToSeconds.plusNanos(111222333)), + toSqlTimestamp(TimestampType.TIMESTAMP_PICOS, dateTimeToSeconds.plusNanos(111222333), 444)); + + for (int i = 0; i < sqlTimestamps.size(); i++) { + SqlTimestamp sqlTimestamp = sqlTimestamps.get(i); + RowType rowType = RowType.rowType( + field("my_ts", TimestampType.createTimestampType(sqlTimestamp.getPrecision()))); + + for (int j = i; j < ions.size(); j++) { + assertValues(rowType, ions.get(j), List.of(sqlTimestamp)); + } + } } @Test @@ -224,17 +364,52 @@ public void testDecimalPrecisionAndScale() "{ amount: 1234.00, big_amount: 1234.00000 }" + "{ amount: 1234d0, big_amount: 1234d0 }" + "{ amount: 12d2, big_amount: 12d2 }" - + "{ amount: 1234.000, big_amount: 1234.000000 }", + + "{ amount: 1234.000, big_amount: 1234.000000 }" + + "{ amount: 1234, big_amount: 1234 }", // these are both IonInts List.of(new SqlDecimal(BigInteger.valueOf(123400), 10, 2), new SqlDecimal(BigInteger.valueOf(123400000), 25, 5)), List.of(new SqlDecimal(BigInteger.valueOf(123400), 10, 2), new SqlDecimal(BigInteger.valueOf(123400000), 25, 5)), List.of(new SqlDecimal(BigInteger.valueOf(120000), 10, 2), new SqlDecimal(BigInteger.valueOf(120000000), 25, 5)), + List.of(new SqlDecimal(BigInteger.valueOf(123400), 10, 2), new SqlDecimal(BigInteger.valueOf(123400000), 25, 5)), List.of(new SqlDecimal(BigInteger.valueOf(123400), 10, 2), new SqlDecimal(BigInteger.valueOf(123400000), 25, 5))); } @Test - public void testOversizeOrOverpreciseDecimals() + public void testNumbersTooBigForShortDecimal() + { + RowType rowType = RowType.rowType( + field("amount", DecimalType.createDecimalType(4, 2))); + + List ions = List.of( + "{ amount: 123.4 }", + "{ amount: 1.234 }", + "{ amount: 123 }", + "{ amount: 1234d-10 }", + "{ amount: 1234d2 }"); + + for (String ionText : ions) { + Assertions.assertThrows(TrinoException.class, () -> + assertValues(rowType, ionText, List.of())); + } + } + + @Test + public void testNumbersTooBigForDecimal128() { - // todo: implement + RowType rowType = RowType.rowType( + field("amount", DecimalType.createDecimalType(20, 2))); + + List ions = List.of( + "{ amount: 12345678901234567890.4 }", + "{ amount: 1.234 }", + "{ amount: 12345678901234567890 }", + "{ amount: 999999999999999999999999999999999999.999 }", // 39 nines + "{ amount: 1234d-10 }", + "{ amount: 1234d22 }"); + + for (String ionText : ions) { + Assertions.assertThrows(TrinoException.class, () -> + assertValues(rowType, ionText, List.of())); + } } @Test @@ -282,7 +457,8 @@ public void testEncodeWithNullNestedField() .put("a", 2) .put("b", 5) .buildOrThrow()); - List row2 = Arrays.asList(31, "somebody", null, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), Arrays.asList(null, "qux"), ImmutableMap.builder() + List row2 = Arrays.asList(31, "somebody", null, new SqlVarbinary(new byte[] {(byte) 0x01, + (byte) 0xaa}), List.of(7, 8, 9), Arrays.asList(null, "qux"), ImmutableMap.builder() .put("foo", 12) .put("bar", 50) .buildOrThrow()); @@ -297,6 +473,12 @@ public void testEncodeWithNullNestedField() private void assertValues(RowType rowType, String ionText, List... expected) throws IOException + { + assertValues(rowType, true, ionText, expected); + } + + private void assertValues(RowType rowType, Boolean strictTlvs, String ionText, List... expected) + throws IOException { List fields = rowType.getFields(); List columns = IntStream.range(0, fields.size()) @@ -306,7 +488,7 @@ private void assertValues(RowType rowType, String ionText, List... expec return new Column(field.getName().get(), field.getType(), i); }) .toList(); - IonDecoder decoder = IonDecoderFactory.buildDecoder(columns); + IonDecoder decoder = IonDecoderFactory.buildDecoder(columns, strictTlvs); PageBuilder pageBuilder = new PageBuilder(expected.length, rowType.getFields().stream().map(RowType.Field::getType).toList()); try (IonReader ionReader = IonReaderBuilder.standard().build(ionText)) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java index f8ddd9d7ee9b..887505dd3ace 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java @@ -137,7 +137,8 @@ public Optional createPageSource( List decoderColumns = projectedReaderColumns.stream() .map(hc -> new Column(hc.getName(), hc.getType(), hc.getBaseHiveColumnIndex())) .toList(); - IonDecoder decoder = IonDecoderFactory.buildDecoder(decoderColumns); + boolean strictPathing = IonReaderOptions.useStrictPathTyping(schema.serdeProperties()); + IonDecoder decoder = IonDecoderFactory.buildDecoder(decoderColumns, strictPathing); IonPageSource pageSource = new IonPageSource(ionReader, countingInputStream::getCount, decoder, pageBuilder); return Optional.of(new ReaderPageSource(pageSource, readerProjections)); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java new file mode 100644 index 000000000000..3c1cc33150b3 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java @@ -0,0 +1,30 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.ion; + +import java.util.Map; + +public final class IonReaderOptions +{ + public static final String STRICT_PATH_TYPING_PROPERTY = "ion.path_extractor.strict"; + public static final String STRICT_PATH_TYPING_DEFAULT = "false"; + + private IonReaderOptions() {} + + static boolean useStrictPathTyping(Map propertiesMap) + { + return Boolean.parseBoolean( + propertiesMap.getOrDefault(STRICT_PATH_TYPING_PROPERTY, STRICT_PATH_TYPING_DEFAULT)); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index b4660c47e305..670426082c3a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -30,6 +30,7 @@ import io.trino.plugin.hive.Schema; import io.trino.plugin.hive.WriterKind; import io.trino.spi.Page; +import io.trino.spi.TrinoException; import io.trino.spi.block.IntArrayBlock; import io.trino.spi.block.RunLengthEncodedBlock; import io.trino.spi.connector.ConnectorPageSource; @@ -42,7 +43,6 @@ import org.junit.jupiter.api.Test; import java.io.IOException; -import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.time.Instant; import java.util.Arrays; @@ -82,18 +82,17 @@ public class IonPageSourceSmokeTest private static final String EXPECTED_TEXT = "{foo:3,bar:6}"; public static final String TEST_ION_LOCATION = "memory:///test.ion"; + public static final List FOO_BAR_COLUMNS = List.of( + toHiveBaseColumnHandle("foo", INTEGER, 0), + toHiveBaseColumnHandle("bar", VARCHAR, 1)); @Test public void testReadTwoValues() throws IOException { - List tableColumns = List.of( - toHiveBaseColumnHandle("foo", INTEGER, 0), - toHiveBaseColumnHandle("bar", VARCHAR, 1)); - assertRowCount( - tableColumns, - tableColumns, + FOO_BAR_COLUMNS, + FOO_BAR_COLUMNS, "{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }", 2); } @@ -112,6 +111,23 @@ public void testReadArray() 2); } + @Test + public void testStrictAndLaxPathTyping() + throws IOException + { + TestFixture defaultFixture = new TestFixture(FOO_BAR_COLUMNS); + defaultFixture.assertRowCount("37 null.timestamp", 2); + + TestFixture laxFixture = new TestFixture(FOO_BAR_COLUMNS); + laxFixture.withStrictPathTyping("false"); + laxFixture.assertRowCount("37 null.timestamp", 2); + + TestFixture strictFixture = new TestFixture(FOO_BAR_COLUMNS); + strictFixture.withStrictPathTyping("true"); + Assertions.assertThrows(TrinoException.class, () -> + strictFixture.assertRowCount("37 null.timestamp", 2)); + } + @Test public void testProjectedColumn() throws IOException @@ -137,11 +153,7 @@ public void testProjectedColumn() public void testPageSourceWithNativeTrinoDisabled() throws IOException { - List tableColumns = List.of( - toHiveBaseColumnHandle("foo", INTEGER, 0), - toHiveBaseColumnHandle("bar", VARCHAR, 1)); - - TestFixture fixture = new TestFixture(tableColumns) + TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) .withNativeIonDisabled(); fixture.writeIonTextFile("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }"); @@ -196,15 +208,7 @@ private void assertRowCount(List tableColumns, List getOptionalPageSource() throws IOException { @@ -313,15 +323,11 @@ ConnectorSession getSession() int writeIonTextFile(String ionText) throws IOException { - final TrinoOutputFile outputFile = fileSystemFactory.create(getSession()).newOutputFile(fileLocation); - int written = 0; - try (OutputStream outputStream = outputFile.create()) { - byte[] bytes = ionText.getBytes(StandardCharsets.UTF_8); - outputStream.write(bytes); - outputStream.flush(); - written = bytes.length; - } - return written; + TrinoOutputFile outputFile = fileSystemFactory.create(getSession()).newOutputFile(fileLocation); + byte[] bytes = ionText.getBytes(StandardCharsets.UTF_8); + outputFile.createOrOverwrite(bytes); + + return bytes.length; } FileWriter getFileWriter() @@ -346,5 +352,19 @@ TrinoInputFile getTrinoInputFile() return fileSystemFactory.create(getSession()) .newInputFile(fileLocation); } + + void assertRowCount(String ionText, int rowCount) + throws IOException + { + writeIonTextFile(ionText); + + try (ConnectorPageSource pageSource = getPageSource()) { + final MaterializedResult result = MaterializedResult.materializeSourceDataStream( + getSession(), + pageSource, + projections.stream().map(HiveColumnHandle::getType).toList()); + Assertions.assertEquals(rowCount, result.getRowCount()); + } + } } } From d09a6e9516b90e9f8acb829b19c72ecd548ba957 Mon Sep 17 00:00:00 2001 From: linlin-s Date: Wed, 18 Dec 2024 11:21:58 -0800 Subject: [PATCH 09/18] Return empty PageSource when there's unsupported Serde property --- .../plugin/hive/ion/IonPageSourceFactory.java | 51 +++++++++++++ .../plugin/hive/ion/IonReaderOptions.java | 8 ++ .../plugin/hive/ion/IonWriterOptions.java | 6 ++ .../hive/ion/IonPageSourceSmokeTest.java | 75 +++++++++++++++++++ 4 files changed, 140 insertions(+) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java index 887505dd3ace..2bc98396f76e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java @@ -15,6 +15,8 @@ import com.amazon.ion.IonReader; import com.amazon.ion.system.IonReaderBuilder; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.io.CountingInputStream; import com.google.inject.Inject; import io.trino.filesystem.Location; @@ -43,8 +45,11 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.OptionalInt; +import java.util.Set; +import java.util.regex.Pattern; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -52,6 +57,19 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT; import static io.trino.plugin.hive.HivePageSourceProvider.projectBaseColumns; import static io.trino.plugin.hive.ReaderPageSource.noProjectionAdaptation; +import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY; +import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY_DEFAULT; +import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN; +import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED; +import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED_DEFAULT; +import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTION_CASE_SENSITIVITY; +import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT; +import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTOR_PROPERTY; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_DEFAULT; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_PROPERTY; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_PROPERTY; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_TIMESTAMP_OFFSET_DEFAULT; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_TIMESTAMP_OFFSET_PROPERTY; import static io.trino.plugin.hive.util.HiveUtil.splitError; public class IonPageSourceFactory @@ -61,6 +79,18 @@ public class IonPageSourceFactory // this is used as a feature flag to enable Ion native trino integration private final boolean nativeTrinoEnabled; + private static final Map TABLE_PROPERTIES = ImmutableMap.of( + FAIL_ON_OVERFLOW_PROPERTY, FAIL_ON_OVERFLOW_PROPERTY_DEFAULT, + IGNORE_MALFORMED, IGNORE_MALFORMED_DEFAULT, + PATH_EXTRACTION_CASE_SENSITIVITY, PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT, + ION_TIMESTAMP_OFFSET_PROPERTY, ION_TIMESTAMP_OFFSET_DEFAULT, + ION_SERIALIZATION_AS_NULL_PROPERTY, ION_SERIALIZATION_AS_NULL_DEFAULT); + + private static final Set COLUMN_PROPERTIES = ImmutableSet.of( + Pattern.compile(FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN), + Pattern.compile(ION_SERIALIZATION_AS_PROPERTY), + Pattern.compile(PATH_EXTRACTOR_PROPERTY)); + @Inject public IonPageSourceFactory(TrinoFileSystemFactory trinoFileSystemFactory, HiveConfig hiveConfig) { @@ -89,6 +119,11 @@ public Optional createPageSource( // on their use case return Optional.empty(); } + + if (schema.serdeProperties().entrySet().stream().filter(entry -> entry.getKey().startsWith("ion.")).anyMatch(this::isUnsupportedProperty)) { + return Optional.empty(); + } + if (!ION_SERDE_CLASS.equals(schema.serializationLibraryName())) { return Optional.empty(); } @@ -147,4 +182,20 @@ public Optional createPageSource( throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, splitError(e, path, start, length), e); } } + + private boolean isUnsupportedProperty(Map.Entry entry) + { + String key = entry.getKey(); + String value = entry.getValue(); + + String propertyDefault = TABLE_PROPERTIES.get(key); + if (propertyDefault != null) { + return !propertyDefault.equals(value); + } + + // For now, any column-specific properties result in an empty PageSource + // since they have no default values for comparison. + return COLUMN_PROPERTIES.stream() + .anyMatch(pattern -> pattern.matcher(key).matches()); + } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java index 3c1cc33150b3..5c510c2a5d09 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java @@ -19,6 +19,14 @@ public final class IonReaderOptions { public static final String STRICT_PATH_TYPING_PROPERTY = "ion.path_extractor.strict"; public static final String STRICT_PATH_TYPING_DEFAULT = "false"; + public static final String PATH_EXTRACTOR_PROPERTY = "ion.\\w+.path_extractor"; + public static final String PATH_EXTRACTION_CASE_SENSITIVITY = "ion.path_extractor.case_sensitive"; + public static final String PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT = "false"; + public static final String FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN = "ion.\\w+.fail_on_overflow"; + public static final String FAIL_ON_OVERFLOW_PROPERTY = "ion.fail_on_overflow"; + public static final String FAIL_ON_OVERFLOW_PROPERTY_DEFAULT = "true"; + public static final String IGNORE_MALFORMED = "ion.ignore_malformed"; + public static final String IGNORE_MALFORMED_DEFAULT = "false"; private IonReaderOptions() {} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java index 41a20df51cb2..b69bb992b16d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java @@ -27,6 +27,12 @@ public final class IonWriterOptions { public static final String ION_ENCODING_PROPERTY = "ion.encoding"; + public static final String ION_TIMESTAMP_OFFSET_PROPERTY = "ion.timestamp.serialization_offset"; + public static final String ION_TIMESTAMP_OFFSET_DEFAULT = "Z"; + public static final String ION_SERIALIZATION_AS_NULL_PROPERTY = "ion.serialize_null"; + public static final String ION_SERIALIZATION_AS_NULL_DEFAULT = "OMIT"; + public static final String ION_SERIALIZATION_AS_PROPERTY = "ion.\\w+.serialize_as"; + public static final String TEXT_ENCODING = "text"; public static final String BINARY_ENCODING = "binary"; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index 670426082c3a..834f64fd81cb 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -41,10 +41,13 @@ import io.trino.testing.MaterializedResult; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.time.Instant; +import java.util.AbstractMap; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -52,6 +55,7 @@ import java.util.Optional; import java.util.OptionalInt; import java.util.stream.Collectors; +import java.util.stream.Stream; import static io.trino.plugin.hive.HivePageSourceProvider.ColumnMapping.buildColumnMappings; import static io.trino.plugin.hive.HiveStorageFormat.ION; @@ -59,8 +63,18 @@ import static io.trino.plugin.hive.HiveTestUtils.projectedColumn; import static io.trino.plugin.hive.HiveTestUtils.toHiveBaseColumnHandle; import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; +import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY; +import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY_DEFAULT; +import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED; +import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED_DEFAULT; +import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTION_CASE_SENSITIVITY; +import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT; import static io.trino.plugin.hive.ion.IonWriterOptions.BINARY_ENCODING; import static io.trino.plugin.hive.ion.IonWriterOptions.ION_ENCODING_PROPERTY; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_DEFAULT; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_PROPERTY; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_TIMESTAMP_OFFSET_DEFAULT; +import static io.trino.plugin.hive.ion.IonWriterOptions.ION_TIMESTAMP_OFFSET_PROPERTY; import static io.trino.plugin.hive.ion.IonWriterOptions.TEXT_ENCODING; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; @@ -161,6 +175,60 @@ public void testPageSourceWithNativeTrinoDisabled() Assertions.assertTrue(connectorPageSource.isEmpty(), "Expected empty page source when native Trino is disabled"); } + private static Stream> propertiesWithDefaults() + { + return Stream.of( + entry(FAIL_ON_OVERFLOW_PROPERTY, FAIL_ON_OVERFLOW_PROPERTY_DEFAULT), + entry(PATH_EXTRACTION_CASE_SENSITIVITY, PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT), + entry(IGNORE_MALFORMED, IGNORE_MALFORMED_DEFAULT), + entry(ION_TIMESTAMP_OFFSET_PROPERTY, ION_TIMESTAMP_OFFSET_DEFAULT), + entry(ION_SERIALIZATION_AS_NULL_PROPERTY, ION_SERIALIZATION_AS_NULL_DEFAULT)); + } + + private static Stream> propertiesWithValues() + { + return Stream.of( + entry(FAIL_ON_OVERFLOW_PROPERTY, "false"), + entry(PATH_EXTRACTION_CASE_SENSITIVITY, "true"), + entry(IGNORE_MALFORMED, "true"), + entry(ION_TIMESTAMP_OFFSET_PROPERTY, "01:00"), + entry(ION_SERIALIZATION_AS_NULL_PROPERTY, "TYPED"), + // These entries represent column-specific properties that are not supported. + // Any presence of these properties in the schema will result in an empty PageSource, + // regardless of their assigned values. + entry("ion.foo.fail_on_overflow", "property_value"), + entry("ion.foo.serialize_as", "property_value"), + entry("ion.foo.path_extractor", "property_value")); + } + + private static Map.Entry entry(String key, String value) + { + return new AbstractMap.SimpleEntry<>(key, value); + } + + @ParameterizedTest + @MethodSource("propertiesWithValues") + void testPropertiesWithValues(Map.Entry property) + throws IOException + { + TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) + .withSerdeProperties(property); + fixture.writeIonTextFile("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }"); + + Optional connectorPageSource = fixture.getOptionalPageSource(); + Assertions.assertTrue(connectorPageSource.isEmpty(), "Expected empty page source when there are unsupported Serde properties"); + } + + @ParameterizedTest + @MethodSource("propertiesWithDefaults") + void testPropertiesWithDefaults(Map.Entry propertyEntry) + throws IOException + { + TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) + .withSerdeProperties(propertyEntry); + fixture.assertRowCount("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }", 2); + } + @Test public void testTextEncoding() throws IOException @@ -267,6 +335,13 @@ TestFixture withStrictPathTyping(String strict) return this; } + TestFixture withSerdeProperties(Map.Entry propertyEntry) + { + // The value of the property is just placeholder + tableProperties.put(propertyEntry.getKey(), propertyEntry.getValue()); + return this; + } + Optional getOptionalPageSource() throws IOException { From 2cfc6737cc3df99ede1555a80207ebe4fcb245db Mon Sep 17 00:00:00 2001 From: Rob Marrowstone <99764876+rmarrowstone@users.noreply.github.com> Date: Wed, 18 Dec 2024 17:12:06 -0800 Subject: [PATCH 10/18] Minor Fixups This commit fixes up a few things I noticed when previewing the PR to Trino. * Column/Field name casing should be preserved when writing * Some missing operational/metrics calls in IonPageSource * Throw clearer Exception for errors in IonFileWriter * Move some tests from TestHiveFileFormats to IonPageSourceSmokeTest * Add test for Timestamp Encoding --- .../hive/formats/ion/IonEncoderFactory.java | 8 +-- .../trino/hive/formats/ion/TestIonFormat.java | 21 ++++++ .../trino/plugin/hive/ion/IonFileWriter.java | 5 +- .../plugin/hive/ion/IonFileWriterFactory.java | 2 - .../trino/plugin/hive/ion/IonPageSource.java | 10 ++- .../plugin/hive/TestHiveFileFormats.java | 68 ++----------------- .../hive/ion/IonPageSourceSmokeTest.java | 10 +++ 7 files changed, 50 insertions(+), 74 deletions(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java index bcb8e469399d..0a75f2ca3504 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java @@ -51,7 +51,6 @@ import java.time.ZoneId; import java.util.Date; import java.util.List; -import java.util.Locale; import java.util.Optional; import java.util.function.IntFunction; @@ -64,7 +63,7 @@ private IonEncoderFactory() {} public static IonEncoder buildEncoder(List columns) { return RowEncoder.forFields(columns.stream() - .map(c -> new RowType.Field(Optional.of(c.name().toLowerCase(Locale.ROOT)), c.type())) + .map(c -> new RowType.Field(Optional.of(c.name()), c.type())) .toList()); } @@ -89,8 +88,7 @@ private static BlockEncoder encoderForType(Type type) case DecimalType t -> decimalEncoder(t); case DateType _ -> dateEncoder; case TimestampType t -> timestampEncoder(t); - case MapType t -> new MapEncoder(t, t.getKeyType(), - encoderForType(t.getValueType())); + case MapType t -> new MapEncoder(t, t.getKeyType(), encoderForType(t.getValueType())); case RowType t -> RowEncoder.forFields(t.getFields()); case ArrayType t -> new ArrayEncoder(wrapEncoder(encoderForType(t.getElementType()))); default -> throw new IllegalArgumentException(String.format("Unsupported type: %s", type)); @@ -119,7 +117,7 @@ private static RowEncoder forFields(List fields) ImmutableList.Builder fieldEncodersBuilder = ImmutableList.builder(); for (RowType.Field field : fields) { - fieldNamesBuilder.add(field.getName().get().toLowerCase(Locale.ROOT)); + fieldNamesBuilder.add(field.getName().get()); fieldEncodersBuilder.add(wrapEncoder(encoderForType(field.getType()))); } diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index e946c3ee23f1..bb9b3539e566 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -435,6 +435,27 @@ public void testEncode() assertIonEquivalence(TEST_COLUMNS, page, ionText); } + @Test + public void testEncodeTimestamp() + throws IOException + { + List timestampColumn = List.of(new Column("my_ts", TimestampType.TIMESTAMP_NANOS, 0)); + Page page = toPage(timestampColumn, List.of( + toSqlTimestamp(TimestampType.TIMESTAMP_NANOS, LocalDateTime.of(2024, 11, 23, 1, 23, 45, 666777888)))); + assertIonEquivalence(timestampColumn, page, "{ my_ts: 2024-11-23T01:23:45.666777888Z }"); + } + + @Test + public void testEncodeMixedCaseColumn() + throws IOException + { + List casedColumn = List.of( + new Column("TheAnswer", INTEGER, 0)); + + Page page = toPage(casedColumn, List.of(42)); + assertIonEquivalence(casedColumn, page, "{ TheAnswer: 42 }"); + } + @Test public void testEncodeWithNullField() throws IOException diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java index 89d0bd408762..693c41238138 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java @@ -33,6 +33,7 @@ import java.util.function.LongSupplier; import static io.trino.plugin.hive.HiveErrorCode.HIVE_WRITER_CLOSE_ERROR; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_WRITER_DATA_ERROR; public class IonFileWriter implements FileWriter @@ -106,7 +107,7 @@ public void rollback() writer.close(); } catch (IOException e) { - throw new RuntimeException(e); + throw new TrinoException(HIVE_WRITER_CLOSE_ERROR, "Error rolling back write to Hive", e); } } @@ -123,7 +124,7 @@ public void appendRows(Page page) pageEncoder.encode(writer, page); } catch (IOException e) { - throw new RuntimeException(e); + throw new TrinoException(HIVE_WRITER_DATA_ERROR, e); } } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java index 5a4f82354aaa..83acc5162327 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java @@ -85,8 +85,6 @@ public Optional createFileWriter( Closeable rollbackAction = () -> fileSystem.deleteFile(location); - // we take the column names from the schema, not what was input - // this is what the LineWriterFactory does, I don't understand why List fileColumnNames = getColumnNames(schema); List fileColumnTypes = getColumnTypes(schema).stream() .map(hiveType -> getType(hiveType, typeManager, getTimestampPrecision(session))) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java index d20a5cc887ff..5fbc9549429a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hive.ion; +import com.amazon.ion.IonBufferConfiguration; import com.amazon.ion.IonReader; import com.amazon.ion.IonType; import io.trino.hive.formats.ion.IonDecoder; @@ -24,9 +25,13 @@ import java.util.OptionalLong; import java.util.function.LongSupplier; +import static io.airlift.slice.SizeOf.instanceSize; + public class IonPageSource implements ConnectorPageSource { + private static final int INSTANCE_SIZE = instanceSize(IonPageSource.class); + private final IonReader ionReader; private final PageBuilder pageBuilder; private final IonDecoder decoder; @@ -86,7 +91,10 @@ public Page getNextPage() @Override public long getMemoryUsage() { - return 4096; + // we don't have the ability to ask an IonReader how many bytes it has buffered + // it will buffer as much as is needed for each top-level-value. + int assumedIonBufferSize = IonBufferConfiguration.DEFAULT.getInitialBufferSize() * 4; + return INSTANCE_SIZE + assumedIonBufferSize + pageBuilder.getRetainedSizeInBytes(); } @Override diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index b9198faf6cf6..a3f0cc59ef39 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -140,7 +140,6 @@ import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; -import static io.trino.plugin.hive.HiveErrorCode.HIVE_WRITER_OPEN_ERROR; import static io.trino.plugin.hive.HivePageSourceProvider.ColumnMapping.buildColumnMappings; import static io.trino.plugin.hive.HiveStorageFormat.AVRO; import static io.trino.plugin.hive.HiveStorageFormat.CSV; @@ -157,8 +156,6 @@ import static io.trino.plugin.hive.HiveTestUtils.getHiveSession; import static io.trino.plugin.hive.HiveTestUtils.mapType; import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_ENCODING_PROPERTY; -import static io.trino.plugin.hive.ion.IonWriterOptions.TEXT_ENCODING; import static io.trino.plugin.hive.util.HiveTypeTranslator.toHiveType; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; @@ -234,7 +231,6 @@ public final class TestHiveFileFormats private static final FileFormatDataSourceStats STATS = new FileFormatDataSourceStats(); private static final ConnectorSession PARQUET_SESSION = getHiveSession(createParquetHiveConfig(false)); private static final ConnectorSession PARQUET_SESSION_USE_NAME = getHiveSession(createParquetHiveConfig(true)); - private static final String ERROR_ENCODING = "error_encoding"; @DataProvider(name = "rowCount") public static Object[][] rowCountProvider() @@ -377,7 +373,8 @@ public void testIonWithBinaryEncoding(int rowCount, long fileSizePadding) throws Exception { List testColumns = TEST_COLUMNS.stream() - // todo: add support for maps to trino impl + // even though maps with text keys work with the native trino impl + // there is an error when testing against the hive serde .filter(tc -> !(tc.type instanceof MapType)) .collect(toList()); @@ -394,54 +391,6 @@ public void testIonWithBinaryEncoding(int rowCount, long fileSizePadding) .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory, hiveConfig)); } - @Test(dataProvider = "validRowAndFileSizePadding") - public void testIonWithTextEncoding(int rowCount, long fileSizePadding) - throws Exception - { - List testColumns = TEST_COLUMNS.stream() - // todo: add support for maps to trino impl - .filter(tc -> !(tc.type instanceof MapType)) - .collect(toList()); - - HiveConfig hiveConfig = new HiveConfig(); - // enable Ion native trino integration for testing while the implementation is in progress - // TODO: In future this flag should change to `true` as default and then the following statement can be removed. - hiveConfig.setIonNativeTrinoEnabled(true); - - assertThatFileFormat(ION) - .withColumns(testColumns) - .withRowsCount(rowCount) - .withFileSizePadding(fileSizePadding) - .withTableProperties(ImmutableMap.of(ION_ENCODING_PROPERTY, TEXT_ENCODING)) - .withFileWriterFactory(fileSystemFactory -> new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) - .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory, hiveConfig)); - } - - @Test(dataProvider = "validRowAndFileSizePadding") - public void testInvalidIonEncoding(int rowCount, long fileSizePadding) - throws Exception - { - List testColumns = TEST_COLUMNS.stream() - // todo: add support for maps to trino impl - .filter(tc -> !(tc.type instanceof MapType)) - .collect(toList()); - - HiveConfig hiveConfig = new HiveConfig(); - // enable Ion native trino integration for testing while the implementation is in progress - // TODO: In future this flag should change to `true` as default and then the following statement can be removed. - hiveConfig.setIonNativeTrinoEnabled(true); - - assertTrinoExceptionThrownBy(() -> assertThatFileFormat(ION) - .withColumns(testColumns) - .withRowsCount(rowCount) - .withFileSizePadding(fileSizePadding) - .withTableProperties(ImmutableMap.of(ION_ENCODING_PROPERTY, ERROR_ENCODING)) - .withFileWriterFactory(fileSystemFactory -> new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) - .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory, hiveConfig))) - .hasErrorCode(HIVE_WRITER_OPEN_ERROR) - .hasMessage("Error creating Ion Output"); - } - @Test(dataProvider = "validRowAndFileSizePadding") public void testRcTextPageSource(int rowCount, long fileSizePadding) throws Exception @@ -1275,7 +1224,6 @@ private static class FileFormatAssertion private boolean skipGenericWrite; private HiveFileWriterFactory fileWriterFactory; private long fileSizePadding; - private Map customTableProperties = ImmutableMap.of(); private final TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); @@ -1333,12 +1281,6 @@ public FileFormatAssertion withRowsCount(int rowsCount) return this; } - public FileFormatAssertion withTableProperties(Map tableProperties) - { - this.customTableProperties = requireNonNull(tableProperties, "customTableProperties is null"); - return this; - } - public FileFormatAssertion withSession(ConnectorSession session) { this.session = requireNonNull(session, "session is null"); @@ -1397,7 +1339,7 @@ private void assertRead(HivePageSourceFactory pageSourceFactory) if (fileWriterFactory == null) { continue; } - createTestFileTrino(location, storageFormat, compressionCodec, writeColumns, session, rowsCount, fileWriterFactory, customTableProperties); + createTestFileTrino(location, storageFormat, compressionCodec, writeColumns, session, rowsCount, fileWriterFactory); } else { if (skipGenericWrite) { @@ -1427,8 +1369,7 @@ private static void createTestFileTrino( List testColumns, ConnectorSession session, int numRows, - HiveFileWriterFactory fileWriterFactory, - Map customTableProperties) + HiveFileWriterFactory fileWriterFactory) { // filter out partition keys, which are not written to the file testColumns = testColumns.stream() @@ -1453,7 +1394,6 @@ private static void createTestFileTrino( Map tableProperties = ImmutableMap.builder() .put(LIST_COLUMNS, testColumns.stream().map(TestColumn::name).collect(Collectors.joining(","))) .put(LIST_COLUMN_TYPES, testColumns.stream().map(TestColumn::type).map(HiveTypeTranslator::toHiveType).map(HiveType::toString).collect(Collectors.joining(","))) - .putAll(customTableProperties) .buildOrThrow(); Optional fileWriter = fileWriterFactory.createFileWriter( diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index 834f64fd81cb..58ba7e743e13 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -251,6 +251,16 @@ public void testBinaryEncoding() assertEncoding(tableColumns, BINARY_ENCODING); } + @Test + public void testBadEncodingName() + throws IOException + { + TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) + .withEncoding("unknown_encoding_name"); + + Assertions.assertThrows(TrinoException.class, fixture::getFileWriter); + } + private void assertEncoding(List tableColumns, String encoding) throws IOException From 135714e6c5c6b9eb74474e72b4220580bf001c74 Mon Sep 17 00:00:00 2001 From: Khushboo <68757952+desaikd@users.noreply.github.com> Date: Tue, 14 Jan 2025 11:26:36 -0800 Subject: [PATCH 11/18] Adds support to string coercions --- .../hive/formats/ion/IonDecoderFactory.java | 53 ++++++++++++++++--- .../trino/hive/formats/ion/TestIonFormat.java | 36 +++++++++++++ 2 files changed, 82 insertions(+), 7 deletions(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java index 6ddfc58a64fd..255882bb43ef 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java @@ -16,7 +16,9 @@ import com.amazon.ion.IonException; import com.amazon.ion.IonReader; import com.amazon.ion.IonType; +import com.amazon.ion.IonWriter; import com.amazon.ion.Timestamp; +import com.amazon.ion.system.IonTextWriterBuilder; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.slice.Slices; @@ -53,6 +55,7 @@ import io.trino.spi.type.VarcharType; import io.trino.spi.type.Varchars; +import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; import java.math.RoundingMode; @@ -127,8 +130,8 @@ private static BlockDecoder decoderForType(Type type) case BooleanType t -> wrapDecoder(boolDecoder, t, IonType.BOOL); case DateType t -> wrapDecoder(dateDecoder, t, IonType.TIMESTAMP); case TimestampType t -> wrapDecoder(timestampDecoder(t), t, IonType.TIMESTAMP); - case VarcharType t -> wrapDecoder(varcharDecoder(t), t, IonType.STRING, IonType.SYMBOL); - case CharType t -> wrapDecoder(charDecoder(t), t, IonType.STRING, IonType.SYMBOL); + case VarcharType t -> wrapDecoder(varcharDecoder(t), t, IonType.values()); + case CharType t -> wrapDecoder(charDecoder(t), t, IonType.values()); case VarbinaryType t -> wrapDecoder(binaryDecoder, t, IonType.BLOB, IonType.CLOB); case RowType t -> wrapDecoder(RowDecoder.forFields(t.getFields()), t, IonType.STRUCT); case ArrayType t -> wrapDecoder(new ArrayDecoder(decoderForType(t.getElementType())), t, IonType.LIST, IonType.SEXP); @@ -148,7 +151,7 @@ private static BlockDecoder decoderForType(Type type) */ private static BlockDecoder wrapDecoder(BlockDecoder decoder, Type trinoType, IonType... allowedTypes) { - Set allowedWithNull = new HashSet<>(Arrays.asList(allowedTypes)); + final Set allowedWithNull = new HashSet<>(Arrays.asList(allowedTypes)); allowedWithNull.add(IonType.NULL); return (reader, builder) -> { @@ -360,16 +363,52 @@ private static BlockDecoder decimalDecoder(DecimalType type) }; } + private static String getCoercedValue(IonReader ionReader) + { + IonTextWriterBuilder textWriterBuilder = IonTextWriterBuilder.standard(); + StringBuilder stringBuilder = new StringBuilder(); + IonWriter writer = textWriterBuilder.build(stringBuilder); + try { + writer.writeValue(ionReader); + } + catch (IOException e) { + throw new RuntimeException(e); + } + return stringBuilder.toString(); + } + private static BlockDecoder varcharDecoder(VarcharType type) { - return (ionReader, blockBuilder) -> - type.writeSlice(blockBuilder, Varchars.truncateToLength(Slices.utf8Slice(ionReader.stringValue()), type)); + return (ionReader, blockBuilder) -> { + IonType valueType = ionReader.getType(); + String value; + + if (valueType == IonType.SYMBOL || valueType == IonType.STRING) { + value = ionReader.stringValue(); + } + else { + // For any types other than IonType.SYMBOL and IonType.STRING, performs text coercion + value = getCoercedValue(ionReader); + } + type.writeSlice(blockBuilder, Varchars.truncateToLength(Slices.utf8Slice(value), type)); + }; } private static BlockDecoder charDecoder(CharType type) { - return (ionReader, blockBuilder) -> - type.writeSlice(blockBuilder, Chars.truncateToLengthAndTrimSpaces(Slices.utf8Slice(ionReader.stringValue()), type)); + return (ionReader, blockBuilder) -> { + IonType valueType = ionReader.getType(); + String value; + + if (valueType == IonType.SYMBOL || valueType == IonType.STRING) { + value = ionReader.stringValue(); + } + else { + // For any types other than IonType.SYMBOL and IonType.STRING, performs text coercion + value = getCoercedValue(ionReader); + } + type.writeSlice(blockBuilder, Chars.truncateToLengthAndTrimSpaces(Slices.utf8Slice(value), type)); + }; } private static final BlockDecoder byteDecoder = (ionReader, blockBuilder) -> diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index bb9b3539e566..ead606fe3e1b 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -186,6 +186,42 @@ public void testCaseInsensitivityOfKeys() List.of(31, "baz")); } + @Test + public void testStringCoercions() + throws IOException + { + assertValues( + RowType.rowType( + field("foo", VARCHAR)), + "{ foo: true }", + List.of("true")); + assertValues( + RowType.rowType( + field("foo", VARCHAR)), + "{ foo: 31 }", + List.of("31")); + assertValues( + RowType.rowType( + field("foo", VARCHAR)), + "{ foo: 31.50 }", + List.of("31.50")); + assertValues( + RowType.rowType( + field("foo", VARCHAR)), + "{ foo: [1, 2, 3] }", + List.of("[1,2,3]")); + assertValues( + RowType.rowType( + field("foo", VARCHAR)), + "{ foo: \"bar\" }", + List.of("bar")); + assertValues( + RowType.rowType( + field("foo", VARCHAR)), + "{ foo: { nested_foo: 12 } }", + List.of("{nested_foo:12}")); + } + @Test public void testCaseInsensitivityOfDuplicateKeys() throws IOException From cafebaffc496decdeda33d7e0f90e7c62157d097 Mon Sep 17 00:00:00 2001 From: Rob Marrowstone <99764876+rmarrowstone@users.noreply.github.com> Date: Tue, 14 Jan 2025 13:32:12 -0800 Subject: [PATCH 12/18] Implement Path Extraction This change implements support for path extraction SerDe Properties. It uses the same ion-java-path-extraction library the Ion Hive SerDe does. Unlike the Hive SerDe, this ensures that the "strict" and more performant path extraction implementation is used. I chose to use the path-extraction in the absence of any defined path extractors. When a path extractor is defined, you have to define all columns as extractions. With the strict implementation, the field lookup is effectively the same as the Decoder here. So given that I would rather cut modality unless there's a really compelling reason. --- lib/trino-hive-formats/pom.xml | 6 ++ .../io/trino/hive/formats/ion/IonDecoder.java | 3 +- .../hive/formats/ion/IonDecoderConfig.java | 50 +++++++++++ .../hive/formats/ion/IonDecoderFactory.java | 88 ++++++++++++------- .../trino/hive/formats/ion/TestIonFormat.java | 80 +++++++++++++++-- .../trino/plugin/hive/ion/IonPageSource.java | 2 +- .../plugin/hive/ion/IonPageSourceFactory.java | 13 ++- .../plugin/hive/ion/IonReaderOptions.java | 27 +++++- .../hive/ion/IonPageSourceSmokeTest.java | 48 ++++++---- 9 files changed, 246 insertions(+), 71 deletions(-) create mode 100644 lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderConfig.java diff --git a/lib/trino-hive-formats/pom.xml b/lib/trino-hive-formats/pom.xml index c8d6fa745f58..72d911457f39 100644 --- a/lib/trino-hive-formats/pom.xml +++ b/lib/trino-hive-formats/pom.xml @@ -24,6 +24,12 @@ 1.11.9 + + com.amazon.ion + ion-java-path-extraction + 1.5.0 + + com.fasterxml.jackson.core jackson-core diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoder.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoder.java index 422ba16fb712..4463bc79c500 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoder.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoder.java @@ -15,7 +15,6 @@ import com.amazon.ion.IonException; import com.amazon.ion.IonReader; -import io.trino.spi.PageBuilder; public interface IonDecoder { @@ -25,6 +24,6 @@ public interface IonDecoder * Expects that the calling code has called IonReader.next() * to position the reader at the value to be decoded. */ - void decode(IonReader reader, PageBuilder builder) + void decode(IonReader reader) throws IonException; } diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderConfig.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderConfig.java new file mode 100644 index 000000000000..755d00ae9010 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderConfig.java @@ -0,0 +1,50 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.hive.formats.ion; + +import java.util.Map; + +/** + * Captures the SerDe properties that affect decoding. + * + * @param pathExtractors Map of column name => ion paths + * for each entry in the map, the value bound to the column will be the result + * of extracting the given search path. + * @param strictTyping whether the path extractions should enforce type expectations. + * this only affects type checking of path extractions; any value decoded into + * a trino column will be correctly typed or coerced for that column. + * @param caseSensitive whether field name matching should be case-sensitive or not. + */ +public record IonDecoderConfig(Map pathExtractors, Boolean strictTyping, Boolean caseSensitive) +{ + static IonDecoderConfig defaultConfig() + { + return new IonDecoderConfig(Map.of(), false, false); + } + + IonDecoderConfig withStrictTyping() + { + return new IonDecoderConfig(pathExtractors, true, caseSensitive); + } + + IonDecoderConfig withCaseSensitive() + { + return new IonDecoderConfig(pathExtractors, strictTyping, true); + } + + IonDecoderConfig withPathExtractors(Map pathExtractors) + { + return new IonDecoderConfig(pathExtractors, strictTyping, caseSensitive); + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java index 255882bb43ef..f3051ef620b7 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java @@ -19,11 +19,15 @@ import com.amazon.ion.IonWriter; import com.amazon.ion.Timestamp; import com.amazon.ion.system.IonTextWriterBuilder; +import com.amazon.ionpathextraction.PathExtractor; +import com.amazon.ionpathextraction.PathExtractorBuilder; +import com.amazon.ionpathextraction.pathcomponents.Text; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.slice.Slices; import io.trino.hive.formats.DistinctMapKeys; import io.trino.hive.formats.line.Column; +import io.trino.spi.PageBuilder; import io.trino.spi.StandardErrorCode; import io.trino.spi.TrinoException; import io.trino.spi.block.ArrayBlockBuilder; @@ -65,8 +69,8 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Optional; import java.util.Set; +import java.util.function.BiFunction; import java.util.function.IntFunction; public class IonDecoderFactory @@ -79,39 +83,66 @@ private IonDecoderFactory() {} * The decoder expects to decode the _current_ Ion Value. * It also expects that the calling code will manage the PageBuilder. *

- * - * @param strictPathing controls behavior when encountering mistyped - * values during path extraction. That is outside (before), the trino - * type model. The ion-hive-serde used path extraction for navigating - * the top-level-values even if no path extractions were configured. - * So, in absence of support for path extraction configurations this - * still affects the handling of mistyped top-level-values. - * todo: revisit the above once path extraction config is supported. */ - public static IonDecoder buildDecoder(List columns, boolean strictPathing) + public static IonDecoder buildDecoder( + List columns, + IonDecoderConfig decoderConfig, + PageBuilder pageBuilder) { - RowDecoder rowDecoder = RowDecoder.forFields( - columns.stream() - .map(c -> new RowType.Field(Optional.of(c.name()), c.type())) - .toList()); + PathExtractorBuilder extractorBuilder = PathExtractorBuilder.standard() + .withMatchCaseInsensitive(!decoderConfig.caseSensitive()); - return (ionReader, pageBuilder) -> { - IonType ionType = ionReader.getType(); - IntFunction blockSelector = pageBuilder::getBlockBuilder; + for (int pos = 0; pos < columns.size(); pos++) { + String name = columns.get(pos).name(); + BlockDecoder decoder = decoderForType(columns.get(pos).type()); + BiFunction callback = callbackFor(decoder, pos); - if (ionType == IonType.STRUCT && !ionReader.isNullValue()) { - rowDecoder.decode(ionReader, blockSelector); + String extractionPath = decoderConfig.pathExtractors().get(name); + if (extractionPath == null) { + extractorBuilder.withSearchPath(List.of(new Text(name)), callback); } - else if (ionType == IonType.STRUCT || ionType == IonType.NULL || !strictPathing) { - rowDecoder.appendNulls(blockSelector); + else { + extractorBuilder.withSearchPath(extractionPath, callback); + } + } + PathExtractor extractor = extractorBuilder.buildStrict(decoderConfig.strictTyping()); + PageExtractionContext context = new PageExtractionContext(pageBuilder, new boolean[columns.size()]); + + return (ionReader) -> { + extractor.matchCurrentValue(ionReader, context); + context.completeRowAndReset(); + }; + } + + private static BiFunction callbackFor(BlockDecoder decoder, int pos) + { + return (ionReader, context) -> { + BlockBuilder blockBuilder = context.pageBuilder.getBlockBuilder(pos); + if (context.encountered[pos]) { + blockBuilder.resetTo(blockBuilder.getPositionCount() - 1); } else { - throw new TrinoException(StandardErrorCode.GENERIC_USER_ERROR, - "Top-level-value of IonType %s is not valid with strict typing.".formatted(ionType)); + context.encountered[pos] = true; } + + decoder.decode(ionReader, context.pageBuilder.getBlockBuilder(pos)); + return 0; }; } + private record PageExtractionContext(PageBuilder pageBuilder, boolean[] encountered) + { + private void completeRowAndReset() + { + for (int i = 0; i < encountered.length; i++) { + if (!encountered[i]) { + pageBuilder.getBlockBuilder(i).appendNull(); + } + encountered[i] = false; + } + } + } + private interface BlockDecoder { void decode(IonReader reader, BlockBuilder builder); @@ -169,10 +200,6 @@ private static BlockDecoder wrapDecoder(BlockDecoder decoder, Type trinoType, Io }; } - /** - * The RowDecoder is used as the BlockDecoder for nested RowTypes and is used for decoding - * top-level structs into pages. - */ private record RowDecoder(Map fieldPositions, List fieldDecoders) implements BlockDecoder { @@ -224,13 +251,6 @@ private void decode(IonReader ionReader, IntFunction blockSelector ionReader.stepOut(); } - - private void appendNulls(IntFunction blockSelector) - { - for (int i = 0; i < fieldDecoders.size(); i++) { - blockSelector.apply(i).appendNull(); - } - } } private static class MapDecoder diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index ead606fe3e1b..3e7dd843c7af 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -19,6 +19,7 @@ import com.amazon.ion.IonWriter; import com.amazon.ion.system.IonReaderBuilder; import com.amazon.ion.system.IonSystemBuilder; +import com.amazon.ionpathextraction.exceptions.PathExtractionException; import com.google.common.collect.ImmutableMap; import io.trino.hive.formats.line.Column; import io.trino.spi.Page; @@ -48,6 +49,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.stream.IntStream; import static io.trino.hive.formats.FormatTestUtils.assertColumnValuesEquals; @@ -108,18 +110,20 @@ public void testVariousTlvsStrict() throws IOException { RowType rowType = RowType.rowType(field("foo", INTEGER), field("bar", VARCHAR)); + IonDecoderConfig decoderConfig = IonDecoderConfig.defaultConfig().withStrictTyping(); List expected = new ArrayList<>(2); expected.add(null); expected.add(null); assertValues(rowType, + decoderConfig, // empty struct, untyped null, struct null, and explicitly typed null null, phew. "{} null null.struct null.null", expected, expected, expected, expected); - Assertions.assertThrows(TrinoException.class, () -> { - assertValues(rowType, "null.int", expected); - assertValues(rowType, "[]", expected); + Assertions.assertThrows(PathExtractionException.class, () -> { + assertValues(rowType, decoderConfig, "null.int", expected); + assertValues(rowType, decoderConfig, "[]", expected); }); } @@ -133,7 +137,6 @@ public void testVariousTlvsLax() expected.add(null); assertValues(rowType, - false, "{} 37 null.list null.struct null spam false", expected, expected, expected, expected, expected, expected, expected); } @@ -236,6 +239,20 @@ public void testCaseInsensitivityOfDuplicateKeys() List.of(5, "baz")); } + @Test + public void testCaseSensitiveExtraction() + throws IOException + { + assertValues( + RowType.rowType( + field("Foo", INTEGER), + field("Bar", VARCHAR)), + IonDecoderConfig.defaultConfig().withCaseSensitive(), + // assumes duplicate fields overwrite, which is asserted in the test above + "{ Bar: baz, bar: blegh, Foo: 31, foo: 67 }", + List.of(31, "baz")); + } + @Test public void testStructWithNullAndMissingValues() throws IOException @@ -448,6 +465,53 @@ public void testNumbersTooBigForDecimal128() } } + @Test + public void testPathExtraction() + throws IOException + { + Map pathExtractions = Map.of("bar", "(foo bar)", "baz", "(foo baz)"); + assertValues( + RowType.rowType(field("qux", BOOLEAN), field("bar", INTEGER), field("baz", VARCHAR)), + IonDecoderConfig.defaultConfig().withPathExtractors(pathExtractions), + "{ foo: { bar: 31, baz: quux }, qux: true }", + List.of(true, 31, "quux")); + } + + @Test + public void testNonStructTlvPathExtraction() + throws IOException + { + Map pathExtractions = Map.of("tlv", "()"); + assertValues( + RowType.rowType(field("tlv", new ArrayType(INTEGER))), + IonDecoderConfig.defaultConfig().withPathExtractors(pathExtractions), + "[13, 17] [19, 23]", + List.of(List.of(13, 17)), + List.of(List.of(19, 23))); + } + + /** + * Shows how users can configure mapping sequence positions from Ion values to a Trino row. + */ + @Test + public void testPositionalPathExtraction() + throws IOException + { + Map pathExtractions = Map.of( + "foo", "(0)", + "bar", "(1)"); + RowType rowType = RowType.rowType( + field("foo", INTEGER), + field("bar", VARCHAR)); + + assertValues( + rowType, + IonDecoderConfig.defaultConfig().withPathExtractors(pathExtractions), + "[13, baz] [17, qux]", + List.of(13, "baz"), + List.of(17, "qux")); + } + @Test public void testEncode() throws IOException @@ -531,10 +595,10 @@ public void testEncodeWithNullNestedField() private void assertValues(RowType rowType, String ionText, List... expected) throws IOException { - assertValues(rowType, true, ionText, expected); + assertValues(rowType, IonDecoderConfig.defaultConfig(), ionText, expected); } - private void assertValues(RowType rowType, Boolean strictTlvs, String ionText, List... expected) + private void assertValues(RowType rowType, IonDecoderConfig config, String ionText, List... expected) throws IOException { List fields = rowType.getFields(); @@ -545,14 +609,14 @@ private void assertValues(RowType rowType, Boolean strictTlvs, String ionText, L return new Column(field.getName().get(), field.getType(), i); }) .toList(); - IonDecoder decoder = IonDecoderFactory.buildDecoder(columns, strictTlvs); PageBuilder pageBuilder = new PageBuilder(expected.length, rowType.getFields().stream().map(RowType.Field::getType).toList()); + IonDecoder decoder = IonDecoderFactory.buildDecoder(columns, config, pageBuilder); try (IonReader ionReader = IonReaderBuilder.standard().build(ionText)) { for (int i = 0; i < expected.length; i++) { assertThat(ionReader.next()).isNotNull(); pageBuilder.declarePosition(); - decoder.decode(ionReader, pageBuilder); + decoder.decode(ionReader); } assertThat(ionReader.next()).isNull(); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java index 5fbc9549429a..b2472b78ea58 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java @@ -112,7 +112,7 @@ private boolean readNextValue() } pageBuilder.declarePosition(); - decoder.decode(ionReader, pageBuilder); + decoder.decode(ionReader); return true; } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java index 2bc98396f76e..d7b051c26896 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java @@ -26,6 +26,7 @@ import io.trino.hive.formats.compression.Codec; import io.trino.hive.formats.compression.CompressionKind; import io.trino.hive.formats.ion.IonDecoder; +import io.trino.hive.formats.ion.IonDecoderConfig; import io.trino.hive.formats.ion.IonDecoderFactory; import io.trino.hive.formats.line.Column; import io.trino.plugin.hive.AcidInfo; @@ -62,9 +63,6 @@ import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN; import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED; import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED_DEFAULT; -import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTION_CASE_SENSITIVITY; -import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT; -import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTOR_PROPERTY; import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_DEFAULT; import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_PROPERTY; import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_PROPERTY; @@ -82,14 +80,12 @@ public class IonPageSourceFactory private static final Map TABLE_PROPERTIES = ImmutableMap.of( FAIL_ON_OVERFLOW_PROPERTY, FAIL_ON_OVERFLOW_PROPERTY_DEFAULT, IGNORE_MALFORMED, IGNORE_MALFORMED_DEFAULT, - PATH_EXTRACTION_CASE_SENSITIVITY, PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT, ION_TIMESTAMP_OFFSET_PROPERTY, ION_TIMESTAMP_OFFSET_DEFAULT, ION_SERIALIZATION_AS_NULL_PROPERTY, ION_SERIALIZATION_AS_NULL_DEFAULT); private static final Set COLUMN_PROPERTIES = ImmutableSet.of( Pattern.compile(FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN), - Pattern.compile(ION_SERIALIZATION_AS_PROPERTY), - Pattern.compile(PATH_EXTRACTOR_PROPERTY)); + Pattern.compile(ION_SERIALIZATION_AS_PROPERTY)); @Inject public IonPageSourceFactory(TrinoFileSystemFactory trinoFileSystemFactory, HiveConfig hiveConfig) @@ -172,8 +168,9 @@ public Optional createPageSource( List decoderColumns = projectedReaderColumns.stream() .map(hc -> new Column(hc.getName(), hc.getType(), hc.getBaseHiveColumnIndex())) .toList(); - boolean strictPathing = IonReaderOptions.useStrictPathTyping(schema.serdeProperties()); - IonDecoder decoder = IonDecoderFactory.buildDecoder(decoderColumns, strictPathing); + + IonDecoderConfig decoderConfig = IonReaderOptions.decoderConfigFor(schema.serdeProperties()); + IonDecoder decoder = IonDecoderFactory.buildDecoder(decoderColumns, decoderConfig, pageBuilder); IonPageSource pageSource = new IonPageSource(ionReader, countingInputStream::getCount, decoder, pageBuilder); return Optional.of(new ReaderPageSource(pageSource, readerProjections)); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java index 5c510c2a5d09..a87a2280fc67 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java @@ -13,13 +13,18 @@ */ package io.trino.plugin.hive.ion; +import com.google.common.collect.ImmutableMap; +import io.trino.hive.formats.ion.IonDecoderConfig; + import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; public final class IonReaderOptions { public static final String STRICT_PATH_TYPING_PROPERTY = "ion.path_extractor.strict"; public static final String STRICT_PATH_TYPING_DEFAULT = "false"; - public static final String PATH_EXTRACTOR_PROPERTY = "ion.\\w+.path_extractor"; + public static final String PATH_EXTRACTOR_PROPERTY = "ion.(\\w+).path_extractor"; public static final String PATH_EXTRACTION_CASE_SENSITIVITY = "ion.path_extractor.case_sensitive"; public static final String PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT = "false"; public static final String FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN = "ion.\\w+.fail_on_overflow"; @@ -28,11 +33,27 @@ public final class IonReaderOptions public static final String IGNORE_MALFORMED = "ion.ignore_malformed"; public static final String IGNORE_MALFORMED_DEFAULT = "false"; + private static final Pattern pathExtractorPattern = Pattern.compile(PATH_EXTRACTOR_PROPERTY); + private IonReaderOptions() {} - static boolean useStrictPathTyping(Map propertiesMap) + public static IonDecoderConfig decoderConfigFor(Map propertiesMap) { - return Boolean.parseBoolean( + ImmutableMap.Builder extractionsBuilder = ImmutableMap.builder(); + + for (Map.Entry property : propertiesMap.entrySet()) { + Matcher matcher = pathExtractorPattern.matcher(property.getKey()); + if (matcher.matches()) { + extractionsBuilder.put(matcher.group(1), property.getValue()); + } + } + + Boolean strictTyping = Boolean.parseBoolean( propertiesMap.getOrDefault(STRICT_PATH_TYPING_PROPERTY, STRICT_PATH_TYPING_DEFAULT)); + Boolean caseSensitive = Boolean.parseBoolean( + propertiesMap.getOrDefault(PATH_EXTRACTION_CASE_SENSITIVITY, PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT)); + + // n.b.: the hive serde overwrote when there were duplicate extractors defined for a column + return new IonDecoderConfig(extractionsBuilder.buildOrThrow(), strictTyping, caseSensitive); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index 58ba7e743e13..a806ad4ace26 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -67,8 +67,6 @@ import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY_DEFAULT; import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED; import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED_DEFAULT; -import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTION_CASE_SENSITIVITY; -import static io.trino.plugin.hive.ion.IonReaderOptions.PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT; import static io.trino.plugin.hive.ion.IonWriterOptions.BINARY_ENCODING; import static io.trino.plugin.hive.ion.IonWriterOptions.ION_ENCODING_PROPERTY; import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_DEFAULT; @@ -130,16 +128,39 @@ public void testStrictAndLaxPathTyping() throws IOException { TestFixture defaultFixture = new TestFixture(FOO_BAR_COLUMNS); - defaultFixture.assertRowCount("37 null.timestamp", 2); + defaultFixture.assertRowCount("37 null.timestamp []", 3); TestFixture laxFixture = new TestFixture(FOO_BAR_COLUMNS); laxFixture.withStrictPathTyping("false"); - laxFixture.assertRowCount("37 null.timestamp", 2); + laxFixture.assertRowCount("37 null.timestamp []", 3); TestFixture strictFixture = new TestFixture(FOO_BAR_COLUMNS); strictFixture.withStrictPathTyping("true"); + Assertions.assertThrows(TrinoException.class, () -> - strictFixture.assertRowCount("37 null.timestamp", 2)); + strictFixture.assertRowCount("37 null.timestamp []", 3)); + } + + @Test + public void testPathExtraction() + throws IOException + { + TestFixture fixture = new TestFixture(List.of(toHiveBaseColumnHandle("bar", INTEGER, 0))) + .withSerdeProperty("ion.bar.path_extractor", "(foo bar)"); + + // these would result in errors if we tried to extract the bar field from the root instead of the nested bar + fixture.assertRowCount("{ foo: { bar: 17 }, bar: not_this_bar } { foo: { bar: 31 }, bar: not_this_bar }", 2); + } + + @Test + public void testCaseSensitive() + throws IOException + { + TestFixture fixture = new TestFixture(List.of(toHiveBaseColumnHandle("bar", INTEGER, 0))) + .withSerdeProperty("ion.path_extractor.case_sensitive", "true"); + + // this would result in errors if we tried to extract the BAR field + fixture.assertRowCount("{ BAR: should_be_skipped } { bar: 17 }", 2); } @Test @@ -179,7 +200,6 @@ private static Stream> propertiesWithDefaults() { return Stream.of( entry(FAIL_ON_OVERFLOW_PROPERTY, FAIL_ON_OVERFLOW_PROPERTY_DEFAULT), - entry(PATH_EXTRACTION_CASE_SENSITIVITY, PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT), entry(IGNORE_MALFORMED, IGNORE_MALFORMED_DEFAULT), entry(ION_TIMESTAMP_OFFSET_PROPERTY, ION_TIMESTAMP_OFFSET_DEFAULT), entry(ION_SERIALIZATION_AS_NULL_PROPERTY, ION_SERIALIZATION_AS_NULL_DEFAULT)); @@ -189,7 +209,6 @@ private static Stream> propertiesWithValues() { return Stream.of( entry(FAIL_ON_OVERFLOW_PROPERTY, "false"), - entry(PATH_EXTRACTION_CASE_SENSITIVITY, "true"), entry(IGNORE_MALFORMED, "true"), entry(ION_TIMESTAMP_OFFSET_PROPERTY, "01:00"), entry(ION_SERIALIZATION_AS_NULL_PROPERTY, "TYPED"), @@ -197,8 +216,8 @@ private static Stream> propertiesWithValues() // Any presence of these properties in the schema will result in an empty PageSource, // regardless of their assigned values. entry("ion.foo.fail_on_overflow", "property_value"), - entry("ion.foo.serialize_as", "property_value"), - entry("ion.foo.path_extractor", "property_value")); + entry("ion.foo.serialize_as", "property_value")); + //entry("ion.foo.path_extractor", "property_value")); } private static Map.Entry entry(String key, String value) @@ -212,7 +231,7 @@ void testPropertiesWithValues(Map.Entry property) throws IOException { TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) - .withSerdeProperties(property); + .withSerdeProperty(property.getKey(), property.getValue()); fixture.writeIonTextFile("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }"); Optional connectorPageSource = fixture.getOptionalPageSource(); @@ -221,11 +240,11 @@ void testPropertiesWithValues(Map.Entry property) @ParameterizedTest @MethodSource("propertiesWithDefaults") - void testPropertiesWithDefaults(Map.Entry propertyEntry) + void testPropertiesWithDefaults(Map.Entry property) throws IOException { TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) - .withSerdeProperties(propertyEntry); + .withSerdeProperty(property.getKey(), property.getValue()); fixture.assertRowCount("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }", 2); } @@ -345,10 +364,9 @@ TestFixture withStrictPathTyping(String strict) return this; } - TestFixture withSerdeProperties(Map.Entry propertyEntry) + TestFixture withSerdeProperty(String key, String value) { - // The value of the property is just placeholder - tableProperties.put(propertyEntry.getKey(), propertyEntry.getValue()); + tableProperties.put(key, value); return this; } From d4f4be32b84948a16224351f7a46b542ff0b33b5 Mon Sep 17 00:00:00 2001 From: Rob Marrowstone <99764876+rmarrowstone@users.noreply.github.com> Date: Tue, 14 Jan 2025 21:01:29 -0800 Subject: [PATCH 13/18] Add Decoding Tests, Minor Fix to MapDecoder This commit gives us complete coverage for the IonDecoderFactory, except for the default branch we need to make the compiler happy. In the process of covering both Char and Varchar Types for Map Keys I discovered we weren't truncating the keys so I fixed that. --- .../hive/formats/ion/IonDecoderFactory.java | 39 ++++--- .../trino/hive/formats/ion/TestIonFormat.java | 108 +++++++++++++++--- 2 files changed, 114 insertions(+), 33 deletions(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java index f3051ef620b7..e32da72b5c81 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java @@ -70,6 +70,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.IntFunction; @@ -166,7 +167,7 @@ private static BlockDecoder decoderForType(Type type) case VarbinaryType t -> wrapDecoder(binaryDecoder, t, IonType.BLOB, IonType.CLOB); case RowType t -> wrapDecoder(RowDecoder.forFields(t.getFields()), t, IonType.STRUCT); case ArrayType t -> wrapDecoder(new ArrayDecoder(decoderForType(t.getElementType())), t, IonType.LIST, IonType.SEXP); - case MapType t -> wrapDecoder(new MapDecoder(t, decoderForType(t.getValueType())), t, IonType.STRUCT); + case MapType t -> wrapDecoder(new MapDecoder(t), t, IonType.STRUCT); default -> throw new IllegalArgumentException(String.format("Unsupported type: %s", type)); }; } @@ -256,24 +257,32 @@ private void decode(IonReader ionReader, IntFunction blockSelector private static class MapDecoder implements BlockDecoder { + private final BiConsumer keyConsumer; private final BlockDecoder valueDecoder; - private final Type keyType; - private final Type valueType; private final DistinctMapKeys distinctMapKeys; private BlockBuilder keyBlockBuilder; private BlockBuilder valueBlockBuilder; - public MapDecoder(MapType mapType, BlockDecoder valueDecoder) + MapDecoder(MapType mapType) { - this.keyType = mapType.getKeyType(); - if (!(keyType instanceof VarcharType _ || keyType instanceof CharType _)) { - throw new UnsupportedOperationException("Unsupported map key type: " + keyType); - } - this.valueType = mapType.getValueType(); - this.valueDecoder = valueDecoder; + Type keyType = mapType.getKeyType(); + Type valueType = mapType.getValueType(); + this.valueDecoder = decoderForType(valueType); this.distinctMapKeys = new DistinctMapKeys(mapType, true); - this.keyBlockBuilder = mapType.getKeyType().createBlockBuilder(null, 128); - this.valueBlockBuilder = mapType.getValueType().createBlockBuilder(null, 128); + this.keyBlockBuilder = keyType.createBlockBuilder(null, 128); + this.valueBlockBuilder = valueType.createBlockBuilder(null, 128); + + this.keyConsumer = switch (keyType) { + case VarcharType t -> { + yield (String fieldName, BlockBuilder blockBuilder) -> + t.writeSlice(blockBuilder, Varchars.truncateToLength(Slices.utf8Slice(fieldName), t)); + } + case CharType t -> { + yield (String fieldName, BlockBuilder blockBuilder) -> + t.writeSlice(blockBuilder, Chars.truncateToLengthAndTrimSpaces(Slices.utf8Slice(fieldName), t)); + } + default -> throw new UnsupportedOperationException("Unsupported map key type: " + keyType); + }; } @Override @@ -282,13 +291,13 @@ public void decode(IonReader ionReader, BlockBuilder builder) ionReader.stepIn(); // buffer the keys and values while (ionReader.next() != null) { - VarcharType.VARCHAR.writeSlice(keyBlockBuilder, Slices.utf8Slice(ionReader.getFieldName())); + keyConsumer.accept(ionReader.getFieldName(), keyBlockBuilder); valueDecoder.decode(ionReader, valueBlockBuilder); } ValueBlock keys = keyBlockBuilder.buildValueBlock(); ValueBlock values = valueBlockBuilder.buildValueBlock(); - keyBlockBuilder = keyType.createBlockBuilder(null, keys.getPositionCount()); - valueBlockBuilder = valueType.createBlockBuilder(null, values.getPositionCount()); + keyBlockBuilder = keyBlockBuilder.newBlockBuilderLike(null); + valueBlockBuilder = valueBlockBuilder.newBlockBuilderLike(null); // copy the distinct key entries to the output boolean[] distinctKeys = distinctMapKeys.selectDistinctKeys(keys); diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index 3e7dd843c7af..a924078a4515 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -29,6 +29,7 @@ import io.trino.spi.type.CharType; import io.trino.spi.type.DateType; import io.trino.spi.type.DecimalType; +import io.trino.spi.type.DoubleType; import io.trino.spi.type.MapType; import io.trino.spi.type.RealType; import io.trino.spi.type.RowType; @@ -37,6 +38,7 @@ import io.trino.spi.type.SqlTimestamp; import io.trino.spi.type.SqlVarbinary; import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Type; import io.trino.spi.type.TypeOperators; import io.trino.spi.type.VarcharType; import org.junit.jupiter.api.Assertions; @@ -44,6 +46,7 @@ import java.io.IOException; import java.math.BigInteger; +import java.nio.charset.StandardCharsets; import java.time.LocalDate; import java.time.LocalDateTime; import java.util.ArrayList; @@ -60,6 +63,8 @@ import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.RowType.field; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TinyintType.TINYINT; import static io.trino.spi.type.VarbinaryType.VARBINARY; import static io.trino.spi.type.VarcharType.VARCHAR; import static org.assertj.core.api.Assertions.assertThat; @@ -95,14 +100,27 @@ public void testSuperBasicStruct() public void testMap() throws IOException { - MapType mapType = new MapType(VARCHAR, INTEGER, TYPE_OPERATORS); + MapType mapType = new MapType(VarcharType.createVarcharType(3), INTEGER, TYPE_OPERATORS); assertValues( RowType.rowType(field("foo", mapType)), - "{ foo: { a: 1, a: 2, b: 5 } }", - List.of(ImmutableMap.builder() - .put("a", 2) - .put("b", 5) - .buildOrThrow())); + "{ foo: { bar: 1, bar: 2, baz: 5, quxx: 8 } } { foo: { bar: 17, baz: 31, qux: 53 } }", + List.of(Map.of("bar", 2, "baz", 5, "qux", 8)), + List.of(Map.of("bar", 17, "baz", 31, "qux", 53))); + + mapType = new MapType(CharType.createCharType(3), INTEGER, TYPE_OPERATORS); + assertValues( + RowType.rowType(field("foo", mapType)), + "{ foo: { bar: 1, bar: 2, baz: 5, quxx: 8 } }", + List.of(Map.of("bar", 2, "baz", 5, "qux", 8))); + } + + @Test + public void testUnsupportedMapKeys() + throws IOException + { + MapType mapType = new MapType(INTEGER, INTEGER, TYPE_OPERATORS); + Assertions.assertThrows(UnsupportedOperationException.class, () -> + assertValues(RowType.rowType(field("bad_map", mapType)), "", List.of())); } @Test @@ -257,16 +275,12 @@ public void testCaseSensitiveExtraction() public void testStructWithNullAndMissingValues() throws IOException { - final List listWithNulls = new ArrayList<>(); - listWithNulls.add(null); - listWithNulls.add(null); - assertValues( RowType.rowType( field("foo", INTEGER), field("bar", VARCHAR)), "{ bar: null.symbol }", - listWithNulls); + Arrays.asList(null, null)); } @Test @@ -303,10 +317,27 @@ public void testNestedStruct() field("name", RowType.rowType( field("first", VARCHAR), field("last", VARCHAR)))), - "{ name: { first: Woody, last: Guthrie } }", + "{ name: { first: Woody, last: Guthrie, superfluous: ignored } }", List.of(List.of("Woody", "Guthrie"))); } + @Test + public void testNestedStructWithDuplicateAndMissingKeys() + throws IOException + { + assertValues( + RowType.rowType( + field("name", RowType.rowType( + field("first", VARCHAR), + field("last", VARCHAR)))), + """ + { name: { last: Godfrey, last: Guthrie } } + { name: { first: Joan, last: Baez } } + """, + List.of(Arrays.asList(null, "Guthrie")), + List.of(List.of("Joan", "Baez"))); + } + @Test public void testStructInList() throws IOException @@ -324,14 +355,55 @@ public void testStructInList() } @Test - public void testIonIntTooLargeForLong() + public void testIntsOfVariousSizes() throws IOException { - Assertions.assertThrows(TrinoException.class, () -> { - assertValues(RowType.rowType(field("my_bigint", BIGINT)), - "{ my_bigint: 18446744073709551786 }", - List.of()); - }); + List ions = List.of( + "{ ion_int: 0x7f }", // < one byte + "{ ion_int: 0x7fff }", // < two bytes + "{ ion_int: 0x7fffffff }", // < four bytes + "{ ion_int: 0x7fffffffffffffff }", // < eight bytes + "{ ion_int: 0x7fffffffffffffff1 }" // > eight bytes + ); + + List intTypes = List.of(TINYINT, SMALLINT, INTEGER, BIGINT); + List expected = List.of((byte) 0x7f, (short) 0x7fff, 0x7fffffff, 0x7fffffffffffffffL); + for (int i = 0; i < intTypes.size(); i++) { + RowType rowType = RowType.rowType(field("ion_int", intTypes.get(i))); + assertValues( + rowType, + ions.get(i), + List.of(expected.get(i))); + + int nextIon = i + 1; + Assertions.assertThrows(TrinoException.class, () -> { + assertValues(rowType, + ions.get(nextIon), + List.of()); + }); + } + } + + @Test + public void testFloat() + throws IOException + { + RowType rowType = RowType.rowType(field("my_double", DoubleType.DOUBLE)); + assertValues( + rowType, + "{ my_double: 4444e-4 }", + List.of(.4444)); + } + + @Test + public void testBytes() + throws IOException + { + RowType rowType = RowType.rowType(field("blobby", VARBINARY)); + assertValues( + rowType, + "{ blobby: {{ YmxvYmJ5IG1jYmxvYmZhY2U= }} }", + List.of(new SqlVarbinary("blobby mcblobface".getBytes(StandardCharsets.UTF_8)))); } @Test From f10e06a476ec6dad00a68aa201d1d7c698699c7f Mon Sep 17 00:00:00 2001 From: Khushboo <68757952+desaikd@users.noreply.github.com> Date: Tue, 14 Jan 2025 21:02:04 -0800 Subject: [PATCH 14/18] Adds encoding tests --- .../trino/hive/formats/ion/TestIonFormat.java | 42 ++++++++++++------- 1 file changed, 28 insertions(+), 14 deletions(-) diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index a924078a4515..2fd4841a1354 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -27,11 +27,9 @@ import io.trino.spi.TrinoException; import io.trino.spi.type.ArrayType; import io.trino.spi.type.CharType; -import io.trino.spi.type.DateType; import io.trino.spi.type.DecimalType; import io.trino.spi.type.DoubleType; import io.trino.spi.type.MapType; -import io.trino.spi.type.RealType; import io.trino.spi.type.RowType; import io.trino.spi.type.SqlDate; import io.trino.spi.type.SqlDecimal; @@ -61,12 +59,16 @@ import static io.trino.hive.formats.FormatTestUtils.toSqlTimestamp; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; import static io.trino.spi.type.RowType.field; import static io.trino.spi.type.SmallintType.SMALLINT; import static io.trino.spi.type.TinyintType.TINYINT; import static io.trino.spi.type.VarbinaryType.VARBINARY; import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.Math.toIntExact; import static org.assertj.core.api.Assertions.assertThat; public class TestIonFormat @@ -82,7 +84,14 @@ public class TestIonFormat new Column("struction", RowType.rowType( field("foo", INTEGER), field("bar", VARCHAR)), 5), - new Column("map", new MapType(VARCHAR, INTEGER, TYPE_OPERATORS), 6)); + new Column("map", new MapType(VARCHAR, INTEGER, TYPE_OPERATORS), 6), + new Column("double_value", DOUBLE, 7), + new Column("decimal_value", DecimalType.createDecimalType(10, 2), 8), + new Column("tiny_int", TINYINT, 9), + new Column("small_int", SMALLINT, 10), + new Column("big_int", BIGINT, 11), + new Column("real_num", REAL, 12), + new Column("date", DATE, 13)); @Test public void testSuperBasicStruct() @@ -410,7 +419,7 @@ public void testBytes() public void testDoubleAsFloat() throws IOException { - RowType rowType = RowType.rowType(field("my_float", RealType.REAL)); + RowType rowType = RowType.rowType(field("my_float", REAL)); assertValues( rowType, "{ my_float: 625e-3 }", @@ -428,7 +437,7 @@ public void testDoubleAsFloat() public void testDateDecoding() throws IOException { - RowType rowType = RowType.rowType(field("my_date", DateType.DATE)); + RowType rowType = RowType.rowType(field("my_date", DATE)); SqlDate expected = new SqlDate((int) LocalDate.of(2022, 2, 22).toEpochDay()); List ions = List.of( @@ -592,15 +601,17 @@ public void testEncode() 3), List.of(51, "baz"), ImmutableMap.builder() .put("a", 2) .put("b", 5) - .buildOrThrow()); + .buildOrThrow(), 5e0, new SqlDecimal(BigInteger.valueOf(123400), 10, 2), (byte) -1, (short) 32767, + 15L, 12.0f, new SqlDate(toIntExact(LocalDate.of(2025, 1, 1).toEpochDay()))); List row2 = List.of(31, "somebody", false, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), List.of(67, "qux"), ImmutableMap.builder() .put("foo", 12) .put("bar", 50) - .buildOrThrow()); + .buildOrThrow(), 5e0, new SqlDecimal(BigInteger.valueOf(123400), 10, 2), (byte) 0, (short) -1 + , 0L, 0.0f, new SqlDate(toIntExact(LocalDate.of(2025, 1, 1).toEpochDay()))); String ionText = """ - { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{ foo:51, bar:"baz"}, map: {a: 2, b: 5}} - { magic_num:31, some_text:"somebody", is_summer:false, byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{ foo:67, bar:"qux"}, map: {foo: 12, bar: 50}} + { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{ foo:51, bar:"baz"}, map: {a: 2, b: 5}, double_value: 5e0, decimal_value: 1234.00, tiny_int: -1, small_int: 32767, big_int: 15, real_num: 12e0, date: 2025-01-01T00:00:00.000Z } + { magic_num:31, some_text:"somebody", is_summer:false, byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{ foo:67, bar:"qux"}, map: {foo: 12, bar: 50}, double_value: 5e0, decimal_value: 1234.00, tiny_int: 0, small_int: -1, big_int: 0, real_num: 0e0, date: 2025-01-01T00:00:00.000Z } """; Page page = toPage(TEST_COLUMNS, row1, row2); @@ -632,7 +643,8 @@ public void testEncodeMixedCaseColumn() public void testEncodeWithNullField() throws IOException { - List row1 = Arrays.asList(null, null, null, null, null, null, null); + List row1 = Arrays.asList(null, null, null, null, null, null, null, null, null, null, null, null, + null, null); String ionText = """ {} """; @@ -649,15 +661,17 @@ public void testEncodeWithNullNestedField() List.of(1, 2, 3), Arrays.asList(null, "baz"), ImmutableMap.builder() .put("a", 2) .put("b", 5) - .buildOrThrow()); + .buildOrThrow(), 5e0, new SqlDecimal(BigInteger.valueOf(123400), 10, 2), (byte) -1, + (short) 32767, 15L, 12.0f, new SqlDate(toIntExact(LocalDate.of(2025, 1, 1).toEpochDay()))); List row2 = Arrays.asList(31, "somebody", null, new SqlVarbinary(new byte[] {(byte) 0x01, (byte) 0xaa}), List.of(7, 8, 9), Arrays.asList(null, "qux"), ImmutableMap.builder() .put("foo", 12) .put("bar", 50) - .buildOrThrow()); + .buildOrThrow(), 5e0, new SqlDecimal(BigInteger.valueOf(123400), 10, 2), (byte) 0, (short) -1, 0L, + 0.0f, new SqlDate(toIntExact(LocalDate.of(2025, 1, 1).toEpochDay()))); String ionText = """ - { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{bar:"baz"}, map: {a: 2, b: 5}} - { magic_num:31, some_text:"somebody", byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{bar:"qux"}, map: {foo: 12, bar: 50}} + { magic_num:17, some_text:"something", is_summer:true, byte_clob:{{/w==}}, sequencer:[1,2,3], struction:{bar:"baz"}, map: {a: 2, b: 5}, double_value: 5e0, decimal_value: 1234.00, tiny_int: -1, small_int: 32767, big_int: 15, real_num: 12e0, date: 2025-01-01T00:00:00.000Z } + { magic_num:31, some_text:"somebody", byte_clob:{{Aao=}}, sequencer:[7,8,9], struction:{bar:"qux"}, map: {foo: 12, bar: 50}, double_value: 5e0, decimal_value: 1234.00, tiny_int: 0, small_int: -1, big_int: 0, real_num: 0e0, date: 2025-01-01T00:00:00.000Z } """; Page page = toPage(TEST_COLUMNS, row1, row2); From 5af2d2e1742087f2fb3a14af4b009b3fec0b334b Mon Sep 17 00:00:00 2001 From: Rob Marrowstone <99764876+rmarrowstone@users.noreply.github.com> Date: Wed, 15 Jan 2025 10:14:20 -0800 Subject: [PATCH 15/18] Trino PR Fixups This commit addresses comments on the PR to the main trino repo. --- lib/trino-hive-formats/pom.xml | 4 ++-- .../java/io/trino/hive/formats/HiveClassNames.java | 1 - plugin/trino-hive/pom.xml | 2 +- .../java/io/trino/plugin/hive/ion/IonFileWriter.java | 4 ++-- .../java/io/trino/plugin/hive/ion/IonPageSource.java | 12 ++++++------ .../trino/plugin/hive/ion/IonPageSourceFactory.java | 11 ++++++----- pom.xml | 2 ++ 7 files changed, 19 insertions(+), 17 deletions(-) diff --git a/lib/trino-hive-formats/pom.xml b/lib/trino-hive-formats/pom.xml index 72d911457f39..6eae3c69c77a 100644 --- a/lib/trino-hive-formats/pom.xml +++ b/lib/trino-hive-formats/pom.xml @@ -21,13 +21,13 @@ com.amazon.ion ion-java - 1.11.9 + ${dep.ion.version} com.amazon.ion ion-java-path-extraction - 1.5.0 + ${dep.ion-path-extraction.version} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java index 01ccab194f98..f381758603f8 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java @@ -30,7 +30,6 @@ public final class HiveClassNames public static final String ION_SERDE_CLASS = "com.amazon.ionhiveserde.IonHiveSerDe"; public static final String ION_INPUT_FORMAT = "com.amazon.ionhiveserde.formats.IonInputFormat"; public static final String ION_OUTPUT_FORMAT = "com.amazon.ionhiveserde.formats.IonOutputFormat"; - public static final String JSON_SERDE_CLASS = "org.apache.hive.hcatalog.data.JsonSerDe"; public static final String LEGACY_JSON_SERDE_CLASS = "org.apache.hadoop.hive.serde2.JsonSerDe"; public static final String OPENX_JSON_SERDE_CLASS = "org.openx.data.jsonserde.JsonSerDe"; diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index 86d38eb7e9c9..60ca167c9f2f 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -18,7 +18,7 @@ com.amazon.ion ion-java - 1.11.9 + ${dep.ion.version} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java index 693c41238138..224bda4302da 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java @@ -95,7 +95,7 @@ public Closeable commit() catch (Exception _) { // ignore } - throw new TrinoException(HIVE_WRITER_CLOSE_ERROR, "Error committing write to Hive", e); + throw new TrinoException(HIVE_WRITER_CLOSE_ERROR, "Error committing write to Ion file", e); } return rollbackAction; } @@ -107,7 +107,7 @@ public void rollback() writer.close(); } catch (IOException e) { - throw new TrinoException(HIVE_WRITER_CLOSE_ERROR, "Error rolling back write to Hive", e); + throw new TrinoException(HIVE_WRITER_CLOSE_ERROR, "Error rolling back write to Ion file", e); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java index b2472b78ea58..ca78a598caa1 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java @@ -16,6 +16,7 @@ import com.amazon.ion.IonBufferConfiguration; import com.amazon.ion.IonReader; import com.amazon.ion.IonType; +import io.trino.hive.formats.TrinoDataInputStream; import io.trino.hive.formats.ion.IonDecoder; import io.trino.spi.Page; import io.trino.spi.PageBuilder; @@ -23,7 +24,6 @@ import java.io.IOException; import java.util.OptionalLong; -import java.util.function.LongSupplier; import static io.airlift.slice.SizeOf.instanceSize; @@ -32,26 +32,26 @@ public class IonPageSource { private static final int INSTANCE_SIZE = instanceSize(IonPageSource.class); + private final TrinoDataInputStream inputStream; private final IonReader ionReader; private final PageBuilder pageBuilder; private final IonDecoder decoder; - private final LongSupplier counter; private int completedPositions; private boolean finished; - public IonPageSource(IonReader ionReader, LongSupplier counter, IonDecoder decoder, PageBuilder pageBuilder) + public IonPageSource(IonReader ionReader, TrinoDataInputStream inputStream, IonDecoder decoder, PageBuilder pageBuilder) { this.ionReader = ionReader; + this.inputStream = inputStream; this.decoder = decoder; this.pageBuilder = pageBuilder; - this.counter = counter; this.completedPositions = 0; } @Override public long getCompletedBytes() { - return counter.getAsLong(); + return inputStream.getReadBytes(); } @Override @@ -63,7 +63,7 @@ public OptionalLong getCompletedPositions() @Override public long getReadTimeNanos() { - return 0; + return inputStream.getReadTimeNanos(); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java index d7b051c26896..146f3153c784 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java @@ -17,12 +17,12 @@ import com.amazon.ion.system.IonReaderBuilder; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.io.CountingInputStream; import com.google.inject.Inject; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; +import io.trino.hive.formats.TrinoDataInputStream; import io.trino.hive.formats.compression.Codec; import io.trino.hive.formats.compression.CompressionKind; import io.trino.hive.formats.ion.IonDecoder; @@ -150,13 +150,14 @@ public Optional createPageSource( try { Optional codec = CompressionKind.forFile(inputFile.location().fileName()) .map(CompressionKind::createCodec); - CountingInputStream countingInputStream = new CountingInputStream(inputFile.newStream()); + + TrinoDataInputStream trinoInputStream = new TrinoDataInputStream(inputFile.newStream()); InputStream inputStream; if (codec.isPresent()) { - inputStream = codec.get().createStreamDecompressor(countingInputStream); + inputStream = codec.get().createStreamDecompressor(inputFile.newStream()); } else { - inputStream = countingInputStream; + inputStream = inputFile.newStream(); } IonReader ionReader = IonReaderBuilder @@ -171,7 +172,7 @@ public Optional createPageSource( IonDecoderConfig decoderConfig = IonReaderOptions.decoderConfigFor(schema.serdeProperties()); IonDecoder decoder = IonDecoderFactory.buildDecoder(decoderColumns, decoderConfig, pageBuilder); - IonPageSource pageSource = new IonPageSource(ionReader, countingInputStream::getCount, decoder, pageBuilder); + IonPageSource pageSource = new IonPageSource(ionReader, trinoInputStream, decoder, pageBuilder); return Optional.of(new ReaderPageSource(pageSource, readerProjections)); } diff --git a/pom.xml b/pom.xml index 37b24b583750..e2d9a6bdd8c9 100644 --- a/pom.xml +++ b/pom.xml @@ -201,6 +201,8 @@ 1.45.3 5.3.2 1.7.1 + 1.11.9 + 1.5.0 5.16.0 2.12.7 0.12.6 From 105e9aee686aec46bf0cf9d0617aae1a8d3f736a Mon Sep 17 00:00:00 2001 From: Rob Marrowstone <99764876+rmarrowstone@users.noreply.github.com> Date: Wed, 15 Jan 2025 11:10:08 -0800 Subject: [PATCH 16/18] Ensure that FileWriterFactory Checks Hive Feature Flag and Unimplemented Props Before this change the FileWriterFactory was not checking if hive.ion.nativetrino is True, nor if there are non-default values for unimplemented SerDe properties. This change fixes that. I combined IonReaderOptions and IonWriterOptions into a single IonSerDeProperties to avoid code duplication around the unimplemented check. --- .../hive/formats/ion/IonEncoderFactory.java | 4 +- .../trino/plugin/hive/ion/IonFileWriter.java | 2 +- .../plugin/hive/ion/IonFileWriterFactory.java | 14 +- .../plugin/hive/ion/IonPageSourceFactory.java | 57 +------ .../plugin/hive/ion/IonReaderOptions.java | 59 -------- .../plugin/hive/ion/IonSerDeProperties.java | 142 ++++++++++++++++++ .../plugin/hive/ion/IonWriterOptions.java | 74 --------- .../io/trino/plugin/hive/HiveTestUtils.java | 2 +- .../plugin/hive/TestHiveFileFormats.java | 2 +- .../hive/ion/IonPageSourceSmokeTest.java | 94 ++++++------ 10 files changed, 214 insertions(+), 236 deletions(-) delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonSerDeProperties.java delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java index 0a75f2ca3504..904450b6cb0f 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonEncoderFactory.java @@ -148,7 +148,9 @@ private void encodeStruct(IonWriter writer, IntFunction blockSelector, in { writer.stepIn(IonType.STRUCT); for (int i = 0; i < fieldEncoders.size(); i++) { - // Omit the filed when the field is null + // fields are omitted by default, as was true in the hive serde. + // there is an unimplemented hive legacy property of `ion.serialize_null` + // that could be used to specify typed or untyped ion nulls instead. Block block = blockSelector.apply(i); if (block.isNull(position)) { continue; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java index 224bda4302da..1e2c4b1ff6d3 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriter.java @@ -51,7 +51,7 @@ public IonFileWriter( Closeable rollbackAction, TypeManager typeManager, Optional compressionKind, - IonWriterOptions.IonEncoding ionEncoding, + IonSerDeProperties.IonEncoding ionEncoding, List columns) throws IOException { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java index 83acc5162327..7290be076b8e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonFileWriterFactory.java @@ -23,6 +23,7 @@ import io.trino.metastore.StorageFormat; import io.trino.plugin.hive.FileWriter; import io.trino.plugin.hive.HiveCompressionCodec; +import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HiveFileWriterFactory; import io.trino.plugin.hive.WriterKind; import io.trino.plugin.hive.acid.AcidTransaction; @@ -42,7 +43,6 @@ import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static io.trino.plugin.hive.HiveErrorCode.HIVE_WRITER_OPEN_ERROR; import static io.trino.plugin.hive.HiveSessionProperties.getTimestampPrecision; -import static io.trino.plugin.hive.ion.IonWriterOptions.getIonEncoding; import static io.trino.plugin.hive.util.HiveTypeUtil.getType; import static io.trino.plugin.hive.util.HiveUtil.getColumnNames; import static io.trino.plugin.hive.util.HiveUtil.getColumnTypes; @@ -52,14 +52,17 @@ public class IonFileWriterFactory { private final TrinoFileSystemFactory fileSystemFactory; private final TypeManager typeManager; + private final boolean nativeTrinoEnabled; @Inject public IonFileWriterFactory( TrinoFileSystemFactory fileSystemFactory, - TypeManager typeManager) + TypeManager typeManager, + HiveConfig hiveConfig) { this.fileSystemFactory = fileSystemFactory; this.typeManager = typeManager; + this.nativeTrinoEnabled = hiveConfig.getIonNativeTrinoEnabled(); } @Override @@ -75,9 +78,12 @@ public Optional createFileWriter( boolean useAcidSchema, WriterKind writerKind) { - if (!ION_OUTPUT_FORMAT.equals(storageFormat.getOutputFormat())) { + if (!nativeTrinoEnabled + || !ION_OUTPUT_FORMAT.equals(storageFormat.getOutputFormat()) + || IonSerDeProperties.hasUnsupportedProperty(schema)) { return Optional.empty(); } + try { TrinoFileSystem fileSystem = fileSystemFactory.create(session); TrinoOutputFile outputFile = fileSystem.newOutputFile(location); @@ -100,7 +106,7 @@ public Optional createFileWriter( rollbackAction, typeManager, compressionCodec.getHiveCompressionKind(), - getIonEncoding(schema), + IonSerDeProperties.getIonEncoding(schema), columns)); } catch (Exception e) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java index 146f3153c784..50fc23c24d6d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java @@ -15,8 +15,6 @@ import com.amazon.ion.IonReader; import com.amazon.ion.system.IonReaderBuilder; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; @@ -46,11 +44,8 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.OptionalInt; -import java.util.Set; -import java.util.regex.Pattern; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -58,35 +53,14 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT; import static io.trino.plugin.hive.HivePageSourceProvider.projectBaseColumns; import static io.trino.plugin.hive.ReaderPageSource.noProjectionAdaptation; -import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY; -import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY_DEFAULT; -import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN; -import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED; -import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED_DEFAULT; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_DEFAULT; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_PROPERTY; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_PROPERTY; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_TIMESTAMP_OFFSET_DEFAULT; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_TIMESTAMP_OFFSET_PROPERTY; import static io.trino.plugin.hive.util.HiveUtil.splitError; public class IonPageSourceFactory implements HivePageSourceFactory { private final TrinoFileSystemFactory trinoFileSystemFactory; - // this is used as a feature flag to enable Ion native trino integration private final boolean nativeTrinoEnabled; - private static final Map TABLE_PROPERTIES = ImmutableMap.of( - FAIL_ON_OVERFLOW_PROPERTY, FAIL_ON_OVERFLOW_PROPERTY_DEFAULT, - IGNORE_MALFORMED, IGNORE_MALFORMED_DEFAULT, - ION_TIMESTAMP_OFFSET_PROPERTY, ION_TIMESTAMP_OFFSET_DEFAULT, - ION_SERIALIZATION_AS_NULL_PROPERTY, ION_SERIALIZATION_AS_NULL_DEFAULT); - - private static final Set COLUMN_PROPERTIES = ImmutableSet.of( - Pattern.compile(FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN), - Pattern.compile(ION_SERIALIZATION_AS_PROPERTY)); - @Inject public IonPageSourceFactory(TrinoFileSystemFactory trinoFileSystemFactory, HiveConfig hiveConfig) { @@ -110,19 +84,12 @@ public Optional createPageSource( boolean originalFile, AcidTransaction transaction) { - if (!this.nativeTrinoEnabled) { - // this allows user to defer to a legacy hive implementation(like ion-hive-serde) or throw an error based - // on their use case + if (!nativeTrinoEnabled + || !ION_SERDE_CLASS.equals(schema.serializationLibraryName()) + || IonSerDeProperties.hasUnsupportedProperty(schema.serdeProperties())) { return Optional.empty(); } - if (schema.serdeProperties().entrySet().stream().filter(entry -> entry.getKey().startsWith("ion.")).anyMatch(this::isUnsupportedProperty)) { - return Optional.empty(); - } - - if (!ION_SERDE_CLASS.equals(schema.serializationLibraryName())) { - return Optional.empty(); - } checkArgument(acidInfo.isEmpty(), "Acid is not supported for Ion files"); // Skip empty inputs @@ -170,7 +137,7 @@ public Optional createPageSource( .map(hc -> new Column(hc.getName(), hc.getType(), hc.getBaseHiveColumnIndex())) .toList(); - IonDecoderConfig decoderConfig = IonReaderOptions.decoderConfigFor(schema.serdeProperties()); + IonDecoderConfig decoderConfig = IonSerDeProperties.decoderConfigFor(schema.serdeProperties()); IonDecoder decoder = IonDecoderFactory.buildDecoder(decoderColumns, decoderConfig, pageBuilder); IonPageSource pageSource = new IonPageSource(ionReader, trinoInputStream, decoder, pageBuilder); @@ -180,20 +147,4 @@ public Optional createPageSource( throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, splitError(e, path, start, length), e); } } - - private boolean isUnsupportedProperty(Map.Entry entry) - { - String key = entry.getKey(); - String value = entry.getValue(); - - String propertyDefault = TABLE_PROPERTIES.get(key); - if (propertyDefault != null) { - return !propertyDefault.equals(value); - } - - // For now, any column-specific properties result in an empty PageSource - // since they have no default values for comparison. - return COLUMN_PROPERTIES.stream() - .anyMatch(pattern -> pattern.matcher(key).matches()); - } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java deleted file mode 100644 index a87a2280fc67..000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonReaderOptions.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.plugin.hive.ion; - -import com.google.common.collect.ImmutableMap; -import io.trino.hive.formats.ion.IonDecoderConfig; - -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -public final class IonReaderOptions -{ - public static final String STRICT_PATH_TYPING_PROPERTY = "ion.path_extractor.strict"; - public static final String STRICT_PATH_TYPING_DEFAULT = "false"; - public static final String PATH_EXTRACTOR_PROPERTY = "ion.(\\w+).path_extractor"; - public static final String PATH_EXTRACTION_CASE_SENSITIVITY = "ion.path_extractor.case_sensitive"; - public static final String PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT = "false"; - public static final String FAIL_ON_OVERFLOW_PROPERTY_WITH_COLUMN = "ion.\\w+.fail_on_overflow"; - public static final String FAIL_ON_OVERFLOW_PROPERTY = "ion.fail_on_overflow"; - public static final String FAIL_ON_OVERFLOW_PROPERTY_DEFAULT = "true"; - public static final String IGNORE_MALFORMED = "ion.ignore_malformed"; - public static final String IGNORE_MALFORMED_DEFAULT = "false"; - - private static final Pattern pathExtractorPattern = Pattern.compile(PATH_EXTRACTOR_PROPERTY); - - private IonReaderOptions() {} - - public static IonDecoderConfig decoderConfigFor(Map propertiesMap) - { - ImmutableMap.Builder extractionsBuilder = ImmutableMap.builder(); - - for (Map.Entry property : propertiesMap.entrySet()) { - Matcher matcher = pathExtractorPattern.matcher(property.getKey()); - if (matcher.matches()) { - extractionsBuilder.put(matcher.group(1), property.getValue()); - } - } - - Boolean strictTyping = Boolean.parseBoolean( - propertiesMap.getOrDefault(STRICT_PATH_TYPING_PROPERTY, STRICT_PATH_TYPING_DEFAULT)); - Boolean caseSensitive = Boolean.parseBoolean( - propertiesMap.getOrDefault(PATH_EXTRACTION_CASE_SENSITIVITY, PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT)); - - // n.b.: the hive serde overwrote when there were duplicate extractors defined for a column - return new IonDecoderConfig(extractionsBuilder.buildOrThrow(), strictTyping, caseSensitive); - } -} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonSerDeProperties.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonSerDeProperties.java new file mode 100644 index 000000000000..aac0d3735636 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonSerDeProperties.java @@ -0,0 +1,142 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.hive.ion; + +import com.amazon.ion.IonWriter; +import com.amazon.ion.system.IonBinaryWriterBuilder; +import com.amazon.ion.system.IonTextWriterBuilder; +import com.google.common.collect.ImmutableMap; +import io.trino.hive.formats.ion.IonDecoderConfig; +import io.trino.spi.TrinoException; + +import java.io.OutputStream; +import java.util.Locale; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNSUPPORTED_FORMAT; + +public final class IonSerDeProperties +{ + // Reader properties + public static final String STRICT_PATH_TYPING_PROPERTY = "ion.path_extractor.strict"; + public static final String STRICT_PATH_TYPING_DEFAULT = "false"; + public static final String PATH_EXTRACTOR_PROPERTY = "ion.(\\w+).path_extractor"; + public static final String PATH_EXTRACTION_CASE_SENSITIVITY = "ion.path_extractor.case_sensitive"; + public static final String PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT = "false"; + private static final Pattern pathExtractorPattern = Pattern.compile(PATH_EXTRACTOR_PROPERTY); + + // unimplemented reader properties + public static final String FAIL_ON_OVERFLOW_PROPERTY = "ion.fail_on_overflow"; + public static final String FAIL_ON_OVERFLOW_PROPERTY_DEFAULT = "true"; + public static final String COLUMN_FAIL_ON_OVERFLOW_PROPERTY = "ion.\\w+.fail_on_overflow"; + public static final String IGNORE_MALFORMED = "ion.ignore_malformed"; + public static final String IGNORE_MALFORMED_DEFAULT = "false"; + + // Writer properties + public static final String ION_ENCODING_PROPERTY = "ion.encoding"; + public static final String TEXT_ENCODING = "text"; + public static final String BINARY_ENCODING = "binary"; + + // unimplemented writer properties + public static final String ION_TIMESTAMP_OFFSET_PROPERTY = "ion.timestamp.serialization_offset"; + public static final String ION_TIMESTAMP_OFFSET_DEFAULT = "Z"; + public static final String ION_SERIALIZE_NULL_AS_PROPERTY = "ion.serialize_null"; + public static final String ION_SERIALIZE_NULL_AS_DEFAULT = "OMIT"; + public static final String ION_SERIALIZE_AS_PROPERTY = "ion.\\w+.serialize_as"; + + private static final Pattern unsupportedPropertiesRegex = Pattern.compile( + ION_SERIALIZE_AS_PROPERTY + "|" + COLUMN_FAIL_ON_OVERFLOW_PROPERTY); + + private static final Map defaultOnlyProperties = Map.of( + // reader properties + FAIL_ON_OVERFLOW_PROPERTY, FAIL_ON_OVERFLOW_PROPERTY_DEFAULT, + IGNORE_MALFORMED, IGNORE_MALFORMED_DEFAULT, + + // writer properties + ION_TIMESTAMP_OFFSET_PROPERTY, ION_TIMESTAMP_OFFSET_DEFAULT, + ION_SERIALIZE_NULL_AS_PROPERTY, ION_SERIALIZE_NULL_AS_DEFAULT); + + private IonSerDeProperties() {} + + public static IonDecoderConfig decoderConfigFor(Map propertiesMap) + { + ImmutableMap.Builder extractionsBuilder = ImmutableMap.builder(); + + for (Map.Entry property : propertiesMap.entrySet()) { + Matcher matcher = pathExtractorPattern.matcher(property.getKey()); + if (matcher.matches()) { + extractionsBuilder.put(matcher.group(1), property.getValue()); + } + } + + Boolean strictTyping = Boolean.parseBoolean( + propertiesMap.getOrDefault(STRICT_PATH_TYPING_PROPERTY, STRICT_PATH_TYPING_DEFAULT)); + Boolean caseSensitive = Boolean.parseBoolean( + propertiesMap.getOrDefault(PATH_EXTRACTION_CASE_SENSITIVITY, PATH_EXTRACTION_CASE_SENSITIVITY_DEFAULT)); + + return new IonDecoderConfig(extractionsBuilder.buildOrThrow(), strictTyping, caseSensitive); + } + + public enum IonEncoding + { + BINARY { + @Override + public IonWriter createWriter(OutputStream outputStream) + { + return IonBinaryWriterBuilder.standard().build(outputStream); + } + }, + + TEXT { + @Override + public IonWriter createWriter(OutputStream outputStream) + { + return IonTextWriterBuilder.minimal().build(outputStream); + } + }; + + public abstract IonWriter createWriter(OutputStream outputStream); + } + + public static IonEncoding getIonEncoding(Map schema) + { + String encodingStr = schema.getOrDefault(ION_ENCODING_PROPERTY, BINARY_ENCODING); + return switch (encodingStr.toLowerCase(Locale.ROOT)) { + case TEXT_ENCODING -> IonEncoding.TEXT; + case BINARY_ENCODING -> IonEncoding.BINARY; + default -> throw new TrinoException(HIVE_UNSUPPORTED_FORMAT, + "Unsupported Ion encoding format: " + encodingStr); + }; + } + + public static boolean hasUnsupportedProperty(Map properties) + { + return properties.entrySet().stream() + .anyMatch((entry) -> { + String key = entry.getKey(); + String value = entry.getValue(); + if (!key.startsWith("ion.")) { + return false; + } + + if (!defaultOnlyProperties.getOrDefault(key, value).equals(value)) { + return true; + } + + return unsupportedPropertiesRegex.matcher(key).matches(); + }); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java deleted file mode 100644 index b69bb992b16d..000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonWriterOptions.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.plugin.hive.ion; - -import com.amazon.ion.IonWriter; -import com.amazon.ion.system.IonBinaryWriterBuilder; -import com.amazon.ion.system.IonTextWriterBuilder; -import io.trino.spi.TrinoException; - -import java.io.OutputStream; -import java.util.Locale; -import java.util.Map; - -import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNSUPPORTED_FORMAT; - -public final class IonWriterOptions -{ - public static final String ION_ENCODING_PROPERTY = "ion.encoding"; - public static final String ION_TIMESTAMP_OFFSET_PROPERTY = "ion.timestamp.serialization_offset"; - public static final String ION_TIMESTAMP_OFFSET_DEFAULT = "Z"; - public static final String ION_SERIALIZATION_AS_NULL_PROPERTY = "ion.serialize_null"; - public static final String ION_SERIALIZATION_AS_NULL_DEFAULT = "OMIT"; - public static final String ION_SERIALIZATION_AS_PROPERTY = "ion.\\w+.serialize_as"; - - public static final String TEXT_ENCODING = "text"; - public static final String BINARY_ENCODING = "binary"; - - public enum IonEncoding - { - BINARY - { - @Override - public IonWriter createWriter(OutputStream outputStream) - { - return IonBinaryWriterBuilder.standard().build(outputStream); - } - }, - - TEXT - { - @Override - public IonWriter createWriter(OutputStream outputStream) - { - return IonTextWriterBuilder.minimal().build(outputStream); - } - }; - - public abstract IonWriter createWriter(OutputStream outputStream); - } - - public static IonEncoding getIonEncoding(Map schema) - { - String encodingStr = schema.getOrDefault(ION_ENCODING_PROPERTY, BINARY_ENCODING); - return switch (encodingStr.toLowerCase(Locale.ROOT)) { - case TEXT_ENCODING -> IonEncoding.TEXT; - case BINARY_ENCODING -> IonEncoding.BINARY; - default -> throw new TrinoException(HIVE_UNSUPPORTED_FORMAT, - "Unsupported Ion encoding format: " + encodingStr); - }; - } - - private IonWriterOptions() {} -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java index b883576a8177..a50c66f67f20 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java @@ -218,7 +218,7 @@ public static Set getDefaultHiveFileWriterFactories(HiveC .add(new RcFileFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, nodeVersion, hiveConfig)) .add(new OrcFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, nodeVersion, new FileFormatDataSourceStats(), new OrcWriterConfig())) .add(new ParquetFileWriterFactory(fileSystemFactory, nodeVersion, TESTING_TYPE_MANAGER, hiveConfig, new FileFormatDataSourceStats())) - .add(new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) + .add(new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, hiveConfig)) .build(); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index a3f0cc59ef39..87e2e81e48bd 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -387,7 +387,7 @@ public void testIonWithBinaryEncoding(int rowCount, long fileSizePadding) .withColumns(testColumns) .withRowsCount(rowCount) .withFileSizePadding(fileSizePadding) - .withFileWriterFactory(fileSystemFactory -> new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER)) + .withFileWriterFactory(fileSystemFactory -> new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, hiveConfig)) .isReadableByPageSource(fileSystemFactory -> new IonPageSourceFactory(fileSystemFactory, hiveConfig)); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index a806ad4ace26..9dca4f2eede0 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -63,17 +63,18 @@ import static io.trino.plugin.hive.HiveTestUtils.projectedColumn; import static io.trino.plugin.hive.HiveTestUtils.toHiveBaseColumnHandle; import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; -import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY; -import static io.trino.plugin.hive.ion.IonReaderOptions.FAIL_ON_OVERFLOW_PROPERTY_DEFAULT; -import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED; -import static io.trino.plugin.hive.ion.IonReaderOptions.IGNORE_MALFORMED_DEFAULT; -import static io.trino.plugin.hive.ion.IonWriterOptions.BINARY_ENCODING; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_ENCODING_PROPERTY; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_DEFAULT; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_SERIALIZATION_AS_NULL_PROPERTY; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_TIMESTAMP_OFFSET_DEFAULT; -import static io.trino.plugin.hive.ion.IonWriterOptions.ION_TIMESTAMP_OFFSET_PROPERTY; -import static io.trino.plugin.hive.ion.IonWriterOptions.TEXT_ENCODING; +import static io.trino.plugin.hive.ion.IonSerDeProperties.BINARY_ENCODING; +import static io.trino.plugin.hive.ion.IonSerDeProperties.FAIL_ON_OVERFLOW_PROPERTY; +import static io.trino.plugin.hive.ion.IonSerDeProperties.FAIL_ON_OVERFLOW_PROPERTY_DEFAULT; +import static io.trino.plugin.hive.ion.IonSerDeProperties.IGNORE_MALFORMED; +import static io.trino.plugin.hive.ion.IonSerDeProperties.IGNORE_MALFORMED_DEFAULT; +import static io.trino.plugin.hive.ion.IonSerDeProperties.ION_ENCODING_PROPERTY; +import static io.trino.plugin.hive.ion.IonSerDeProperties.ION_SERIALIZE_NULL_AS_DEFAULT; +import static io.trino.plugin.hive.ion.IonSerDeProperties.ION_SERIALIZE_NULL_AS_PROPERTY; +import static io.trino.plugin.hive.ion.IonSerDeProperties.ION_TIMESTAMP_OFFSET_DEFAULT; +import static io.trino.plugin.hive.ion.IonSerDeProperties.ION_TIMESTAMP_OFFSET_PROPERTY; +import static io.trino.plugin.hive.ion.IonSerDeProperties.STRICT_PATH_TYPING_PROPERTY; +import static io.trino.plugin.hive.ion.IonSerDeProperties.TEXT_ENCODING; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; import static io.trino.spi.type.BooleanType.BOOLEAN; @@ -131,11 +132,11 @@ public void testStrictAndLaxPathTyping() defaultFixture.assertRowCount("37 null.timestamp []", 3); TestFixture laxFixture = new TestFixture(FOO_BAR_COLUMNS); - laxFixture.withStrictPathTyping("false"); + laxFixture.withSerdeProperty(STRICT_PATH_TYPING_PROPERTY, "false"); laxFixture.assertRowCount("37 null.timestamp []", 3); TestFixture strictFixture = new TestFixture(FOO_BAR_COLUMNS); - strictFixture.withStrictPathTyping("true"); + strictFixture.withSerdeProperty(STRICT_PATH_TYPING_PROPERTY, "true"); Assertions.assertThrows(TrinoException.class, () -> strictFixture.assertRowCount("37 null.timestamp []", 3)); @@ -185,15 +186,20 @@ public void testProjectedColumn() } @Test - public void testPageSourceWithNativeTrinoDisabled() + public void testNativeTrinoDisabled() throws IOException { TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) .withNativeIonDisabled(); - fixture.writeIonTextFile("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }"); - Optional connectorPageSource = fixture.getOptionalPageSource(); - Assertions.assertTrue(connectorPageSource.isEmpty(), "Expected empty page source when native Trino is disabled"); + Assertions.assertTrue( + fixture.getOptionalFileWriter().isEmpty(), + "Expected empty file writer when native trino is disabled"); + + fixture.writeIonTextFile(""); + Assertions.assertTrue( + fixture.getOptionalPageSource().isEmpty(), + "Expected empty page source when native trino is disabled"); } private static Stream> propertiesWithDefaults() @@ -202,7 +208,7 @@ private static Stream> propertiesWithDefaults() entry(FAIL_ON_OVERFLOW_PROPERTY, FAIL_ON_OVERFLOW_PROPERTY_DEFAULT), entry(IGNORE_MALFORMED, IGNORE_MALFORMED_DEFAULT), entry(ION_TIMESTAMP_OFFSET_PROPERTY, ION_TIMESTAMP_OFFSET_DEFAULT), - entry(ION_SERIALIZATION_AS_NULL_PROPERTY, ION_SERIALIZATION_AS_NULL_DEFAULT)); + entry(ION_SERIALIZE_NULL_AS_PROPERTY, ION_SERIALIZE_NULL_AS_DEFAULT)); } private static Stream> propertiesWithValues() @@ -211,13 +217,12 @@ private static Stream> propertiesWithValues() entry(FAIL_ON_OVERFLOW_PROPERTY, "false"), entry(IGNORE_MALFORMED, "true"), entry(ION_TIMESTAMP_OFFSET_PROPERTY, "01:00"), - entry(ION_SERIALIZATION_AS_NULL_PROPERTY, "TYPED"), + entry(ION_SERIALIZE_NULL_AS_PROPERTY, "TYPED"), // These entries represent column-specific properties that are not supported. // Any presence of these properties in the schema will result in an empty PageSource, // regardless of their assigned values. entry("ion.foo.fail_on_overflow", "property_value"), entry("ion.foo.serialize_as", "property_value")); - //entry("ion.foo.path_extractor", "property_value")); } private static Map.Entry entry(String key, String value) @@ -232,10 +237,15 @@ void testPropertiesWithValues(Map.Entry property) { TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) .withSerdeProperty(property.getKey(), property.getValue()); - fixture.writeIonTextFile("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }"); - Optional connectorPageSource = fixture.getOptionalPageSource(); - Assertions.assertTrue(connectorPageSource.isEmpty(), "Expected empty page source when there are unsupported Serde properties"); + Assertions.assertTrue( + fixture.getOptionalFileWriter().isEmpty(), + "Expected empty file writer when there are unsupported Serde properties"); + + fixture.writeIonTextFile("{ }"); + Assertions.assertTrue( + fixture.getOptionalPageSource().isEmpty(), + "Expected empty page source when there are unsupported Serde properties"); } @ParameterizedTest @@ -245,7 +255,15 @@ void testPropertiesWithDefaults(Map.Entry property) { TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) .withSerdeProperty(property.getKey(), property.getValue()); - fixture.assertRowCount("{ foo: 31, bar: baz } { foo: 31, bar: \"baz\" }", 2); + + Assertions.assertTrue( + fixture.getOptionalFileWriter().isPresent(), + "Expected present file writer when there are unsupported Serde properties"); + + fixture.writeIonTextFile(""); + Assertions.assertTrue( + fixture.getOptionalPageSource().isPresent(), + "Expected present page source when there are unsupported Serde properties"); } @Test @@ -275,7 +293,7 @@ public void testBadEncodingName() throws IOException { TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS) - .withEncoding("unknown_encoding_name"); + .withSerdeProperty(ION_ENCODING_PROPERTY, "unknown_encoding_name"); Assertions.assertThrows(TrinoException.class, fixture::getFileWriter); } @@ -285,7 +303,7 @@ private void assertEncoding(List tableColumns, throws IOException { TestFixture fixture = new TestFixture(tableColumns) - .withEncoding(encoding); + .withSerdeProperty(ION_ENCODING_PROPERTY, encoding); writeTestData(fixture.getFileWriter()); byte[] inputStreamBytes = fixture.getTrinoInputFile() @@ -346,24 +364,12 @@ private static class TestFixture hiveConfig.setIonNativeTrinoEnabled(true); } - TestFixture withEncoding(String encoding) - { - tableProperties.put(ION_ENCODING_PROPERTY, encoding); - return this; - } - TestFixture withNativeIonDisabled() { hiveConfig.setIonNativeTrinoEnabled(false); return this; } - TestFixture withStrictPathTyping(String strict) - { - tableProperties.put(IonReaderOptions.STRICT_PATH_TYPING_PROPERTY, strict); - return this; - } - TestFixture withSerdeProperty(String key, String value) { tableProperties.put(key, value); @@ -433,9 +439,9 @@ int writeIonTextFile(String ionText) return bytes.length; } - FileWriter getFileWriter() + Optional getOptionalFileWriter() { - return new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER) + return new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, hiveConfig) .createFileWriter( fileLocation, columns.stream().map(HiveColumnHandle::getName).collect(toList()), @@ -446,8 +452,12 @@ FileWriter getFileWriter() OptionalInt.empty(), NO_ACID_TRANSACTION, false, - WriterKind.INSERT) - .orElseThrow(); + WriterKind.INSERT); + } + + FileWriter getFileWriter() + { + return getOptionalFileWriter().orElseThrow(); } TrinoInputFile getTrinoInputFile() From f6ed5943c23622f75b798a53a1e2a8b7706e0651 Mon Sep 17 00:00:00 2001 From: Rob Marrowstone Date: Wed, 22 Jan 2025 10:41:30 -0800 Subject: [PATCH 17/18] Fix Telemetry Methods in IonPageSource This change fixes issues with the collection of readTimeNanos and completedBytes from the IonPageSource. It includes tests to verify behavior for both compressed and uncompressed inputs. --- .../trino/plugin/hive/ion/IonPageSource.java | 19 +++++--- .../plugin/hive/ion/IonPageSourceFactory.java | 14 +++--- .../hive/ion/IonPageSourceSmokeTest.java | 46 +++++++++++++++++++ 3 files changed, 65 insertions(+), 14 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java index ca78a598caa1..6c1ab4cca7a0 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSource.java @@ -16,7 +16,6 @@ import com.amazon.ion.IonBufferConfiguration; import com.amazon.ion.IonReader; import com.amazon.ion.IonType; -import io.trino.hive.formats.TrinoDataInputStream; import io.trino.hive.formats.ion.IonDecoder; import io.trino.spi.Page; import io.trino.spi.PageBuilder; @@ -24,6 +23,7 @@ import java.io.IOException; import java.util.OptionalLong; +import java.util.function.LongSupplier; import static io.airlift.slice.SizeOf.instanceSize; @@ -32,26 +32,28 @@ public class IonPageSource { private static final int INSTANCE_SIZE = instanceSize(IonPageSource.class); - private final TrinoDataInputStream inputStream; private final IonReader ionReader; + private final LongSupplier counter; private final PageBuilder pageBuilder; private final IonDecoder decoder; + private long readTimeNanos; private int completedPositions; private boolean finished; - public IonPageSource(IonReader ionReader, TrinoDataInputStream inputStream, IonDecoder decoder, PageBuilder pageBuilder) + public IonPageSource(IonReader ionReader, LongSupplier counter, IonDecoder decoder, PageBuilder pageBuilder) { this.ionReader = ionReader; - this.inputStream = inputStream; + this.counter = counter; this.decoder = decoder; this.pageBuilder = pageBuilder; this.completedPositions = 0; + this.readTimeNanos = 0; } @Override public long getCompletedBytes() { - return inputStream.getReadBytes(); + return counter.getAsLong(); } @Override @@ -63,7 +65,7 @@ public OptionalLong getCompletedPositions() @Override public long getReadTimeNanos() { - return inputStream.getReadTimeNanos(); + return readTimeNanos; } @Override @@ -106,13 +108,18 @@ public void close() private boolean readNextValue() { + long start = System.nanoTime(); final IonType type = ionReader.next(); + if (type == null) { + readTimeNanos += System.nanoTime() - start; return false; } pageBuilder.declarePosition(); decoder.decode(ionReader); + + readTimeNanos += System.nanoTime() - start; return true; } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java index 50fc23c24d6d..7c5150eec1fd 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ion/IonPageSourceFactory.java @@ -15,12 +15,12 @@ import com.amazon.ion.IonReader; import com.amazon.ion.system.IonReaderBuilder; +import com.google.common.io.CountingInputStream; import com.google.inject.Inject; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; -import io.trino.hive.formats.TrinoDataInputStream; import io.trino.hive.formats.compression.Codec; import io.trino.hive.formats.compression.CompressionKind; import io.trino.hive.formats.ion.IonDecoder; @@ -118,18 +118,15 @@ public Optional createPageSource( Optional codec = CompressionKind.forFile(inputFile.location().fileName()) .map(CompressionKind::createCodec); - TrinoDataInputStream trinoInputStream = new TrinoDataInputStream(inputFile.newStream()); + CountingInputStream countingInputStream = new CountingInputStream(inputFile.newStream()); InputStream inputStream; if (codec.isPresent()) { - inputStream = codec.get().createStreamDecompressor(inputFile.newStream()); + inputStream = codec.get().createStreamDecompressor(countingInputStream); } else { - inputStream = inputFile.newStream(); + inputStream = countingInputStream; } - IonReader ionReader = IonReaderBuilder - .standard() - .build(inputStream); PageBuilder pageBuilder = new PageBuilder(projectedReaderColumns.stream() .map(HiveColumnHandle::getType) .toList()); @@ -139,7 +136,8 @@ public Optional createPageSource( IonDecoderConfig decoderConfig = IonSerDeProperties.decoderConfigFor(schema.serdeProperties()); IonDecoder decoder = IonDecoderFactory.buildDecoder(decoderColumns, decoderConfig, pageBuilder); - IonPageSource pageSource = new IonPageSource(ionReader, trinoInputStream, decoder, pageBuilder); + IonReader ionReader = IonReaderBuilder.standard().build(inputStream); + IonPageSource pageSource = new IonPageSource(ionReader, countingInputStream::getCount, decoder, pageBuilder); return Optional.of(new ReaderPageSource(pageSource, readerProjections)); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java index 9dca4f2eede0..a5f15a59a86d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/ion/IonPageSourceSmokeTest.java @@ -16,11 +16,14 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; import io.trino.filesystem.memory.MemoryFileSystemFactory; +import io.trino.hive.formats.compression.CompressionKind; import io.trino.metastore.HiveType; import io.trino.plugin.hive.FileWriter; import io.trino.plugin.hive.HiveColumnHandle; @@ -185,6 +188,36 @@ public void testProjectedColumn() 1); } + @Test + public void testPageSourceTelemetryUncompressed() + throws IOException + { + TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS); + int bytes = fixture.writeIonTextFile("{ foo: 17, bar: baz } { foo: 31, bar: qux }"); + + ConnectorPageSource pageSource = fixture.getPageSource(); + + Assertions.assertNotNull(pageSource.getNextPage()); + Assertions.assertEquals(2, pageSource.getCompletedPositions().getAsLong()); + Assertions.assertEquals(bytes, pageSource.getCompletedBytes()); + Assertions.assertTrue(pageSource.getReadTimeNanos() > 0); + } + + @Test + public void testPageSourceTelemetryCompressed() + throws IOException + { + TestFixture fixture = new TestFixture(FOO_BAR_COLUMNS); + int bytes = fixture.writeZstdCompressedIonText("{ foo: 17, bar: baz } { foo: 31, bar: qux }"); + + ConnectorPageSource pageSource = fixture.getPageSource(); + + Assertions.assertNotNull(pageSource.getNextPage()); + Assertions.assertEquals(2, pageSource.getCompletedPositions().getAsLong()); + Assertions.assertEquals(bytes, pageSource.getCompletedBytes()); + Assertions.assertTrue(pageSource.getReadTimeNanos() > 0); + } + @Test public void testNativeTrinoDisabled() throws IOException @@ -439,6 +472,19 @@ int writeIonTextFile(String ionText) return bytes.length; } + int writeZstdCompressedIonText(String ionText) + throws IOException + { + fileLocation = Location.of(TEST_ION_LOCATION + ".zst"); + TrinoOutputFile outputFile = fileSystemFactory.create(getSession()).newOutputFile(fileLocation); + + Slice textSlice = Slices.wrappedBuffer(ionText.getBytes(StandardCharsets.UTF_8)); + Slice compressed = CompressionKind.ZSTD.createCodec().createValueCompressor().compress(textSlice); + outputFile.createOrOverwrite(compressed.getBytes()); + + return compressed.length(); + } + Optional getOptionalFileWriter() { return new IonFileWriterFactory(fileSystemFactory, TESTING_TYPE_MANAGER, hiveConfig) From 5c0950a983f1cff138f2ca469aea5e937a11f55d Mon Sep 17 00:00:00 2001 From: Rob Marrowstone Date: Tue, 28 Jan 2025 17:01:18 -0800 Subject: [PATCH 18/18] Preserve Legacy Behavior for Duplicate Fields in Nested Rows This change makes it so that nested rows keep only the first of duplicate fields. This is different from top-level struct fields and maps, which both keep the last value. I'd like to have a SerDe property to manage this behavior but don't want to add any more complexity at this point. --- .../hive/formats/ion/IonDecoderFactory.java | 7 ++-- .../trino/hive/formats/ion/TestIonFormat.java | 35 +++++++++++-------- 2 files changed, 22 insertions(+), 20 deletions(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java index e32da72b5c81..d31963bd9301 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/ion/IonDecoderFactory.java @@ -235,13 +235,10 @@ private void decode(IonReader ionReader, IntFunction blockSelector continue; } final BlockBuilder blockBuilder = blockSelector.apply(fieldIndex); - if (encountered[fieldIndex]) { - blockBuilder.resetTo(blockBuilder.getPositionCount() - 1); - } - else { + if (!encountered[fieldIndex]) { + fieldDecoders.get(fieldIndex).decode(ionReader, blockBuilder); encountered[fieldIndex] = true; } - fieldDecoders.get(fieldIndex).decode(ionReader, blockBuilder); } for (int i = 0; i < encountered.length; i++) { diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java index 2fd4841a1354..527075fa0300 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/ion/TestIonFormat.java @@ -292,20 +292,6 @@ public void testStructWithNullAndMissingValues() Arrays.asList(null, null)); } - @Test - public void testStructWithDuplicateKeys() - throws IOException - { - // this test is not making a value judgement; capturing the last - // is not necessarily the "right" behavior. the test just - // documents what the behavior is, which is based on the behavior - // of the hive serde, and is consistent with the trino json parser. - assertValues( - RowType.rowType(field("foo", INTEGER)), - "{ foo: 17, foo: 31, foo: 53 } { foo: 67 }", - List.of(53), List.of(67)); - } - @Test public void testNestedList() throws IOException @@ -330,6 +316,25 @@ public void testNestedStruct() List.of(List.of("Woody", "Guthrie"))); } + /** + * The Ion Hive SerDe captures the last value for fields with + * duplicate keys. There is different behavior for nested Rows, + * which you can see below. + */ + @Test + public void testTopLevelStructWithDuplicateKeys() + throws IOException + { + assertValues( + RowType.rowType(field("foo", INTEGER)), + "{ foo: 17, foo: 31, foo: 53 } { foo: 67 }", + List.of(53), List.of(67)); + } + + /** + * The Ion Hive SerDe captures the first value for duplicate fields in + * nested Rows, so that is what we default to here. + */ @Test public void testNestedStructWithDuplicateAndMissingKeys() throws IOException @@ -340,7 +345,7 @@ public void testNestedStructWithDuplicateAndMissingKeys() field("first", VARCHAR), field("last", VARCHAR)))), """ - { name: { last: Godfrey, last: Guthrie } } + { name: { last: Guthrie, last: Godfrey } } { name: { first: Joan, last: Baez } } """, List.of(Arrays.asList(null, "Guthrie")),