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