diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index ae23d5e10dd8..7a222915d8dd 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -351,6 +351,7 @@ jobs: !:trino-jdbc, !:trino-kafka, !:trino-lakehouse, + !:trino-lance, !:trino-main, !:trino-mariadb, !:trino-memory, @@ -472,6 +473,7 @@ jobs: - { modules: plugin/trino-ignite } - { modules: plugin/trino-kafka } - { modules: plugin/trino-lakehouse } + - { modules: plugin/trino-lance } - { modules: plugin/trino-mariadb } - { modules: plugin/trino-mongodb } - { modules: plugin/trino-mysql } @@ -893,6 +895,7 @@ jobs: - suite-storage-formats-detailed - suite-parquet - suite-oauth2 + - suite-lance - suite-ldap - suite-compatibility - suite-all-connectors-smoke diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index 97a366b39290..aedc8d966ca1 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -157,6 +157,12 @@ + + + + + + diff --git a/docs/src/main/sphinx/connector.md b/docs/src/main/sphinx/connector.md index a954b30cf059..13130b947854 100644 --- a/docs/src/main/sphinx/connector.md +++ b/docs/src/main/sphinx/connector.md @@ -26,6 +26,7 @@ Ignite JMX Kafka Lakehouse +Lance Loki MariaDB Memory diff --git a/docs/src/main/sphinx/connector/lance.md b/docs/src/main/sphinx/connector/lance.md new file mode 100644 index 000000000000..61b5267fcd96 --- /dev/null +++ b/docs/src/main/sphinx/connector/lance.md @@ -0,0 +1,46 @@ +# Lance connector + +## General configuration + +To configure the Lance connector, create a catalog properties file `etc/catalog/example.properties` with the following content + +```text +connector.name=lance +``` + +You must configure a [namespace](https://lancedb.github.io/lance/format/namespace/) type. +Currently only [directory namespace](https://lancedb.github.io/lance/format/namespace/impls/dir/) is supported. + +```text +lance.namespace.type=directory +``` + +## Lance Namespace configuration +### Directory namespace +Lance directory namespace is a lightweight and simple single-level Lance namespace that contains only a list of tables. All tables reside in the default namespace. + +The following configuration properties are available: + +:::{list-table} +:widths: 30, 10, 60 +:header-rows: 1 + +* - Property name + - Required + - Description +* - `lance.namespace.directory.warehouse.location` + - Yes + - The root directory URI of the namespace where tables are stored. +::: + + +## File system access configuration + +The connector supports accessing the following file systems: + +* [](/object-storage/file-system-azure) +* [](/object-storage/file-system-gcs) +* [](/object-storage/file-system-s3) +* [](/object-storage/file-system-hdfs) + +You must enable and configure the specific file system access. diff --git a/lib/trino-lance-file/pom.xml b/lib/trino-lance-file/pom.xml new file mode 100644 index 000000000000..3e177e2c0d13 --- /dev/null +++ b/lib/trino-lance-file/pom.xml @@ -0,0 +1,179 @@ + + + 4.0.0 + + + io.trino + trino-root + 478-SNAPSHOT + ../../pom.xml + + + trino-lance-file + Trino - Lance file format + + + true + ${air.test.jvm.additional-arguments.default} + --add-opens=java.base/java.nio=ALL-UNNAMED + --sun-misc-unsafe-memory-access=allow + + + + + build.buf.gen + lancedb_lance_protocolbuffers_java + + + + com.google.errorprone + error_prone_annotations + + + + com.google.guava + guava + + + + com.google.protobuf + protobuf-java + + + + io.github.luohao + fastlanes-java + 1 + + + + io.trino + trino-filesystem + + + + io.trino + trino-memory-context + + + + io.trino + trino-plugin-toolkit + + + + it.unimi.dsi + fastutil + + + + jakarta.annotation + jakarta.annotation-api + + + + io.airlift + slice + provided + + + + io.trino + trino-spi + provided + + + + org.openjdk.jol + jol-core + provided + + + + com.lancedb + lance-core + test + + + + io.airlift + junit-extensions + test + + + + io.trino + trino-main + test + + + + io.trino + trino-testing-services + test + + + + org.apache.arrow + arrow-memory-core + test + + + + org.apache.arrow + arrow-vector + test + + + + org.apache.calcite.avatica + avatica-core + 1.25.0 + test + + + + org.assertj + assertj-core + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.junit.jupiter + junit-jupiter-engine + test + + + + org.openjdk.jmh + jmh-core + test + + + + org.openjdk.jmh + jmh-generator-annprocess + test + + + + + + + true + + + false + + buf + Buf Maven Repository + https://buf.build/gen/maven + + + diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/AbstractLanceDataSource.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/AbstractLanceDataSource.java new file mode 100644 index 000000000000..985ad112f005 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/AbstractLanceDataSource.java @@ -0,0 +1,118 @@ +/* + * 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.lance.file; + +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; + +import java.io.IOException; + +import static java.lang.Math.min; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public abstract class AbstractLanceDataSource + implements LanceDataSource +{ + private final LanceDataSourceId id; + private final long estimatedSize; + private long readTimeNanos; + private long readBytes; + + public AbstractLanceDataSource(LanceDataSourceId id, long estimatedSize) + { + this.id = requireNonNull(id, "id is null"); + this.estimatedSize = estimatedSize; + } + + @Override + public LanceDataSourceId getId() + { + return id; + } + + @Override + public long getReadBytes() + { + return readBytes; + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos; + } + + @Override + public long getEstimatedSize() + { + return estimatedSize; + } + + @Override + public Slice readTail(int length) + throws IOException + { + long start = System.nanoTime(); + + Slice tailSlice = readTailInternal(length); + + readTimeNanos += System.nanoTime() - start; + readBytes += tailSlice.length(); + + return tailSlice; + } + + protected Slice readTailInternal(int length) + throws IOException + { + int readSize = toIntExact(min(estimatedSize, length)); + return readFully(estimatedSize - readSize, readSize); + } + + @Override + public long getRetainedSize() + { + return 0; + } + + @Override + public final Slice readFully(long position, int length) + throws IOException + { + byte[] buffer = new byte[length]; + readFully(position, buffer, 0, length); + return Slices.wrappedBuffer(buffer); + } + + protected abstract void readInternal(long position, byte[] buffer, int bufferOffset, int bufferLength) + throws IOException; + + private void readFully(long position, byte[] buffer, int bufferOffset, int bufferLength) + throws IOException + { + long start = System.nanoTime(); + + readInternal(position, buffer, bufferOffset, bufferLength); + + readTimeNanos += System.nanoTime() - start; + readBytes += bufferLength; + } + + @Override + public final String toString() + { + return id.toString(); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/FileLanceDataSource.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/FileLanceDataSource.java new file mode 100644 index 000000000000..59d9974f5806 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/FileLanceDataSource.java @@ -0,0 +1,47 @@ +/* + * 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.lance.file; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.RandomAccessFile; + +public class FileLanceDataSource + extends AbstractLanceDataSource +{ + private final RandomAccessFile input; + + public FileLanceDataSource(File path) + throws FileNotFoundException + { + super(new LanceDataSourceId(path.getPath()), path.length()); + this.input = new RandomAccessFile(path, "r"); + } + + @Override + protected void readInternal(long position, byte[] buffer, int bufferOffset, int bufferLength) + throws IOException + { + input.seek(position); + input.readFully(buffer, bufferOffset, bufferLength); + } + + @Override + public void close() + throws IOException + { + input.close(); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceDataSource.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceDataSource.java new file mode 100644 index 000000000000..daf21e8c83ed --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceDataSource.java @@ -0,0 +1,45 @@ +/* + * 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.lance.file; + +import io.airlift.slice.Slice; + +import java.io.Closeable; +import java.io.IOException; + +public interface LanceDataSource + extends Closeable +{ + LanceDataSourceId getId(); + + long getReadBytes(); + + long getReadTimeNanos(); + + long getEstimatedSize(); + + long getRetainedSize(); + + Slice readTail(int length) + throws IOException; + + Slice readFully(long position, int length) + throws IOException; + + @Override + default void close() + throws IOException + { + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceDataSourceId.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceDataSourceId.java new file mode 100644 index 000000000000..f4bbf1dfe22b --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceDataSourceId.java @@ -0,0 +1,53 @@ +/* + * 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.lance.file; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public final class LanceDataSourceId +{ + private final String id; + + public LanceDataSourceId(String id) + { + this.id = requireNonNull(id, "id is null"); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LanceDataSourceId that = (LanceDataSourceId) o; + return Objects.equals(id, that.id); + } + + @Override + public int hashCode() + { + return Objects.hash(id); + } + + @Override + public String toString() + { + return id; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceReader.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceReader.java new file mode 100644 index 000000000000..d8a4ac85b6f7 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/LanceReader.java @@ -0,0 +1,338 @@ +/* + * 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.lance.file; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import com.google.errorprone.annotations.CheckReturnValue; +import io.airlift.slice.Slice; +import io.trino.lance.file.v2.metadata.ColumnMetadata; +import io.trino.lance.file.v2.metadata.DiskRange; +import io.trino.lance.file.v2.metadata.Field; +import io.trino.lance.file.v2.metadata.FileVersion; +import io.trino.lance.file.v2.metadata.Footer; +import io.trino.lance.file.v2.metadata.LanceTypeUtil; +import io.trino.lance.file.v2.reader.ColumnReader; +import io.trino.lance.file.v2.reader.Range; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.spi.Page; +import io.trino.spi.block.Block; +import io.trino.spi.block.LongArrayBlock; +import io.trino.spi.connector.SourcePage; +import jakarta.annotation.Nullable; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.ObjLongConsumer; +import java.util.stream.IntStream; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.lance.file.v2.metadata.DiskRange.BUFFER_DESCRIPTOR_SIZE; +import static java.lang.Math.min; +import static java.lang.Math.toIntExact; +import static java.util.Objects.checkIndex; + +public class LanceReader + implements Closeable +{ + public static final int FOOTER_LEN = 40; + public static final int MAX_BATCH_SIZE = 8 * 1024; + + private final Footer footer; + private final FileVersion fileVersion; + private final Map columnMetadata; + private final List fields; + private final ColumnReader[] columnReaders; + private final long numRows; + + private int currentPageId; + private long currentRowId; + private long currentBatchSize; + + public LanceReader(LanceDataSource dataSource, + List columnIds, + Optional> requestRanges, + AggregatedMemoryContext memoryUsage) + throws IOException + { + // read footer + Slice data = dataSource.readTail(FOOTER_LEN); + this.footer = Footer.from(data); + this.fileVersion = FileVersion.fromMajorMinor(footer.getMajorVersion(), footer.getMinorVersion()); + + // read Global Buffer Offset Table + Slice bufferOffsetTableSlice = dataSource.readFully(footer.getGlobalBuffOffsetStart(), footer.getNumGlobalBuffers() * BUFFER_DESCRIPTOR_SIZE); + List bufferOffsets = IntStream.range(0, footer.getNumGlobalBuffers()).boxed() + .map(i -> new DiskRange(bufferOffsetTableSlice.getLong(BUFFER_DESCRIPTOR_SIZE * i), bufferOffsetTableSlice.getLong(BUFFER_DESCRIPTOR_SIZE * i + 8))) + .collect(toImmutableList()); + if (bufferOffsets.size() == 0) { + throw new RuntimeException("File did not contain any buffers"); + } + + // read global schema + DiskRange schemaBufferLocation = bufferOffsets.get(0); + // prefetch all metadata + Slice metadataSlice = dataSource.readTail(toIntExact(dataSource.getEstimatedSize() - schemaBufferLocation.position())); + // read file descriptor + Slice schemaSlice = metadataSlice.slice(0, toIntExact(schemaBufferLocation.length())); + build.buf.gen.lance.file.FileDescriptor fileDescriptor = build.buf.gen.lance.file.FileDescriptor.parseFrom(schemaSlice.toByteBuffer()); + checkArgument(fileDescriptor.hasSchema(), "FileDescriptor does not contain a schema"); + this.fields = toFields(fileDescriptor.getSchema()); + List ranges = requestRanges.orElse(ImmutableList.of(new Range(0, fileDescriptor.getLength()))); + this.numRows = ranges.stream() + .mapToLong(Range::length) + .sum(); + // read Column Metadata Offset Table + Slice columnMetadataOffsetsSlice = metadataSlice.slice(toIntExact(footer.getColumnMetadataOffsetsStart() - schemaBufferLocation.position()), toIntExact(footer.getGlobalBuffOffsetStart() - footer.getColumnMetadataOffsetsStart())); + List columnMetadataOffsets = IntStream.range(0, footer.getNumColumns()).boxed() + .map(i -> { + long position = columnMetadataOffsetsSlice.getLong(i * BUFFER_DESCRIPTOR_SIZE); + return new DiskRange(position, columnMetadataOffsetsSlice.getLong(i * BUFFER_DESCRIPTOR_SIZE + 8)); + }) + .collect(toImmutableList()); + + // read Column Metadata + Slice columnMetadataSlice = metadataSlice.slice(toIntExact(footer.getColumnMetadataStart() - schemaBufferLocation.position()), toIntExact(footer.getColumnMetadataOffsetsStart() - footer.getColumnMetadataStart())); + List metadata = IntStream.range(0, footer.getNumColumns()).boxed() + .map(i -> { + DiskRange offset = columnMetadataOffsets.get(i); + Slice message = columnMetadataSlice.slice(toIntExact(offset.position() - footer.getColumnMetadataStart()), toIntExact(offset.length())); + return ColumnMetadata.from(i, message); + }) + .collect(toImmutableList()); + + Map fieldIdMap = LanceTypeUtil.visit(fields, new LanceTypeUtil.FieldIdToColumnIndexVisitor()); + this.columnMetadata = fieldIdMap.entrySet().stream() + .collect(toImmutableMap( + Map.Entry::getKey, + entry -> metadata.get(entry.getValue()))); + + this.columnReaders = fields.stream() + .filter(field -> columnIds.contains(field.id())) + .map(field -> + ColumnReader.createColumnReader( + dataSource, + field, + columnMetadata, + ranges, + memoryUsage)) + .collect(toImmutableList()) + .toArray(ColumnReader[]::new); + } + + public static List toFields(build.buf.gen.lance.file.Schema schema) + { + return toFields(schema.getFieldsList()); + } + + public static List toFields(List fieldsProto) + { + Map fieldMap = Maps.newHashMapWithExpectedSize(fieldsProto.size()); + for (build.buf.gen.lance.file.Field proto : fieldsProto) { + fieldMap.put(proto.getId(), Field.fromProto(proto)); + } + List fields = new ArrayList<>(); + for (Map.Entry entry : fieldMap.entrySet()) { + int parentId = fieldMap.get(entry.getKey()).parentId(); + Field field = entry.getValue(); + if (parentId == -1) { + fields.add(field); + } + else { + fieldMap.get(parentId).addChild(field); + } + } + return fields; + } + + public SourcePage nextSourcePage() + { + currentRowId += currentBatchSize; + currentBatchSize = 0; + + // return null if no more rows + if (currentRowId >= numRows) { + return null; + } + + // TODO: add exponential growth of batch size to unblock consumer faster + currentBatchSize = min(MAX_BATCH_SIZE, numRows - currentRowId); + for (ColumnReader reader : columnReaders) { + if (reader != null) { + reader.prepareNextRead(toIntExact(currentBatchSize)); + } + } + + currentPageId++; + return new LanceSourcePage(toIntExact(currentBatchSize)); + } + + public List getFields() + { + return fields; + } + + public Footer getFooter() + { + return footer; + } + + @Override + public void close() + throws IOException + { + } + + public FileVersion getFileVersion() + { + return fileVersion; + } + + private record SelectedPositions(int positionCount, @Nullable int[] positions) + { + @CheckReturnValue + public Block apply(Block block) + { + if (positions == null) { + return block; + } + return block.getPositions(positions, 0, positionCount); + } + + public Block createRowNumberBlock(long filePosition) + { + long[] rowNumbers = new long[positionCount]; + for (int i = 0; i < positionCount; i++) { + int position = positions == null ? i : positions[i]; + rowNumbers[i] = filePosition + position; + } + return new LongArrayBlock(positionCount, Optional.empty(), rowNumbers); + } + + @CheckReturnValue + public SelectedPositions selectPositions(int[] positions, int offset, int size) + { + if (this.positions == null) { + for (int i = 0; i < size; i++) { + checkIndex(offset + i, positionCount); + } + return new SelectedPositions(size, Arrays.copyOfRange(positions, offset, offset + size)); + } + + int[] newPositions = new int[size]; + for (int i = 0; i < size; i++) { + newPositions[i] = this.positions[positions[offset + i]]; + } + return new SelectedPositions(size, newPositions); + } + } + + private class LanceSourcePage + implements SourcePage + { + private final int expectedPageId = currentPageId; + private final Block[] blocks = new Block[columnReaders.length]; + private SelectedPositions selectedPositions; + private long sizeInBytes; + private long retainedSizeInBytes; + + public LanceSourcePage(int positionCount) + { + selectedPositions = new SelectedPositions(positionCount, null); + } + + @Override + public int getPositionCount() + { + return selectedPositions.positionCount(); + } + + @Override + public long getSizeInBytes() + { + return sizeInBytes; + } + + @Override + public long getRetainedSizeInBytes() + { + return retainedSizeInBytes; + } + + @Override + public void retainedBytesForEachPart(ObjLongConsumer consumer) + { + for (Block block : blocks) { + if (block != null) { + block.retainedBytesForEachPart(consumer); + } + } + } + + @Override + public int getChannelCount() + { + return blocks.length; + } + + @Override + public Block getBlock(int channel) + { + checkState(currentPageId == expectedPageId); + checkIndex(channel, blocks.length); + + Block block = blocks[channel]; + if (block == null) { + block = columnReaders[channel].read().block(); + block = selectedPositions.apply(block); + } + blocks[channel] = block; + sizeInBytes += block.getSizeInBytes(); + retainedSizeInBytes += block.getRetainedSizeInBytes(); + return block; + } + + @Override + public Page getPage() + { + for (int i = 0; i < blocks.length; i++) { + getBlock(i); + } + return new Page(selectedPositions.positionCount(), blocks); + } + + @Override + public void selectPositions(int[] positions, int offset, int size) + { + selectedPositions = selectedPositions.selectPositions(positions, offset, size); + retainedSizeInBytes = 0; + for (int i = 0; i < blocks.length; i++) { + Block block = blocks[i]; + if (block != null) { + block = selectedPositions.apply(block); + retainedSizeInBytes += block.getRetainedSizeInBytes(); + blocks[i] = block; + } + } + } + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/TrinoLanceDataSource.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/TrinoLanceDataSource.java new file mode 100644 index 000000000000..89692165d5a7 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/TrinoLanceDataSource.java @@ -0,0 +1,64 @@ +/* + * 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.lance.file; + +import io.airlift.slice.Slice; +import io.trino.filesystem.TrinoInput; +import io.trino.filesystem.TrinoInputFile; +import io.trino.plugin.base.metrics.FileFormatDataSourceStats; + +import java.io.IOException; + +import static java.util.Objects.requireNonNull; + +public class TrinoLanceDataSource + extends AbstractLanceDataSource +{ + private final FileFormatDataSourceStats stats; + private final TrinoInput input; + + public TrinoLanceDataSource(TrinoInputFile file, FileFormatDataSourceStats stats) + throws IOException + { + super(new LanceDataSourceId(file.location().toString()), file.length()); + this.stats = requireNonNull(stats, "stats is null"); + this.input = file.newInput(); + } + + @Override + public void close() + throws IOException + { + input.close(); + } + + @Override + protected Slice readTailInternal(int length) + throws IOException + { + long readStart = System.nanoTime(); + Slice tail = input.readTail(length); + stats.readDataBytesPerSecond(tail.length(), System.nanoTime() - readStart); + return tail; + } + + @Override + protected void readInternal(long position, byte[] buffer, int bufferOffset, int bufferLength) + throws IOException + { + long readStart = System.nanoTime(); + input.readFully(position, buffer, bufferOffset, bufferLength); + stats.readDataBytesPerSecond(bufferLength, System.nanoTime() - readStart); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/BlockDecoder.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/BlockDecoder.java new file mode 100644 index 000000000000..82f81fc5d3bf --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/BlockDecoder.java @@ -0,0 +1,23 @@ +/* + * 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.lance.file.v2.encoding; + +import io.airlift.slice.Slice; + +public interface BlockDecoder +{ + void init(Slice slice, int numValues); + + void read(int sourceIndex, T destination, int destinationIndex, int length); +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FixedSizeListEncoding.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FixedSizeListEncoding.java new file mode 100644 index 000000000000..40abe5befd1c --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FixedSizeListEncoding.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.lance.file.v2.encoding; + +import io.trino.lance.file.v2.reader.BufferAdapter; + +public class FixedSizeListEncoding + implements LanceEncoding +{ + @Override + public BufferAdapter getBufferAdapter() + { + throw new UnsupportedOperationException("getBufferAdapter is not supported for " + getClass().getSimpleName()); + } + + @Override + public MiniBlockDecoder getMiniBlockDecoder() + { + throw new UnsupportedOperationException("getMiniBlockDecoder is not supported for " + getClass().getSimpleName()); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FlatValueEncoding.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FlatValueEncoding.java new file mode 100644 index 000000000000..33f44dbd05bb --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FlatValueEncoding.java @@ -0,0 +1,265 @@ +/* + * 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.lance.file.v2.encoding; + +import io.airlift.slice.Slice; +import io.trino.lance.file.v2.reader.BufferAdapter; +import io.trino.spi.block.ByteArrayBlock; +import io.trino.spi.block.IntArrayBlock; +import io.trino.spi.block.LongArrayBlock; +import io.trino.spi.block.ShortArrayBlock; +import io.trino.spi.block.ValueBlock; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.lance.file.v2.reader.ByteArrayBufferAdapter.BYTE_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.IntArrayBufferAdapter.INT_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.LongArrayBufferAdapter.LONG_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.ShortArrayBufferAdapter.SHORT_ARRAY_BUFFER_ADAPTER; + +public class FlatValueEncoding + implements LanceEncoding +{ + private final int bytesPerValue; + + public FlatValueEncoding(int bytesPerValue) + { + checkArgument(bytesPerValue > 0, "bytesPerValue must be greater than 0"); + this.bytesPerValue = bytesPerValue; + } + + public int getBytesPerValue() + { + return bytesPerValue; + } + + @Override + public ValueBlock decodeBlock(Slice slice, int count) + { + return decode(slice, count); + } + + @Override + public BufferAdapter getBufferAdapter() + { + return switch (bytesPerValue) { + case 1 -> BYTE_ARRAY_BUFFER_ADAPTER; + case 2 -> SHORT_ARRAY_BUFFER_ADAPTER; + case 4 -> INT_ARRAY_BUFFER_ADAPTER; + case 8 -> LONG_ARRAY_BUFFER_ADAPTER; + default -> throw new IllegalArgumentException("Invalid bytesPerValue: " + bytesPerValue); + }; + } + + @Override + public MiniBlockDecoder getMiniBlockDecoder() + { + return switch (bytesPerValue) { + case 1 -> new ByteMiniBlockDecoder(); + case 2 -> new ShortMiniBlockDecoder(); + case 4 -> new IntMiniBlockDecoder(); + case 8 -> new LongMiniBlockDecoder(); + default -> throw new IllegalArgumentException("Invalid bytesPerValue: " + bytesPerValue); + }; + } + + @Override + public BlockDecoder getBlockDecoder() + { + return switch (bytesPerValue) { + case 1 -> new ByteBlockDecoder(); + case 2 -> new ShortBlockDecoder(); + case 4 -> new IntBlockDecoder(); + case 8 -> new LongBlockDecoder(); + default -> throw new IllegalArgumentException("Invalid bytesPerValue: " + bytesPerValue); + }; + } + + private ValueBlock decode(Slice slice, int count) + { + return switch (bytesPerValue) { + case 1 -> new ByteArrayBlock(count, Optional.empty(), slice.getBytes(0, count)); + case 2 -> new ShortArrayBlock(count, Optional.empty(), slice.getShorts(0, count)); + case 4 -> new IntArrayBlock(count, Optional.empty(), slice.getInts(0, count)); + case 8 -> new LongArrayBlock(count, Optional.empty(), slice.getLongs(0, count)); + default -> throw new IllegalArgumentException("Invalid bytesPerValue: " + bytesPerValue); + }; + } + + public class ByteBlockDecoder + implements BlockDecoder + { + private Slice slice; + private int numValues; + + @Override + public void init(Slice slice, int numValues) + { + checkArgument(bytesPerValue == 1); + this.slice = slice; + this.numValues = numValues; + } + + @Override + public void read(int sourceIndex, byte[] destination, int destinationIndex, int length) + { + checkArgument(sourceIndex + length <= numValues); + slice.getBytes(sourceIndex, destination, destinationIndex, length); + } + } + + public class ShortBlockDecoder + implements BlockDecoder + { + private Slice slice; + private int numValues; + + @Override + public void init(Slice slice, int numValues) + { + checkArgument(bytesPerValue == 2); + this.slice = slice; + this.numValues = numValues; + } + + @Override + public void read(int sourceIndex, short[] destination, int destinationIndex, int length) + { + checkArgument(sourceIndex + length <= numValues); + slice.getShorts(sourceIndex * Short.BYTES, destination, destinationIndex, length); + } + } + + public class IntBlockDecoder + implements BlockDecoder + { + private Slice slice; + private int numValues; + + @Override + public void init(Slice slice, int numValues) + { + checkArgument(bytesPerValue == 4); + this.slice = slice; + this.numValues = numValues; + } + + @Override + public void read(int sourceIndex, int[] destination, int destinationIndex, int length) + { + checkArgument(sourceIndex + length <= numValues); + slice.getInts(sourceIndex * Integer.BYTES, destination, destinationIndex, length); + } + } + + public class LongBlockDecoder + implements BlockDecoder + { + private Slice slice; + private int numValues; + + @Override + public void init(Slice slice, int numValues) + { + checkArgument(bytesPerValue == 8); + this.slice = slice; + this.numValues = numValues; + } + + @Override + public void read(int sourceIndex, long[] destination, int destinationIndex, int length) + { + checkArgument(sourceIndex + length <= numValues); + slice.getLongs(sourceIndex * Long.BYTES, destination, destinationIndex, length); + } + } + + public class ByteMiniBlockDecoder + implements MiniBlockDecoder + { + private final ByteBlockDecoder blockDecoder = new ByteBlockDecoder(); + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + blockDecoder.init(slices.getFirst(), numValues); + } + + @Override + public void read(int sourceIndex, byte[] destination, int destinationIndex, int length) + { + blockDecoder.read(sourceIndex, destination, destinationIndex, length); + } + } + + public class ShortMiniBlockDecoder + implements MiniBlockDecoder + { + private final ShortBlockDecoder blockDecoder = new ShortBlockDecoder(); + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + blockDecoder.init(slices.getFirst(), numValues); + } + + @Override + public void read(int sourceIndex, short[] destination, int destinationIndex, int length) + { + blockDecoder.read(sourceIndex, destination, destinationIndex, length); + } + } + + public class IntMiniBlockDecoder + implements MiniBlockDecoder + { + private final IntBlockDecoder blockDecoder = new IntBlockDecoder(); + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + blockDecoder.init(slices.getFirst(), numValues); + } + + @Override + public void read(int sourceIndex, int[] destination, int destinationIndex, int length) + { + blockDecoder.read(sourceIndex, destination, destinationIndex, length); + } + } + + public class LongMiniBlockDecoder + implements MiniBlockDecoder + { + private final LongBlockDecoder blockDecoder = new LongBlockDecoder(); + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + blockDecoder.init(slices.getFirst(), numValues); + } + + @Override + public void read(int sourceIndex, long[] destination, int destinationIndex, int length) + { + blockDecoder.read(sourceIndex, destination, destinationIndex, length); + } + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FsstEncoding.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FsstEncoding.java new file mode 100644 index 000000000000..5790bca32bdf --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/FsstEncoding.java @@ -0,0 +1,129 @@ +/* + * 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.lance.file.v2.encoding; + +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.trino.lance.file.v2.reader.BinaryBuffer; +import io.trino.lance.file.v2.reader.BufferAdapter; +import it.unimi.dsi.fastutil.bytes.ByteArrayList; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.lance.file.v2.reader.BinaryBufferAdapter.VARIABLE_BINARY_BUFFER_ADAPTER; +import static java.lang.Math.toIntExact; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class FsstEncoding + implements LanceEncoding +{ + public static final int FSST_MAX_SYMBOLS = 256; + public static final int FSST_MAX_SYMBOL_SIZE = 8; + public static final int FSST_SYMBOL_TABLE_SIZE = Long.BYTES + FSST_MAX_SYMBOLS * FSST_MAX_SYMBOL_SIZE + FSST_MAX_SYMBOLS; + public static final long FSST_SYMBOL_TABLE_MAGIC = 0x46535354L << 32; + public static final short FSST_ESC = 255; + + private final long header; + private final int numSymbols; + private final Slice symbols; + private final short[] lengths; + private final LanceEncoding valueEncoding; + + public FsstEncoding(LanceEncoding valueEncoding, Slice symbolTableSlice) + { + checkArgument(symbolTableSlice.length() == FSST_SYMBOL_TABLE_SIZE, format("FSST symbol table must have %d bytes", FSST_SYMBOL_TABLE_SIZE)); + checkArgument((symbolTableSlice.getLong(0) & FSST_SYMBOL_TABLE_MAGIC) == FSST_SYMBOL_TABLE_MAGIC, "Invalid header in FSST symbol table"); + this.header = symbolTableSlice.getLong(0); + this.numSymbols = toIntExact(header & 0xFF); + this.symbols = symbolTableSlice.slice(Long.BYTES, FSST_MAX_SYMBOL_SIZE * numSymbols); + this.lengths = new short[FSST_MAX_SYMBOLS]; + for (int i = 0; i < numSymbols; i++) { + this.lengths[i] = symbolTableSlice.getUnsignedByte(Long.BYTES + numSymbols * FSST_MAX_SYMBOL_SIZE + i); + } + this.valueEncoding = requireNonNull(valueEncoding, "valueEncoding is null"); + } + + @Override + public BufferAdapter getBufferAdapter() + { + return VARIABLE_BINARY_BUFFER_ADAPTER; + } + + @Override + public MiniBlockDecoder getMiniBlockDecoder() + { + return new FsstMiniBlockDecoder(valueEncoding.getMiniBlockDecoder()); + } + + public static FsstEncoding fromProto(build.buf.gen.lance.encodings21.Fsst proto) + { + LanceEncoding valueEncoding = LanceEncoding.fromProto(proto.getValues()); + return new FsstEncoding(valueEncoding, Slices.wrappedBuffer(proto.getSymbolTable().toByteArray())); + } + + public class FsstMiniBlockDecoder + implements MiniBlockDecoder + { + private final MiniBlockDecoder valueDecoder; + + public FsstMiniBlockDecoder(MiniBlockDecoder valueDecoder) + { + this.valueDecoder = valueDecoder; + } + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1, "FSST encoded block has exactly one buffer"); + valueDecoder.init(slices, numValues); + } + + @Override + public void read(int sourceIndex, BinaryBuffer destination, int destinationIndex, int length) + { + // TODO: performance optimizations with vectorization, loop unrolling and reduced memory copy + BinaryBuffer inputs = VARIABLE_BINARY_BUFFER_ADAPTER.createBuffer(length); + valueDecoder.read(sourceIndex, inputs, destinationIndex, length); + for (int i = 0; i < length; i++) { + destination.add(decompress(inputs.get(i)), destinationIndex + i); + } + } + + public Slice decompress(Slice input) + { + if ((header & (1 << 24)) == 0) { + return input.copy(); + } + + ByteArrayList output = new ByteArrayList(FSST_MAX_SYMBOL_SIZE * input.length()); + int count = 0; + for (int i = 0; i < input.length(); i++) { + short code = input.getUnsignedByte(i); + if (code < FSST_ESC) { + int size = lengths[code]; + output.addElements(count, symbols.getBytes(code * FSST_MAX_SYMBOL_SIZE, size)); + count += size; + } + else { + i++; + output.add(input.getByte(i)); + count++; + } + } + return Slices.wrappedBuffer(output.toByteArray()); + } + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/InlineBitpackingEncoding.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/InlineBitpackingEncoding.java new file mode 100644 index 000000000000..89874fe4fa93 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/InlineBitpackingEncoding.java @@ -0,0 +1,298 @@ +/* + * 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.lance.file.v2.encoding; + +import com.github.luohao.fastlanes.bitpack.VectorBytePacker; +import com.github.luohao.fastlanes.bitpack.VectorIntegerPacker; +import com.github.luohao.fastlanes.bitpack.VectorLongPacker; +import com.github.luohao.fastlanes.bitpack.VectorShortPacker; +import io.airlift.slice.Slice; +import io.trino.lance.file.v2.reader.BufferAdapter; +import io.trino.spi.block.ByteArrayBlock; +import io.trino.spi.block.IntArrayBlock; +import io.trino.spi.block.LongArrayBlock; +import io.trino.spi.block.ShortArrayBlock; +import io.trino.spi.block.ValueBlock; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.lance.file.v2.reader.ByteArrayBufferAdapter.BYTE_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.IntArrayBufferAdapter.INT_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.LongArrayBufferAdapter.LONG_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.ShortArrayBufferAdapter.SHORT_ARRAY_BUFFER_ADAPTER; +import static java.lang.Math.toIntExact; + +public class InlineBitpackingEncoding + implements LanceEncoding +{ + public static final int MAX_ELEMENTS_PER_CHUNK = 1024; + private final int uncompressedBitWidth; + + public InlineBitpackingEncoding(int uncompressedBitWidth) + { + this.uncompressedBitWidth = uncompressedBitWidth; + } + + @Override + public ValueBlock decodeBlock(Slice slice, int count) + { + return decode(slice, count); + } + + @Override + public MiniBlockDecoder getMiniBlockDecoder() + { + return switch (uncompressedBitWidth) { + case 8 -> new ByteMiniBlockDecoder(); + case 16 -> new ShortMiniBlockDecoder(); + case 32 -> new IntMiniBlockDecoder(); + case 64 -> new LongMiniBlockDecoder(); + default -> throw new IllegalStateException("Unexpected uncompressedBitWidth: " + uncompressedBitWidth); + }; + } + + @Override + public BlockDecoder getBlockDecoder() + { + return switch (uncompressedBitWidth) { + case 8 -> new ByteBlockDecoder(); + case 16 -> new ShortBlockDecoder(); + case 32 -> new IntBlockDecoder(); + case 64 -> new LongBlockDecoder(); + default -> throw new IllegalStateException("Unexpected uncompressedBitWidth: " + uncompressedBitWidth); + }; + } + + @Override + public BufferAdapter getBufferAdapter() + { + return switch (uncompressedBitWidth) { + case 8 -> BYTE_ARRAY_BUFFER_ADAPTER; + case 16 -> SHORT_ARRAY_BUFFER_ADAPTER; + case 32 -> INT_ARRAY_BUFFER_ADAPTER; + case 64 -> LONG_ARRAY_BUFFER_ADAPTER; + default -> throw new IllegalStateException("Unexpected uncompressedBitWidth: " + uncompressedBitWidth); + }; + } + + public ValueBlock decode(Slice slice, int count) + { + return switch (uncompressedBitWidth) { + case 8 -> { + int bitWidth = slice.getUnsignedByte(0); + byte[] input = slice.getBytes(1, MAX_ELEMENTS_PER_CHUNK * bitWidth / uncompressedBitWidth); + byte[] output = new byte[MAX_ELEMENTS_PER_CHUNK]; + VectorBytePacker.unpack(input, bitWidth, output); + yield new ByteArrayBlock(count, Optional.empty(), output); + } + case 16 -> { + int bitWidth = slice.getUnsignedShort(0); + short[] input = slice.getShorts(2, MAX_ELEMENTS_PER_CHUNK * bitWidth / uncompressedBitWidth); + short[] output = new short[MAX_ELEMENTS_PER_CHUNK]; + VectorShortPacker.unpack(input, bitWidth, output); + yield new ShortArrayBlock(count, Optional.empty(), output); + } + case 32 -> { + long bitWidth = slice.getUnsignedInt(0); + int[] input = slice.getInts(4, toIntExact(MAX_ELEMENTS_PER_CHUNK * bitWidth / uncompressedBitWidth)); + int[] output = new int[MAX_ELEMENTS_PER_CHUNK]; + VectorIntegerPacker.unpack(input, toIntExact(bitWidth), output); + yield new IntArrayBlock(count, Optional.empty(), output); + } + case 64 -> { + long bitWidth = slice.getLong(0); + long[] input = slice.getLongs(8, toIntExact(MAX_ELEMENTS_PER_CHUNK * bitWidth / uncompressedBitWidth)); + long[] output = new long[MAX_ELEMENTS_PER_CHUNK]; + VectorLongPacker.unpack(input, toIntExact(bitWidth), output); + yield new LongArrayBlock(count, Optional.empty(), output); + } + default -> throw new IllegalStateException("Unexpected uncompressedBitWidth: " + uncompressedBitWidth); + }; + } + + public int getUncompressedBitWidth() + { + return uncompressedBitWidth; + } + + public class ByteBlockDecoder + implements BlockDecoder + { + private int numValues; + private final byte[] data = new byte[MAX_ELEMENTS_PER_CHUNK]; + + @Override + public void init(Slice slice, int numValues) + { + this.numValues = numValues; + int bitWidth = slice.getUnsignedByte(0); + VectorBytePacker.unpack(slice.getBytes(1, MAX_ELEMENTS_PER_CHUNK * bitWidth / uncompressedBitWidth), bitWidth, data); + } + + @Override + public void read(int sourceIndex, byte[] destination, int destinationIndex, int length) + { + checkArgument(sourceIndex + length <= numValues); + System.arraycopy(data, sourceIndex, destination, destinationIndex, length); + } + } + + public class ShortBlockDecoder + implements BlockDecoder + { + private int numValues; + private final short[] data = new short[MAX_ELEMENTS_PER_CHUNK]; + + @Override + public void init(Slice slice, int numValues) + { + this.numValues = numValues; + int bitWidth = slice.getUnsignedByte(0); + VectorShortPacker.unpack(slice.getShorts(2, MAX_ELEMENTS_PER_CHUNK * bitWidth / uncompressedBitWidth), bitWidth, data); + } + + @Override + public void read(int sourceIndex, short[] destination, int destinationIndex, int length) + { + checkArgument(sourceIndex + length <= numValues); + System.arraycopy(data, sourceIndex, destination, destinationIndex, length); + } + } + + public class IntBlockDecoder + implements BlockDecoder + { + private int numValues; + private final int[] data = new int[MAX_ELEMENTS_PER_CHUNK]; + + @Override + public void init(Slice slice, int numValues) + { + this.numValues = numValues; + int bitWidth = slice.getUnsignedByte(0); + VectorIntegerPacker.unpack(slice.getInts(4, MAX_ELEMENTS_PER_CHUNK * bitWidth / uncompressedBitWidth), bitWidth, data); + } + + @Override + public void read(int sourceIndex, int[] destination, int destinationIndex, int length) + { + checkArgument(sourceIndex + length <= numValues); + System.arraycopy(data, sourceIndex, destination, destinationIndex, length); + } + } + + public class LongBlockDecoder + implements BlockDecoder + { + private int numValues; + private final long[] data = new long[MAX_ELEMENTS_PER_CHUNK]; + + @Override + public void init(Slice slice, int numValues) + { + this.numValues = numValues; + int bitWidth = slice.getUnsignedByte(0); + VectorLongPacker.unpack(slice.getLongs(8, MAX_ELEMENTS_PER_CHUNK * bitWidth / uncompressedBitWidth), bitWidth, data); + } + + @Override + public void read(int sourceIndex, long[] destination, int destinationIndex, int length) + { + checkArgument(sourceIndex + length <= numValues); + System.arraycopy(data, sourceIndex, destination, destinationIndex, length); + } + } + + public class ByteMiniBlockDecoder + implements MiniBlockDecoder + { + private final ByteBlockDecoder blockDecoder = new ByteBlockDecoder(); + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + checkArgument(numValues <= MAX_ELEMENTS_PER_CHUNK); + blockDecoder.init(slices.getFirst(), numValues); + } + + @Override + public void read(int sourceIndex, byte[] destination, int destinationIndex, int length) + { + blockDecoder.read(sourceIndex, destination, destinationIndex, length); + } + } + + public class ShortMiniBlockDecoder + implements MiniBlockDecoder + { + private final ShortBlockDecoder blockDecoder = new ShortBlockDecoder(); + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + checkArgument(numValues <= MAX_ELEMENTS_PER_CHUNK); + blockDecoder.init(slices.getFirst(), numValues); + } + + @Override + public void read(int sourceIndex, short[] destination, int destinationIndex, int length) + { + blockDecoder.read(sourceIndex, destination, destinationIndex, length); + } + } + + public class IntMiniBlockDecoder + implements MiniBlockDecoder + { + private final IntBlockDecoder blockDecoder = new IntBlockDecoder(); + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + checkArgument(numValues <= MAX_ELEMENTS_PER_CHUNK); + blockDecoder.init(slices.getFirst(), numValues); + } + + @Override + public void read(int sourceIndex, int[] destination, int destinationIndex, int length) + { + blockDecoder.read(sourceIndex, destination, destinationIndex, length); + } + } + + public class LongMiniBlockDecoder + implements MiniBlockDecoder + { + private final LongBlockDecoder blockDecoder = new LongBlockDecoder(); + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + checkArgument(numValues <= MAX_ELEMENTS_PER_CHUNK); + blockDecoder.init(slices.getFirst(), numValues); + } + + @Override + public void read(int sourceIndex, long[] destination, int destinationIndex, int length) + { + blockDecoder.read(sourceIndex, destination, destinationIndex, length); + } + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/LanceEncoding.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/LanceEncoding.java new file mode 100644 index 000000000000..c0100480dfe6 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/LanceEncoding.java @@ -0,0 +1,51 @@ +/* + * 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.lance.file.v2.encoding; + +import io.airlift.slice.Slice; +import io.trino.lance.file.v2.reader.BufferAdapter; +import io.trino.spi.block.ValueBlock; + +import static java.lang.Math.toIntExact; + +public interface LanceEncoding +{ + static LanceEncoding fromProto(build.buf.gen.lance.encodings21.CompressiveEncoding proto) + { + return switch (proto.getCompressionCase()) { + case FLAT -> new FlatValueEncoding(toIntExact(proto.getFlat().getBitsPerValue() / Byte.SIZE)); + case INLINE_BITPACKING -> new InlineBitpackingEncoding(toIntExact(proto.getInlineBitpacking().getUncompressedBitsPerValue())); + case VARIABLE -> new VariableEncoding(); + case FIXED_SIZE_LIST -> new FixedSizeListEncoding(); + case RLE -> RunLengthEncoding.from(proto.getRle()); + case FSST -> FsstEncoding.fromProto(proto.getFsst()); + case OUT_OF_LINE_BITPACKING -> OutOfLineBitpackingEncoding.fromProto(proto.getOutOfLineBitpacking()); + default -> throw new IllegalArgumentException("Invalid encoding: " + proto.getCompressionCase()); + }; + } + + default ValueBlock decodeBlock(Slice slice, int count) + { + throw new UnsupportedOperationException("decodeBlock is not supported for " + getClass().getSimpleName()); + } + + BufferAdapter getBufferAdapter(); + + MiniBlockDecoder getMiniBlockDecoder(); + + default BlockDecoder getBlockDecoder() + { + throw new UnsupportedOperationException("getBlockDecoder is not supported for " + getClass().getSimpleName()); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/MiniBlockDecoder.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/MiniBlockDecoder.java new file mode 100644 index 000000000000..f223f56b8845 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/MiniBlockDecoder.java @@ -0,0 +1,25 @@ +/* + * 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.lance.file.v2.encoding; + +import io.airlift.slice.Slice; + +import java.util.List; + +public interface MiniBlockDecoder +{ + void init(List slices, int numValues); + + void read(int sourceIndex, T destination, int destinationIndex, int length); +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/OutOfLineBitpackingEncoding.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/OutOfLineBitpackingEncoding.java new file mode 100644 index 000000000000..dba8491a2e50 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/OutOfLineBitpackingEncoding.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.lance.file.v2.encoding; + +import build.buf.gen.lance.encodings21.OutOfLineBitpacking; +import com.github.luohao.fastlanes.bitpack.VectorShortPacker; +import io.airlift.slice.Slice; +import io.trino.lance.file.v2.reader.BufferAdapter; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.Math.toIntExact; + +public class OutOfLineBitpackingEncoding + implements LanceEncoding +{ + public static final int ELEMENTS_PER_CHUNK = 1024; + private final int uncompressedBitsPerValue; + private final int compressedBitsPerValue; + private final int chunkSize; + + public OutOfLineBitpackingEncoding(int uncompressedBitsPerValue, int compressedBitsPerValue) + { + this.uncompressedBitsPerValue = uncompressedBitsPerValue; + this.compressedBitsPerValue = compressedBitsPerValue; + this.chunkSize = (ELEMENTS_PER_CHUNK * compressedBitsPerValue + Byte.SIZE - 1) / Byte.SIZE; + } + + @Override + public BufferAdapter getBufferAdapter() + { + throw new UnsupportedOperationException("getBufferAdapter not supported for OutOfLineBitpacking"); + } + + @Override + public MiniBlockDecoder getMiniBlockDecoder() + { + throw new UnsupportedOperationException("getMiniBlockDecoder not supported for OutOfLineBitpacking"); + } + + @Override + public BlockDecoder getBlockDecoder() + { + return switch (uncompressedBitsPerValue) { + case 16 -> new ShortBlockDecoder(); + default -> throw new IllegalStateException("Unexpected value: " + uncompressedBitsPerValue); + }; + } + + public class ShortBlockDecoder + implements BlockDecoder + { + private short[] values; + + @Override + public void init(Slice slice, int count) + { + this.values = new short[count]; + int numFullChunks = count / ELEMENTS_PER_CHUNK; + int numTailValues = count % ELEMENTS_PER_CHUNK; + int currentOffset = 0; + short[] buffer = new short[ELEMENTS_PER_CHUNK]; + + for (int chunk = 0; chunk < numFullChunks; chunk++) { + Slice chunkSlice = slice.slice(chunkSize * chunk, chunkSize); + VectorShortPacker.unpack(chunkSlice.getShorts(0, chunkSize / Short.BYTES), compressedBitsPerValue, buffer); + System.arraycopy(buffer, 0, values, currentOffset, ELEMENTS_PER_CHUNK); + currentOffset += ELEMENTS_PER_CHUNK; + } + if (numTailValues > 0) { + int tailByteOffset = chunkSize * numFullChunks; + int tailSize = slice.length() - tailByteOffset; + Slice tailSlice = slice.slice(tailByteOffset, tailSize); + if (tailSize * Byte.SIZE == numTailValues * uncompressedBitsPerValue) { + tailSlice.getShorts(0, values, currentOffset, numTailValues); + } + else { + checkArgument(tailSize == chunkSize, "tail chunk size must be equal to full chunk size if bitpacked"); + VectorShortPacker.unpack(tailSlice.getShorts(0, chunkSize / Short.BYTES), compressedBitsPerValue, buffer); + System.arraycopy(buffer, 0, values, currentOffset, numTailValues); + } + } + } + + @Override + public void read(int sourceIndex, short[] destination, int destinationIndex, int length) + { + System.arraycopy(values, sourceIndex, destination, destinationIndex, length); + } + } + + public static OutOfLineBitpackingEncoding fromProto(OutOfLineBitpacking proto) + { + checkArgument(proto.hasValues()); + checkArgument(proto.getValues().hasFlat()); + int uncompressedBitsPerValue = toIntExact(proto.getUncompressedBitsPerValue()); + int compressedBitsPerValue = toIntExact(proto.getValues().getFlat().getBitsPerValue()); + return new OutOfLineBitpackingEncoding(uncompressedBitsPerValue, compressedBitsPerValue); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/RunLengthEncoding.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/RunLengthEncoding.java new file mode 100644 index 000000000000..896edb5fc437 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/RunLengthEncoding.java @@ -0,0 +1,189 @@ +/* + * 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.lance.file.v2.encoding; + +import io.airlift.slice.Slice; +import io.trino.lance.file.v2.reader.BufferAdapter; + +import java.util.Arrays; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.lance.file.v2.reader.ByteArrayBufferAdapter.BYTE_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.IntArrayBufferAdapter.INT_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.LongArrayBufferAdapter.LONG_ARRAY_BUFFER_ADAPTER; +import static io.trino.lance.file.v2.reader.ShortArrayBufferAdapter.SHORT_ARRAY_BUFFER_ADAPTER; +import static java.lang.Math.toIntExact; + +public class RunLengthEncoding + implements LanceEncoding +{ + private final int bitsPerValue; + + public RunLengthEncoding(int bitsPerValue) + { + this.bitsPerValue = bitsPerValue; + } + + @Override + public BufferAdapter getBufferAdapter() + { + return switch (bitsPerValue) { + case 8 -> BYTE_ARRAY_BUFFER_ADAPTER; + case 16 -> SHORT_ARRAY_BUFFER_ADAPTER; + case 32 -> INT_ARRAY_BUFFER_ADAPTER; + case 64 -> LONG_ARRAY_BUFFER_ADAPTER; + default -> throw new IllegalStateException("Unexpected uncompressedBitWidth: " + bitsPerValue); + }; + } + + @Override + public MiniBlockDecoder getMiniBlockDecoder() + { + return switch (bitsPerValue) { + case 8 -> new ByteRunLengthDecoder(bitsPerValue); + case 16 -> new ShortRunLengthDecoder(bitsPerValue); + case 32 -> new IntegerRunLengthDecoder(bitsPerValue); + case 64 -> new LongRunLengthDecoder(bitsPerValue); + default -> throw new IllegalStateException("Unexpected bitsPerValue: " + bitsPerValue); + }; + } + + public static RunLengthEncoding from(build.buf.gen.lance.encodings21.Rle proto) + { + checkArgument(proto.getValues().hasFlat(), "value buffer only supports flat encoding"); + checkArgument(proto.getRunLengths().hasFlat(), "length buffer only supports flat encoding"); + build.buf.gen.lance.encodings21.Flat valueEncoding = proto.getValues().getFlat(); + return new RunLengthEncoding(toIntExact(valueEncoding.getBitsPerValue())); + } + + public abstract static class RunLengthDecoder + implements MiniBlockDecoder + { + private final int bitsPerValue; + protected Slice valueSlice; + protected Slice lengthSlice; + + protected RunLengthDecoder(int bitsPerValue) + { + this.bitsPerValue = bitsPerValue; + } + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 2, "RLE miniblock has exact 2 buffers"); + checkArgument(slices.get(0).length() / (bitsPerValue / Byte.SIZE) == slices.get(1).length(), "values buffer and length buffer do not match"); + this.valueSlice = slices.get(0); + this.lengthSlice = slices.get(1); + } + + @Override + public void read(int sourceIndex, T destination, int destinationIndex, int length) + { + int remainingSkip = sourceIndex; + int runOffset = 0; + int runLength = lengthSlice.getUnsignedByte(runOffset); + while (remainingSkip > 0) { + if (remainingSkip >= runLength) { + remainingSkip -= runLength; + runOffset++; + runLength = lengthSlice.getUnsignedByte(runOffset); + } + else { + runLength -= remainingSkip; + remainingSkip = 0; + } + } + int destinationPosition = destinationIndex; + int remaining = length; + + while (remaining > 0) { + if (runLength == 0) { + runOffset++; + runLength = lengthSlice.getUnsignedByte(runOffset); + } + + int n = Math.min(runLength, remaining); + fill(runOffset, destination, destinationPosition, n); + + runLength -= n; + remaining -= n; + destinationPosition += n; + } + } + + protected abstract void fill(int runOffset, T destination, int offset, int length); + } + + public static class ByteRunLengthDecoder + extends RunLengthDecoder + { + protected ByteRunLengthDecoder(int bitsPerValue) + { + super(bitsPerValue); + } + + @Override + protected void fill(int runOffset, byte[] destination, int offset, int length) + { + Arrays.fill(destination, offset, offset + length, valueSlice.getByte(runOffset)); + } + } + + public static class ShortRunLengthDecoder + extends RunLengthDecoder + { + protected ShortRunLengthDecoder(int bitsPerValue) + { + super(bitsPerValue); + } + + @Override + protected void fill(int runOffset, short[] destination, int offset, int length) + { + Arrays.fill(destination, offset, offset + length, valueSlice.getShort(runOffset * Short.BYTES)); + } + } + + public static class IntegerRunLengthDecoder + extends RunLengthDecoder + { + protected IntegerRunLengthDecoder(int bitsPerValue) + { + super(bitsPerValue); + } + + @Override + protected void fill(int runOffset, int[] destination, int offset, int length) + { + Arrays.fill(destination, offset, offset + length, valueSlice.getInt(runOffset * Integer.BYTES)); + } + } + + public static class LongRunLengthDecoder + extends RunLengthDecoder + { + protected LongRunLengthDecoder(int bitsPerValue) + { + super(bitsPerValue); + } + + @Override + protected void fill(int runOffset, long[] destination, int offset, int length) + { + Arrays.fill(destination, offset, offset + length, valueSlice.getLong(runOffset * Long.BYTES)); + } + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/VariableEncoding.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/VariableEncoding.java new file mode 100644 index 000000000000..b27cb0d6e2b1 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/encoding/VariableEncoding.java @@ -0,0 +1,93 @@ +/* + * 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.lance.file.v2.encoding; + +import io.airlift.slice.Slice; +import io.trino.lance.file.v2.reader.BinaryBuffer; +import io.trino.lance.file.v2.reader.BufferAdapter; +import io.trino.spi.block.ValueBlock; +import io.trino.spi.block.VariableWidthBlock; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.EMPTY_SLICE; +import static io.trino.lance.file.v2.reader.BinaryBufferAdapter.VARIABLE_BINARY_BUFFER_ADAPTER; +import static java.lang.Math.toIntExact; + +public class VariableEncoding + implements LanceEncoding +{ + @Override + public ValueBlock decodeBlock(Slice slice, int count) + { + // caller verifies count is smaller than the number of values in the buffer + return decode(slice, count); + } + + @Override + public MiniBlockDecoder getMiniBlockDecoder() + { + return new VariableBinaryDecoder(); + } + + @Override + public BufferAdapter getBufferAdapter() + { + return VARIABLE_BINARY_BUFFER_ADAPTER; + } + + private ValueBlock decode(Slice slice, int count) + { + if (count == 0) { + return new VariableWidthBlock(0, EMPTY_SLICE, new int[1], Optional.empty()); + } + + int bitsPerOffset = toIntExact(slice.getUnsignedInt(0)); + if (bitsPerOffset == 32) { + int offsetStart = 8; + long bytesStartOffset = slice.getUnsignedInt(4); + int[] offsets = slice.getInts(offsetStart, count + 1); + Slice data = slice.slice(toIntExact(bytesStartOffset), toIntExact(slice.length() - bytesStartOffset)); + return new VariableWidthBlock(count, data, offsets, Optional.empty()); + } + else { + throw new UnsupportedOperationException("Unsupported bits per offset: " + bitsPerOffset); + } + } + + public static class VariableBinaryDecoder + implements MiniBlockDecoder + { + private Slice slice; + private int[] offsets; + + @Override + public void init(List slices, int numValues) + { + checkArgument(slices.size() == 1); + slice = slices.get(0); + offsets = slice.getInts(0, numValues + 1); + } + + @Override + public void read(int sourceIndex, BinaryBuffer destination, int destinationIndex, int length) + { + for (int i = 0; i < length; i++) { + destination.add(slice.slice(offsets[sourceIndex + i], offsets[sourceIndex + i + 1] - offsets[sourceIndex + i]), destinationIndex + i); + } + } + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/AllNullLayout.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/AllNullLayout.java new file mode 100644 index 000000000000..270f113a73ff --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/AllNullLayout.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.lance.file.v2.metadata; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +public record AllNullLayout(List layers) + implements PageLayout +{ + public AllNullLayout + { + layers = ImmutableList.copyOf(layers); + } + + public static AllNullLayout fromProto(build.buf.gen.lance.encodings21.AllNullLayout proto) + { + return new AllNullLayout(RepDefLayer.fromProtoList(proto.getLayersList())); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/ColumnMetadata.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/ColumnMetadata.java new file mode 100644 index 000000000000..d6540b639735 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/ColumnMetadata.java @@ -0,0 +1,91 @@ +/* + * 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.lance.file.v2.metadata; + +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Any; +import com.google.protobuf.InvalidProtocolBufferException; +import io.airlift.slice.Slice; + +import java.util.List; +import java.util.stream.IntStream; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; + +public record ColumnMetadata(int index, List pages, List bufferOffsets) +{ + public ColumnMetadata + { + pages = ImmutableList.copyOf(pages); + bufferOffsets = ImmutableList.copyOf(bufferOffsets); + } + + public static ColumnMetadata from(int columnIndex, Slice data) + { + checkArgument(data != null, "data is null"); + + build.buf.gen.lance.file.v2.ColumnMetadata proto; + try { + proto = build.buf.gen.lance.file.v2.ColumnMetadata.parseFrom(data.toByteBuffer()); + } + catch (InvalidProtocolBufferException e) { + throw new RuntimeException("Failed to fromProto ColumnMetadata proto: " + e); + } + + List pages = proto.getPagesList().stream() + .map(page -> { + long numRows = page.getLength(); + long priority = page.getPriority(); + int bufferCount = page.getBufferOffsetsList().size(); + List buffers = IntStream.range(0, bufferCount).boxed() + .map(i -> { + long position = page.getBufferOffsets(i); + return new DiskRange(position, page.getBufferSizes(i)); + }) + .collect(toImmutableList()); + return new PageMetadata(numRows, priority, getPageLayout(page), buffers); + }) + .collect(toImmutableList()); + + int bufferCount = proto.getBufferOffsetsList().size(); + List buffers = IntStream.range(0, bufferCount).boxed() + .map(index -> { + long position = proto.getBufferOffsets(index); + return new DiskRange(position, proto.getBufferSizes(index)); + }) + .collect(toImmutableList()); + return new ColumnMetadata(columnIndex, pages, buffers); + } + + private static PageLayout getPageLayout(build.buf.gen.lance.file.v2.ColumnMetadata.Page page) + { + checkArgument(page.hasEncoding(), "Page has no encoding"); + build.buf.gen.lance.file.v2.Encoding encoding = page.getEncoding(); + return switch (encoding.getLocationCase()) { + case DIRECT -> { + try { + Any any = Any.parseFrom(encoding.getDirect().getEncoding().toByteArray()); + build.buf.gen.lance.encodings21.PageLayout layout = any.unpack(build.buf.gen.lance.encodings21.PageLayout.class); + yield PageLayout.fromProto(layout); + } + catch (InvalidProtocolBufferException e) { + throw new IllegalArgumentException("Failed to parse from proto message", e); + } + } + case INDIRECT -> throw new UnsupportedOperationException("Indirect encoding not supported"); + default -> throw new UnsupportedOperationException("Invalid encoding: " + encoding); + }; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/DiskRange.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/DiskRange.java new file mode 100644 index 000000000000..52cc1901b7cc --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/DiskRange.java @@ -0,0 +1,19 @@ +/* + * 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.lance.file.v2.metadata; + +public record DiskRange(long position, long length) +{ + public static final int BUFFER_DESCRIPTOR_SIZE = 16; +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/Field.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/Field.java new file mode 100644 index 000000000000..04d0234f9e1e --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/Field.java @@ -0,0 +1,116 @@ +/* + * 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.lance.file.v2.metadata; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.protobuf.ByteString; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.Type; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.spi.type.BigintType.BIGINT; +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.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.util.Objects.requireNonNull; + +public record Field(String name, int id, int parentId, String logicalType, Map metadata, boolean nullable, List children) +{ + public Field + { + requireNonNull(name, "name is null"); + requireNonNull(logicalType, "logicalType is null"); + metadata = ImmutableMap.copyOf(metadata); + children = ImmutableList.copyOf(children); + } + + public static Field fromProto(build.buf.gen.lance.file.Field proto) + { + ImmutableMap.Builder metadataBuilder = ImmutableMap.builder(); + for (Map.Entry entry : proto.getMetadataMap().entrySet()) { + metadataBuilder.put(entry.getKey(), entry.getValue().toStringUtf8()); + } + Map metadata = metadataBuilder.buildOrThrow(); + return new Field(proto.getName(), + proto.getId(), + proto.getParentId(), + proto.getLogicalType(), + metadata, + proto.getNullable(), + new ArrayList()); + } + + public void addChild(Field child) + { + this.children.add(child); + } + + public boolean isLeaf() + { + return children.isEmpty(); + } + + public Type toTrinoType() + { + return switch (LogicalType.from(logicalType)) { + case LogicalType.Int8Type _ -> TINYINT; + case LogicalType.Int16Type _ -> SMALLINT; + case LogicalType.Int32Type _ -> INTEGER; + case LogicalType.Int64Type _ -> BIGINT; + case LogicalType.FloatType _ -> REAL; + case LogicalType.DoubleType _ -> DOUBLE; + case LogicalType.StringType _ -> VARCHAR; + case LogicalType.BinaryType _ -> VARBINARY; + case LogicalType.StructType _ -> { + List fields = children.stream() + .map(field -> RowType.field(field.name, field.toTrinoType())) + .collect(toImmutableList()); + yield RowType.from(fields); + } + case LogicalType.ListType _ -> { + checkArgument(children.size() == 1); + yield new ArrayType(children.get(0).toTrinoType()); + } + case LogicalType.DateType _ -> DATE; + default -> throw new IllegalArgumentException("Unsupported type: " + logicalType); + }; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("id", id) + .add("parentId", parentId) + .add("logicalType", logicalType) + .add("metadata", metadata) + .add("nullable", nullable) + .add("children", children) + .toString(); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/FileVersion.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/FileVersion.java new file mode 100644 index 000000000000..91898b13bb0d --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/FileVersion.java @@ -0,0 +1,28 @@ +/* + * 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.lance.file.v2.metadata; + +public enum FileVersion +{ + // Only v2.1+ formats are supported + V2_1; + + public static FileVersion fromMajorMinor(int major, int minor) + { + if (major == 2 && minor == 1) { + return V2_1; + } + throw new UnsupportedOperationException("Unsupported major version: " + major + "." + minor); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/Footer.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/Footer.java new file mode 100644 index 000000000000..47bc6789bc01 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/Footer.java @@ -0,0 +1,116 @@ +/* + * 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.lance.file.v2.metadata; + +import io.airlift.slice.Slice; + +import static com.google.common.base.MoreObjects.toStringHelper; + +public final class Footer +{ + public static final int COLUMN_METADATA_START_POS = 0; + public static final int COLUMN_METADATA_OFFSETS_START_POS = COLUMN_METADATA_START_POS + 8; + public static final int GLOBAL_BUFF_OFFSETS_START_POS = COLUMN_METADATA_OFFSETS_START_POS + 8; + public static final int NUM_GLOBAL_BUFFERS_POS = GLOBAL_BUFF_OFFSETS_START_POS + 8; + public static final int NUM_COLUMNS_POS = NUM_GLOBAL_BUFFERS_POS + 4; + public static final int MAJOR_VERSION_POS = NUM_COLUMNS_POS + 4; + public static final int MINOR_VERSION_POS = MAJOR_VERSION_POS + 2; + + private final long columnMetadataStart; + private final long columnMetadataOffsetsStart; + private final long globalBuffOffsetStart; + private final int numGlobalBuffers; + private final int numColumns; + private final short majorVersion; + private final short minorVersion; + + public Footer( + long columnMetadataStart, + long columnMetadataOffsetsStart, + long globalBuffOffsetStart, + int numGlobalBuffers, + int numColumns, + short majorVersion, + short minorVersion) + { + this.columnMetadataStart = columnMetadataStart; + this.columnMetadataOffsetsStart = columnMetadataOffsetsStart; + this.globalBuffOffsetStart = globalBuffOffsetStart; + this.numGlobalBuffers = numGlobalBuffers; + this.numColumns = numColumns; + this.majorVersion = majorVersion; + this.minorVersion = minorVersion; + } + + public static Footer from(Slice data) + { + long columnMetaStart = data.getLong(COLUMN_METADATA_START_POS); + long columnMetaOffsetsStart = data.getLong(COLUMN_METADATA_OFFSETS_START_POS); + long globalBuffOffsetStart = data.getLong(GLOBAL_BUFF_OFFSETS_START_POS); + int numGlobalBuffers = data.getInt(NUM_GLOBAL_BUFFERS_POS); + int numColumns = data.getInt(NUM_COLUMNS_POS); + short majorVersion = data.getShort(MAJOR_VERSION_POS); + short minorVersion = data.getShort(MINOR_VERSION_POS); + return new Footer(columnMetaStart, columnMetaOffsetsStart, globalBuffOffsetStart, numGlobalBuffers, numColumns, majorVersion, minorVersion); + } + + public long getColumnMetadataStart() + { + return columnMetadataStart; + } + + public long getColumnMetadataOffsetsStart() + { + return columnMetadataOffsetsStart; + } + + public long getGlobalBuffOffsetStart() + { + return globalBuffOffsetStart; + } + + public int getNumGlobalBuffers() + { + return numGlobalBuffers; + } + + public int getNumColumns() + { + return numColumns; + } + + public int getMajorVersion() + { + return majorVersion; + } + + public int getMinorVersion() + { + return minorVersion; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("columnMetadataStart", columnMetadataStart) + .add("columnMetadataOffsetsStart", columnMetadataOffsetsStart) + .add("globalBuffOffsetStart", globalBuffOffsetStart) + .add("numGlobalBuffers", numGlobalBuffers) + .add("numColumns", numColumns) + .add("majorVersion", majorVersion) + .add("minorVersion", minorVersion) + .toString(); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/FullZipLayout.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/FullZipLayout.java new file mode 100644 index 000000000000..c5cafc8b5d87 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/FullZipLayout.java @@ -0,0 +1,62 @@ +/* + * 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.lance.file.v2.metadata; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public record FullZipLayout( + int numRepBits, + int numDeflBits, + io.trino.lance.file.v2.metadata.FullZipLayout.Block block, + int numItems, + int numVisibleItems, + List repDefLayers) + implements PageLayout +{ + public FullZipLayout + { + requireNonNull(block, "chunkSize is null"); + repDefLayers = ImmutableList.copyOf(repDefLayers); + } + + public static FullZipLayout fromProto(build.buf.gen.lance.encodings21.FullZipLayout proto) + { + Block block = switch (proto.getDetailsCase()) { + case BITS_PER_VALUE -> new Block.FixedWidthBlock(proto.getBitsPerValue()); + case BITS_PER_OFFSET -> new Block.VariableWidthBlock(proto.getBitsPerOffset()); + default -> throw new IllegalArgumentException("Unexpected details case: " + proto.getDetailsCase()); + }; + return new FullZipLayout( + proto.getBitsRep(), + proto.getBitsDef(), + block, + proto.getNumItems(), + proto.getNumVisibleItems(), + RepDefLayer.fromProtoList(proto.getLayersList())); + } + + public sealed interface Block + permits Block.FixedWidthBlock, Block.VariableWidthBlock + { + record FixedWidthBlock(int bitsPerValue) + implements Block {} + + record VariableWidthBlock(int bitsPerOffset) + implements Block {} + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/LanceTypeUtil.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/LanceTypeUtil.java new file mode 100644 index 000000000000..380430c64325 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/LanceTypeUtil.java @@ -0,0 +1,105 @@ +/* + * 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.lance.file.v2.metadata; + +import com.google.common.collect.ImmutableMap; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.google.common.base.Verify.verify; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.lance.file.v2.metadata.LogicalType.LogicalTypeKind.STRUCT; + +public final class LanceTypeUtil +{ + private LanceTypeUtil() {} + + public static class FieldVisitor + { + public T struct(Field field, List fieldResults) + { + return null; + } + + public T list(Field field, T elementResult) + { + return null; + } + + public T primitive(Field primitive) + { + return null; + } + } + + public static T visit(Field field, FieldVisitor visitor) + { + return switch (LogicalType.from(field.logicalType())) { + case LogicalType.Int8Type _, + LogicalType.Int16Type _, + LogicalType.Int32Type _, + LogicalType.Int64Type _, + LogicalType.FloatType _, + LogicalType.DoubleType _, + LogicalType.StringType _, + LogicalType.BinaryType _, + LogicalType.DateType _ -> visitor.primitive(field); + case LogicalType.StructType _ -> { + List results = new ArrayList<>(field.children().size()); + for (Field child : field.children()) { + results.add(visit(child, visitor)); + } + yield visitor.struct(field, results); + } + case LogicalType.ListType _ -> { + verify(field.children().size() == 1); + T result = visit(field.children().get(0), visitor); + yield visitor.list(field, result); + } + case LogicalType.FixedSizeListType _ -> throw new UnsupportedOperationException("FIXED LIST TYPES not yet supported"); + }; + } + + public static T visit(List fields, FieldVisitor visitor) + { + Field rootStruct = new Field("root_", -1, -1, STRUCT.name(), Map.of(), false, fields); + return visit(rootStruct, visitor); + } + + public static class FieldIdToColumnIndexVisitor + extends FieldVisitor> + { + private int current; + + @Override + public Map primitive(Field primitive) + { + return ImmutableMap.of(primitive.id(), current++); + } + + @Override + public Map struct(Field struct, List> fieldResults) + { + return fieldResults.stream().flatMap(result -> result.entrySet().stream()).collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + @Override + public Map list(Field field, Map elementResult) + { + return elementResult; + } + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/LogicalType.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/LogicalType.java new file mode 100644 index 000000000000..eb2a83dbdcea --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/LogicalType.java @@ -0,0 +1,141 @@ +/* + * 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.lance.file.v2.metadata; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public sealed interface LogicalType + permits + LogicalType.Int8Type, + LogicalType.Int16Type, + LogicalType.Int32Type, + LogicalType.Int64Type, + LogicalType.FloatType, + LogicalType.DoubleType, + LogicalType.StringType, + LogicalType.BinaryType, + LogicalType.FixedSizeListType, + LogicalType.ListType, + LogicalType.StructType, + LogicalType.DateType +{ + static LogicalType from(String type) + { + requireNonNull(type, "type is null"); + checkArgument(!type.isEmpty(), "type is empty"); + String[] components = type.toUpperCase(ENGLISH).split(":"); + LogicalTypeKind kind = LogicalTypeKind.valueOf(components[0]); + return switch (kind) { + case INT8 -> Int8Type.INT8_TYPE; + case INT16 -> Int16Type.INT16_TYPE; + case INT32 -> Int32Type.INT32_TYPE; + case INT64 -> Int64Type.INT64_TYPE; + case FLOAT -> FloatType.FLOAT_TYPE; + case DOUBLE -> DoubleType.DOUBLE_TYPE; + case STRING -> StringType.STRING_TYPE; + case BINARY -> BinaryType.BINARY_TYPE; + case FIXED_SIZE_LIST -> { + checkArgument(components.length == 3, "FixedSizeList type signature must have exactly 3 components"); + int size = Integer.parseInt(components[components.length - 1]); + LogicalTypeKind dataType = LogicalTypeKind.valueOf(components[1]); + yield new FixedSizeListType(dataType, size); + } + case LIST -> new ListType(); + case STRUCT -> new StructType(); + case DATE32 -> { + checkArgument(components.length == 2, "DATE32 signature must have exactly 2 components"); + checkArgument(components[1].toLowerCase(ENGLISH).equals("day"), "only supports date32:day"); + yield new DateType(); + } + }; + } + + enum LogicalTypeKind + { + INT8, + INT16, + INT32, + INT64, + FLOAT, + DOUBLE, + STRING, + BINARY, + FIXED_SIZE_LIST, + LIST, + STRUCT, + DATE32 + } + + record Int8Type() + implements LogicalType + { + public static final Int8Type INT8_TYPE = new Int8Type(); + } + + record Int16Type() + implements LogicalType + { + public static final Int16Type INT16_TYPE = new Int16Type(); + } + + record Int32Type() + implements LogicalType + { + public static final Int32Type INT32_TYPE = new Int32Type(); + } + + record Int64Type() + implements LogicalType + { + public static final Int64Type INT64_TYPE = new Int64Type(); + } + + record FloatType() + implements LogicalType + { + public static final FloatType FLOAT_TYPE = new FloatType(); + } + + record DoubleType() + implements LogicalType + { + public static final DoubleType DOUBLE_TYPE = new DoubleType(); + } + + record StringType() + implements LogicalType + { + public static final StringType STRING_TYPE = new StringType(); + } + + record BinaryType() + implements LogicalType + { + public static final BinaryType BINARY_TYPE = new BinaryType(); + } + + record FixedSizeListType(LogicalTypeKind kind, int size) + implements LogicalType {} + + record ListType() + implements LogicalType {} + + record StructType() + implements LogicalType {} + + record DateType() + implements LogicalType {} +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/MiniBlockLayout.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/MiniBlockLayout.java new file mode 100644 index 000000000000..12e2d9144d0d --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/MiniBlockLayout.java @@ -0,0 +1,61 @@ +/* + * 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.lance.file.v2.metadata; + +import com.google.common.collect.ImmutableList; +import io.trino.lance.file.v2.encoding.LanceEncoding; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public record MiniBlockLayout( + LanceEncoding valueEncoding, + Optional repetitionEncoding, + Optional definitionEncoding, + Optional dictionaryEncoding, + Optional numDictionaryItems, + List layers, + long numBuffers, + int repIndexDepth, + long numItems) + implements PageLayout +{ + public MiniBlockLayout + { + requireNonNull(valueEncoding, "valueEncoding is null"); + requireNonNull(repetitionEncoding, "repetitionEncoding is null"); + requireNonNull(definitionEncoding, "definitionEncoding is null"); + requireNonNull(dictionaryEncoding, "dictionaryEncoding is null"); + requireNonNull(numDictionaryItems, "numDictionaryItems is null"); + layers = ImmutableList.copyOf(layers); + } + + public static MiniBlockLayout fromProto(build.buf.gen.lance.encodings21.MiniBlockLayout proto) + { + checkArgument(proto.hasValueCompression()); + return new MiniBlockLayout( + LanceEncoding.fromProto(proto.getValueCompression()), + Optional.ofNullable(proto.hasRepCompression() ? LanceEncoding.fromProto(proto.getRepCompression()) : null), + Optional.ofNullable(proto.hasDefCompression() ? LanceEncoding.fromProto(proto.getDefCompression()) : null), + Optional.ofNullable(proto.hasDictionary() ? LanceEncoding.fromProto(proto.getDictionary()) : null), + Optional.ofNullable(proto.hasDictionary() ? proto.getNumDictionaryItems() : null), + RepDefLayer.fromProtoList(proto.getLayersList()), + proto.getNumBuffers(), + proto.getRepetitionIndexDepth(), + proto.getNumItems()); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/PageLayout.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/PageLayout.java new file mode 100644 index 000000000000..d32bf7886208 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/PageLayout.java @@ -0,0 +1,28 @@ +/* + * 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.lance.file.v2.metadata; + +public sealed interface PageLayout + permits MiniBlockLayout, AllNullLayout, FullZipLayout +{ + static PageLayout fromProto(build.buf.gen.lance.encodings21.PageLayout proto) + { + return switch (proto.getLayoutCase()) { + case MINI_BLOCK_LAYOUT -> MiniBlockLayout.fromProto(proto.getMiniBlockLayout()); + case ALL_NULL_LAYOUT -> AllNullLayout.fromProto(proto.getAllNullLayout()); + case FULL_ZIP_LAYOUT -> FullZipLayout.fromProto(proto.getFullZipLayout()); + default -> throw new RuntimeException("Unknown layout: " + proto.getLayoutCase()); + }; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/PageMetadata.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/PageMetadata.java new file mode 100644 index 000000000000..9993ff7f38ca --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/PageMetadata.java @@ -0,0 +1,29 @@ +/* + * 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.lance.file.v2.metadata; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public record PageMetadata(long numRows, long priority, PageLayout layout, List bufferOffsets) +{ + public PageMetadata + { + requireNonNull(layout, "layout is null"); + bufferOffsets = ImmutableList.copyOf(bufferOffsets); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/RepDefLayer.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/RepDefLayer.java new file mode 100644 index 000000000000..0a74a597fcda --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/metadata/RepDefLayer.java @@ -0,0 +1,71 @@ +/* + * 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.lance.file.v2.metadata; + +import java.util.List; + +import static com.google.common.collect.ImmutableList.toImmutableList; + +public enum RepDefLayer +{ + ALL_VALID_ITEM, + ALL_VALID_LIST, + NULLABLE_ITEM, + NULLABLE_LIST, + EMPTYABLE_LIST, + NULLABLE_AND_EMPTYABLE_LIST; + + public static RepDefLayer fromProto(build.buf.gen.lance.encodings21.RepDefLayer proto) + { + return switch (proto) { + case REPDEF_ALL_VALID_ITEM -> ALL_VALID_ITEM; + case REPDEF_ALL_VALID_LIST -> ALL_VALID_LIST; + case REPDEF_NULLABLE_ITEM -> NULLABLE_ITEM; + case REPDEF_NULLABLE_LIST -> NULLABLE_LIST; + case REPDEF_EMPTYABLE_LIST -> EMPTYABLE_LIST; + case REPDEF_NULL_AND_EMPTY_LIST -> NULLABLE_AND_EMPTYABLE_LIST; + default -> throw new IllegalArgumentException("Unknown RepDefLayer: " + proto); + }; + } + + public static List fromProtoList(List protos) + { + return protos.stream().map(RepDefLayer::fromProto).collect(toImmutableList()); + } + + public boolean isList() + { + return switch (this) { + case ALL_VALID_LIST, NULLABLE_LIST, EMPTYABLE_LIST, NULLABLE_AND_EMPTYABLE_LIST -> true; + default -> false; + }; + } + + public boolean isAllValid() + { + return switch (this) { + case ALL_VALID_ITEM, ALL_VALID_LIST, EMPTYABLE_LIST -> true; + default -> false; + }; + } + + public int numDefLevels() + { + return switch (this) { + case ALL_VALID_ITEM, ALL_VALID_LIST -> 0; + case NULLABLE_ITEM, NULLABLE_LIST, EMPTYABLE_LIST -> 1; + case NULLABLE_AND_EMPTYABLE_LIST -> 2; + }; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ArrayBufferAdapter.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ArrayBufferAdapter.java new file mode 100644 index 000000000000..e7182357dd9d --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ArrayBufferAdapter.java @@ -0,0 +1,41 @@ +/* + * 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.lance.file.v2.reader; + +import java.util.List; + +import static java.lang.Math.toIntExact; + +public abstract class ArrayBufferAdapter + implements BufferAdapter +{ + protected abstract int getLength(T buffer); + + @Override + public T merge(List buffers) + { + long totalSize = 0; + for (T buffer : buffers) { + totalSize += getLength(buffer); + } + T result = createBuffer(toIntExact(totalSize)); + + int offset = 0; + for (T buffer : buffers) { + copy(buffer, 0, result, offset, getLength(buffer)); + offset += getLength(buffer); + } + return result; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BaseUnraveler.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BaseUnraveler.java new file mode 100644 index 000000000000..da1aa2646596 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BaseUnraveler.java @@ -0,0 +1,230 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.lance.file.v2.metadata.RepDefLayer; +import it.unimi.dsi.fastutil.booleans.BooleanArrayList; +import it.unimi.dsi.fastutil.ints.IntArrayList; + +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; +import static io.trino.lance.file.v2.metadata.RepDefLayer.ALL_VALID_ITEM; +import static io.trino.lance.file.v2.metadata.RepDefLayer.NULLABLE_ITEM; + +// Lance uses repetition/definition levels similar to Parquet for nested data. +// Key difference is that Lance uses a layered system with 6 RepDefLayer types: +// - ALL_VALID_ITEM/ALL_VALID_LIST: no nulls/empties (0 def levels) +// - NULLABLE_ITEM/NULLABLE_LIST: nulls only (1 def level) +// - EMPTYABLE_LIST: empty lists only (1 def level) +// - NULLABLE_AND_EMPTYABLE_LIST: both null and empty lists (2 def levels) +public class BaseUnraveler + implements RepetitionDefinitionUnraveler +{ + private final int[] repetitionLevels; + private final int[] definitionLevels; + private final RepDefLayer[] layers; + private final int[] levelsToRepetition; + + private int currentDefinitionLevel; + private int currentRepetitionLevel; + private int currentLayer; + + public BaseUnraveler(int[] repetitionLevels, int[] definitionLevels, RepDefLayer[] layers) + { + this.repetitionLevels = repetitionLevels.length == 0 ? null : repetitionLevels; + this.definitionLevels = definitionLevels.length == 0 ? null : definitionLevels; + this.layers = layers; + + IntArrayList levelsToRepetitionBuilder = new IntArrayList(); + int repetitionCounter = 0; + levelsToRepetitionBuilder.add(0); + for (RepDefLayer layer : layers) { + switch (layer) { + case ALL_VALID_ITEM, ALL_VALID_LIST -> {} + case NULLABLE_ITEM -> levelsToRepetitionBuilder.add(repetitionCounter); + case NULLABLE_LIST, EMPTYABLE_LIST -> { + repetitionCounter++; + levelsToRepetitionBuilder.add(repetitionCounter); + } + case NULLABLE_AND_EMPTYABLE_LIST -> { + repetitionCounter++; + levelsToRepetitionBuilder.add(repetitionCounter); + levelsToRepetitionBuilder.add(repetitionCounter); + } + } + } + this.levelsToRepetition = levelsToRepetitionBuilder.toIntArray(); + this.currentDefinitionLevel = 0; + this.currentRepetitionLevel = 0; + this.currentLayer = 0; + } + + @Override + public boolean isAllValid() + { + return layers[currentLayer].isAllValid(); + } + + @Override + public void skipValidity() + { + verify(layers[currentLayer] == ALL_VALID_ITEM); + currentLayer++; + } + + @Override + public long maxLists() + { + checkArgument(layers[currentLayer] != NULLABLE_ITEM, "Current layer must not be a struct or primitive"); + return repetitionLevels.length; + } + + @Override + public BlockPositions calculateOffsets() + { + int validLevel = currentDefinitionLevel; + int nullLevel; + int emptyLevel; + + RepDefLayer currentMeaning = layers[currentLayer]; + switch (currentMeaning) { + case NULLABLE_LIST: + currentDefinitionLevel++; + nullLevel = validLevel + 1; + emptyLevel = 0; + break; + case EMPTYABLE_LIST: + currentDefinitionLevel++; + nullLevel = 0; + emptyLevel = validLevel + 1; + break; + case NULLABLE_AND_EMPTYABLE_LIST: + currentDefinitionLevel += 2; + nullLevel = validLevel + 1; + emptyLevel = validLevel + 2; + break; + case ALL_VALID_LIST: + nullLevel = 0; + emptyLevel = 0; + break; + default: + throw new IllegalStateException("Unexpected layer for list"); + } + currentLayer++; + + int maxLevel = Math.max(nullLevel, emptyLevel); + int upperNull = maxLevel; + + for (int i = currentLayer; i < layers.length; i++) { + RepDefLayer level = layers[i]; + switch (level) { + case NULLABLE_ITEM: + maxLevel++; + break; + case ALL_VALID_ITEM: + break; + default: + i = layers.length; + break; + } + } + + IntArrayList offsets = new IntArrayList(); + int currentOffset = 0; + + if (definitionLevels != null) { + BooleanArrayList nullList = new BooleanArrayList(); + + if (repetitionLevels.length != definitionLevels.length) { + throw new IllegalStateException("Rep and def levels must have same length"); + } + + for (int i = 0; i < repetitionLevels.length; i++) { + int repetitionLevel = repetitionLevels[i]; + + if (repetitionLevel > currentRepetitionLevel) { + int definitionLevel = definitionLevels[i]; + if (definitionLevel == 0) { + // Valid list + offsets.add(currentOffset); + currentOffset += 1; + nullList.add(false); + } + else if (definitionLevel > maxLevel) { + // Invisible at this level + } + else if (definitionLevel == nullLevel || definitionLevel > upperNull) { + // Null list + offsets.add(currentOffset); + nullList.add(true); + } + else if (definitionLevel == emptyLevel) { + // Empty list + offsets.add(currentOffset); + nullList.add(false); + } + else { + // New valid list starting with null item + offsets.add(currentOffset); + currentOffset += 1; + nullList.add(false); + } + } + else if (repetitionLevel == currentRepetitionLevel) { + currentOffset += 1; + } + } + + offsets.add(currentOffset); + currentRepetitionLevel++; + return new BlockPositions(Optional.of(nullList.toBooleanArray()), offsets.toIntArray()); + } + else { + for (int repetitionLevel : repetitionLevels) { + if (repetitionLevel > currentRepetitionLevel) { + offsets.add(currentOffset); + currentOffset += 1; + } + else if (repetitionLevel == currentRepetitionLevel) { + currentOffset += 1; + } + } + + offsets.add(currentOffset); + currentRepetitionLevel++; + return new BlockPositions(Optional.empty(), offsets.toIntArray()); + } + } + + @Override + public Optional calculateNulls() + { + if (layers[currentLayer] == ALL_VALID_ITEM) { + currentLayer++; + return Optional.empty(); + } + BooleanArrayList nullList = new BooleanArrayList(); + + for (int level : definitionLevels) { + if (levelsToRepetition[level] <= currentRepetitionLevel) { + nullList.add(level > currentDefinitionLevel); + } + } + currentLayer++; + currentDefinitionLevel++; + return Optional.of(nullList.toBooleanArray()); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BinaryBuffer.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BinaryBuffer.java new file mode 100644 index 000000000000..0d2ddd6bf3a7 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BinaryBuffer.java @@ -0,0 +1,105 @@ +/* + * 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.lance.file.v2.reader; + +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; + +import java.util.ArrayList; +import java.util.List; + +import static io.airlift.slice.SizeOf.sizeOf; +import static java.util.Objects.requireNonNull; + +public class BinaryBuffer +{ + private final List chunks; + private final int[] offsets; + + public BinaryBuffer(int valueCount) + { + this(new ArrayList<>(), new int[valueCount + 1]); + } + + private BinaryBuffer(List chunks, int[] offsets) + { + this.offsets = requireNonNull(offsets, "offsets is null"); + this.chunks = requireNonNull(chunks, "chunks is null"); + } + + public void add(byte[] source, int offset) + { + add(Slices.wrappedBuffer(source), offset); + } + + public void add(Slice slice, int offset) + { + chunks.add(slice); + offsets[offset + 1] = offsets[offset] + slice.length(); + } + + public Slice get(int index) + { + return chunks.get(index); + } + + public void addChunk(Slice slice) + { + chunks.add(slice); + } + + public Slice getSlice() + { + return asSlice(); + } + + public Slice asSlice() + { + if (chunks.size() == 1) { + return chunks.getFirst(); + } + int totalLength = 0; + for (Slice chunk : chunks) { + totalLength += chunk.length(); + } + Slice slice = Slices.allocate(totalLength); + int offset = 0; + for (Slice chunk : chunks) { + slice.setBytes(offset, chunk); + offset += chunk.length(); + } + chunks.clear(); + chunks.add(slice); + return slice; + } + + public int[] getOffsets() + { + return offsets; + } + + public int getValueCount() + { + return offsets.length - 1; + } + + public long getRetainedSize() + { + long chunksSizeInBytes = 0; + for (Slice slice : chunks) { + chunksSizeInBytes += slice.getRetainedSize(); + } + return sizeOf(offsets) + chunksSizeInBytes; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BinaryBufferAdapter.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BinaryBufferAdapter.java new file mode 100644 index 000000000000..dff5ad4a94a9 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BinaryBufferAdapter.java @@ -0,0 +1,83 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.spi.block.Block; +import io.trino.spi.block.VariableWidthBlock; + +import java.util.List; +import java.util.Optional; + +public class BinaryBufferAdapter + implements BufferAdapter +{ + public static final BinaryBufferAdapter VARIABLE_BINARY_BUFFER_ADAPTER = new BinaryBufferAdapter(); + + @Override + public BinaryBuffer createBuffer(int size) + { + return new BinaryBuffer(size); + } + + @Override + public void copy(BinaryBuffer source, int sourceIndex, BinaryBuffer destination, int destinationIndex, int length) + { + throw new UnsupportedOperationException(); + } + + @Override + public BinaryBuffer merge(List buffers) + { + if (buffers.isEmpty()) { + return new BinaryBuffer(0); + } + + int valueCount = 0; + for (BinaryBuffer binaryBuffer : buffers) { + valueCount += binaryBuffer.getValueCount(); + } + BinaryBuffer result = new BinaryBuffer(valueCount); + for (BinaryBuffer binaryBuffer : buffers) { + result.addChunk(binaryBuffer.asSlice()); + } + int[] resultOffsets = result.getOffsets(); + int[] firstOffsets = buffers.get(0).getOffsets(); + System.arraycopy(firstOffsets, 0, resultOffsets, 0, firstOffsets.length); + + int dataOffset = firstOffsets[firstOffsets.length - 1]; + int outputArrayOffset = firstOffsets.length; + for (int i = 1; i < buffers.size(); i++) { + int[] currentOffsets = buffers.get(i).getOffsets(); + for (int j = 1; j < currentOffsets.length; j++) { + resultOffsets[outputArrayOffset + j - 1] = dataOffset + currentOffsets[j]; + } + outputArrayOffset += currentOffsets.length - 1; + dataOffset = resultOffsets[outputArrayOffset - 1]; + } + + return result; + } + + @Override + public Block createBlock(BinaryBuffer buffer, Optional valueIsNull) + { + return new VariableWidthBlock(buffer.getValueCount(), buffer.getSlice(), buffer.getOffsets(), valueIsNull); + } + + @Override + public long getRetainedBytes(BinaryBuffer buffer) + { + return buffer.getRetainedSize(); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BufferAdapter.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BufferAdapter.java new file mode 100644 index 000000000000..9dec2fac5ab0 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/BufferAdapter.java @@ -0,0 +1,37 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.spi.block.Block; + +import java.util.List; +import java.util.Optional; + +public interface BufferAdapter +{ + T createBuffer(int size); + + void copy(T source, int sourceIndex, T destination, int destinationIndex, int length); + + T merge(List buffers); + + Block createBlock(T buffer, Optional valueIsNull); + + long getRetainedBytes(T buffer); + + default Block createDictionaryBlock(T buffer, Block dictionary, Optional valueIsNull) + { + throw new UnsupportedOperationException(); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ByteArrayBufferAdapter.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ByteArrayBufferAdapter.java new file mode 100644 index 000000000000..6d9e41cd41cc --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ByteArrayBufferAdapter.java @@ -0,0 +1,55 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.spi.block.Block; +import io.trino.spi.block.ByteArrayBlock; + +import java.util.Optional; + +public class ByteArrayBufferAdapter + extends ArrayBufferAdapter +{ + public static final ByteArrayBufferAdapter BYTE_ARRAY_BUFFER_ADAPTER = new ByteArrayBufferAdapter(); + + @Override + public byte[] createBuffer(int size) + { + return new byte[size]; + } + + @Override + public void copy(byte[] source, int sourceOffset, byte[] destination, int destinationOffset, int length) + { + System.arraycopy(source, sourceOffset, destination, destinationOffset, length); + } + + @Override + public Block createBlock(byte[] buffer, Optional valueIsNull) + { + return new ByteArrayBlock(buffer.length, valueIsNull, buffer); + } + + @Override + public long getRetainedBytes(byte[] buffer) + { + return buffer.length; + } + + @Override + protected int getLength(byte[] buffer) + { + return buffer.length; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ChunkMetadata.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ChunkMetadata.java new file mode 100644 index 000000000000..ea7530694586 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ChunkMetadata.java @@ -0,0 +1,16 @@ +/* + * 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.lance.file.v2.reader; + +public record ChunkMetadata(long numValues, long chunkSizeBytes, long offsetBytes) {} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ColumnReader.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ColumnReader.java new file mode 100644 index 000000000000..ffdad600f286 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ColumnReader.java @@ -0,0 +1,52 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.lance.file.LanceDataSource; +import io.trino.lance.file.v2.metadata.ColumnMetadata; +import io.trino.lance.file.v2.metadata.Field; +import io.trino.lance.file.v2.metadata.LogicalType; +import io.trino.memory.context.AggregatedMemoryContext; + +import java.util.List; +import java.util.Map; + +public interface ColumnReader +{ + static ColumnReader createColumnReader(LanceDataSource dataSource, + Field field, + Map columnMetadata, + List readRanges, + AggregatedMemoryContext memoryContext) + { + return switch (LogicalType.from(field.logicalType())) { + case LogicalType.Int8Type _, + LogicalType.Int16Type _, + LogicalType.Int32Type _, + LogicalType.Int64Type _, + LogicalType.FloatType _, + LogicalType.DoubleType _, + LogicalType.StringType _, + LogicalType.BinaryType _, + LogicalType.DateType _ -> new PrimitiveColumnReader(dataSource, field, columnMetadata.get(field.id()), readRanges, memoryContext); + case LogicalType.ListType _ -> new ListColumnReader(dataSource, field, columnMetadata, readRanges, memoryContext); + case LogicalType.StructType _ -> new StructColumnReader(dataSource, field, columnMetadata, readRanges, memoryContext); + default -> throw new RuntimeException("Unsupported logical type: " + field.logicalType()); + }; + } + + void prepareNextRead(int batchSize); + + DecodedPage read(); +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/CompositeUnraveler.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/CompositeUnraveler.java new file mode 100644 index 000000000000..df0ff1ea70de --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/CompositeUnraveler.java @@ -0,0 +1,106 @@ +/* + * 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.lance.file.v2.reader; + +import it.unimi.dsi.fastutil.booleans.BooleanArrayList; +import it.unimi.dsi.fastutil.booleans.BooleanList; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; + +import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class CompositeUnraveler + implements RepetitionDefinitionUnraveler +{ + private final List unravelers; + + public CompositeUnraveler(List unravelers) + { + this.unravelers = requireNonNull(unravelers, "unravelers is null"); + } + + @Override + public BlockPositions calculateOffsets() + { + boolean allValid = isAllValid(); + IntArrayList offsetBuffer = new IntArrayList(); + BooleanArrayList nullBuffer = new BooleanArrayList(); + for (RepetitionDefinitionUnraveler unraveler : unravelers) { + BlockPositions positions = unraveler.calculateOffsets(); + if (offsetBuffer.isEmpty()) { + offsetBuffer.addAll(IntList.of(positions.offsets())); + } + else { + int lastOffset = offsetBuffer.popInt(); + for (int i = 0; i < positions.offsets().length; i++) { + offsetBuffer.add(positions.offsets()[i] + lastOffset); + } + } + if (positions.nulls().isPresent()) { + nullBuffer.addAll(BooleanList.of(positions.nulls().get())); + } + else if (!allValid) { + for (int i = 0; i < positions.offsets().length - 1; i++) { + nullBuffer.add(false); + } + } + } + if (allValid) { + return new BlockPositions(Optional.empty(), offsetBuffer.toIntArray()); + } + return new BlockPositions(Optional.of(nullBuffer.toBooleanArray()), offsetBuffer.toIntArray()); + } + + @Override + public Optional calculateNulls() + { + if (isAllValid()) { + skipValidity(); + return Optional.empty(); + } + else { + BooleanArrayList validity = unravelers.stream() + .map(unraveler -> unraveler.calculateNulls().orElseThrow()) + .collect(BooleanArrayList::new, + (buffer, array) -> { + for (boolean value : array) { + buffer.add(value); + } + }, + BooleanArrayList::addAll); + return Optional.of(validity.toBooleanArray()); + } + } + + @Override + public boolean isAllValid() + { + return unravelers.stream().allMatch(RepetitionDefinitionUnraveler::isAllValid); + } + + @Override + public void skipValidity() + { + unravelers.forEach(RepetitionDefinitionUnraveler::skipValidity); + } + + @Override + public long maxLists() + { + return unravelers.stream().map(RepetitionDefinitionUnraveler::maxLists).mapToLong(Long::longValue).sum(); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/DataValuesBuffer.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/DataValuesBuffer.java new file mode 100644 index 000000000000..93956ae4bc4f --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/DataValuesBuffer.java @@ -0,0 +1,77 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.lance.file.v2.metadata.RepDefLayer; +import io.trino.spi.block.Block; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class DataValuesBuffer +{ + private final BufferAdapter bufferAdapter; + private final List valueBuffers = new ArrayList<>(); + + public DataValuesBuffer(BufferAdapter bufferAdapter) + { + this.bufferAdapter = requireNonNull(bufferAdapter, "bufferAdapter is null"); + } + + public void append(T buffer) + { + valueBuffers.add(buffer); + } + + public T getMergedValues() + { + if (valueBuffers.size() == 1) { + return valueBuffers.get(0); + } + return bufferAdapter.merge(valueBuffers); + } + + public DecodedPage createDecodedPage(int[] definitions, + int[] repetitions, + List layers, + Optional dictionary) + { + T mergedValues = getMergedValues(); + BaseUnraveler unraveler = new BaseUnraveler(repetitions, definitions, layers.toArray(RepDefLayer[]::new)); + + Optional isNull = unraveler.calculateNulls(); + if (dictionary.isEmpty()) { + return new DecodedPage(bufferAdapter.createBlock(mergedValues, isNull), unraveler); + } + return new DecodedPage(bufferAdapter.createDictionaryBlock(mergedValues, dictionary.get(), isNull), unraveler); + } + + public void reset() + { + valueBuffers.clear(); + } + + public boolean isEmpty() + { + return valueBuffers.isEmpty(); + } + + public long getRetainedBytes() + { + return valueBuffers.stream().mapToLong(bufferAdapter::getRetainedBytes).sum(); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/DecodedPage.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/DecodedPage.java new file mode 100644 index 000000000000..3964297e4801 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/DecodedPage.java @@ -0,0 +1,27 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.spi.block.Block; + +import static java.util.Objects.requireNonNull; + +public record DecodedPage(Block block, RepetitionDefinitionUnraveler unraveler) +{ + public DecodedPage + { + requireNonNull(block, "block is null"); + requireNonNull(unraveler, "unraveler is null"); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/IntArrayBufferAdapter.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/IntArrayBufferAdapter.java new file mode 100644 index 000000000000..fc9900b8c11c --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/IntArrayBufferAdapter.java @@ -0,0 +1,70 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.spi.block.Block; +import io.trino.spi.block.DictionaryBlock; +import io.trino.spi.block.IntArrayBlock; + +import java.util.Optional; + +public class IntArrayBufferAdapter + extends ArrayBufferAdapter +{ + public static final IntArrayBufferAdapter INT_ARRAY_BUFFER_ADAPTER = new IntArrayBufferAdapter(); + + @Override + public int[] createBuffer(int size) + { + return new int[size]; + } + + @Override + public void copy(int[] source, int sourceOffset, int[] destination, int destinationOffset, int length) + { + System.arraycopy(source, sourceOffset, destination, destinationOffset, length); + } + + @Override + public Block createBlock(int[] buffer, Optional valueIsNull) + { + return new IntArrayBlock(buffer.length, valueIsNull, buffer); + } + + @Override + public long getRetainedBytes(int[] buffer) + { + return (long) buffer.length * Integer.BYTES; + } + + @Override + protected int getLength(int[] buffer) + { + return buffer.length; + } + + @Override + public Block createDictionaryBlock(int[] ids, Block dictionary, Optional valueIsNull) + { + if (valueIsNull.isPresent()) { + boolean[] isNull = valueIsNull.get(); + for (int i = 0; i < ids.length; i++) { + if (isNull[i]) { + ids[i] = dictionary.getPositionCount() - 1; + } + } + } + return DictionaryBlock.create(ids.length, dictionary, ids); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ListColumnReader.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ListColumnReader.java new file mode 100644 index 000000000000..f9e8293889a1 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ListColumnReader.java @@ -0,0 +1,66 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.lance.file.LanceDataSource; +import io.trino.lance.file.v2.metadata.ColumnMetadata; +import io.trino.lance.file.v2.metadata.Field; +import io.trino.lance.file.v2.reader.RepetitionDefinitionUnraveler.BlockPositions; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.spi.block.ArrayBlock; +import io.trino.spi.block.Block; + +import java.util.List; +import java.util.Map; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; +import static java.util.Objects.requireNonNull; + +public class ListColumnReader + implements ColumnReader +{ + private final ColumnReader childColumnReader; + + private int nextBatchSize; + + public ListColumnReader(LanceDataSource dataSource, + Field field, + Map columnMetadata, + List ranges, + AggregatedMemoryContext memoryContext) + { + requireNonNull(field, "field is null"); + checkArgument(field.children().size() == 1, "list should have only one child filed"); + this.childColumnReader = ColumnReader.createColumnReader(dataSource, field.children().getFirst(), columnMetadata, ranges, memoryContext); + } + + @Override + public void prepareNextRead(int batchSize) + { + childColumnReader.prepareNextRead(batchSize); + nextBatchSize = batchSize; + } + + @Override + public DecodedPage read() + { + DecodedPage decodedChild = childColumnReader.read(); + RepetitionDefinitionUnraveler unraveler = decodedChild.unraveler(); + BlockPositions positions = unraveler.calculateOffsets(); + verify(nextBatchSize == positions.offsets().length - 1); + Block arrayBlock = ArrayBlock.fromElementBlock(nextBatchSize, positions.nulls(), positions.offsets(), decodedChild.block()); + return new DecodedPage(arrayBlock, unraveler); + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/LongArrayBufferAdapter.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/LongArrayBufferAdapter.java new file mode 100644 index 000000000000..4d677343bf67 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/LongArrayBufferAdapter.java @@ -0,0 +1,55 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.spi.block.Block; +import io.trino.spi.block.LongArrayBlock; + +import java.util.Optional; + +public class LongArrayBufferAdapter + extends ArrayBufferAdapter +{ + public static final LongArrayBufferAdapter LONG_ARRAY_BUFFER_ADAPTER = new LongArrayBufferAdapter(); + + @Override + public long[] createBuffer(int size) + { + return new long[size]; + } + + @Override + public void copy(long[] source, int sourceOffset, long[] destination, int destinationOffset, int length) + { + System.arraycopy(source, sourceOffset, destination, destinationOffset, length); + } + + @Override + public Block createBlock(long[] buffer, Optional valueIsNull) + { + return new LongArrayBlock(buffer.length, valueIsNull, buffer); + } + + @Override + public long getRetainedBytes(long[] buffer) + { + return (long) buffer.length * Long.BYTES; + } + + @Override + protected int getLength(long[] buffer) + { + return buffer.length; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/MiniBlockPageReader.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/MiniBlockPageReader.java new file mode 100644 index 000000000000..b486fe4addfc --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/MiniBlockPageReader.java @@ -0,0 +1,559 @@ +/* + * 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.lance.file.v2.reader; + +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import io.trino.lance.file.LanceDataSource; +import io.trino.lance.file.v2.encoding.BlockDecoder; +import io.trino.lance.file.v2.encoding.LanceEncoding; +import io.trino.lance.file.v2.encoding.MiniBlockDecoder; +import io.trino.lance.file.v2.metadata.DiskRange; +import io.trino.lance.file.v2.metadata.MiniBlockLayout; +import io.trino.lance.file.v2.metadata.RepDefLayer; +import io.trino.lance.file.v2.reader.RepetitionIndex.RepIndexBlock; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.memory.context.LocalMemoryContext; +import io.trino.spi.block.Block; +import io.trino.spi.block.ValueBlock; +import io.trino.spi.type.Type; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.Verify.verify; +import static io.trino.lance.file.v2.metadata.RepDefLayer.NULLABLE_ITEM; +import static io.trino.lance.file.v2.reader.IntArrayBufferAdapter.INT_ARRAY_BUFFER_ADAPTER; +import static java.lang.Math.toIntExact; + +public class MiniBlockPageReader + implements PageReader +{ + public static final int MINIBLOCK_ALIGNMENT = 8; + + private final LanceDataSource dataSource; + private final Optional repetitionEncoding; + private final Optional definitionEncoding; + private final Optional dictionaryEncoding; + private final Optional numDictionaryItems; + private final Optional dictionaryBlock; + private final LanceEncoding valueEncoding; + private final int repetitionIndexDepth; + private final List repetitionIndex; + private final List layers; + private final int maxVisibleDefinition; + private final long numBuffers; + private final long numRows; + private final List chunks; + private final BufferAdapter valueBufferAdapter; + private final DataValuesBuffer valuesBuffer; + private final DataValuesBuffer repetitionBuffer; + private final DataValuesBuffer definitionBuffer; + // memory usage for current miniblock page + private final LocalMemoryContext memoryUsage; + + private long levelOffset; + + public MiniBlockPageReader(LanceDataSource dataSource, + Type type, + MiniBlockLayout layout, + List bufferOffsets, + long numRows, + AggregatedMemoryContext memoryContext) + { + this.dataSource = dataSource; + this.repetitionEncoding = layout.repetitionEncoding(); + this.definitionEncoding = layout.definitionEncoding(); + this.dictionaryEncoding = layout.dictionaryEncoding(); + this.numDictionaryItems = layout.numDictionaryItems(); + this.valueEncoding = layout.valueEncoding(); + this.repetitionIndexDepth = layout.repIndexDepth(); + this.layers = layout.layers(); + this.maxVisibleDefinition = layers.stream().takeWhile(layer -> !layer.isList()).mapToInt(RepDefLayer::numDefLevels).sum(); + this.numBuffers = layout.numBuffers(); + this.numRows = numRows; + try { + // build chunk meta + // bufferOffsets[0] = chunk metadata buffer + // bufferOffset[1] = value buffer + DiskRange chunkMetadataBuf = bufferOffsets.get(0); + DiskRange valueBuf = bufferOffsets.get(1); + Slice chunkMetadataSlice = dataSource.readFully(chunkMetadataBuf.position(), toIntExact(chunkMetadataBuf.length())); + int numWords = chunkMetadataSlice.length() / 2; + ImmutableList.Builder chunkMetadataBuilder = ImmutableList.builder(); + long count = 0; + long offset = valueBuf.position(); + for (int i = 0; i < numWords; i++) { + int word = chunkMetadataSlice.getUnsignedShort(i * 2); + int logNumValues = word & 0xF; + int dividedBytes = word >>> 4; + int chunkSizeBytes = (dividedBytes + 1) * MINIBLOCK_ALIGNMENT; + long numValues = i < numWords - 1 ? 1 << logNumValues : layout.numItems() - count; + count += numValues; + + chunkMetadataBuilder.add(new ChunkMetadata(numValues, chunkSizeBytes, offset)); + offset += chunkSizeBytes; + } + this.chunks = chunkMetadataBuilder.build(); + // load dictionary + if (dictionaryEncoding.isPresent()) { + DiskRange dictionaryRange = bufferOffsets.get(2); + Slice dictionarySlice = dataSource.readFully(dictionaryRange.position(), toIntExact(dictionaryRange.length())); + ValueBlock dictionary = dictionaryEncoding.get().decodeBlock(dictionarySlice, toIntExact(numDictionaryItems.get())); + // if a block is nullable, we append null to the end of dictionary + if (layers.stream().anyMatch(layer -> (layer == NULLABLE_ITEM))) { + dictionaryBlock = Optional.of(dictionary.copyWithAppendedNull()); + } + else { + dictionaryBlock = Optional.of(dictionary); + } + } + else { + dictionaryBlock = Optional.empty(); + } + // load repetition index + if (repetitionIndexDepth > 0) { + DiskRange repetitionIndexRange = bufferOffsets.getLast(); + verify(repetitionIndexRange.length() % 8 == 0); + Slice repetitionIndexSlice = dataSource.readFully(repetitionIndexRange.position(), toIntExact(repetitionIndexRange.length())); + repetitionIndex = RepetitionIndex.from(repetitionIndexSlice, repetitionIndexDepth); + } + else { + repetitionIndex = RepetitionIndex.defaultIndex(chunks); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + valueBufferAdapter = valueEncoding.getBufferAdapter(); + valuesBuffer = new DataValuesBuffer(valueBufferAdapter); + repetitionBuffer = new DataValuesBuffer<>(INT_ARRAY_BUFFER_ADAPTER); + definitionBuffer = new DataValuesBuffer<>(INT_ARRAY_BUFFER_ADAPTER); + memoryUsage = memoryContext.newLocalMemoryContext(MiniBlockPageReader.class.getSimpleName()); + + levelOffset = 0; + } + + public static int padding(int n) + { + return (MINIBLOCK_ALIGNMENT - (n & (MINIBLOCK_ALIGNMENT - 1))) & (MINIBLOCK_ALIGNMENT - 1); + } + + @Override + public DecodedPage decodeRanges(List ranges) + { + valuesBuffer.reset(); + repetitionBuffer.reset(); + definitionBuffer.reset(); + levelOffset = 0; + + for (Range range : ranges) { + long rowsNeeded = range.length(); + boolean needPreamble = false; + + // find first chunk that has row >= range.start + int blockIndex = Collections.binarySearch(repetitionIndex, range.start(), (block, key) -> Long.compare(((RepIndexBlock) block).firstRow(), (long) key)); + if (blockIndex >= 0) { + while (blockIndex > 0 && repetitionIndex.get(blockIndex - 1).firstRow() == range.start()) { + blockIndex--; + } + } + else { + blockIndex = -(blockIndex + 1) - 1; + } + + long toSkip = range.start() - repetitionIndex.get(blockIndex).firstRow(); + while (rowsNeeded > 0 || needPreamble) { + RepIndexBlock chunkIndexBlock = repetitionIndex.get(blockIndex); + long rowsAvailable = chunkIndexBlock.startCount() - toSkip; + + // handle preamble only blocks (rowsAvailable == 0) + if (rowsAvailable == 0 && toSkip == 0) { + if (chunkIndexBlock.hasPreamble() && needPreamble) { + readChunk(chunks.get(blockIndex), chunkIndexBlock, toSkip, 0, PreambleAction.TAKE, false); + if (chunkIndexBlock.startCount() > 0 || blockIndex == repetitionIndex.size() - 1) { + needPreamble = false; + } + } + blockIndex++; + continue; + } + + if (rowsAvailable == 0 && toSkip > 0) { + toSkip -= chunkIndexBlock.startCount(); + blockIndex++; + continue; + } + + long rowsToTake = Math.min(rowsNeeded, rowsAvailable); + rowsNeeded -= rowsToTake; + + boolean takeTrailer = false; + PreambleAction preambleAction; + if (chunkIndexBlock.hasPreamble()) { + if (needPreamble) { + preambleAction = PreambleAction.TAKE; + } + else { + preambleAction = PreambleAction.SKIP; + } + } + else { + preambleAction = PreambleAction.ABSENT; + } + long fullRowsToTake = rowsToTake; + + if (rowsToTake == rowsAvailable && chunkIndexBlock.hasTrailer()) { + takeTrailer = true; + needPreamble = true; + fullRowsToTake--; + } + else { + needPreamble = false; + } + readChunk(chunks.get(blockIndex), chunkIndexBlock, toSkip, fullRowsToTake, preambleAction, takeTrailer); + + toSkip = 0; + blockIndex++; + } + } + memoryUsage.setBytes(getRetainedBytes()); + return valuesBuffer.createDecodedPage(definitionBuffer.getMergedValues(), repetitionBuffer.getMergedValues(), layers, dictionaryBlock); + } + + private long getRetainedBytes() + { + long retainedBytes = 0; + if (dictionaryBlock.isPresent()) { + retainedBytes += dictionaryBlock.get().getRetainedSizeInBytes(); + } + retainedBytes += valuesBuffer.getRetainedBytes() + repetitionBuffer.getRetainedBytes() + definitionBuffer.getRetainedBytes(); + return retainedBytes; + } + + public static SelectedRanges mapRange(Range rowRange, int[] rep, int[] def, int maxRepetitionLevel, int maxVisibleDefinition, int numItems, PreambleAction preambleAction) + { + if (rep == null) { + // if there is no repetition, item and level range are the same as row range + return new SelectedRanges(rowRange, rowRange); + } + + int itemsInPreamble = 0; + int firstRowStart = -1; + switch (preambleAction) { + case SKIP, TAKE: { + if (def != null) { + for (int i = 0; i < rep.length; i++) { + if (rep[i] == maxRepetitionLevel) { + firstRowStart = i; + break; + } + if (def[i] <= maxVisibleDefinition) { + itemsInPreamble++; + } + } + } + else { + for (int i = 0; i < rep.length; i++) { + if (rep[i] == maxRepetitionLevel) { + firstRowStart = i; + break; + } + } + itemsInPreamble = Math.min(firstRowStart, rep.length); + } + // chunk is entirely preamble + if (firstRowStart == -1) { + return new SelectedRanges(new Range(0, numItems), new Range(0, rep.length)); + } + break; + } + case ABSENT: { + firstRowStart = 0; + break; + } + } + + // handle preamble only blocks + if (rowRange.start() == rowRange.end()) { + return new SelectedRanges(new Range(0, itemsInPreamble), new Range(0, firstRowStart)); + } + + // we are reading at least 1 full row if we reach here + verify(rowRange.length() > 0); + int rowsSeen = 0; + int newStart = firstRowStart; + int newLevelsStart = firstRowStart; + + if (def != null) { + long leadInvisSeen = 0; + if (rowRange.start() > 0) { + if (def[firstRowStart] > maxVisibleDefinition) { + leadInvisSeen += 1; + } + for (int i = firstRowStart + 1; i < def.length; i++) { + if (rep[i] == maxRepetitionLevel) { + rowsSeen++; + if (rowsSeen == rowRange.start()) { + newStart = i - toIntExact(leadInvisSeen); + newLevelsStart = i; + break; + } + if (def[i] > maxVisibleDefinition) { + leadInvisSeen++; + } + } + } + } + + rowsSeen++; + long newEnd = Long.MAX_VALUE; + long newLevelsEnd = rep.length; + long trailInvisSeen = def[newLevelsStart] > maxVisibleDefinition ? 1 : 0; + + for (int i = newLevelsStart + 1; i < def.length; i++) { + int repLevel = rep[i]; + int defLevel = def[i]; + if (repLevel == maxRepetitionLevel) { + rowsSeen++; + if (rowsSeen == rowRange.end() + 1) { + newEnd = i - leadInvisSeen - trailInvisSeen; + newLevelsEnd = i; + break; + } + if (defLevel > maxVisibleDefinition) { + trailInvisSeen++; + } + } + } + if (newEnd == Long.MAX_VALUE) { + newLevelsEnd = rep.length; + newEnd = rep.length - (leadInvisSeen + trailInvisSeen); + } + verify(newEnd != Long.MAX_VALUE); + if (preambleAction == PreambleAction.TAKE) { + newLevelsStart = 0; + newStart = 0; + } + + return new SelectedRanges(new Range(newStart, newEnd), new Range(newLevelsStart, newLevelsEnd)); + } + else { + if (rowRange.start() > 0) { + for (int i = firstRowStart + 1; i < rep.length; i++) { + if (rep[i] == maxRepetitionLevel) { + rowsSeen++; + if (rowsSeen == rowRange.start()) { + newStart = i; + break; + } + } + } + } + long end = rep.length; + if (rowRange.end() < numItems) { + for (int i = toIntExact(firstRowStart + newStart + 1); i < rep.length; i++) { + if (rep[i] == maxRepetitionLevel) { + rowsSeen++; + if (rowsSeen == rowRange.end()) { + end = i; + break; + } + } + } + } + if (preambleAction == PreambleAction.TAKE) { + newStart = 0; + } + return new SelectedRanges(new Range(newStart, end), new Range(newStart, end)); + } + } + + public void readChunk(ChunkMetadata chunk, RepIndexBlock chunkIndex, long rowsToSkip, long rowsToTake, PreambleAction preambleAction, boolean takeTrailer) + { + try { + ChunkReader chunkReader = new ChunkReader(dataSource.readFully(chunk.offsetBytes(), toIntExact(chunk.chunkSizeBytes())), toIntExact(chunk.numValues()), valueBufferAdapter); + + SelectedRanges selectedRanges = mapRange(new Range(rowsToSkip, rowsToSkip + rowsToTake + (takeTrailer ? 1 : 0)), + chunkReader.readRepetitionLevels(), + chunkReader.readDefinitionLevels(), + toIntExact(layers.stream().filter(RepDefLayer::isList).count()), + maxVisibleDefinition, + chunkReader.getNumValues(), + preambleAction); + Range itemRange = selectedRanges.itemRange(); + Range levelRange = selectedRanges.levelRange(); + + if (!repetitionEncoding.isEmpty()) { + if (repetitionBuffer.isEmpty() && levelOffset > 0) { + repetitionBuffer.append(new int[toIntExact(levelOffset)]); + } + repetitionBuffer.append(Arrays.copyOfRange(chunkReader.readRepetitionLevels(), toIntExact(levelRange.start()), toIntExact(levelRange.end()))); + } + + if (!definitionEncoding.isEmpty()) { + if (definitionBuffer.isEmpty() && levelOffset > 0) { + definitionBuffer.append(new int[toIntExact(levelOffset)]); + } + definitionBuffer.append(Arrays.copyOfRange(chunkReader.readDefinitionLevels(), toIntExact(levelRange.start()), toIntExact(levelRange.end()))); + } + + levelOffset += levelRange.length(); + chunkReader.readValues(toIntExact(itemRange.start()), toIntExact(itemRange.length())); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private int[] loadLevels(LanceEncoding encoding, Slice slice, int numLevels) + { + int[] levels = new int[numLevels]; + BlockDecoder levelsDecoder = encoding.getBlockDecoder(); + short[] buffer = new short[numLevels]; + levelsDecoder.init(slice, numLevels); + levelsDecoder.read(0, buffer, 0, numLevels); + for (int i = 0; i < numLevels; i++) { + levels[i] = buffer[i] & 0xFFFF; + } + return levels; + } + + public long getNumRows() + { + return numRows; + } + + public enum PreambleAction + { + TAKE, SKIP, ABSENT + } + + public record SelectedRanges(Range itemRange, Range levelRange) + { + @Override + public int hashCode() + { + return Objects.hash(itemRange, levelRange); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + SelectedRanges other = (SelectedRanges) obj; + return Objects.equals(itemRange, other.itemRange) && Objects.equals(levelRange, other.levelRange); + } + } + + public class ChunkReader + { + private final BufferAdapter bufferAdapter; + private final List buffers; + private final MiniBlockDecoder valueDecoder; + private final int numValues; + private final int[] repetitions; + private final int[] definitions; + + public ChunkReader(Slice chunk, int numValues, BufferAdapter bufferAdapter) + { + this.bufferAdapter = bufferAdapter; + this.numValues = numValues; + // decode header + int offset = 0; + int numLevels = chunk.getUnsignedShort(offset); + offset += 2; + + int repetitionSize = 0; + if (repetitionEncoding.isPresent()) { + repetitionSize = toIntExact(chunk.getUnsignedShort(offset)); + offset += 2; + } + + int definitionSize = 0; + if (definitionEncoding.isPresent()) { + definitionSize = toIntExact(chunk.getUnsignedShort(offset)); + offset += 2; + } + + int[] bufferSizes = new int[toIntExact(numBuffers)]; + for (int i = 0; i < numBuffers; i++) { + bufferSizes[i] = chunk.getUnsignedShort(offset); + offset += 2; + } + offset += padding(offset); + + // load repetition/definition levels + if (repetitionEncoding.isPresent()) { + repetitions = loadLevels(repetitionEncoding.get(), chunk.slice(offset, repetitionSize), numLevels); + offset += repetitionSize; + offset += padding(offset); + } + else { + repetitions = null; + } + if (definitionEncoding.isPresent()) { + definitions = loadLevels(definitionEncoding.get(), chunk.slice(offset, definitionSize), numLevels); + offset += definitionSize; + offset += padding(offset); + } + else { + definitions = null; + } + + // load data buffers + ImmutableList.Builder builder = ImmutableList.builder(); + for (int i = 0; i < bufferSizes.length; i++) { + int size = bufferSizes[i]; + builder.add(chunk.slice(offset, size)); + offset += size; + offset += padding(offset); + } + buffers = builder.build(); + valueDecoder = valueEncoding.getMiniBlockDecoder(); + valueDecoder.init(buffers, numValues); + } + + public void readValues(int offset, int count) + { + T output = bufferAdapter.createBuffer(count); + valueDecoder.read(offset, output, 0, count); + valuesBuffer.append(output); + } + + public int[] readDefinitionLevels() + { + return definitions; + } + + public int[] readRepetitionLevels() + { + return repetitions; + } + + public int getNumValues() + { + return numValues; + } + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/PageReader.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/PageReader.java new file mode 100644 index 000000000000..f3f3238d0168 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/PageReader.java @@ -0,0 +1,21 @@ +/* + * 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.lance.file.v2.reader; + +import java.util.List; + +public interface PageReader +{ + DecodedPage decodeRanges(List ranges); +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/PrimitiveColumnReader.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/PrimitiveColumnReader.java new file mode 100644 index 000000000000..8a637552ed89 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/PrimitiveColumnReader.java @@ -0,0 +1,178 @@ +/* + * 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.lance.file.v2.reader; + +import com.google.common.collect.ImmutableList; +import io.trino.lance.file.LanceDataSource; +import io.trino.lance.file.v2.metadata.ColumnMetadata; +import io.trino.lance.file.v2.metadata.Field; +import io.trino.lance.file.v2.metadata.MiniBlockLayout; +import io.trino.lance.file.v2.metadata.PageLayout; +import io.trino.lance.file.v2.metadata.PageMetadata; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.spi.block.Block; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.type.Type; + +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Verify.verify; +import static java.lang.Math.max; +import static java.lang.Math.min; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public class PrimitiveColumnReader + implements ColumnReader +{ + private final LanceDataSource dataSource; + private final Type type; + private final List pages; + private final List ranges; + private final AggregatedMemoryContext aggregatedMemoryContext; + + private PageReader pageReader; + private int nextBatchSize; + + private long globalRowOffset; // global row number + private int pageIndex; // current page being processed + private long pageOffset; + private int rangeIndex; // current range being processed + private long rangeOffset; + + public PrimitiveColumnReader( + LanceDataSource dataSource, + Field field, + ColumnMetadata columnMetadata, + List ranges, + AggregatedMemoryContext memoryContext) + { + requireNonNull(field, "field is null"); + this.dataSource = requireNonNull(dataSource, "dataSource is null"); + this.type = field.toTrinoType(); + this.pages = requireNonNull(columnMetadata.pages(), "pages is null"); + this.ranges = requireNonNull(ranges, "ranges is null"); + this.aggregatedMemoryContext = requireNonNull(memoryContext, "memoryContext is null"); + + this.globalRowOffset = 0; + this.pageIndex = 0; + this.rangeIndex = 0; + } + + @Override + public void prepareNextRead(int batchSize) + { + nextBatchSize = batchSize; + } + + @Override + public DecodedPage read() + { + if (rangeIndex >= ranges.size()) { + throw new RuntimeException("no more ranges to read, something went wrong in LanceReader"); + } + + BlockBuilder blockBuilder = type.createBlockBuilder(null, nextBatchSize); + int rowCount = 0; + Range currentRange = ranges.get(rangeIndex); + PageMetadata currentPage = pages.get(pageIndex); + List decodedPages = new ArrayList<>(); + while (rowCount < nextBatchSize) { + // move to next page + while (currentPage.numRows() + globalRowOffset <= currentRange.start() + rangeOffset) { + globalRowOffset += currentPage.numRows(); + advancePage(); + currentPage = pages.get(pageIndex); + } + + // find all ranges in current page + ImmutableList.Builder builder = ImmutableList.builder(); + long remaining = nextBatchSize - rowCount; + while (remaining > 0 && currentPage.numRows() + globalRowOffset > currentRange.start() + rangeOffset) { + long start = max(currentRange.start() + rangeOffset, globalRowOffset); + long startInPage = start - globalRowOffset; + long endInPage = min(startInPage + min(currentRange.length() - rangeOffset, remaining), currentPage.numRows()); + boolean lastInPage = (endInPage + globalRowOffset) >= currentRange.end(); + builder.add(new Range(startInPage, endInPage)); + remaining -= endInPage - startInPage; + pageOffset = endInPage - globalRowOffset; + rangeOffset = endInPage - currentRange.start(); + + if (lastInPage) { + rangeIndex++; + rangeOffset = 0; + if (rangeIndex == ranges.size()) { + break; + } + currentRange = ranges.get(rangeIndex); + } + else { + break; + } + } + + // decode the page with ranges for current batch + if (pageReader == null) { + pageReader = createPageReader(currentPage); + } + DecodedPage decodedPage = pageReader.decodeRanges(builder.build()); + decodedPages.add(decodedPage); + long numRowsRead = nextBatchSize - rowCount - remaining; + rowCount += toIntExact(numRowsRead); + if (pageOffset >= currentPage.numRows()) { + globalRowOffset += currentPage.numRows(); + advancePage(); + } + } + + if (decodedPages.size() > 1) { + // merge all decoded pages + List unravelers = new ArrayList<>(); + for (DecodedPage decodedPage : decodedPages) { + Block block = decodedPage.block(); + for (int i = 0; i < block.getPositionCount(); i++) { + if (block.isNull(i)) { + blockBuilder.appendNull(); + } + else { + blockBuilder.append(block.getUnderlyingValueBlock(), block.getUnderlyingValuePosition(i)); + } + } + unravelers.add(decodedPage.unraveler()); + } + + return new DecodedPage(blockBuilder.build(), new CompositeUnraveler(unravelers)); + } + verify(!decodedPages.isEmpty()); + return decodedPages.getFirst(); + } + + private PageReader createPageReader(PageMetadata pageMetadata) + { + PageLayout layout = pageMetadata.layout(); + return switch (layout) { + case MiniBlockLayout miniBlockLayout -> + new MiniBlockPageReader(dataSource, type, miniBlockLayout, pageMetadata.bufferOffsets(), pageMetadata.numRows(), aggregatedMemoryContext); + default -> throw new IllegalArgumentException("Unsupported PageLayout: " + layout); + }; + } + + private void advancePage() + { + pageIndex++; + pageOffset = 0; + pageReader = null; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/Range.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/Range.java new file mode 100644 index 000000000000..cd294ed5572d --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/Range.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.lance.file.v2.reader; + +import static com.google.common.base.Preconditions.checkArgument; + +public record Range(long start, long end) +{ + public Range + { + checkArgument(start >= 0, "start must be greater than or equal to zero"); + checkArgument(end >= start, "end must be greater start"); + } + + public long length() + { + return end - start; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/RepetitionDefinitionUnraveler.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/RepetitionDefinitionUnraveler.java new file mode 100644 index 000000000000..6b40971c2c94 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/RepetitionDefinitionUnraveler.java @@ -0,0 +1,31 @@ +/* + * 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.lance.file.v2.reader; + +import java.util.Optional; + +public interface RepetitionDefinitionUnraveler +{ + record BlockPositions(Optional nulls, int[] offsets) {} + + BlockPositions calculateOffsets(); + + Optional calculateNulls(); + + boolean isAllValid(); + + void skipValidity(); + + long maxLists(); +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/RepetitionIndex.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/RepetitionIndex.java new file mode 100644 index 000000000000..ee1b857aa490 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/RepetitionIndex.java @@ -0,0 +1,60 @@ +/* + * 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.lance.file.v2.reader; + +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; + +public class RepetitionIndex +{ + private RepetitionIndex() {} + + public static List from(Slice slice, int depth) + { + checkArgument(depth > 0, "depth must be positive"); + boolean hasPreamble = false; + long offset = 0; + ImmutableList.Builder builder = ImmutableList.builder(); + int stride = (depth + 1) * Long.BYTES; + for (int i = 0; i < slice.length() / stride; i++) { + long endCount = slice.getLong(i * stride); + long partialCount = slice.getLong(i * stride + Long.BYTES); + boolean hasTrailer = partialCount > 0; + long startCount = endCount + (hasTrailer ? 1 : 0) - (hasPreamble ? 1 : 0); + builder.add(new RepIndexBlock(offset, startCount, hasPreamble, hasTrailer)); + hasPreamble = hasTrailer; + offset += startCount; + } + return builder.build(); + } + + public static List defaultIndex(List chunks) + { + long offset = 0; + ImmutableList.Builder builder = ImmutableList.builder(); + for (ChunkMetadata chunk : chunks) { + long count = chunk.numValues(); + builder.add(new RepIndexBlock(offset, count, false, false)); + offset += count; + } + return builder.build(); + } + + // startCount is number of rows starts in this block(i.e., include trailer but not preamble) + public record RepIndexBlock(long firstRow, long startCount, boolean hasPreamble, boolean hasTrailer) {} +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ShortArrayBufferAdapter.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ShortArrayBufferAdapter.java new file mode 100644 index 000000000000..89fca60f2d04 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/ShortArrayBufferAdapter.java @@ -0,0 +1,55 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.spi.block.Block; +import io.trino.spi.block.ShortArrayBlock; + +import java.util.Optional; + +public class ShortArrayBufferAdapter + extends ArrayBufferAdapter +{ + public static final ShortArrayBufferAdapter SHORT_ARRAY_BUFFER_ADAPTER = new ShortArrayBufferAdapter(); + + @Override + public short[] createBuffer(int size) + { + return new short[size]; + } + + @Override + public void copy(short[] source, int sourceOffset, short[] destination, int destinationOffset, int length) + { + System.arraycopy(source, sourceOffset, destination, destinationOffset, length); + } + + @Override + public Block createBlock(short[] buffer, Optional valueIsNull) + { + return new ShortArrayBlock(buffer.length, valueIsNull, buffer); + } + + @Override + public long getRetainedBytes(short[] buffer) + { + return (long) buffer.length * Short.BYTES; + } + + @Override + protected int getLength(short[] buffer) + { + return buffer.length; + } +} diff --git a/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/StructColumnReader.java b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/StructColumnReader.java new file mode 100644 index 000000000000..d5727ec5d3c2 --- /dev/null +++ b/lib/trino-lance-file/src/main/java/io/trino/lance/file/v2/reader/StructColumnReader.java @@ -0,0 +1,67 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.lance.file.LanceDataSource; +import io.trino.lance.file.v2.metadata.ColumnMetadata; +import io.trino.lance.file.v2.metadata.Field; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.spi.block.Block; +import io.trino.spi.block.RowBlock; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static com.google.common.collect.ImmutableList.toImmutableList; + +public class StructColumnReader + implements ColumnReader +{ + private final ColumnReader[] childColumnReaders; + private int nextBatchSize; + + public StructColumnReader(LanceDataSource dataSource, Field field, Map columnMetadata, List ranges, AggregatedMemoryContext memoryContext) + { + ColumnReader[] childReaders = new ColumnReader[field.children().size()]; + for (int i = 0; i < childReaders.length; i++) { + childReaders[i] = ColumnReader.createColumnReader(dataSource, field.children().get(i), columnMetadata, ranges, memoryContext); + } + this.childColumnReaders = childReaders; + } + + @Override + public void prepareNextRead(int batchSize) + { + for (ColumnReader childColumnReader : childColumnReaders) { + childColumnReader.prepareNextRead(batchSize); + } + nextBatchSize = batchSize; + } + + @Override + public DecodedPage read() + { + List decodedPages = Arrays.stream(childColumnReaders).map(ColumnReader::read).collect(toImmutableList()); + + Block[] fieldBlocks = decodedPages.stream().map(DecodedPage::block).toArray(Block[]::new); + // repetition/definition levels should be identical across all children + RepetitionDefinitionUnraveler unraveler = decodedPages.getFirst().unraveler(); + if (unraveler.isAllValid()) { + unraveler.skipValidity(); + return new DecodedPage(RowBlock.fromFieldBlocks(nextBatchSize, fieldBlocks), unraveler); + } + return new DecodedPage(RowBlock.fromNotNullSuppressedFieldBlocks(nextBatchSize, unraveler.calculateNulls(), fieldBlocks), unraveler); + } +} diff --git a/lib/trino-lance-file/src/test/java/io/trino/lance/file/BenchmarkColumnReaders.java b/lib/trino-lance-file/src/test/java/io/trino/lance/file/BenchmarkColumnReaders.java new file mode 100644 index 000000000000..f97a39a63528 --- /dev/null +++ b/lib/trino-lance-file/src/test/java/io/trino/lance/file/BenchmarkColumnReaders.java @@ -0,0 +1,323 @@ +/* + * 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.lance.file; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.block.Block; +import io.trino.spi.connector.SourcePage; +import io.trino.spi.type.Type; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OperationsPerInvocation; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +import static com.google.common.collect.Lists.newArrayList; +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static io.trino.jmh.Benchmarks.benchmark; +import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.nio.file.Files.createTempDirectory; +import static java.util.Collections.nCopies; +import static java.util.UUID.randomUUID; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.openjdk.jmh.annotations.Scope.Thread; + +@State(Thread) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@Fork(3) +@Warmup(iterations = 30, time = 500, timeUnit = MILLISECONDS) +@Measurement(iterations = 20, time = 500, timeUnit = MILLISECONDS) +@BenchmarkMode(Mode.AverageTime) +@OperationsPerInvocation(BenchmarkColumnReaders.ROWS) +public class BenchmarkColumnReaders +{ + public static final int ROWS = 10_000_000; + + @Benchmark + public Object readBigIntJNI(BigIntBenchmarkData data) + throws Exception + { + try (LanceJNIReader reader = data.createJNIReader()) { + return readColumnJNI(reader); + } + } + + @Benchmark + public Object readBigInt(BigIntBenchmarkData data) + throws Exception + { + try (LanceReader reader = data.createReader()) { + return readColumn(reader); + } + } + + @Benchmark + public Object readVarcharJNI(VarcharBenchmarkData data) + throws Exception + { + try (LanceJNIReader reader = data.createJNIReader()) { + return readColumnJNI(reader); + } + } + + @Benchmark + public Object readVarchar(VarcharBenchmarkData data) + throws Exception + { + try (LanceReader reader = data.createReader()) { + return readColumn(reader); + } + } + + @Benchmark + public Object readListJNI(ListBenchmarkData data) + throws Exception + { + try (LanceJNIReader reader = data.createJNIReader()) { + return readColumnJNI(reader); + } + } + + @Benchmark + public Object readList(ListBenchmarkData data) + throws Exception + { + try (LanceReader reader = data.createReader()) { + return readColumn(reader); + } + } + + @Benchmark + public Object readStructJNI(StructBenchmarkData data) + throws Exception + { + try (LanceJNIReader reader = data.createJNIReader()) { + return readColumnJNI(reader); + } + } + + @Benchmark + public Object readStruct(StructBenchmarkData data) + throws Exception + { + try (LanceReader reader = data.createReader()) { + return readColumn(reader); + } + } + + private Object readColumn(LanceReader reader) + throws IOException + { + List blocks = new ArrayList<>(); + for (SourcePage page = reader.nextSourcePage(); page != null; page = reader.nextSourcePage()) { + blocks.add(page.getBlock(0)); + } + return blocks; + } + + private Object readColumnJNI(LanceJNIReader reader) + throws IOException + { + List blocks = new ArrayList<>(); + for (SourcePage page = reader.nextSourcePage(); page != null; page = reader.nextSourcePage()) { + blocks.add(page.getBlock(0)); + } + return blocks; + } + + @State(Thread) + public static class ListBenchmarkData + extends BenchmarkData + { + public static final Type LIST_TYPE = LanceTester.arrayType(BIGINT); + public static final int MAX_LIST_SIZE = 32; + + @Setup + public void setup() + throws Exception + { + setup(LIST_TYPE, createValues(), true); + } + + private Iterator createValues() + { + List> values = new ArrayList<>(); + for (int i = 0; i < ROWS; i++) { + if (i % 9 == 0) { + values.add(null); + } + else { + List row = new ArrayList<>(); + for (int j = 0; j < i % MAX_LIST_SIZE; j++) { + if (j % 7 == 0) { + row.add(null); + } + else { + row.add(ThreadLocalRandom.current().nextLong()); + } + } + values.add(row); + } + } + return values.iterator(); + } + } + + @State(Thread) + public static class StructBenchmarkData + extends BenchmarkData + { + public static final int NUM_FILEDS = 3; + public static final Type STRUCT_TYPE = LanceTester.rowType(BIGINT, BIGINT, BIGINT); + + @Setup + public void setup() + throws Exception + { + setup(STRUCT_TYPE, createValues(), true); + } + + private Iterator createValues() + { + List> values = new ArrayList<>(); + for (int i = 0; i < ROWS; i++) { + if (i % 9 == 0) { + values.add(null); + } + else { + values.add(nCopies(NUM_FILEDS, (long) i)); + } + } + return values.iterator(); + } + } + + @State(Thread) + public static class BigIntBenchmarkData + extends BenchmarkData + { + @Setup + public void setup() + throws Exception + { + setup(BIGINT, createValues(), true); + } + + private Iterator createValues() + { + List values = new ArrayList<>(); + for (int i = 0; i < ROWS; ++i) { + long value = ThreadLocalRandom.current().nextLong(); + if (value % 7 == 0) { + values.add(null); + } + else { + values.add(value); + } + } + return values.iterator(); + } + } + + @State(Thread) + public static class VarcharBenchmarkData + extends BenchmarkData + { + @Setup + public void setup() + throws Exception + { + setup(VARCHAR, createValues(), true); + } + + private Iterator createValues() + { + List values = new ArrayList<>(); + for (int i = 0; i < ROWS; ++i) { + long value = ThreadLocalRandom.current().nextLong(); + if (value % 7 == 0) { + values.add(null); + } + else { + values.add(Long.toString(value)); + } + } + return values.iterator(); + } + } + + public abstract static class BenchmarkData + { + protected final Random random = new Random(0); + private Type type; + private Path temporaryDirectory; + private File lanceFile; + private LanceDataSource dataSource; + + public void setup(Type type, Iterator values, boolean nullable) + throws Exception + { + this.type = type; + temporaryDirectory = createTempDirectory(null); + lanceFile = temporaryDirectory.resolve(randomUUID().toString()).toFile(); + LanceTester.writeLanceColumnJNI(lanceFile, type, newArrayList(values), nullable); + dataSource = new FileLanceDataSource(lanceFile); + } + + @TearDown + public void tearDown() + throws IOException + { + deleteRecursively(temporaryDirectory, ALLOW_INSECURE); + } + + LanceReader createReader() + throws IOException + { + return new LanceReader(dataSource, ImmutableList.of(0), Optional.empty(), newSimpleAggregatedMemoryContext()); + } + + public LanceJNIReader createJNIReader() + throws IOException + { + return new LanceJNIReader(lanceFile, ImmutableList.of(type.getDisplayName()), ImmutableList.of(type)); + } + } + + public static void main(String[] args) + throws Exception + { + benchmark(BenchmarkColumnReaders.class).run(); + } +} diff --git a/lib/trino-lance-file/src/test/java/io/trino/lance/file/LanceJNIReader.java b/lib/trino-lance-file/src/test/java/io/trino/lance/file/LanceJNIReader.java new file mode 100644 index 000000000000..55ad66ae858b --- /dev/null +++ b/lib/trino-lance-file/src/test/java/io/trino/lance/file/LanceJNIReader.java @@ -0,0 +1,216 @@ +/* + * 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.lance.file; + +import com.lancedb.lance.file.LanceFileReader; +import io.airlift.slice.Slice; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.TrinoException; +import io.trino.spi.block.ArrayBlockBuilder; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.RowBlockBuilder; +import io.trino.spi.connector.SourcePage; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarbinaryType; +import io.trino.spi.type.VarcharType; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BigIntVector; +import org.apache.arrow.vector.BitVector; +import org.apache.arrow.vector.DateDayVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.Float8Vector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.ipc.ArrowReader; +import org.apache.arrow.vector.util.TransferPair; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.function.Consumer; + +import static io.airlift.slice.Slices.wrappedBuffer; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.TimeType.TIME_MICROS; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_MICROSECOND; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static org.apache.arrow.vector.complex.BaseRepeatedValueVector.OFFSET_WIDTH; + +public class LanceJNIReader + implements Closeable +{ + public static final int READ_BATCH_SIZE = 8192; + private final List columnTypes; + private final BufferAllocator allocator; + private final ArrowReader arrowReader; + private final PageBuilder pageBuilder; + + public LanceJNIReader(File file, List columnNames, List columnTypes) + throws IOException + { + this.columnTypes = requireNonNull(columnTypes, "columnTypes is null"); + this.allocator = new RootAllocator(); + LanceFileReader lanceReader = LanceFileReader.open(file.getPath(), allocator); + this.arrowReader = lanceReader.readAll(columnNames, null, READ_BATCH_SIZE); + this.pageBuilder = new PageBuilder(columnTypes); + } + + public SourcePage nextSourcePage() + throws IOException + { + if (!arrowReader.loadNextBatch()) { + return null; + } + VectorSchemaRoot batch = arrowReader.getVectorSchemaRoot(); + convert(batch); + Page page = pageBuilder.build(); + pageBuilder.reset(); + return SourcePage.create(page); + } + + public void convert(VectorSchemaRoot vectorSchemaRoot) + { + pageBuilder.declarePositions(vectorSchemaRoot.getRowCount()); + List fieldVectors = vectorSchemaRoot.getFieldVectors(); + + for (int column = 0; column < columnTypes.size(); column++) { + convertType(pageBuilder.getBlockBuilder(column), columnTypes.get(column), fieldVectors.get(column), 0, fieldVectors.get(column).getValueCount()); + } + vectorSchemaRoot.clear(); + } + + private void convertType(BlockBuilder output, Type type, FieldVector vector, int offset, int length) + { + Class javaType = type.getJavaType(); + try { + if (javaType == boolean.class) { + writeVectorValues(output, vector, index -> type.writeBoolean(output, ((BitVector) vector).get(index) == 1), offset, length); + } + else if (javaType == long.class) { + if (type.equals(BIGINT)) { + writeVectorValues(output, vector, index -> type.writeLong(output, ((BigIntVector) vector).get(index)), offset, length); + } + else if (type.equals(INTEGER)) { + writeVectorValues(output, vector, index -> type.writeLong(output, ((IntVector) vector).get(index)), offset, length); + } + else if (type.equals(DATE)) { + writeVectorValues(output, vector, index -> type.writeLong(output, ((DateDayVector) vector).get(index)), offset, length); + } + else if (type.equals(TIME_MICROS)) { + writeVectorValues(output, vector, index -> type.writeLong(output, ((TimeMicroVector) vector).get(index) * PICOSECONDS_PER_MICROSECOND), offset, length); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, format("Unhandled type for %s: %s", javaType.getSimpleName(), type)); + } + } + else if (javaType == double.class) { + writeVectorValues(output, vector, index -> type.writeDouble(output, ((Float8Vector) vector).get(index)), offset, length); + } + else if (javaType == Slice.class) { + writeVectorValues(output, vector, index -> writeSlice(output, type, vector, index), offset, length); + } + else if (type instanceof ArrayType arrayType) { + writeVectorValues(output, vector, index -> writeArrayBlock(output, arrayType, vector, index), offset, length); + } + else if (type instanceof RowType rowType) { + writeVectorValues(output, vector, index -> writeRowBlock(output, rowType, vector, index), offset, length); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, format("Unhandled type for %s: %s", javaType.getSimpleName(), type)); + } + } + catch (ClassCastException ex) { + throw new TrinoException(GENERIC_INTERNAL_ERROR, format("Unhandled type for %s: %s", javaType.getSimpleName(), type), ex); + } + } + + private static void writeSlice(BlockBuilder output, Type type, FieldVector vector, int index) + { + if (type instanceof VarcharType) { + byte[] slice = ((VarCharVector) vector).get(index); + type.writeSlice(output, wrappedBuffer(slice)); + } + else if (type instanceof VarbinaryType) { + byte[] slice = ((VarBinaryVector) vector).get(index); + type.writeSlice(output, wrappedBuffer(slice)); + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, "Unhandled type for Slice: " + type.getTypeSignature()); + } + } + + private static void writeVectorValues(BlockBuilder output, FieldVector vector, Consumer consumer, int offset, int length) + { + for (int i = offset; i < offset + length; i++) { + if (vector.isNull(i)) { + output.appendNull(); + } + else { + consumer.accept(i); + } + } + } + + private void writeRowBlock(BlockBuilder output, RowType rowType, FieldVector vector, int index) + { + List fields = rowType.getFields(); + ((RowBlockBuilder) output).buildEntry(fieldBuilders -> { + for (int i = 0; i < fields.size(); i++) { + RowType.Field field = fields.get(i); + FieldVector innerVector = ((StructVector) vector).getChild(field.getName().orElse("field" + i)); + convertType(fieldBuilders.get(i), field.getType(), innerVector, index, 1); + } + }); + } + + private void writeArrayBlock(BlockBuilder output, ArrayType arrayType, FieldVector vector, int index) + { + Type elementType = arrayType.getElementType(); + ((ArrayBlockBuilder) output).buildEntry(elementBuilder -> { + ArrowBuf offsetBuffer = vector.getOffsetBuffer(); + + int start = offsetBuffer.getInt((long) index * OFFSET_WIDTH); + int end = offsetBuffer.getInt((long) (index + 1) * OFFSET_WIDTH); + + FieldVector innerVector = ((ListVector) vector).getDataVector(); + + TransferPair transferPair = innerVector.getTransferPair(allocator); + transferPair.splitAndTransfer(start, end - start); + try (FieldVector sliced = (FieldVector) transferPair.getTo()) { + convertType(elementBuilder, elementType, sliced, 0, sliced.getValueCount()); + } + }); + } + + @Override + public void close() + throws IOException + { + } +} diff --git a/lib/trino-lance-file/src/test/java/io/trino/lance/file/LanceTester.java b/lib/trino-lance-file/src/test/java/io/trino/lance/file/LanceTester.java new file mode 100644 index 000000000000..2bdd2ad4c37c --- /dev/null +++ b/lib/trino-lance-file/src/test/java/io/trino/lance/file/LanceTester.java @@ -0,0 +1,545 @@ +/* + * 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.lance.file; + +import com.google.common.collect.AbstractIterator; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.lancedb.lance.WriteParams; +import com.lancedb.lance.file.LanceFileWriter; +import io.trino.spi.block.Block; +import io.trino.spi.connector.SourcePage; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.DoubleType; +import io.trino.spi.type.IntegerType; +import io.trino.spi.type.NamedTypeSignature; +import io.trino.spi.type.RealType; +import io.trino.spi.type.RowFieldName; +import io.trino.spi.type.RowType; +import io.trino.spi.type.SmallintType; +import io.trino.spi.type.StandardTypes; +import io.trino.spi.type.TinyintType; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeSignatureParameter; +import io.trino.spi.type.VarbinaryType; +import io.trino.spi.type.VarcharType; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BigIntVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.Float4Vector; +import org.apache.arrow.vector.Float8Vector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.SmallIntVector; +import org.apache.arrow.vector.TinyIntVector; +import org.apache.arrow.vector.VarBinaryVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.complex.impl.UnionListWriter; +import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Random; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.Lists.newArrayList; +import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; +import static java.util.Collections.nCopies; +import static java.util.stream.Collectors.toList; +import static org.assertj.core.api.Assertions.assertThat; + +public class LanceTester +{ + public static final int MAX_LIST_SIZE = 256; + + private static final Random random = new Random(); + + private static void assertFileContentsTrino(Type type, TempFile tempFile, List expectedValues) + throws IOException + { + try (LanceReader lanceReader = createLanceReader(tempFile)) { + Iterator iterator = expectedValues.iterator(); + + int rowsProcessed = 0; + for (SourcePage page = lanceReader.nextSourcePage(); page != null; page = lanceReader.nextSourcePage()) { + int batchSize = page.getPositionCount(); + Block block = page.getBlock(0); + List data = new ArrayList<>(block.getPositionCount()); + for (int position = 0; position < block.getPositionCount(); position++) { + data.add(type.getObjectValue(block, position)); + } + for (int i = 0; i < batchSize; i++) { + assertThat(iterator.hasNext()).isTrue(); + Object expected = iterator.next(); + Object actual = data.get(i); + assertColumnValueEquals(type, actual, expected); + } + rowsProcessed += batchSize; + } + assertThat(iterator.hasNext()).isFalse(); + assertThat(lanceReader.nextSourcePage()).isNull(); + assertThat(rowsProcessed).isEqualTo(expectedValues.size()); + } + } + + private static void assertColumnValueEquals(Type type, Object actual, Object expected) + { + if (expected == null) { + assertThat(actual).isNull(); + return; + } + + if (type instanceof RowType) { + List fieldTypes = type.getTypeParameters(); + List actualRow = (List) actual; + List expectedRow = (List) expected; + assertThat(actualRow).hasSize(fieldTypes.size()); + assertThat(actualRow).hasSize(expectedRow.size()); + for (int fieldId = 0; fieldId < actualRow.size(); fieldId++) { + Type fieldType = fieldTypes.get(fieldId); + Object actualElement = actualRow.get(fieldId); + Object expectedElement = expectedRow.get(fieldId); + assertColumnValueEquals(fieldType, actualElement, expectedElement); + } + } + else if (!Objects.equals(actual, expected)) { + assertThat(actual).isEqualTo(expected); + } + } + + private static LanceReader createLanceReader(TempFile tempFile) + throws IOException + { + LanceDataSource dataSource = new FileLanceDataSource(tempFile.getFile()); + return new LanceReader(dataSource, ImmutableList.of(0), Optional.empty(), newSimpleAggregatedMemoryContext()); + } + + public static void writeLanceColumnJNI(File outputFile, Type type, List values, boolean nullable) + throws Exception + { + BufferAllocator allocator = new RootAllocator(); + LanceFileWriter writer = LanceFileWriter.open(outputFile.getPath(), allocator, null, Optional.of(WriteParams.LanceFileVersion.V2_1), ImmutableMap.of()); + String columnName = type.getDisplayName(); + Field field = toArrowField(columnName, type, nullable); + Schema schema = new Schema(ImmutableList.of(field), null); + List> data = values.stream().map(Arrays::asList).collect(toImmutableList()); + VectorSchemaRoot root = writeVectorSchemaRoot(schema, data, allocator); + writer.write(root); + writer.close(); + } + + private static ArrowType toArrowPrimitiveType(Type type) + { + return switch (type) { + case TinyintType _ -> new ArrowType.Int(8, true); + case SmallintType _ -> new ArrowType.Int(16, true); + case IntegerType _ -> new ArrowType.Int(32, true); + case BigintType _ -> new ArrowType.Int(64, true); + case RealType _ -> new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE); + case DoubleType _ -> new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE); + case VarcharType _ -> new ArrowType.Utf8(); + case VarbinaryType _ -> new ArrowType.Binary(); + default -> throw new UnsupportedOperationException(); + }; + } + + public static Field toArrowField(String name, Type type, boolean nullable) + { + return switch (type) { + case TinyintType _, SmallintType _, IntegerType _, BigintType _, RealType _, DoubleType _, VarcharType _, VarbinaryType _ -> + new Field(name, nullable ? FieldType.nullable(toArrowPrimitiveType(type)) : FieldType.notNullable(toArrowPrimitiveType(type)), ImmutableList.of()); + case RowType row -> { + List childFields = row.getFields().stream().map(child -> toArrowField(child.getName().orElse(""), child.getType(), nullable)).collect(toImmutableList()); + yield new Field(name, nullable ? FieldType.nullable(ArrowType.Struct.INSTANCE) : FieldType.notNullable(ArrowType.Struct.INSTANCE), childFields); + } + case ArrayType array -> { + Field element = toArrowField("element", array.getElementType(), true); + yield new Field(name, nullable ? FieldType.nullable(ArrowType.List.INSTANCE) : FieldType.notNullable(ArrowType.List.INSTANCE), ImmutableList.of(element)); + } + default -> throw new UnsupportedOperationException(); + }; + } + + private static List insertNullEvery(int n, List iterable) + { + return newArrayList(() -> new AbstractIterator() + { + private final Iterator delegate = iterable.iterator(); + private int position; + + @Override + protected T computeNext() + { + position++; + if (position > n) { + position = 0; + return null; + } + + if (!delegate.hasNext()) { + return endOfData(); + } + + return delegate.next(); + } + }); + } + + public void testRoundTrip(Type type, List readValues) + throws Exception + { + testRoundTripType(type, true, insertNullEvery(5, readValues)); + testRoundTripType(type, false, readValues); + testSimpleStructRoundTrip(type, readValues); + testSimpleListRoundTrip(type, readValues); + } + + public void testRoundTripType(Type type, boolean nullable, List readValues) + throws Exception + { + // For non-nullable tests, filter out null values + List filteredValues = nullable ? readValues : readValues.stream().filter(Objects::nonNull).collect(toList()); + assertRoundTrip(type, type, filteredValues, filteredValues, nullable); + } + + private void testSimpleListRoundTrip(Type type, List values) + throws Exception + { + Type arrayType = arrayType(type); + List data = values.stream().filter(Objects::nonNull).map(value -> insertNullEvery(9, nCopies(random.nextInt(MAX_LIST_SIZE), value))).collect(toImmutableList()); + testRoundTripType(arrayType, false, data); + testRoundTripType(arrayType, true, insertNullEvery(7, data)); + } + + public void testLongListRoundTrip(Type type, List values) + throws Exception + { + Type arrayType = arrayType(type); + List data = values.stream().filter(Objects::nonNull).map(value -> insertNullEvery(9, nCopies(random.nextInt(2048, 10000), value))).collect(toImmutableList()); + testRoundTripType(arrayType, false, data); + testRoundTripType(arrayType, true, insertNullEvery(7, data)); + } + + private void testSimpleStructRoundTrip(Type type, List values) + throws Exception + { + Type rowType = rowType(type, type, type); + testRoundTripType(rowType, false, values.stream().map(value -> List.of(value, value, value)).collect(toList())); + testRoundTripType(rowType, true, insertNullEvery(7, values.stream().map(value -> List.of(value, value, value)).collect(toList()))); + } + + private void assertRoundTrip(Type writeType, Type readType, List writeValues, List readValues, boolean nullable) + throws Exception + { + // write w/ JNI writer, read w/ LanceReader + try (TempFile file = new TempFile()) { + writeLanceColumnJNI(file.getFile(), writeType, writeValues, nullable); + assertFileContentsTrino(readType, file, readValues); + } + } + + public static Type arrayType(Type elementType) + { + return TESTING_TYPE_MANAGER.getParameterizedType(StandardTypes.ARRAY, ImmutableList.of(TypeSignatureParameter.typeParameter(elementType.getTypeSignature()))); + } + + public static Type rowType(Type... fieldTypes) + { + ImmutableList.Builder typeSignatureParameters = ImmutableList.builder(); + for (int i = 0; i < fieldTypes.length; i++) { + String fieldName = "field_" + i; + Type fieldType = fieldTypes[i]; + typeSignatureParameters.add(TypeSignatureParameter.namedTypeParameter(new NamedTypeSignature(Optional.of(new RowFieldName(fieldName)), fieldType.getTypeSignature()))); + } + return TESTING_TYPE_MANAGER.getParameterizedType(StandardTypes.ROW, typeSignatureParameters.build()); + } + + public static VectorSchemaRoot writeVectorSchemaRoot(Schema schema, List> data, BufferAllocator allocator) + { + VectorSchemaRoot root = VectorSchemaRoot.create(schema, allocator); + root.allocateNew(); + + int rowCount = data.size(); + List fields = schema.getFields(); + + for (int fieldIndex = 0; fieldIndex < fields.size(); fieldIndex++) { + Field field = fields.get(fieldIndex); + FieldVector vector = root.getVector(field.getName()); + + List columnData = new ArrayList<>(); + for (List row : data) { + columnData.add(row.get(fieldIndex)); + } + + writeFieldVector(vector, field, columnData); + } + + root.setRowCount(rowCount); + return root; + } + + private static void writeFieldVector(FieldVector vector, Field field, List data) + { + ArrowType arrowType = field.getType(); + + switch (arrowType) { + case ArrowType.Int intType -> writeIntVector(vector, data, intType); + case ArrowType.FloatingPoint floatingPointType -> writeFloatingPointVector(vector, data, floatingPointType); + case ArrowType.Utf8 _ -> writeStringVector((VarCharVector) vector, data); + case ArrowType.Binary _ -> writeBinaryVector((VarBinaryVector) vector, data); + case ArrowType.Struct _ -> writeStructVector((StructVector) vector, field, data); + case ArrowType.List _ -> writeListVector((ListVector) vector, field, data); + case null, default -> throw new UnsupportedOperationException("Unsupported Arrow type: " + arrowType); + } + + vector.setValueCount(data.size()); + } + + private static void writeIntVector(FieldVector vector, List data, ArrowType.Int intType) + { + int bitWidth = intType.getBitWidth(); + + for (int i = 0; i < data.size(); i++) { + Object value = data.get(i); + if (value == null) { + vector.setNull(i); + } + else { + Number num = (Number) value; + switch (bitWidth) { + case 8: + ((TinyIntVector) vector).setSafe(i, num.byteValue()); + break; + case 16: + ((SmallIntVector) vector).setSafe(i, num.shortValue()); + break; + case 32: + ((IntVector) vector).setSafe(i, num.intValue()); + break; + case 64: + ((BigIntVector) vector).setSafe(i, num.longValue()); + break; + default: + throw new UnsupportedOperationException("Unsupported bit width: " + bitWidth); + } + } + } + } + + private static void writeFloatingPointVector(FieldVector vector, List data, ArrowType.FloatingPoint floatType) + { + FloatingPointPrecision precision = floatType.getPrecision(); + + for (int i = 0; i < data.size(); i++) { + Object value = data.get(i); + if (value == null) { + vector.setNull(i); + } + else { + Number num = (Number) value; + switch (precision) { + case SINGLE: + ((Float4Vector) vector).setSafe(i, num.floatValue()); + break; + case DOUBLE: + ((Float8Vector) vector).setSafe(i, num.doubleValue()); + break; + default: + throw new UnsupportedOperationException("Unsupported precision: " + precision); + } + } + } + } + + private static void writeStringVector(VarCharVector vector, List data) + { + for (int i = 0; i < data.size(); i++) { + Object value = data.get(i); + if (value == null) { + vector.setNull(i); + } + else { + String str = value.toString(); + vector.setSafe(i, str.getBytes(StandardCharsets.UTF_8)); + } + } + } + + private static void writeBinaryVector(VarBinaryVector vector, List data) + { + for (int i = 0; i < data.size(); i++) { + Object value = data.get(i); + if (value == null) { + vector.setNull(i); + } + else { + byte[] bytes = (byte[]) value; + vector.setSafe(i, bytes); + } + } + } + + private static void writeStructVector(StructVector vector, Field field, List data) + { + List childFields = field.getChildren(); + + for (int i = 0; i < data.size(); i++) { + Object value = data.get(i); + if (value == null) { + vector.setNull(i); + } + else { + List structData = (List) value; + vector.setIndexDefined(i); + + // Populate child vectors + for (int childIndex = 0; childIndex < childFields.size(); childIndex++) { + Field childField = childFields.get(childIndex); + FieldVector childVector = vector.getChild(childField.getName()); + Object childValue = structData.get(childIndex); + + if (childValue == null) { + childVector.setNull(i); + } + else { + // For primitive types in struct, we need to handle them individually + writeStructChildVector(childVector, childField, i, childValue); + } + } + } + } + + // Set value count for child vectors + for (Field childField : childFields) { + FieldVector childVector = vector.getChild(childField.getName()); + childVector.setValueCount(data.size()); + } + } + + private static void writeStructChildVector(FieldVector childVector, Field childField, int index, Object value) + { + ArrowType arrowType = childField.getType(); + + switch (arrowType) { + case ArrowType.Int intType -> { + Number num = (Number) value; + switch (intType.getBitWidth()) { + case 8: + ((TinyIntVector) childVector).setSafe(index, num.byteValue()); + break; + case 16: + ((SmallIntVector) childVector).setSafe(index, num.shortValue()); + break; + case 32: + ((IntVector) childVector).setSafe(index, num.intValue()); + break; + case 64: + ((BigIntVector) childVector).setSafe(index, num.longValue()); + break; + default: + throw new UnsupportedOperationException("Unsupported bit width: " + intType.getBitWidth()); + } + } + case ArrowType.FloatingPoint floatType -> { + Number num = (Number) value; + switch (floatType.getPrecision()) { + case SINGLE: + ((Float4Vector) childVector).setSafe(index, num.floatValue()); + break; + case DOUBLE: + ((Float8Vector) childVector).setSafe(index, num.doubleValue()); + break; + default: + throw new UnsupportedOperationException("Unsupported precision: " + floatType.getPrecision()); + } + } + case ArrowType.Utf8 _ -> { + String str = value.toString(); + ((VarCharVector) childVector).setSafe(index, str.getBytes(StandardCharsets.UTF_8)); + } + case ArrowType.Binary _ -> { + byte[] bytes = (byte[]) value; + ((VarBinaryVector) childVector).setSafe(index, bytes); + } + case null, default -> throw new UnsupportedOperationException("Unsupported ArrowType " + arrowType + " in struct field"); + } + } + + public static void writeListVector(ListVector vector, Field field, List data) + { + checkArgument(field.getChildren().size() == 1, "List field must have a single child"); + Field elementField = field.getChildren().getFirst(); + vector.setInitialCapacity(data.size()); + UnionListWriter writer = vector.getWriter(); + for (int i = 0; i < data.size(); i++) { + Object list = data.get(i); + writer.setPosition(i); + if (list == null) { + writer.writeNull(); + continue; + } + writer.startList(); + for (Object value : (List) list) { + if (value == null) { + if (!elementField.isNullable()) { + throw new IllegalArgumentException("Cannot write null value to non-nullable array element field: " + elementField.getName()); + } + writer.writeNull(); + } + else { + switch (elementField.getType()) { + case ArrowType.Int intType -> { + switch (intType.getBitWidth()) { + case 8 -> writer.writeTinyInt((byte) value); + case 16 -> writer.writeSmallInt((short) value); + case 32 -> writer.writeInt((int) value); + case 64 -> writer.writeBigInt((long) value); + } + } + case ArrowType.FloatingPoint floatType -> { + switch (floatType.getPrecision()) { + case SINGLE -> writer.writeFloat4((float) value); + case DOUBLE -> writer.writeFloat8((double) value); + default -> throw new UnsupportedOperationException("Unsupported floatType " + floatType.getPrecision()); + } + } + case ArrowType.Utf8 _ -> writer.writeVarChar((String) value); + case ArrowType.Binary _ -> writer.writeVarBinary((byte[]) value); + default -> throw new IllegalStateException("Unexpected value: " + elementField.getFieldType()); + } + } + } + writer.endList(); + } + writer.setValueCount(data.size()); + vector.setValueCount(data.size()); + } +} diff --git a/lib/trino-lance-file/src/test/java/io/trino/lance/file/TempFile.java b/lib/trino-lance-file/src/test/java/io/trino/lance/file/TempFile.java new file mode 100644 index 000000000000..591f7640fda6 --- /dev/null +++ b/lib/trino-lance-file/src/test/java/io/trino/lance/file/TempFile.java @@ -0,0 +1,49 @@ +/* + * 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.lance.file; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; + +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static java.nio.file.Files.createTempDirectory; + +public class TempFile + implements Closeable +{ + private final Path tempDir; + private final File file; + + public TempFile() + throws IOException + { + tempDir = createTempDirectory(null); + file = tempDir.resolve("data.lance").toFile(); + } + + public File getFile() + { + return file; + } + + @Override + public void close() + throws IOException + { + deleteRecursively(tempDir, ALLOW_INSECURE); + } +} diff --git a/lib/trino-lance-file/src/test/java/io/trino/lance/file/TestLanceReader.java b/lib/trino-lance-file/src/test/java/io/trino/lance/file/TestLanceReader.java new file mode 100644 index 000000000000..acf6372eac90 --- /dev/null +++ b/lib/trino-lance-file/src/test/java/io/trino/lance/file/TestLanceReader.java @@ -0,0 +1,156 @@ +/* + * 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.lance.file; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.collect.Iterables.concat; +import static com.google.common.collect.Iterables.cycle; +import static com.google.common.collect.Iterables.limit; +import static com.google.common.collect.Lists.newArrayList; +import static io.trino.spi.type.BigintType.BIGINT; +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.SmallintType.SMALLINT; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.util.Collections.nCopies; +import static java.util.stream.Collectors.toList; + +final class TestLanceReader +{ + private final LanceTester tester = new LanceTester(); + + private static List doubleSequence(double start, double step, int items) + { + List values = new ArrayList<>(items); + double nextValue = start; + for (int i = 0; i < items; i++) { + values.add(nextValue); + nextValue += step; + } + return values; + } + + private static List floatSequence(float start, float step, int items) + { + ImmutableList.Builder values = ImmutableList.builder(); + float nextValue = start; + for (int i = 0; i < items; i++) { + values.add(nextValue); + nextValue += step; + } + return values.build(); + } + + @Test + void testSmallNumericShortSequence() + throws Exception + { + testRoundTripNumeric(limit(cycle(ImmutableList.of(1, 3, 5, 7, 11, 13, 17, 19, 23, 27)), 7)); + } + + @Test + void testSmallNumericLongSequence() + throws Exception + { + testRoundTripNumeric(limit(cycle(ImmutableList.of(1, 3, 5, 7, 11, 13, 17, 19, 23, 27)), 30_000)); + } + + @Test + void testLargeNumeric() + throws Exception + { + testRoundTripNumeric(limit(cycle(ImmutableList.of(Long.MAX_VALUE, Long.MAX_VALUE - 1, Long.MAX_VALUE - 2)), 30_000)); + } + + @Test + void testDoubleSequence() + throws Exception + { + tester.testRoundTrip(DOUBLE, doubleSequence(0, 0.1, 30_000)); + } + + @Test + void testFloatSequence() + throws Exception + { + tester.testRoundTrip(REAL, floatSequence(0.0f, 0.1f, 30_000)); + } + + @Test + void testStringLargeDictionary() + throws Exception + { + tester.testRoundTrip(VARCHAR, newArrayList(limit(cycle(doubleSequence(1.0, 0.001, 257)), 30_000)).stream().map(Object::toString).collect(toList())); + } + + @Test + void testStringSequence() + throws Exception + { + tester.testRoundTrip(VARCHAR, newArrayList(doubleSequence(1.0, 10.01, 30_000)).stream().map(Object::toString).collect(toList())); + } + + @Test + void testStringDictionarySequence() + throws Exception + { + tester.testRoundTrip(VARCHAR, newArrayList(limit(cycle(ImmutableList.of(1, 3, 5, 7, 11, 13, 17)), 30_000)).stream().map(Object::toString).collect(toList())); + } + + @Test + void testStringStrideDictionary() + throws Exception + { + tester.testRoundTrip(VARCHAR, newArrayList(concat(ImmutableList.of("a"), nCopies(9999, "123"), ImmutableList.of("b"), nCopies(9999, "123")))); + } + + @Test + void testStringConstant() + throws Exception + { + tester.testRoundTrip(VARCHAR, newArrayList(nCopies(99999, "123"))); + } + + @Test + void testLongList() + throws Exception + { + // test preamble only chunks + tester.testLongListRoundTrip(BIGINT, newArrayList(limit(cycle(ImmutableList.of(1, 3, 5, 7, 11, 13, 17)), 10_000)).stream().map(Number::longValue).collect(toList())); + } + + private void testRoundTripNumeric(Iterable values) + throws Exception + { + List writeValues = ImmutableList.copyOf(values).stream().map(Number::longValue).collect(toList()); + + tester.testRoundTrip(TINYINT, writeValues.stream().map(Long::byteValue) + .collect(toList())); + + tester.testRoundTrip(SMALLINT, writeValues.stream().map(Long::shortValue) + .collect(toList())); + + tester.testRoundTrip(INTEGER, writeValues.stream().map(Long::intValue) + .collect(toList())); + + tester.testRoundTrip(BIGINT, writeValues); + } +} diff --git a/lib/trino-lance-file/src/test/java/io/trino/lance/file/v2/reader/TestMiniblockMapRange.java b/lib/trino-lance-file/src/test/java/io/trino/lance/file/v2/reader/TestMiniblockMapRange.java new file mode 100644 index 000000000000..20f0366a281b --- /dev/null +++ b/lib/trino-lance-file/src/test/java/io/trino/lance/file/v2/reader/TestMiniblockMapRange.java @@ -0,0 +1,204 @@ +/* + * 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.lance.file.v2.reader; + +import io.trino.lance.file.v2.reader.MiniBlockPageReader.PreambleAction; +import io.trino.lance.file.v2.reader.MiniBlockPageReader.SelectedRanges; +import org.junit.jupiter.api.Test; + +import java.util.function.BiFunction; +import java.util.function.Function; + +import static io.trino.lance.file.v2.reader.MiniBlockPageReader.mapRange; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestMiniblockMapRange +{ + @Test + void testMiddleNull() + { + // NULL in the middle + // [[A, B, C], [D, E], NULL, [F, G, H]] + int[] repetitions = new int[] {1, 0, 0, 1, 0, 1, 1, 0, 0}; + int[] definitions = new int[] {0, 0, 0, 0, 0, 1, 0, 0, 0}; + int maxVisibleDef = 0; + int maxRepetitionLevel = 1; + int totalItems = 8; + Function calculate = range -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, PreambleAction.ABSENT); + + assertThat(calculate.apply(new Range(0, 1))).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 3))); + assertThat(calculate.apply(new Range(1, 2))).isEqualTo(new SelectedRanges(new Range(3, 5), new Range(3, 5))); + assertThat(calculate.apply(new Range(2, 3))).isEqualTo(new SelectedRanges(new Range(5, 5), new Range(5, 6))); + assertThat(calculate.apply(new Range(3, 4))).isEqualTo(new SelectedRanges(new Range(5, 8), new Range(6, 9))); + assertThat(calculate.apply(new Range(3, 4))).isEqualTo(new SelectedRanges(new Range(5, 8), new Range(6, 9))); + assertThat(calculate.apply(new Range(0, 2))).isEqualTo(new SelectedRanges(new Range(0, 5), new Range(0, 5))); + assertThat(calculate.apply(new Range(0, 2))).isEqualTo(new SelectedRanges(new Range(0, 5), new Range(0, 5))); + assertThat(calculate.apply(new Range(1, 3))).isEqualTo(new SelectedRanges(new Range(3, 5), new Range(3, 6))); + assertThat(calculate.apply(new Range(2, 4))).isEqualTo(new SelectedRanges(new Range(5, 8), new Range(5, 9))); + assertThat(calculate.apply(new Range(0, 3))).isEqualTo(new SelectedRanges(new Range(0, 5), new Range(0, 6))); + assertThat(calculate.apply(new Range(1, 4))).isEqualTo(new SelectedRanges(new Range(3, 8), new Range(3, 9))); + assertThat(calculate.apply(new Range(0, 4))).isEqualTo(new SelectedRanges(new Range(0, 8), new Range(0, 9))); + } + + @Test + void testLeadingNull() + { + // NULL at the begining + // [NULL, [A, B], [C]] + int[] repetitions = new int[] {1, 1, 0, 1}; + int[] definitions = new int[] {1, 0, 0, 0}; + int maxVisibleDef = 0; + int maxRepetitionLevel = 1; + int totalItems = 3; + Function calculate = range -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, PreambleAction.ABSENT); + + assertThat(calculate.apply(new Range(0, 1))).isEqualTo(new SelectedRanges(new Range(0, 0), new Range(0, 1))); + assertThat(calculate.apply(new Range(1, 2))).isEqualTo(new SelectedRanges(new Range(0, 2), new Range(1, 3))); + assertThat(calculate.apply(new Range(2, 3))).isEqualTo(new SelectedRanges(new Range(2, 3), new Range(3, 4))); + assertThat(calculate.apply(new Range(0, 2))).isEqualTo(new SelectedRanges(new Range(0, 2), new Range(0, 3))); + assertThat(calculate.apply(new Range(1, 3))).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(1, 4))); + assertThat(calculate.apply(new Range(0, 3))).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 4))); + } + + @Test + void testTrailingNull() + { + // Null at end + // [[A], [B, C], NULL] + int[] repetitions = new int[] {1, 1, 0, 1}; + int[] definitions = new int[] {0, 0, 0, 1}; + int maxVisibleDef = 0; + int maxRepetitionLevel = 1; + int totalItems = 3; + Function calculate = range -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, PreambleAction.ABSENT); + + assertThat(calculate.apply(new Range(1, 2))).isEqualTo(new SelectedRanges(new Range(1, 3), new Range(1, 3))); + assertThat(calculate.apply(new Range(2, 3))).isEqualTo(new SelectedRanges(new Range(3, 3), new Range(3, 4))); + assertThat(calculate.apply(new Range(0, 2))).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 3))); + assertThat(calculate.apply(new Range(1, 3))).isEqualTo(new SelectedRanges(new Range(1, 3), new Range(1, 4))); + assertThat(calculate.apply(new Range(0, 3))).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 4))); + } + + @Test + void testNoNulls() + { + // No nulls, with repetition + // [[A, B], [C, D], [E, F]] + int[] repetitions = new int[] {1, 0, 1, 0, 1, 0}; + int[] definitions = null; + int maxVisibleDef = 0; + int maxRepetitionLevel = 1; + int totalItems = 6; + Function calculate = range -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, PreambleAction.ABSENT); + + assertThat(calculate.apply(new Range(0, 1))).isEqualTo(new SelectedRanges(new Range(0, 2), new Range(0, 2))); + assertThat(calculate.apply(new Range(1, 2))).isEqualTo(new SelectedRanges(new Range(2, 4), new Range(2, 4))); + assertThat(calculate.apply(new Range(2, 3))).isEqualTo(new SelectedRanges(new Range(4, 6), new Range(4, 6))); + assertThat(calculate.apply(new Range(0, 2))).isEqualTo(new SelectedRanges(new Range(0, 4), new Range(0, 4))); + assertThat(calculate.apply(new Range(1, 3))).isEqualTo(new SelectedRanges(new Range(2, 6), new Range(2, 6))); + assertThat(calculate.apply(new Range(0, 3))).isEqualTo(new SelectedRanges(new Range(0, 6), new Range(0, 6))); + } + + @Test + void testNoRepetitions() + { + // No repetition, with nulls + // [A, B, NULL, C] + int[] repetitions = null; + int[] definitions = new int[] {0, 0, 1, 0}; + int maxVisibleDef = 1; + int maxRepetitionLevel = 1; + int totalItems = 4; + Function calculate = range -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, PreambleAction.ABSENT); + + assertThat(calculate.apply(new Range(0, 1))).isEqualTo(new SelectedRanges(new Range(0, 1), new Range(0, 1))); + assertThat(calculate.apply(new Range(1, 2))).isEqualTo(new SelectedRanges(new Range(1, 2), new Range(1, 2))); + assertThat(calculate.apply(new Range(2, 3))).isEqualTo(new SelectedRanges(new Range(2, 3), new Range(2, 3))); + assertThat(calculate.apply(new Range(0, 2))).isEqualTo(new SelectedRanges(new Range(0, 2), new Range(0, 2))); + assertThat(calculate.apply(new Range(1, 3))).isEqualTo(new SelectedRanges(new Range(1, 3), new Range(1, 3))); + assertThat(calculate.apply(new Range(0, 3))).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 3))); + } + + @Test + void testTrailingNullWithPreamble() + { + // [[..., A] [B, C], NULL] + int[] repetitions = new int[] {0, 1, 0, 1}; + int[] definitions = new int[] {0, 0, 0, 1}; + int maxVisibleDef = 0; + int maxRepetitionLevel = 1; + int totalItems = 3; + BiFunction calculate = (range, preambleAction) -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, preambleAction); + + assertThat(calculate.apply(new Range(0, 1), PreambleAction.TAKE)).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 3))); + assertThat(calculate.apply(new Range(0, 2), PreambleAction.TAKE)).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 4))); + assertThat(calculate.apply(new Range(1, 2), PreambleAction.SKIP)).isEqualTo(new SelectedRanges(new Range(3, 3), new Range(3, 4))); + assertThat(calculate.apply(new Range(0, 2), PreambleAction.SKIP)).isEqualTo(new SelectedRanges(new Range(1, 3), new Range(1, 4))); + } + + @Test + void testPreambleWithMiddleNull() + { + // [[..., A], NULL, [D, E]] + int[] repetitions = new int[] {0, 1, 1, 0}; + int[] definitions = new int[] {0, 1, 0, 0}; + int maxVisibleDef = 0; + int maxRepetitionLevel = 1; + int totalItems = 4; + BiFunction calculate = (range, preambleAction) -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, preambleAction); + + assertThat(calculate.apply(new Range(0, 1), PreambleAction.TAKE)).isEqualTo(new SelectedRanges(new Range(0, 1), new Range(0, 2))); + assertThat(calculate.apply(new Range(0, 2), PreambleAction.TAKE)).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 4))); + assertThat(calculate.apply(new Range(0, 1), PreambleAction.SKIP)).isEqualTo(new SelectedRanges(new Range(1, 1), new Range(1, 2))); + assertThat(calculate.apply(new Range(1, 2), PreambleAction.SKIP)).isEqualTo(new SelectedRanges(new Range(1, 3), new Range(2, 4))); + assertThat(calculate.apply(new Range(0, 2), PreambleAction.SKIP)).isEqualTo(new SelectedRanges(new Range(1, 3), new Range(1, 4))); + } + + @Test + void testPreambleWithoutDefinition() + { + // [[..., A] [B, C], [D]] + int[] repetitions = new int[] {0, 1, 0, 1}; + int[] definitions = null; + int maxVisibleDef = 0; + int maxRepetitionLevel = 1; + int totalItems = 4; + BiFunction calculate = (range, preambleAction) -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, preambleAction); + + assertThat(calculate.apply(new Range(0, 1), PreambleAction.TAKE)).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 3))); + assertThat(calculate.apply(new Range(0, 2), PreambleAction.TAKE)).isEqualTo(new SelectedRanges(new Range(0, 4), new Range(0, 4))); + assertThat(calculate.apply(new Range(0, 1), PreambleAction.SKIP)).isEqualTo(new SelectedRanges(new Range(1, 3), new Range(1, 3))); + assertThat(calculate.apply(new Range(1, 2), PreambleAction.SKIP)).isEqualTo(new SelectedRanges(new Range(3, 4), new Range(3, 4))); + assertThat(calculate.apply(new Range(0, 2), PreambleAction.SKIP)).isEqualTo(new SelectedRanges(new Range(1, 4), new Range(1, 4))); + } + + @Test + void testEmptyList() + { + // [[] [A], [B, C]] + int[] repetitions = new int[] {1, 1, 1, 0}; + int[] definitions = new int[] {1, 0, 0, 0}; + int maxVisibleDef = 0; + int maxRepetitionLevel = 1; + int totalItems = 3; + Function calculate = (range) -> mapRange(range, repetitions, definitions, maxRepetitionLevel, maxVisibleDef, totalItems, PreambleAction.ABSENT); + + assertThat(calculate.apply(new Range(0, 1))).isEqualTo(new SelectedRanges(new Range(0, 0), new Range(0, 1))); + assertThat(calculate.apply(new Range(1, 2))).isEqualTo(new SelectedRanges(new Range(0, 1), new Range(1, 2))); + assertThat(calculate.apply(new Range(2, 3))).isEqualTo(new SelectedRanges(new Range(1, 3), new Range(2, 4))); + assertThat(calculate.apply(new Range(0, 2))).isEqualTo(new SelectedRanges(new Range(0, 1), new Range(0, 2))); + assertThat(calculate.apply(new Range(1, 3))).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(1, 4))); + assertThat(calculate.apply(new Range(0, 3))).isEqualTo(new SelectedRanges(new Range(0, 3), new Range(0, 4))); + } +} diff --git a/lib/trino-lance-file/src/test/java/io/trino/lance/file/v2/reader/TestRepDefUnravler.java b/lib/trino-lance-file/src/test/java/io/trino/lance/file/v2/reader/TestRepDefUnravler.java new file mode 100644 index 000000000000..de5b36631160 --- /dev/null +++ b/lib/trino-lance-file/src/test/java/io/trino/lance/file/v2/reader/TestRepDefUnravler.java @@ -0,0 +1,211 @@ +/* + * 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.lance.file.v2.reader; + +import com.google.common.collect.ImmutableList; +import io.trino.lance.file.v2.metadata.RepDefLayer; +import io.trino.lance.file.v2.reader.RepetitionDefinitionUnraveler.BlockPositions; +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static io.trino.lance.file.v2.metadata.RepDefLayer.ALL_VALID_ITEM; +import static io.trino.lance.file.v2.metadata.RepDefLayer.ALL_VALID_LIST; +import static io.trino.lance.file.v2.metadata.RepDefLayer.EMPTYABLE_LIST; +import static io.trino.lance.file.v2.metadata.RepDefLayer.NULLABLE_AND_EMPTYABLE_LIST; +import static io.trino.lance.file.v2.metadata.RepDefLayer.NULLABLE_ITEM; +import static io.trino.lance.file.v2.metadata.RepDefLayer.NULLABLE_LIST; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestRepDefUnravler +{ + @Test + void testBasicRepDef() + { + // [[I], [I, I]], NULL, [[NULL, NULL], NULL, [NULL, I, I, NULL]] + BaseUnraveler unraveler = new BaseUnraveler(new int[] {2, 1, 0, 2, 2, 0, 1, 1, 0, 0, 0}, + new int[] {0, 0, 0, 3, 1, 1, 2, 1, 0, 0, 1}, + new RepDefLayer[] {NULLABLE_ITEM, NULLABLE_LIST, NULLABLE_LIST}); + + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isPresent()).isTrue(); + assertThat(isNull.get()).isEqualTo(new boolean[] {false, false, false, true, true, true, false, false, true}); + + BlockPositions innerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, false, false, true, false}), new int[] {0, 1, 3, 5, 5, 9}), innerPositions); + BlockPositions outerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, true, false}), new int[] {0, 2, 2, 5}), outerPositions); + } + + @Test + void testEmptyListNoNull() + { + BaseUnraveler unraveler = new BaseUnraveler(new int[] {1, 0, 0, 0, 1, 1, 1, 0}, new int[] {0, 0, 0, 0, 1, 1, 0, 0}, new RepDefLayer[] {ALL_VALID_ITEM, + EMPTYABLE_LIST}); + + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isEmpty()).isTrue(); + BlockPositions positions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, false, false, false}), new int[] {0, 4, 4, 4, 6}), positions); + } + + @Test + void testNullList() + { + // nullable list + BaseUnraveler unraveler = new BaseUnraveler(new int[] {1, 0, 1, 1, 0, 0}, new int[] {0, 0, 2, 0, 1, 0}, new RepDefLayer[] {NULLABLE_ITEM, NULLABLE_LIST}); + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isPresent()).isTrue(); + assertThat(isNull.get()).isEqualTo(new boolean[] {false, false, false, true, false}); + BlockPositions positions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, true, false}), new int[] {0, 2, 2, 5}), positions); + } + + @Test + void testEmptyableList() + { + // emptyable list + BaseUnraveler unraveler = new BaseUnraveler(new int[] {1, 0, 1, 1, 0, 0}, new int[] {0, 0, 2, 0, 1, 0}, new RepDefLayer[] {NULLABLE_ITEM, EMPTYABLE_LIST}); + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isPresent()).isTrue(); + assertThat(isNull.get()).isEqualTo(new boolean[] {false, false, false, true, false}); + BlockPositions positions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, false, false}), new int[] {0, 2, 2, 5}), positions); + } + + @Test + void testEmptyListAtEnd() + { + // last item is an empty list + BaseUnraveler unraveler = new BaseUnraveler(new int[] {1, 0, 1, 0, 0, 1}, new int[] {0, 0, 0, 1, 0, 2}, new RepDefLayer[] {NULLABLE_ITEM, EMPTYABLE_LIST}); + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isPresent()).isTrue(); + assertThat(isNull.get()).isEqualTo(new boolean[] {false, false, false, true, false}); + BlockPositions positions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, false, false}), new int[] {0, 2, 5, 5}), positions); + } + + @Test + void testAllValid() + { + BaseUnraveler unraveler = new BaseUnraveler(new int[] {2, 1, 0, 2, 0, 2, 0, 1, 0}, + new int[0], + new RepDefLayer[] {ALL_VALID_ITEM, ALL_VALID_LIST, ALL_VALID_LIST}); + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isEmpty()).isTrue(); + BlockPositions innerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.empty(), new int[] {0, 1, 3, 5, 7, 9}), innerPositions); + BlockPositions outerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.empty(), new int[] {0, 2, 3, 5}), outerPositions); + } + + @Test + void testOnlyEmptyLists() + { + BaseUnraveler unraveler = new BaseUnraveler(new int[] {1, 0, 0, 0, 1, 1, 1, 0}, + new int[] {0, 0, 0, 0, 1, 1, 0, 0}, + new RepDefLayer[] {ALL_VALID_ITEM, EMPTYABLE_LIST}); + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isEmpty()).isTrue(); + BlockPositions innerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, false, false, false}), new int[] {0, 4, 4, 4, 6}), innerPositions); + } + + @Test + void testOnlyNullLists() + { + BaseUnraveler unraveler = new BaseUnraveler(new int[] {1, 0, 0, 0, 1, 1, 1, 0}, + new int[] {0, 0, 0, 0, 1, 1, 0, 0}, + new RepDefLayer[] {ALL_VALID_ITEM, NULLABLE_LIST}); + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isEmpty()).isTrue(); + BlockPositions innerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, true, true, false}), new int[] {0, 4, 4, 4, 6}), innerPositions); + } + + @Test + void testNullAndEmptyLists() + { + BaseUnraveler unraveler = new BaseUnraveler(new int[] {1, 0, 0, 0, 1, 1, 1, 0}, + new int[] {0, 0, 0, 0, 1, 2, 0, 0}, + new RepDefLayer[] {ALL_VALID_ITEM, NULLABLE_AND_EMPTYABLE_LIST}); + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isEmpty()).isTrue(); + BlockPositions innerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, true, false, false}), new int[] {0, 4, 4, 4, 6}), innerPositions); + } + + @Test + void testNoRep() + { + BaseUnraveler unraveler = new BaseUnraveler(new int[0], new int[] {2, 2, 0, 0, 1}, new RepDefLayer[] {NULLABLE_ITEM, NULLABLE_ITEM, ALL_VALID_ITEM}); + Optional innerNulls = unraveler.calculateNulls(); + assertThat(innerNulls.isPresent()).isTrue(); + assertThat(innerNulls.get()).isEqualTo(new boolean[] {true, true, false, false, true}); + Optional middleNulls = unraveler.calculateNulls(); + assertThat(middleNulls.isPresent()).isTrue(); + assertThat(middleNulls.get()).isEqualTo(new boolean[] {true, true, false, false, false}); + Optional outerNulls = unraveler.calculateNulls(); + assertThat(outerNulls.isEmpty()).isTrue(); + } + + @Test + void testNullsInStruct() + { + BaseUnraveler unraveler = new BaseUnraveler(new int[] {1, 1, 1}, new int[] {1, 2, 1}, new RepDefLayer[] {ALL_VALID_ITEM, NULLABLE_LIST, NULLABLE_ITEM}); + Optional innerNulls = unraveler.calculateNulls(); + assertThat(innerNulls.isEmpty()).isTrue(); + BlockPositions positions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {true, true, true}), new int[] {0, 0, 0, 0}), positions); + Optional outerNulls = unraveler.calculateNulls(); + assertThat(outerNulls.isPresent()).isTrue(); + assertThat(outerNulls.get()).isEqualTo(new boolean[] {false, true, false}); + } + + @Test + void testListEndsWithNull() + { + BaseUnraveler unraveler = new BaseUnraveler(new int[] {2, 2, 2}, new int[] {0, 1, 2}, new RepDefLayer[] {ALL_VALID_ITEM, NULLABLE_LIST, NULLABLE_LIST}); + Optional innerNulls = unraveler.calculateNulls(); + assertThat(innerNulls.isEmpty()).isTrue(); + BlockPositions innerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, true}), new int[] {0, 1, 1}), innerPositions); + BlockPositions outerPositions = unraveler.calculateOffsets(); + assertBlockPositionsEqual(new BlockPositions(Optional.of(new boolean[] {false, false, true}), new int[] {0, 1, 2, 2}), outerPositions); + } + + @Test + void testCompositeUnravel() + { + CompositeUnraveler unraveler = new CompositeUnraveler(ImmutableList.of( + new BaseUnraveler(new int[] {1, 0, 1, 1, 0, 0}, new int[] {0, 0, 1, 0, 0, 0}, new RepDefLayer[] {ALL_VALID_ITEM, NULLABLE_LIST}), + new BaseUnraveler(new int[] {1, 1, 0, 1, 0, 1, 0, 1, 0}, new int[0], new RepDefLayer[] {ALL_VALID_ITEM, ALL_VALID_LIST}))); + + Optional isNull = unraveler.calculateNulls(); + assertThat(isNull.isEmpty()).isTrue(); + BlockPositions positions = unraveler.calculateOffsets(); + assertThat(positions.offsets()).isEqualTo(new int[] {0, 2, 2, 5, 6, 8, 10, 12, 14}); + assertThat(positions.nulls().isPresent()).isTrue(); + assertThat(positions.nulls().get()).isEqualTo(new boolean[] {false, true, false, false, false, false, false, false}); + } + + private static void assertBlockPositionsEqual(BlockPositions expected, BlockPositions actual) + { + assertThat(actual.nulls().isPresent()).isEqualTo(expected.nulls().isPresent()); + if (expected.nulls().isPresent()) { + assertThat(actual.nulls().get()).isEqualTo(expected.nulls().get()); + } + assertThat(actual.offsets()).isEqualTo(expected.offsets()); + } +} diff --git a/plugin/trino-lance/pom.xml b/plugin/trino-lance/pom.xml new file mode 100644 index 000000000000..fbcfc78f3fa7 --- /dev/null +++ b/plugin/trino-lance/pom.xml @@ -0,0 +1,249 @@ + + + 4.0.0 + + + io.trino + trino-root + 478-SNAPSHOT + ../../pom.xml + + + trino-lance + trino-plugin + Trino - Lance table connector + + + true + ${air.test.jvm.additional-arguments.default} + --add-opens=java.base/java.nio=ALL-UNNAMED + --sun-misc-unsafe-memory-access=allow + + + + + build.buf.gen + lancedb_lance_protocolbuffers_java + + + + com.google.guava + guava + + + + com.google.inject + guice + classes + + + + com.google.protobuf + protobuf-java + + + + io.airlift + bootstrap + + + + io.airlift + configuration + + + + io.trino + trino-filesystem + + + + io.trino + trino-filesystem-manager + + + + io.trino + trino-lance-file + + + + io.trino + trino-memory-context + + + + io.trino + trino-plugin-toolkit + + + + jakarta.validation + jakarta.validation-api + + + + org.weakref + jmxutils + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + io.airlift + slice + provided + + + + io.opentelemetry + opentelemetry-api + provided + + + + io.opentelemetry + opentelemetry-api-incubator + provided + + + + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi + provided + + + + com.google.errorprone + error_prone_annotations + runtime + + + + io.airlift + log + runtime + + + + io.airlift + log-manager + runtime + + + + io.airlift + units + runtime + + + + com.lancedb + lance-core + test + + + + io.airlift + junit-extensions + test + + + + io.airlift + testing + test + + + + io.trino + trino-client + test + + + + io.trino + trino-main + test + + + + io.trino + trino-main + test-jar + test + + + + io.trino + trino-testing + test + + + + io.trino + trino-tpch + test + + + + io.trino.tpch + tpch + test + + + + org.apache.arrow + arrow-c-data + test + + + + org.apache.arrow + arrow-memory-core + test + + + + org.apache.arrow + arrow-vector + test + + + + org.assertj + assertj-core + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + + + + true + + + false + + buf + Buf Maven Repository + https://buf.build/gen/maven + + + diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceColumnHandle.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceColumnHandle.java new file mode 100644 index 000000000000..add6d5b4d20a --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceColumnHandle.java @@ -0,0 +1,29 @@ +/* + * 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.lance; + +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.type.Type; + +import static java.util.Objects.requireNonNull; + +public record LanceColumnHandle(int id, String name, Type type) + implements ColumnHandle +{ + public LanceColumnHandle + { + requireNonNull(name, "name is null"); + requireNonNull(type, "type is null"); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceConnector.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceConnector.java new file mode 100644 index 000000000000..aabfb23c4e53 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceConnector.java @@ -0,0 +1,75 @@ +/* + * 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.lance; + +import com.google.inject.Inject; +import io.airlift.bootstrap.LifeCycleManager; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.transaction.IsolationLevel; + +import static io.trino.plugin.lance.LanceTransactionHandle.INSTANCE; +import static java.util.Objects.requireNonNull; + +public class LanceConnector + implements Connector +{ + private final LifeCycleManager lifeCycleManager; + private final LanceMetadata metadata; + private final LanceSplitManager splitManager; + private final LancePageSourceProvider pageSourceProvider; + + @Inject + public LanceConnector(LifeCycleManager lifeCycleManager, LanceMetadata metadata, LanceSplitManager splitManager, LancePageSourceProvider pageSourceProvider) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) + { + return INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorSession session, ConnectorTransactionHandle transactionHandle) + { + return metadata; + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return pageSourceProvider; + } + + @Override + public final void shutdown() + { + lifeCycleManager.stop(); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceConnectorFactory.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceConnectorFactory.java new file mode 100644 index 000000000000..f051fc5a3120 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceConnectorFactory.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.lance; + +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.trace.Tracer; +import io.trino.filesystem.manager.FileSystemModule; +import io.trino.spi.NodeManager; +import io.trino.spi.catalog.CatalogName; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.spi.type.TypeManager; + +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public class LanceConnectorFactory + implements ConnectorFactory +{ + public static final String NAME = "lance"; + + @Override + public String getName() + { + return NAME; + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + requireNonNull(config, "config is null"); + + Bootstrap app = new Bootstrap( + binder -> binder.bind(TypeManager.class).toInstance(context.getTypeManager()), + binder -> binder.bind(NodeManager.class).toInstance(context.getNodeManager()), + new FileSystemModule(catalogName, context.getCurrentNode().isCoordinator(), context.getOpenTelemetry(), true), + binder -> { + binder.bind(OpenTelemetry.class).toInstance(context.getOpenTelemetry()); + binder.bind(Tracer.class).toInstance(context.getTracer()); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); + }, + new LanceModule()); + + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(LanceConnector.class); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceErrorCode.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceErrorCode.java new file mode 100644 index 000000000000..f9073ae43fae --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceErrorCode.java @@ -0,0 +1,44 @@ +/* + * 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.lance; + +import io.trino.spi.ErrorCode; +import io.trino.spi.ErrorCodeSupplier; +import io.trino.spi.ErrorType; + +import static io.trino.spi.ErrorType.EXTERNAL; +import static io.trino.spi.ErrorType.USER_ERROR; + +public enum LanceErrorCode + implements ErrorCodeSupplier +{ + LANCE_TABLE_NOT_FOUND(1, EXTERNAL), + LANCE_INVALID_METADATA(2, EXTERNAL), + LANCE_BAD_DATA(3, EXTERNAL), + LANCE_SPLIT_ERROR(4, EXTERNAL), + LANCE_INVALID_VERSION_NUMBER(11, USER_ERROR); + + private final ErrorCode errorCode; + + LanceErrorCode(int code, ErrorType errorType) + { + errorCode = new ErrorCode(code + 0x0514_0000, name(), errorType, errorType == USER_ERROR); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceMetadata.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceMetadata.java new file mode 100644 index 000000000000..79eb657cc4f1 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceMetadata.java @@ -0,0 +1,161 @@ +/* + * 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.lance; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.trino.lance.file.v2.metadata.Field; +import io.trino.plugin.lance.catalog.BaseTable; +import io.trino.plugin.lance.catalog.TrinoCatalog; +import io.trino.plugin.lance.metadata.Manifest; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.ConnectorTableVersion; +import io.trino.spi.connector.RelationColumnsMetadata; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SchemaTablePrefix; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.UnaryOperator; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.type.BigintType.BIGINT; +import static java.util.Objects.requireNonNull; + +public class LanceMetadata + implements ConnectorMetadata +{ + private final TrinoCatalog catalog; + + @Inject + public LanceMetadata(TrinoCatalog catalog) + { + this.catalog = requireNonNull(catalog, "catalog is null"); + } + + private static long getSnapshotIdFromVersion(ConnectorTableVersion version) + { + io.trino.spi.type.Type versionType = version.getVersionType(); + return switch (version.getPointerType()) { + // TODO: list and search versions to do temporal time travel + case TEMPORAL -> throw new TrinoException(NOT_SUPPORTED, "This connector does not support versioned tables with temporal version"); + case TARGET_ID -> { + if (versionType != BIGINT) { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support versioned tables: unsupported type for table version " + versionType.getDisplayName()); + } + // TODO: support String type target id + yield (long) version.getVersion(); + } + }; + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return catalog.listNamespaces(session); + } + + @Override + public List listTables(ConnectorSession session, Optional schemaName) + { + return catalog.listTables(session, schemaName); + } + + @Override + public Iterator streamRelationColumns(ConnectorSession session, Optional schemaName, UnaryOperator> relationFilter) + { + Map relationColumns = new HashMap<>(); + + SchemaTablePrefix prefix = schemaName.map(SchemaTablePrefix::new) + .orElseGet(SchemaTablePrefix::new); + + requireNonNull(prefix, "prefix is null"); + List schemaTableNames; + if (prefix.getTable().isEmpty()) { + schemaTableNames = catalog.listTables(session, prefix.getSchema()); + } + else { + schemaTableNames = ImmutableList.of(prefix.toSchemaTableName()); + } + + for (SchemaTableName tableName : schemaTableNames) { + Optional table = catalog.loadTable(session, tableName); + if (table.isPresent()) { + Manifest manifest = table.get().loadManifest(Optional.empty()); + List columns = manifest.fields().stream().map(field -> new ColumnMetadata(field.name(), field.toTrinoType())).collect(toImmutableList()); + relationColumns.put(tableName, RelationColumnsMetadata.forTable(tableName, columns)); + } + } + + return relationFilter.apply(relationColumns.keySet()).stream() + .map(relationColumns::get) + .iterator(); + } + + @Override + public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName, Optional startVersion, Optional endVersion) + { + if (startVersion.isPresent()) { + throw new TrinoException(NOT_SUPPORTED, "Read table with startRowPosition version is not supported"); + } + Optional loadedTable = catalog.loadTable(session, tableName); + if (loadedTable.isEmpty()) { + return null; + } + BaseTable baseTable = loadedTable.get(); + Optional version; + if (endVersion.isPresent()) { + version = Optional.of(getSnapshotIdFromVersion(endVersion.get())); + } + else { + version = Optional.empty(); + } + return new LanceTableHandle(tableName, baseTable.loadManifest(version), baseTable.getTableLocation().toString()); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) + { + LanceTableHandle table = (LanceTableHandle) tableHandle; + + List columns = table.manifest().fields().stream().map(field -> new ColumnMetadata(field.name(), field.toTrinoType())).collect(toImmutableList()); + return new ConnectorTableMetadata(table.name(), columns); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + LanceTableHandle table = (LanceTableHandle) tableHandle; + return table.manifest().fields().stream().collect(toImmutableMap(Field::name, field -> new LanceColumnHandle(field.id(), field.name(), field.toTrinoType()))); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + LanceColumnHandle column = (LanceColumnHandle) columnHandle; + return new ColumnMetadata(column.name(), column.type()); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceModule.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceModule.java new file mode 100644 index 000000000000..264be1e0fc62 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceModule.java @@ -0,0 +1,40 @@ +/* + * 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.lance; + +import com.google.inject.Binder; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.base.metrics.FileFormatDataSourceStats; +import io.trino.plugin.lance.catalog.LanceNamespaceModule; + +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class LanceModule + extends AbstractConfigurationAwareModule +{ + @Override + public void setup(Binder binder) + { + binder.bind(LanceConnector.class).in(Scopes.SINGLETON); + binder.bind(LanceMetadata.class).in(Scopes.SINGLETON); + binder.bind(LanceSplitManager.class).in(Scopes.SINGLETON); + binder.bind(LancePageSourceProvider.class).in(Scopes.SINGLETON); + + binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); + newExporter(binder).export(FileFormatDataSourceStats.class).withGeneratedName(); + + install(new LanceNamespaceModule()); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePageSource.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePageSource.java new file mode 100644 index 000000000000..c7188b782bfc --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePageSource.java @@ -0,0 +1,95 @@ +/* + * 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.lance; + +import io.trino.lance.file.LanceDataSource; +import io.trino.lance.file.LanceReader; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.SourcePage; + +import java.io.IOException; +import java.io.UncheckedIOException; + +import static java.util.Objects.requireNonNull; + +public class LancePageSource + implements ConnectorPageSource +{ + private final LanceReader reader; + private final LanceDataSource dataSource; + private final AggregatedMemoryContext memoryContext; + + private boolean closed; + + public LancePageSource( + LanceReader reader, + LanceDataSource dataSource, + AggregatedMemoryContext memoryContext) + { + this.reader = requireNonNull(reader, "reader is null"); + this.dataSource = requireNonNull(dataSource, "dataSource is null"); + this.memoryContext = requireNonNull(memoryContext, "memoryContext is null"); + } + + @Override + public long getCompletedBytes() + { + return dataSource.getReadBytes(); + } + + @Override + public long getReadTimeNanos() + { + return dataSource.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return closed; + } + + @Override + public SourcePage getNextSourcePage() + { + SourcePage page = reader.nextSourcePage(); + if (closed || page == null) { + close(); + return null; + } + return page; + } + + @Override + public long getMemoryUsage() + { + return memoryContext.getBytes(); + } + + @Override + public void close() + { + if (closed) { + return; + } + closed = true; + try { + reader.close(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePageSourceProvider.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePageSourceProvider.java new file mode 100644 index 000000000000..8f388a360e62 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePageSourceProvider.java @@ -0,0 +1,124 @@ +/* + * 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.lance; + +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +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.lance.file.LanceDataSource; +import io.trino.lance.file.LanceReader; +import io.trino.lance.file.TrinoLanceDataSource; +import io.trino.lance.file.v2.reader.Range; +import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.plugin.base.metrics.FileFormatDataSourceStats; +import io.trino.plugin.lance.metadata.Fragment; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.DynamicFilter; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; +import static io.trino.plugin.lance.LanceErrorCode.LANCE_BAD_DATA; +import static io.trino.plugin.lance.LanceErrorCode.LANCE_SPLIT_ERROR; +import static io.trino.plugin.lance.catalog.BaseTable.DATA_DIR; +import static io.trino.plugin.lance.catalog.BaseTable.LANCE_SUFFIX; +import static java.util.Objects.requireNonNull; + +public class LancePageSourceProvider + implements ConnectorPageSourceProvider +{ + private final TrinoFileSystemFactory fileSystemFactory; + private final FileFormatDataSourceStats stats; + + @Inject + public LancePageSourceProvider( + TrinoFileSystemFactory fileSystemFactory, + FileFormatDataSourceStats stats) + { + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.stats = requireNonNull(stats, "stats is null"); + } + + public ConnectorPageSource createFilePageSource( + ConnectorSession session, + Location path, + List columns, + long start, + long end) + { + if (!path.fileName().endsWith(LANCE_SUFFIX)) { + throw new TrinoException(LANCE_BAD_DATA, "Unsupported file suffix: path=%s".formatted(path.fileName())); + } + LanceDataSource lanceDataSource; + try { + TrinoFileSystem fileSystem = fileSystemFactory.create(session); + TrinoInputFile inputFile = fileSystem.newInputFile(path); + lanceDataSource = new TrinoLanceDataSource(inputFile, stats); + List readColumnIds = columns.stream() + .map(LanceColumnHandle.class::cast) + .map(LanceColumnHandle::id) + .collect(toImmutableList()); + AggregatedMemoryContext memoryUsage = newSimpleAggregatedMemoryContext(); + LanceReader reader = new LanceReader(lanceDataSource, readColumnIds, Optional.of(ImmutableList.of(new Range(start, end))), memoryUsage); + return new LancePageSource(reader, lanceDataSource, memoryUsage); + } + catch (IOException e) { + throw new TrinoException(LANCE_SPLIT_ERROR, e); + } + } + + public ConnectorPageSource createFragmentPageSource(ConnectorSession session, + LanceTableHandle table, + Fragment fragment, + List columns, + long start, + long end) + { + // TODO: support multiple files in a fragment + checkArgument(fragment.files().size() == 1, "only one file per fragment is supported"); + Fragment.DataFile dataFile = fragment.files().getFirst(); + return createFilePageSource(session, Location.of(Joiner.on("/").join(table.tablePath(), DATA_DIR, dataFile.path())), columns, start, end); + } + + @Override + public ConnectorPageSource createPageSource( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit connectorSplit, + ConnectorTableHandle table, + List columns, + DynamicFilter dynamicFilter) + { + checkArgument(table instanceof LanceTableHandle); + checkArgument(connectorSplit instanceof LanceSplit); + LanceTableHandle lanceTable = (LanceTableHandle) table; + LanceSplit split = (LanceSplit) connectorSplit; + return createFragmentPageSource(session, lanceTable, split.fragment(), columns, split.startRowPosition(), split.endRowPosition()); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePlugin.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePlugin.java new file mode 100644 index 000000000000..cd9349a34b08 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LancePlugin.java @@ -0,0 +1,28 @@ +/* + * 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.lance; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; + +public class LancePlugin + implements Plugin +{ + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new LanceConnectorFactory()); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplit.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplit.java new file mode 100644 index 000000000000..1c50de7bf1b3 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplit.java @@ -0,0 +1,22 @@ +/* + * 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.lance; + +import io.trino.plugin.lance.metadata.Fragment; +import io.trino.spi.connector.ConnectorSplit; + +public record LanceSplit(Fragment fragment, long startRowPosition, long endRowPosition) + implements ConnectorSplit +{ +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplitManager.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplitManager.java new file mode 100644 index 000000000000..8e647c7179a1 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplitManager.java @@ -0,0 +1,40 @@ +/* + * 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.lance; + +import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.DynamicFilter; + +public class LanceSplitManager + implements ConnectorSplitManager +{ + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle tableHandle, + DynamicFilter dynamicFilter, + Constraint constraint) + { + LanceTableHandle table = (LanceTableHandle) tableHandle; + LanceSplitSource splitSource = new LanceSplitSource(table.manifest().fragments()); + return new ClassLoaderSafeConnectorSplitSource(splitSource, LanceSplitManager.class.getClassLoader()); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplitSource.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplitSource.java new file mode 100644 index 000000000000..9ff31f5d49c3 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceSplitSource.java @@ -0,0 +1,103 @@ +/* + * 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.lance; + +import io.trino.plugin.lance.metadata.Fragment; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorSplitSource; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; + +public class LanceSplitSource + implements ConnectorSplitSource +{ + public static final long MAX_ROWS_PER_SPLIT = 97L; + + private final SplitIterator splitIterator; + + public LanceSplitSource(List fragments) + { + requireNonNull(fragments, "fragments is null"); + this.splitIterator = new SplitIterator(fragments); + } + + @Override + public CompletableFuture getNextBatch(int maxSize) + { + List splits = new ArrayList<>(maxSize); + while (splits.size() < maxSize && splitIterator.hasNext()) { + splits.add(splitIterator.next()); + } + return completedFuture(new ConnectorSplitBatch(splits, isFinished())); + } + + @Override + public void close() + { + } + + @Override + public boolean isFinished() + { + return !splitIterator.hasNext(); + } + + public static class SplitIterator + implements Iterator + { + private final List fragments; + + private int fragmentIndex; + private long offsetInFragment; + + public SplitIterator(List fragments) + { + this.fragments = requireNonNull(fragments, "fragments is null"); + } + + @Override + public boolean hasNext() + { + return fragmentIndex < fragments.size(); + } + + @Override + public LanceSplit next() + { + if (!hasNext()) { + return null; + } + Fragment fragment = fragments.get(fragmentIndex); + long rowsLeft = fragment.physicalRows() - offsetInFragment; + long length = Math.min(rowsLeft, MAX_ROWS_PER_SPLIT); + + long start = offsetInFragment; + long end = offsetInFragment + length; // exclusive + + offsetInFragment += length; + if (offsetInFragment >= fragment.physicalRows()) { + fragmentIndex++; + offsetInFragment = 0; + } + + return new LanceSplit(fragment, start, end); + } + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceTableHandle.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceTableHandle.java new file mode 100644 index 000000000000..92f08ae18312 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceTableHandle.java @@ -0,0 +1,36 @@ +/* + * 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.lance; + +import io.trino.plugin.lance.metadata.Manifest; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.SchemaTableName; + +import static java.util.Objects.requireNonNull; + +public record LanceTableHandle(SchemaTableName name, Manifest manifest, String tablePath) + implements ConnectorTableHandle +{ + public LanceTableHandle + { + requireNonNull(name, "name is null"); + requireNonNull(manifest, "manifest is null"); + } + + @Override + public String toString() + { + return name.toString(); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceTransactionHandle.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceTransactionHandle.java new file mode 100644 index 000000000000..ef3aaa6ed54e --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/LanceTransactionHandle.java @@ -0,0 +1,22 @@ +/* + * 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.lance; + +import io.trino.spi.connector.ConnectorTransactionHandle; + +public enum LanceTransactionHandle + implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/BaseTable.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/BaseTable.java new file mode 100644 index 000000000000..c36318814eb0 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/BaseTable.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.lance.catalog; + +import io.airlift.slice.Slice; +import io.trino.filesystem.FileEntry; +import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoInputFile; +import io.trino.plugin.lance.metadata.Manifest; +import io.trino.spi.TrinoException; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.math.BigInteger; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static io.trino.plugin.lance.LanceErrorCode.LANCE_INVALID_VERSION_NUMBER; +import static java.lang.Math.toIntExact; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class BaseTable +{ + public static final String VERSIONS_DIR = "_versions"; + public static final String DATA_DIR = "data"; + public static final String MANIFEST_SUFFIX = ".manifest"; + public static final String LANCE_SUFFIX = ".lance"; + public static final String DETACHED_VERSION_PREFIX = "d"; + public static final int MANIFEST_V2_FILE_NAME_LENGTH = 20 + MANIFEST_SUFFIX.length(); + private static final BigInteger U64_MAX = BigInteger.valueOf(2).pow(64).subtract(BigInteger.ONE); + + private final String schema; + private final String name; + private final TrinoFileSystem fileSystem; + private final Location tableLocation; + + public BaseTable(String schema, String name, TrinoFileSystem fileSystem, Location tableLocation) + { + this.schema = requireNonNull(schema, "schema is null"); + this.name = requireNonNull(name, "name is null"); + this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); + this.tableLocation = requireNonNull(tableLocation, "location is null"); + } + + public static long parseManifestVersion(String fileName) + { + checkArgument(fileName.endsWith(MANIFEST_SUFFIX), "manifest file name must ends with .manifest"); + String version = fileName.substring(0, fileName.length() - MANIFEST_SUFFIX.length()); + if (fileName.startsWith(DETACHED_VERSION_PREFIX) || fileName.length() == MANIFEST_V2_FILE_NAME_LENGTH) { + return U64_MAX.subtract(new BigInteger(version)).longValueExact(); + } + return Long.parseLong(version); + } + + public Location getTableLocation() + { + return tableLocation; + } + + public Manifest loadManifest(Optional version) + { + Optional manifestLocation = findManifest(version); + if (manifestLocation.isEmpty()) { + throw new TrinoException(LANCE_INVALID_VERSION_NUMBER, format("Manifest not found for version: %s", version)); + } + TrinoInputFile file = fileSystem.newInputFile(manifestLocation.get()); + Slice slice; + try { + slice = file.newInput().readFully(0, toIntExact(file.length())); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + return Manifest.from(slice); + } + + private Optional findManifest(Optional version) + { + long current = -1; + try { + FileIterator files = fileSystem.listFiles(tableLocation.appendPath(VERSIONS_DIR)); + while (files.hasNext()) { + FileEntry file = files.next(); + String fileName = file.location().fileName(); + checkState( + fileName.endsWith(MANIFEST_SUFFIX), + "Manifest file [%s] does not endRowPosition with .manifest", + file.location().toString()); + long manifestVersion = parseManifestVersion(fileName); + if (version.isPresent() && manifestVersion > version.get()) { + continue; + } + if (manifestVersion > current) { + current = manifestVersion; + } + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + + if (current == -1) { + return Optional.empty(); + } + return Optional.of(tableLocation.appendPath(VERSIONS_DIR).appendPath(current + MANIFEST_SUFFIX)); + } + + public String getSchema() + { + return schema; + } + + public String getName() + { + return name; + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/LanceNamespaceModule.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/LanceNamespaceModule.java new file mode 100644 index 000000000000..0a4f681d7178 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/LanceNamespaceModule.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.lance.catalog; + +import com.google.inject.Binder; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.lance.catalog.namespace.DirectoryNamespaceModule; + +public class LanceNamespaceModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + install(switch (buildConfigObject(NamespaceTypeConfig.class).getNamespaceType()) { + case DIRECTORY -> new DirectoryNamespaceModule(); + }); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/NamespaceTypeConfig.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/NamespaceTypeConfig.java new file mode 100644 index 000000000000..161198793e3d --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/NamespaceTypeConfig.java @@ -0,0 +1,40 @@ +/* + * 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.lance.catalog; + +import io.airlift.configuration.Config; +import jakarta.validation.constraints.NotNull; + +public class NamespaceTypeConfig +{ + public enum NamespaceType + { + DIRECTORY; + } + + private NamespaceType namespaceType; + + @NotNull + public NamespaceType getNamespaceType() + { + return namespaceType; + } + + @Config("lance.namespace.type") + public NamespaceTypeConfig setNamespaceType(NamespaceTypeConfig.NamespaceType namespaceType) + { + this.namespaceType = namespaceType; + return this; + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/TrinoCatalog.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/TrinoCatalog.java new file mode 100644 index 000000000000..ad3a261f5183 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/TrinoCatalog.java @@ -0,0 +1,29 @@ +/* + * 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.lance.catalog; + +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; + +import java.util.List; +import java.util.Optional; + +public interface TrinoCatalog +{ + List listNamespaces(ConnectorSession session); + + List listTables(ConnectorSession session, Optional namespace); + + Optional loadTable(ConnectorSession session, SchemaTableName schemaTableName); +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespace.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespace.java new file mode 100644 index 000000000000..f758b6de729f --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespace.java @@ -0,0 +1,106 @@ +/* + * 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.lance.catalog.namespace; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.plugin.lance.catalog.BaseTable; +import io.trino.plugin.lance.catalog.TrinoCatalog; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class DirectoryNamespace + implements TrinoCatalog +{ + public static final String DEFAULT_NAMESPACE = "default"; + private final TrinoFileSystemFactory fileSystemFactory; + private final Location warehouseLocation; + + @Inject + public DirectoryNamespace(TrinoFileSystemFactory fileSystemFactory, DirectoryNamespaceConfig config) + { + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.warehouseLocation = Location.of(config.getWarehouseLocation()); + } + + private static String directoryName(Location directory, Location location) + { + String prefix = directory.path(); + if (!prefix.isEmpty() && !prefix.endsWith("/")) { + prefix += "/"; + } + String path = location.path(); + verify(path.endsWith("/"), "path does not endRowPosition with slash: %s", location); + verify(path.startsWith(prefix), "path [%s] is not a child of directory [%s]", location, directory); + return path.substring(prefix.length(), path.length() - 1); + } + + @Override + public List listNamespaces(ConnectorSession session) + { + return List.of(DEFAULT_NAMESPACE); + } + + @Override + public List listTables(ConnectorSession session, Optional namespace) + { + checkArgument(namespace.isEmpty() || namespace.get().equals(DEFAULT_NAMESPACE)); + TrinoFileSystem fileSystem = fileSystemFactory.create(session); + ImmutableList.Builder builder = ImmutableList.builder(); + try { + for (Location location : fileSystem.listDirectories(warehouseLocation)) { + String directoryName = directoryName(warehouseLocation, location); + if (directoryName.endsWith(BaseTable.LANCE_SUFFIX)) { + String tableName = directoryName.substring(0, directoryName.length() - BaseTable.LANCE_SUFFIX.length()); + builder.add(new SchemaTableName(DEFAULT_NAMESPACE, tableName)); + } + } + } + catch (IOException e) { + throw new RuntimeException(format("Failed to list tables under %s:", warehouseLocation), e); + } + + return builder.build(); + } + + @Override + public Optional loadTable(ConnectorSession session, SchemaTableName schemaTableName) + { + checkArgument(schemaTableName.getSchemaName().equals(DEFAULT_NAMESPACE)); + Location tableLocation = warehouseLocation.appendPath(schemaTableName.getTableName() + BaseTable.LANCE_SUFFIX); + TrinoFileSystem fileSystem = fileSystemFactory.create(session); + try { + Optional tableExists = fileSystem.directoryExists(tableLocation); + if (tableExists.isPresent() && tableExists.get()) { + return Optional.of(new BaseTable(schemaTableName.getSchemaName(), schemaTableName.getTableName(), fileSystem, tableLocation)); + } + return Optional.empty(); + } + catch (IOException e) { + throw new RuntimeException(format("Failed to check if table exists under %s:", warehouseLocation), e); + } + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespaceConfig.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespaceConfig.java new file mode 100644 index 000000000000..a3b91e4a2c26 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespaceConfig.java @@ -0,0 +1,35 @@ +/* + * 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.lance.catalog.namespace; + +import io.airlift.configuration.Config; +import jakarta.validation.constraints.NotEmpty; + +public class DirectoryNamespaceConfig +{ + private String warehouseLocation; + + @NotEmpty + public String getWarehouseLocation() + { + return warehouseLocation; + } + + @Config("lance.namespace.directory.warehouse.location") + public DirectoryNamespaceConfig setWarehouseLocation(String warehouseLocation) + { + this.warehouseLocation = warehouseLocation; + return this; + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespaceModule.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespaceModule.java new file mode 100644 index 000000000000..e7ee65757bce --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/catalog/namespace/DirectoryNamespaceModule.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.plugin.lance.catalog.namespace; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.trino.plugin.lance.catalog.TrinoCatalog; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class DirectoryNamespaceModule + implements Module +{ + @Override + public void configure(Binder binder) + { + configBinder(binder).bindConfig(DirectoryNamespaceConfig.class); + binder.bind(TrinoCatalog.class).to(DirectoryNamespace.class).in(Scopes.SINGLETON); + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/metadata/Fragment.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/metadata/Fragment.java new file mode 100644 index 000000000000..06d10b571f84 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/metadata/Fragment.java @@ -0,0 +1,61 @@ +/* + * 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.lance.metadata; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Objects.requireNonNull; + +public record Fragment(long id, List files, long physicalRows) +{ + // TODO: support deletion files + + public Fragment + { + files = ImmutableList.copyOf(files); + } + + public static Fragment from(build.buf.gen.lance.table.DataFragment proto) + { + if (proto.hasDeletionFile()) { + throw new UnsupportedOperationException("Deletion files are not supported"); + } + List files = proto.getFilesList().stream() + .map(DataFile::from) + .collect(toImmutableList()); + return new Fragment(proto.getId(), files, proto.getPhysicalRows()); + } + + public record DataFile(String path, List fields, List columnIndices, long fileMajorVersion, long fileMinorVersion) + { + public DataFile + { + requireNonNull(path, "path is null"); + requireNonNull(fields, "fields is null"); + requireNonNull(columnIndices, "columnIndices is null"); + } + + public static DataFile from(build.buf.gen.lance.table.DataFile proto) + { + return new DataFile(proto.getPath(), + proto.getFieldsList(), + proto.getColumnIndicesList(), + proto.getFileMajorVersion(), + proto.getFileMinorVersion()); + } + } +} diff --git a/plugin/trino-lance/src/main/java/io/trino/plugin/lance/metadata/Manifest.java b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/metadata/Manifest.java new file mode 100644 index 000000000000..e03f78fbd137 --- /dev/null +++ b/plugin/trino-lance/src/main/java/io/trino/plugin/lance/metadata/Manifest.java @@ -0,0 +1,73 @@ +/* + * 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.lance.metadata; + +import com.google.common.collect.ImmutableList; +import com.google.protobuf.InvalidProtocolBufferException; +import io.airlift.slice.Slice; +import io.trino.lance.file.v2.metadata.Field; +import io.trino.spi.TrinoException; + +import java.util.Arrays; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.lance.file.LanceReader.toFields; +import static io.trino.plugin.lance.LanceErrorCode.LANCE_INVALID_METADATA; +import static java.lang.Math.toIntExact; +import static java.nio.charset.StandardCharsets.US_ASCII; + +public record Manifest(List fields, List fragments, long version, long maxFragmentId) +{ + private static final int MIN_FILE_SIZE = 16; + private static final byte[] MAGIC = "LANC".getBytes(US_ASCII); + + public Manifest + { + fields = ImmutableList.copyOf(fields); + fragments = ImmutableList.copyOf(fragments); + } + + public static Manifest from(Slice slice) + { + checkArgument(slice.length() > MIN_FILE_SIZE, "Invalid manifest size: %s", slice); + int length = slice.length(); + byte[] magic = slice.getBytes(length - MAGIC.length, MAGIC.length); + if (!Arrays.equals(MAGIC, magic)) { + throw new TrinoException(LANCE_INVALID_METADATA, "Invalid MAGIC in manifest footer"); + } + long position = slice.getLong(toIntExact(length - 16)); + int recordedLength = slice.getInt(toIntExact(position)); + if (recordedLength != length - position - 20) { + throw new TrinoException(LANCE_INVALID_METADATA, "Invalid manifest proto message length: " + recordedLength); + } + build.buf.gen.lance.table.Manifest proto; + try { + proto = build.buf.gen.lance.table.Manifest.parseFrom(slice.slice(toIntExact(position + 4), recordedLength).toByteBuffer()); + } + catch (InvalidProtocolBufferException e) { + throw new TrinoException(LANCE_INVALID_METADATA, e); + } + + List fragments = proto.getFragmentsList().stream() + .map(Fragment::from) + .collect(toImmutableList()); + + return new Manifest(toFields(proto.getFieldsList()), + fragments, + proto.getVersion(), + proto.getMaxFragmentId()); + } +} diff --git a/plugin/trino-lance/src/test/java/io/trino/plugin/lance/LanceLoader.java b/plugin/trino-lance/src/test/java/io/trino/plugin/lance/LanceLoader.java new file mode 100644 index 000000000000..7f03fbdd78b0 --- /dev/null +++ b/plugin/trino-lance/src/test/java/io/trino/plugin/lance/LanceLoader.java @@ -0,0 +1,217 @@ +/* + * 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.lance; + +import com.lancedb.lance.Dataset; +import com.lancedb.lance.WriteParams; +import io.trino.Session; +import io.trino.client.Column; +import io.trino.client.QueryStatusInfo; +import io.trino.client.ResultRows; +import io.trino.server.testing.TestingTrinoServer; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.DateType; +import io.trino.spi.type.DoubleType; +import io.trino.spi.type.IntegerType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; +import io.trino.testing.AbstractTestingTrinoClient; +import io.trino.testing.ResultsSession; +import org.apache.arrow.c.ArrowArrayStream; +import org.apache.arrow.c.Data; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BigIntVector; +import org.apache.arrow.vector.DateDayVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.Float8Vector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.ipc.ArrowReader; +import org.apache.arrow.vector.types.DateUnit; +import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.util.DateTimeUtils.parseDate; + +public class LanceLoader + extends AbstractTestingTrinoClient +{ + private final String tablePath; + private final RootAllocator allocator = new RootAllocator(); + + public LanceLoader(TestingTrinoServer trinoServer, Session defaultSession, String tablePath) + { + super(trinoServer, defaultSession); + this.tablePath = tablePath; + } + + @Override + protected ResultsSession getResultSession(Session session) + { + return new LanceLoadingSession(); + } + + private class LanceLoadingSession + implements ResultsSession + { + private final AtomicReference schema = new AtomicReference<>(); + private final AtomicReference> types = new AtomicReference<>(); + private final AtomicReference>> data = new AtomicReference<>(); + + @Override + public void addResults(QueryStatusInfo statusInfo, ResultRows rows) + { + if (schema.get() == null && statusInfo.getColumns() != null) { + types.set(statusInfo.getColumns().stream().map(LanceLoader.this::getTrinoType).collect(toImmutableList())); + data.set(new ArrayList<>()); + schema.set(getArrowSchema(statusInfo.getColumns())); + } + + if (rows.isNull()) { + return; + } + + checkState(schema.get() != null, "rows received without schema set"); + for (List row : rows) { + data.get().add(row); + } + } + + @Override + public Void build(Map setSessionProperties, Set resetSessionProperties) + { + try (VectorSchemaRoot root = VectorSchemaRoot.create(schema.get(), allocator)) { + root.allocateNew(); + List fields = schema.get().getFields(); + List fieldVectors = fields.stream().map(root::getVector).collect(toImmutableList()); + + for (int i = 0; i < data.get().size(); i++) { + List row = data.get().get(i); + for (int filedIdx = 0; filedIdx < fields.size(); filedIdx++) { + Type fieldType = types.get().get(filedIdx); + Object value = row.get(filedIdx); + switch (fieldType) { + case IntegerType _ -> ((IntVector) fieldVectors.get(filedIdx)).setSafe(i, ((Number) value).intValue()); + case BigintType _ -> ((BigIntVector) fieldVectors.get(filedIdx)).setSafe(i, ((Number) value).longValue()); + case DoubleType _ -> ((Float8Vector) fieldVectors.get(filedIdx)).setSafe(i, ((Number) value).doubleValue()); + case VarcharType _ -> ((VarCharVector) fieldVectors.get(filedIdx)).setSafe(i, ((String) value).getBytes(StandardCharsets.UTF_8)); + case DateType _ -> ((DateDayVector) fieldVectors.get(filedIdx)).setSafe(i, parseDate((String) value)); + default -> throw new IllegalStateException("Unsupported fieldType: " + fieldType); + } + } + } + root.setRowCount(data.get().size()); + + try (ArrowReader reader = new SimpleArrowReader(root, allocator); ArrowArrayStream stream = ArrowArrayStream.allocateNew(allocator)) { + Data.exportArrayStream(allocator, reader, stream); + WriteParams writeParams = new WriteParams.Builder() + .withDataStorageVersion(WriteParams.LanceFileVersion.V2_1) + .build(); + Dataset dataset = Dataset.create(allocator, stream, tablePath, writeParams); + dataset.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + return null; + } + + private static class SimpleArrowReader + extends ArrowReader + { + private final VectorSchemaRoot root; + private boolean batchLoaded; + + public SimpleArrowReader(VectorSchemaRoot root, BufferAllocator allocator) + { + super(allocator); + this.root = root; + batchLoaded = false; + } + + @Override + public boolean loadNextBatch() + { + if (!batchLoaded) { + batchLoaded = true; + return true; + } + return false; + } + + @Override + public long bytesRead() + { + return root.getFieldVectors().stream() + .mapToLong(FieldVector::getBufferSize) + .sum(); + } + + @Override + protected void closeReadSource() {} + + @Override + protected Schema readSchema() + { + return root.getSchema(); + } + + @Override + public VectorSchemaRoot getVectorSchemaRoot() + { + return root; + } + } + } + + private Schema getArrowSchema(List columns) + { + List fields = columns.stream().map(column -> new Field(column.getName(), FieldType.notNullable(getArrowType(column)), null)).collect(toImmutableList()); + return new Schema(fields); + } + + private Type getTrinoType(Column column) + { + return getServer().getPlannerContext().getTypeManager().fromSqlType(column.getType()); + } + + private ArrowType getArrowType(Column column) + { + Type trinoType = getTrinoType(column); + return switch (trinoType) { + case IntegerType _ -> new ArrowType.Int(32, true); + case BigintType _ -> new ArrowType.Int(64, true); + case DoubleType _ -> new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE); + case DateType _ -> new ArrowType.Date(DateUnit.DAY); + case VarcharType _ -> new ArrowType.Utf8(); + default -> throw new IllegalStateException("Unsupported type: " + trinoType); + }; + } +} diff --git a/plugin/trino-lance/src/test/java/io/trino/plugin/lance/LanceQueryRunner.java b/plugin/trino-lance/src/test/java/io/trino/plugin/lance/LanceQueryRunner.java new file mode 100644 index 000000000000..cf11733dca56 --- /dev/null +++ b/plugin/trino-lance/src/test/java/io/trino/plugin/lance/LanceQueryRunner.java @@ -0,0 +1,128 @@ +/* + * 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.lance; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.trino.plugin.base.util.Closables; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingTrinoClient; +import io.trino.tpch.TpchTable; + +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; + +import static io.airlift.units.Duration.nanosSince; +import static io.trino.plugin.lance.catalog.BaseTable.LANCE_SUFFIX; +import static io.trino.testing.TestingSession.testSessionBuilder; +import static java.nio.file.Files.createTempDirectory; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.SECONDS; + +public class LanceQueryRunner +{ + public static final String LANCE_CATALOG = "lance"; + + private static final Logger log = Logger.get(LanceQueryRunner.class); + + private LanceQueryRunner() {} + + public static Builder builder(String warehousePath) + { + return new Builder(warehousePath); + } + + public static class Builder + extends DistributedQueryRunner.Builder + { + private final String warehousePath; + private List> initialTables = ImmutableList.of(); + + protected Builder(String warehousePath) + { + super(testSessionBuilder().setCatalog(LANCE_CATALOG).build()); + this.warehousePath = requireNonNull(warehousePath, "warehousePath is null"); + } + + public Builder setInitialTables(List> initialTables) + { + this.initialTables = ImmutableList.copyOf(initialTables); + return this; + } + + private static void loadTpchTable(TestingTrinoClient trinoClient, TpchTable table, String warehousePath) + { + long start = System.nanoTime(); + Path tablePath = Paths.get(warehousePath, table.getTableName() + LANCE_SUFFIX); + log.info("Running import for %s", table.getTableName()); + LanceLoader tpchLoader = new LanceLoader(trinoClient.getServer(), trinoClient.getDefaultSession(), tablePath.toString()); + tpchLoader.execute("SELECT * FROM tpch.tiny." + table.getTableName().toLowerCase(ENGLISH)); + log.info("Imported %s to %s in %s", table.getTableName(), tablePath, nanosSince(start).convertToMostSuccinctTimeUnit()); + } + + @Override + public DistributedQueryRunner build() + throws Exception + { + DistributedQueryRunner queryRunner = super.build(); + + try { + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + queryRunner.installPlugin(new LancePlugin()); + queryRunner.createCatalog(LANCE_CATALOG, "lance", + ImmutableMap.of("lance.namespace.type", "directory", + "lance.namespace.directory.warehouse.location", "local:///", + "fs.native-local.enabled", "true", + "local.location", warehousePath)); + + TestingTrinoClient trinoClient = queryRunner.getClient(); + log.info("Loading data..."); + long startTime = System.nanoTime(); + for (TpchTable table : initialTables) { + loadTpchTable(trinoClient, table, warehousePath); + } + log.info("Loading complete in %s", nanosSince(startTime).toString(SECONDS)); + + return queryRunner; + } + catch (Throwable e) { + Closables.closeAllSuppress(e, queryRunner); + throw e; + } + } + } + + // LanceQueryRunner requires the following additional JVM options: + // - --add-opens=java.base/java.nio=ALL-UNNAMED + // - --sun-misc-unsafe-memory-access=allow + public static void main(String[] args) + throws Exception + { + Path warehousePath = createTempDirectory(null); + QueryRunner queryRunner = LanceQueryRunner.builder(warehousePath.toString()) + .addCoordinatorProperty("http-server.http.port", "8080") + .setInitialTables(TpchTable.getTables()) + .build(); + + Logger log = Logger.get(LanceQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/plugin/trino-lance/src/test/java/io/trino/plugin/lance/TestLanceConnectorTest.java b/plugin/trino-lance/src/test/java/io/trino/plugin/lance/TestLanceConnectorTest.java new file mode 100644 index 000000000000..9c7202d4b862 --- /dev/null +++ b/plugin/trino-lance/src/test/java/io/trino/plugin/lance/TestLanceConnectorTest.java @@ -0,0 +1,131 @@ +/* + * 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.lance; + +import io.trino.Session; +import io.trino.testing.BaseConnectorTest; +import io.trino.testing.MaterializedResult; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; +import org.junit.jupiter.api.Test; + +import java.nio.file.Files; +import java.nio.file.Path; + +import static io.trino.plugin.lance.LanceQueryRunner.LANCE_CATALOG; +import static io.trino.plugin.lance.catalog.namespace.DirectoryNamespace.DEFAULT_NAMESPACE; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.testing.MaterializedResult.resultBuilder; +import static java.lang.String.format; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestLanceConnectorTest + extends BaseConnectorTest +{ + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + return switch (connectorBehavior) { + case SUPPORTS_ADD_COLUMN, + SUPPORTS_ADD_FIELD, + SUPPORTS_AGGREGATION_PUSHDOWN, + SUPPORTS_COMMENT_ON_TABLE, + SUPPORTS_CREATE_MATERIALIZED_VIEW, + SUPPORTS_CREATE_OR_REPLACE_TABLE, + SUPPORTS_CREATE_SCHEMA, + SUPPORTS_CREATE_TABLE, + SUPPORTS_CREATE_VIEW, + SUPPORTS_DELETE, + SUPPORTS_DEREFERENCE_PUSHDOWN, + SUPPORTS_DROP_FIELD, + SUPPORTS_DYNAMIC_FILTER_PUSHDOWN, + SUPPORTS_INSERT, + SUPPORTS_JOIN_PUSHDOWN, + SUPPORTS_MERGE, + SUPPORTS_NATIVE_QUERY, + SUPPORTS_PREDICATE_PUSHDOWN, + SUPPORTS_RENAME_COLUMN, + SUPPORTS_RENAME_TABLE, + SUPPORTS_TOPN_PUSHDOWN, + SUPPORTS_TOPN_PUSHDOWN_WITH_VARCHAR, + SUPPORTS_UPDATE -> false; + default -> super.hasBehavior(connectorBehavior); + }; + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + Path catalogDir = Files.createTempDirectory("lance-catalog"); + return LanceQueryRunner.builder(catalogDir.toString()) + .setInitialTables(REQUIRED_TPCH_TABLES) + .build(); + } + + @Test + @Override + public void testDescribeTable() + { + // Lance only supports variable width VARCHAR + MaterializedResult expectedColumns = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "", "") + .row("custkey", "bigint", "", "") + .row("orderstatus", "varchar", "", "") + .row("totalprice", "double", "", "") + .row("orderdate", "date", "", "") + .row("orderpriority", "varchar", "", "") + .row("clerk", "varchar", "", "") + .row("shippriority", "integer", "", "") + .row("comment", "varchar", "", "") + .build(); + MaterializedResult actualColumns = computeActual("DESCRIBE orders"); + assertThat(actualColumns).isEqualTo(expectedColumns); + } + + @Test + @Override + public void testShowCreateTable() + { + String catalog = getSession().getCatalog().orElseThrow(); + String schema = getSession().getSchema().orElseThrow(); + // Lance only supports variable width VARCHAR + assertThat(computeScalar("SHOW CREATE TABLE orders")) + .isEqualTo(format( + """ + CREATE TABLE %s.%s.orders ( + orderkey bigint, + custkey bigint, + orderstatus varchar, + totalprice double, + orderdate date, + orderpriority varchar, + clerk varchar, + shippriority integer, + comment varchar + )\ + """, + catalog, + schema)); + } + + @Override + protected Session getSession() { + // Lance only supports variable width VARCHAR + return Session.builder(super.getSession()) + .setCatalog(LANCE_CATALOG) + .setSchema(DEFAULT_NAMESPACE) + .build(); + } +} diff --git a/plugin/trino-lance/src/test/java/io/trino/plugin/lance/TestLancePlugin.java b/plugin/trino-lance/src/test/java/io/trino/plugin/lance/TestLancePlugin.java new file mode 100644 index 000000000000..93bde7b94766 --- /dev/null +++ b/plugin/trino-lance/src/test/java/io/trino/plugin/lance/TestLancePlugin.java @@ -0,0 +1,34 @@ +/* + * 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.lance; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.testing.TestingConnectorContext; +import org.junit.jupiter.api.Test; + +final class TestLancePlugin +{ + @Test + void testDictionaryNamespace() + { + ConnectorFactory factory = new LanceConnectorFactory(); + factory.create( + "test", + ImmutableMap.of("lance.namespace.type", "directory", + "lance.namespace.directory.warehouse.location", "s3://test-bucket/"), + new TestingConnectorContext()) + .shutdown(); + } +} diff --git a/pom.xml b/pom.xml index 7ce62ff6855d..1db7a6b6d22a 100644 --- a/pom.xml +++ b/pom.xml @@ -52,6 +52,7 @@ lib/trino-geospatial-toolkit lib/trino-hdfs lib/trino-hive-formats + lib/trino-lance-file lib/trino-matching lib/trino-memory-context lib/trino-metastore @@ -88,6 +89,7 @@ plugin/trino-kafka plugin/trino-kafka-event-listener plugin/trino-lakehouse + plugin/trino-lance plugin/trino-ldap-group-provider plugin/trino-loki plugin/trino-mariadb @@ -305,6 +307,12 @@ import + + build.buf.gen + lancedb_lance_protocolbuffers_java + 25.8.0.1.20251015015903.554c918d1601 + + com.adobe.testing s3mock-testcontainers @@ -1205,6 +1213,18 @@ ${project.version} + + io.trino + trino-lance + ${project.version} + + + + io.trino + trino-lance-file + ${project.version} + + io.trino trino-main @@ -2393,6 +2413,13 @@ provided + + com.lancedb + lance-core + 0.38.3-beta.4 + test + + io.confluent kafka-json-schema-serializer diff --git a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java index 941642a94daf..436ac38bf661 100644 --- a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java +++ b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java @@ -67,6 +67,7 @@ public final class TestGroups public static final String CASSANDRA = "cassandra"; public static final String POSTGRESQL = "postgresql"; public static final String SQLSERVER = "sqlserver"; + public static final String LANCE = "lance"; public static final String LDAP = "ldap"; public static final String LDAP_AND_FILE = "ldap_and_file"; public static final String LDAP_CLI = "ldap_cli"; diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java index ee3de89bd9f0..59a75ad54625 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java @@ -60,6 +60,7 @@ public void extendEnvironment(Environment.Builder builder) "iceberg", "ignite", "kafka", + "lance", "loki", "mariadb", "memory", diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeLanceMinio.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeLanceMinio.java new file mode 100644 index 000000000000..12ce768fb4a5 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeLanceMinio.java @@ -0,0 +1,74 @@ +/* + * 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.tests.product.launcher.env.environment; + +import com.google.inject.Inject; +import io.trino.tests.product.launcher.docker.DockerFiles; +import io.trino.tests.product.launcher.env.Environment; +import io.trino.tests.product.launcher.env.EnvironmentProvider; +import io.trino.tests.product.launcher.env.common.Minio; +import io.trino.tests.product.launcher.env.common.StandardMultinode; +import io.trino.tests.product.launcher.env.common.TestsEnvironment; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.attribute.FileAttribute; +import java.nio.file.attribute.PosixFilePermission; +import java.nio.file.attribute.PosixFilePermissions; +import java.util.Set; + +import static io.trino.tests.product.launcher.env.EnvironmentContainers.TESTS; +import static io.trino.tests.product.launcher.env.common.Minio.MINIO_CONTAINER_NAME; +import static org.testcontainers.utility.MountableFile.forHostPath; + +@TestsEnvironment +public class EnvMultinodeLanceMinio + extends EnvironmentProvider +{ + private static final String S3_BUCKET_NAME = "test-bucket"; + + private final DockerFiles.ResourceProvider configDir; + + @Inject + public EnvMultinodeLanceMinio(StandardMultinode standardMultinode, Minio minio, DockerFiles dockerFiles) + { + super(standardMultinode, minio); + this.configDir = dockerFiles.getDockerFilesHostDirectory("conf/environment/multinode-lance-minio"); + } + + @Override + public void extendEnvironment(Environment.Builder builder) + { + builder.configureContainer(TESTS, dockerContainer -> dockerContainer.withEnv("S3_BUCKET", S3_BUCKET_NAME)); + + // initialize buckets in minio + FileAttribute> posixFilePermissions = PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rw-r--r--")); + Path minioBucketDirectory; + try { + minioBucketDirectory = Files.createTempDirectory("test-bucket-contents", posixFilePermissions); + minioBucketDirectory.toFile().deleteOnExit(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + builder.configureContainer(MINIO_CONTAINER_NAME, container -> + { + container.withCopyFileToContainer(forHostPath(minioBucketDirectory), "/data/" + S3_BUCKET_NAME); + }); + + builder.addConnector("lance", forHostPath(configDir.getPath("lance.properties"))); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteLance.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteLance.java new file mode 100644 index 000000000000..6cd359a7849c --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteLance.java @@ -0,0 +1,38 @@ +/* + * 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.tests.product.launcher.suite.suites; + +import com.google.common.collect.ImmutableList; +import io.trino.tests.product.launcher.env.EnvironmentConfig; +import io.trino.tests.product.launcher.env.environment.EnvMultinodeLanceMinio; +import io.trino.tests.product.launcher.suite.Suite; +import io.trino.tests.product.launcher.suite.SuiteTestRun; + +import java.util.List; + +import static io.trino.tests.product.TestGroups.CONFIGURED_FEATURES; +import static io.trino.tests.product.TestGroups.LANCE; + +public class SuiteLance + extends Suite +{ + @Override + public List getTestRuns(EnvironmentConfig config) + { + return ImmutableList.of( + SuiteTestRun.testOnEnvironment(EnvMultinodeLanceMinio.class) + .withGroups(CONFIGURED_FEATURES, LANCE) + .build()); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/lance.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/lance.properties new file mode 100644 index 000000000000..fc17510b271f --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/lance.properties @@ -0,0 +1,5 @@ +connector.name=lance +lance.namespace.type=directory +lance.namespace.directory.warehouse.location=local:/// +fs.native-local.enabled=true +local.location=/tmp diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-lance-minio/jvm.config b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-lance-minio/jvm.config new file mode 100644 index 000000000000..f8a4471c3f1f --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-lance-minio/jvm.config @@ -0,0 +1,17 @@ +-server +--add-opens=java.base/java.nio=ALL-UNNAMED +-Xmx2G +-XX:G1HeapRegionSize=32M +-XX:+ExplicitGCInvokesConcurrent +-XX:+ExitOnOutOfMemoryError +-XX:+HeapDumpOnOutOfMemoryError +-XX:-OmitStackTraceInFastThrow +-XX:ReservedCodeCacheSize=150M +-XX:PerMethodRecompilationCutoff=10000 +-XX:PerBytecodeRecompilationCutoff=10000 +-Djdk.attach.allowAttachSelf=true +-Duser.timezone=Asia/Kathmandu +-XX:ErrorFile=/docker/logs/product-tests-presto-jvm-error-file.log +# Allow loading dynamic agent used by JOL +-XX:+EnableDynamicAgentLoading +-XX:+ExitOnOutOfMemoryError diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-lance-minio/lance.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-lance-minio/lance.properties new file mode 100644 index 000000000000..5f9bbaeb9b42 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-lance-minio/lance.properties @@ -0,0 +1,10 @@ +connector.name=lance +lance.namespace.type=directory +lance.namespace.directory.warehouse.location=s3://test-bucket/ +fs.native-s3.enabled=true +fs.hadoop.enabled=false +s3.region=us-east-1 +s3.aws-access-key=minio-access-key +s3.aws-secret-key=minio-secret-key +s3.endpoint=http://minio:9080/ +s3.path-style-access=true diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/_transactions/0-8483f31a-6840-4ccf-bd77-fe8771f028f6.txn b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/_transactions/0-8483f31a-6840-4ccf-bd77-fe8771f028f6.txn new file mode 100644 index 000000000000..a4b54a1a03fe Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/_transactions/0-8483f31a-6840-4ccf-bd77-fe8771f028f6.txn differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/_versions/1.manifest b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/_versions/1.manifest new file mode 100644 index 000000000000..eb84855741ad Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/_versions/1.manifest differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/data/110000100010000110111110339d6c4f5d967d46819f76af34.lance b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/data/110000100010000110111110339d6c4f5d967d46819f76af34.lance new file mode 100644 index 000000000000..076bdca44587 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/customer.lance/data/110000100010000110111110339d6c4f5d967d46819f76af34.lance differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/_transactions/0-893147af-ba65-453a-81cc-fdf3d01808f6.txn b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/_transactions/0-893147af-ba65-453a-81cc-fdf3d01808f6.txn new file mode 100644 index 000000000000..ba5e7c3cf6d8 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/_transactions/0-893147af-ba65-453a-81cc-fdf3d01808f6.txn differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/_versions/1.manifest b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/_versions/1.manifest new file mode 100644 index 000000000000..c201d2ba5df5 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/_versions/1.manifest differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/data/0010100110011010011110119b80734e3e8c4f704b78798de0.lance b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/data/0010100110011010011110119b80734e3e8c4f704b78798de0.lance new file mode 100644 index 000000000000..4cad78da4d03 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/lineitem.lance/data/0010100110011010011110119b80734e3e8c4f704b78798de0.lance differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/_transactions/0-d2a0ad2c-cf99-4cc0-9248-efc56064166e.txn b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/_transactions/0-d2a0ad2c-cf99-4cc0-9248-efc56064166e.txn new file mode 100644 index 000000000000..adea81f05889 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/_transactions/0-d2a0ad2c-cf99-4cc0-9248-efc56064166e.txn differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/_versions/1.manifest b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/_versions/1.manifest new file mode 100644 index 000000000000..49617ac05835 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/_versions/1.manifest differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/data/011111100101110011000011696483447eaa8c9bf4c06d19fe.lance b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/data/011111100101110011000011696483447eaa8c9bf4c06d19fe.lance new file mode 100644 index 000000000000..53bb7efcc060 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/nation.lance/data/011111100101110011000011696483447eaa8c9bf4c06d19fe.lance differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/_transactions/0-dfdc0b84-f9ed-459a-8783-121a3236dce6.txn b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/_transactions/0-dfdc0b84-f9ed-459a-8783-121a3236dce6.txn new file mode 100644 index 000000000000..06fc07458e53 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/_transactions/0-dfdc0b84-f9ed-459a-8783-121a3236dce6.txn differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/_versions/1.manifest b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/_versions/1.manifest new file mode 100644 index 000000000000..ed7a474b8a31 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/_versions/1.manifest differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/data/001011111110010001100111c381b24d52bb6106586f1c201b.lance b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/data/001011111110010001100111c381b24d52bb6106586f1c201b.lance new file mode 100644 index 000000000000..69a8b5233690 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/orders.lance/data/001011111110010001100111c381b24d52bb6106586f1c201b.lance differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/_transactions/0-e0e07d71-e090-4237-894f-06b18b3a3946.txn b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/_transactions/0-e0e07d71-e090-4237-894f-06b18b3a3946.txn new file mode 100644 index 000000000000..c9db57b04232 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/_transactions/0-e0e07d71-e090-4237-894f-06b18b3a3946.txn differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/_versions/1.manifest b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/_versions/1.manifest new file mode 100644 index 000000000000..beb4f42699c1 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/_versions/1.manifest differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/data/00010001010011010011100035c2e344a2acb9cbf303f1e27c.lance b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/data/00010001010011010011100035c2e344a2acb9cbf303f1e27c.lance new file mode 100644 index 000000000000..700eb81ad3c6 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/part.lance/data/00010001010011010011100035c2e344a2acb9cbf303f1e27c.lance differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/_transactions/0-dc23e88b-e510-44a8-8b14-7411025c1478.txn b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/_transactions/0-dc23e88b-e510-44a8-8b14-7411025c1478.txn new file mode 100644 index 000000000000..75db8534428f Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/_transactions/0-dc23e88b-e510-44a8-8b14-7411025c1478.txn differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/_versions/1.manifest b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/_versions/1.manifest new file mode 100644 index 000000000000..0c00334e4093 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/_versions/1.manifest differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/data/01110100000111110100101019d20447e5bc7fdf2e9728fc4b.lance b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/data/01110100000111110100101019d20447e5bc7fdf2e9728fc4b.lance new file mode 100644 index 000000000000..44c4d7ca82cc Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/partsupp.lance/data/01110100000111110100101019d20447e5bc7fdf2e9728fc4b.lance differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/_transactions/0-d6847cbc-38db-4edf-8c9b-a120aeb9b3a1.txn b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/_transactions/0-d6847cbc-38db-4edf-8c9b-a120aeb9b3a1.txn new file mode 100644 index 000000000000..689c71d7dd8f Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/_transactions/0-d6847cbc-38db-4edf-8c9b-a120aeb9b3a1.txn differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/_versions/1.manifest b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/_versions/1.manifest new file mode 100644 index 000000000000..d531b8f97b5d Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/_versions/1.manifest differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/data/11010011101110110010111156e9134a45ad829b2ddf7836cf.lance b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/data/11010011101110110010111156e9134a45ad829b2ddf7836cf.lance new file mode 100644 index 000000000000..76a9b6ce5e3b Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/region.lance/data/11010011101110110010111156e9134a45ad829b2ddf7836cf.lance differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/_transactions/0-d9a3660a-f3fd-4f64-87ca-0901dfd66807.txn b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/_transactions/0-d9a3660a-f3fd-4f64-87ca-0901dfd66807.txn new file mode 100644 index 000000000000..8b7ac538765a Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/_transactions/0-d9a3660a-f3fd-4f64-87ca-0901dfd66807.txn differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/_versions/1.manifest b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/_versions/1.manifest new file mode 100644 index 000000000000..d0caa361e631 Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/_versions/1.manifest differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/data/0110100010001110111101105403ba4e0ba474a0e08a90e967.lance b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/data/0110100010001110111101105403ba4e0ba474a0e08a90e967.lance new file mode 100644 index 000000000000..d0cc2596f99c Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/lance/supplier.lance/data/0110100010001110111101105403ba4e0ba474a0e08a90e967.lance differ diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/lance/TestLance.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/lance/TestLance.java new file mode 100644 index 000000000000..ef5fa481357b --- /dev/null +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/lance/TestLance.java @@ -0,0 +1,36 @@ +/* + * 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.tests.product.lance; + +import io.trino.tempto.ProductTest; +import io.trino.tempto.query.QueryResult; +import org.testng.annotations.Test; + +import static io.trino.tempto.assertions.QueryAssert.Row.row; +import static io.trino.tests.product.TestGroups.LANCE; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; +import static io.trino.tests.product.utils.QueryExecutors.onTrino; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestLance + extends ProductTest +{ + @Test(groups = {LANCE, PROFILE_SPECIFIC_TESTS}) + public void testShowSchemas() + { + // A simple product test to ensure no class loader issue. + QueryResult result = onTrino().executeQuery("SHOW SCHEMAS FROM lance"); + assertThat(result).containsOnly(row("default"), row("information_schema")); + } +}