lance_index/vector/
storage.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

//! Vector Storage, holding (quantized) vectors and providing distance calculation.

use std::collections::HashMap;
use std::{any::Any, sync::Arc};

use arrow::array::AsArray;
use arrow::compute::concat_batches;
use arrow::datatypes::UInt64Type;
use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array};
use arrow_schema::{Field, SchemaRef};
use deepsize::DeepSizeOf;
use futures::prelude::stream::TryStreamExt;
use lance_arrow::RecordBatchExt;
use lance_core::{Error, Result};
use lance_encoding::decoder::FilterExpression;
use lance_file::v2::reader::FileReader;
use lance_io::ReadBatchParams;
use lance_linalg::distance::DistanceType;
use prost::Message;
use snafu::{location, Location};

use crate::{
    pb,
    vector::{
        ivf::storage::{IvfModel, IVF_METADATA_KEY},
        quantizer::Quantization,
    },
    INDEX_METADATA_SCHEMA_KEY,
};

use super::quantizer::Quantizer;
use super::DISTANCE_TYPE_KEY;

/// <section class="warning">
///  Internal API
///
///  API stability is not guaranteed
/// </section>
pub trait DistCalculator {
    fn distance(&self, id: u32) -> f32;
    fn distance_all(&self) -> Vec<f32>;
    fn prefetch(&self, _id: u32) {}
}

pub const STORAGE_METADATA_KEY: &str = "storage_metadata";

/// Vector Storage is the abstraction to store the vectors.
///
/// It can be in-memory or on-disk, raw vector or quantized vectors.
///
/// It abstracts away the logic to compute the distance between vectors.
///
/// TODO: should we rename this to "VectorDistance"?;
///
/// <section class="warning">
///  Internal API
///
///  API stability is not guaranteed
/// </section>
pub trait VectorStore: Send + Sync + Sized + Clone {
    type DistanceCalculator<'a>: DistCalculator
    where
        Self: 'a;

    /// Create a [VectorStore] from a [RecordBatch].
    /// The batch should consist of row IDs and quantized vector.
    fn try_from_batch(batch: RecordBatch, distance_type: DistanceType) -> Result<Self>;

    fn as_any(&self) -> &dyn Any;

    fn schema(&self) -> &SchemaRef;

    fn to_batches(&self) -> Result<impl Iterator<Item = RecordBatch> + Send>;

    fn remap(&self, mapping: &HashMap<u64, Option<u64>>) -> Result<Self> {
        let batches = self
            .to_batches()?
            .map(|b| {
                let mut indices = Vec::with_capacity(b.num_rows());
                let mut new_row_ids = Vec::with_capacity(b.num_rows());

                let row_ids = b.column(0).as_primitive::<UInt64Type>().values();
                for (i, row_id) in row_ids.iter().enumerate() {
                    match mapping.get(row_id) {
                        Some(Some(new_id)) => {
                            indices.push(i as u32);
                            new_row_ids.push(*new_id);
                        }
                        Some(None) => {}
                        None => {
                            indices.push(i as u32);
                            new_row_ids.push(*row_id);
                        }
                    }
                }

                let indices = UInt32Array::from(indices);
                let new_row_ids = Arc::new(UInt64Array::from(new_row_ids));
                let new_vectors = arrow::compute::take(b.column(1), &indices, None)?;

                Ok(RecordBatch::try_new(
                    self.schema().clone(),
                    vec![new_row_ids, new_vectors],
                )?)
            })
            .collect::<Result<Vec<_>>>()?;

        let batch = concat_batches(self.schema(), batches.iter())?;
        Self::try_from_batch(batch, self.distance_type())
    }

    fn len(&self) -> usize;

    /// Returns true if this graph is empty.
    fn is_empty(&self) -> bool {
        self.len() == 0
    }

    /// Return [DistanceType].
    fn distance_type(&self) -> DistanceType;

    /// Get the lance ROW ID from one vector.
    fn row_id(&self, id: u32) -> u64;

    fn row_ids(&self) -> impl Iterator<Item = &u64>;

    /// Append Raw [RecordBatch] into the Storage.
    /// The storage implement will perform quantization if necessary.
    fn append_batch(&self, batch: RecordBatch, vector_column: &str) -> Result<Self>;

    /// Create a [DistCalculator] to compute the distance between the query.
    ///
    /// Using dist calculator can be more efficient as it can pre-compute some
    /// values.
    fn dist_calculator(&self, query: ArrayRef) -> Self::DistanceCalculator<'_>;

    fn dist_calculator_from_id(&self, id: u32) -> Self::DistanceCalculator<'_>;

    fn distance_between(&self, a: u32, b: u32) -> f32;

    fn dist_calculator_from_native(&self, _query: ArrayRef) -> Self::DistanceCalculator<'_> {
        todo!("Implement this")
    }
}

pub struct StorageBuilder<Q: Quantization> {
    column: String,
    distance_type: DistanceType,
    quantizer: Q,
}

impl<Q: Quantization> StorageBuilder<Q> {
    pub fn new(column: String, distance_type: DistanceType, quantizer: Q) -> Self {
        Self {
            column,
            distance_type,
            quantizer,
        }
    }

    pub fn build(&self, batch: &RecordBatch) -> Result<Q::Storage> {
        let vectors = batch.column_by_name(&self.column).ok_or(Error::Schema {
            message: format!("column {} not found", self.column),
            location: location!(),
        })?;
        let code_array = self.quantizer.quantize(vectors.as_ref())?;
        let batch = batch
            .try_with_column(
                Field::new(
                    self.quantizer.column(),
                    code_array.data_type().clone(),
                    true,
                ),
                code_array,
            )?
            .drop_column(&self.column)?;
        let batch = batch.add_metadata(
            STORAGE_METADATA_KEY.to_owned(),
            self.quantizer.metadata(None)?.to_string(),
        )?;
        Q::Storage::try_from_batch(batch, self.distance_type)
    }
}

/// Loader to load partitioned PQ storage from disk.
#[derive(Debug)]
pub struct IvfQuantizationStorage {
    reader: FileReader,

    distance_type: DistanceType,
    metadata: Vec<String>,

    ivf: IvfModel,
}

impl DeepSizeOf for IvfQuantizationStorage {
    fn deep_size_of_children(&self, context: &mut deepsize::Context) -> usize {
        self.metadata.deep_size_of_children(context) + self.ivf.deep_size_of_children(context)
    }
}

#[allow(dead_code)]
impl IvfQuantizationStorage {
    /// Open a Loader.
    ///
    ///
    pub async fn try_new(reader: FileReader) -> Result<Self> {
        let schema = reader.schema();

        let distance_type = DistanceType::try_from(
            schema
                .metadata
                .get(DISTANCE_TYPE_KEY)
                .ok_or(Error::Index {
                    message: format!("{} not found", INDEX_METADATA_SCHEMA_KEY),
                    location: location!(),
                })?
                .as_str(),
        )?;

        let ivf_pos = schema
            .metadata
            .get(IVF_METADATA_KEY)
            .ok_or(Error::Index {
                message: format!("{} not found", IVF_METADATA_KEY),
                location: location!(),
            })?
            .parse()
            .map_err(|e| Error::Index {
                message: format!("Failed to decode IVF metadata: {}", e),
                location: location!(),
            })?;
        let ivf_bytes = reader.read_global_buffer(ivf_pos).await?;
        let ivf = IvfModel::try_from(pb::Ivf::decode(ivf_bytes)?)?;

        let metadata: Vec<String> = serde_json::from_str(
            schema
                .metadata
                .get(STORAGE_METADATA_KEY)
                .ok_or(Error::Index {
                    message: format!("{} not found", STORAGE_METADATA_KEY),
                    location: location!(),
                })?
                .as_str(),
        )?;
        Ok(Self {
            reader,
            distance_type,
            metadata,
            ivf,
        })
    }

    pub fn quantizer<Q: Quantization>(&self) -> Result<Quantizer> {
        let metadata = serde_json::from_str(&self.metadata[0])?;
        Q::from_metadata(&metadata, self.distance_type)
    }

    pub fn schema(&self) -> SchemaRef {
        Arc::new(self.reader.schema().as_ref().into())
    }

    /// Get the number of partitions in the storage.
    pub fn num_partitions(&self) -> usize {
        self.ivf.num_partitions()
    }

    pub async fn load_partition<Q: Quantization>(&self, part_id: usize) -> Result<Q::Storage> {
        let range = self.ivf.row_range(part_id);
        let batch = if range.is_empty() {
            let schema = self.reader.schema();
            let arrow_schema = arrow_schema::Schema::from(schema.as_ref());
            RecordBatch::new_empty(Arc::new(arrow_schema))
        } else {
            let batches = self
                .reader
                .read_stream(
                    ReadBatchParams::Range(range),
                    u32::MAX,
                    16,
                    FilterExpression::no_filter(),
                )?
                .try_collect::<Vec<_>>()
                .await?;
            let schema = Arc::new(self.reader.schema().as_ref().into());
            concat_batches(&schema, batches.iter())?
        };
        let batch = batch.add_metadata(
            STORAGE_METADATA_KEY.to_owned(),
            // TODO: this is a hack, cause the metadata is just the quantizer metadata
            // it's all the same for all partitions, so now we store only one copy of it
            self.metadata[0].clone(),
        )?;
        Q::Storage::try_from_batch(batch, self.distance_type)
    }
}