mirror of
				https://github.com/meilisearch/meilisearch.git
				synced 2025-10-25 21:16:28 +00:00 
			
		
		
		
	WIP rebase on main
This commit is contained in:
		| @@ -108,6 +108,10 @@ impl Data { | ||||
|         Ok(self.index_controller.get_all_stats().await?) | ||||
|     } | ||||
|  | ||||
|     pub async fn dump(&self) -> anyhow::Result<String> { | ||||
|         Ok(self.index_controller.dump(self.options.dumps_dir.clone()).await?) | ||||
|     } | ||||
|  | ||||
|     #[inline] | ||||
|     pub fn http_payload_size_limit(&self) -> usize { | ||||
|         self.options.http_payload_size_limit.get_bytes() as usize | ||||
|   | ||||
| @@ -1,16 +1,20 @@ | ||||
| mod v1; | ||||
| mod v2; | ||||
|  | ||||
| use std::{fs::File, path::{Path}, sync::Arc}; | ||||
| use std::{collections::HashSet, fs::{File}, path::{Path, PathBuf}, sync::Arc}; | ||||
|  | ||||
| use anyhow::bail; | ||||
| use chrono::Utc; | ||||
| use heed::EnvOpenOptions; | ||||
| use log::{error, info}; | ||||
| use milli::update::{IndexDocumentsMethod, UpdateBuilder, UpdateFormat}; | ||||
| use serde::{Deserialize, Serialize}; | ||||
| use tempfile::TempDir; | ||||
| use tokio::task::spawn_blocking; | ||||
| use tokio::fs; | ||||
| use uuid::Uuid; | ||||
|  | ||||
| use super::IndexMetadata; | ||||
| use super::{IndexController, IndexMetadata, update_actor::UpdateActorHandle, uuid_resolver::UuidResolverHandle}; | ||||
| use crate::index::Index; | ||||
| use crate::index_controller::uuid_resolver; | ||||
| use crate::helpers::compression; | ||||
| @@ -22,7 +26,7 @@ enum DumpVersion { | ||||
| } | ||||
|  | ||||
| impl DumpVersion { | ||||
|     // const CURRENT: Self = Self::V2; | ||||
|     const CURRENT: Self = Self::V2; | ||||
|  | ||||
|     /// Select the good importation function from the `DumpVersion` of metadata | ||||
|     pub fn import_index(self, size: usize, dump_path: &Path, index_path: &Path) -> anyhow::Result<()> { | ||||
| @@ -42,7 +46,6 @@ pub struct Metadata { | ||||
| } | ||||
|  | ||||
| impl Metadata { | ||||
|     /* | ||||
|     /// Create a Metadata with the current dump version of meilisearch. | ||||
|     pub fn new(indexes: Vec<IndexMetadata>, db_version: String) -> Self { | ||||
|         Metadata { | ||||
| @@ -51,7 +54,6 @@ impl Metadata { | ||||
|             dump_version: DumpVersion::CURRENT, | ||||
|         } | ||||
|     } | ||||
|     */ | ||||
|  | ||||
|     /// Extract Metadata from `metadata.json` file present at provided `dir_path` | ||||
|     fn from_path(dir_path: &Path) -> anyhow::Result<Self> { | ||||
| @@ -63,105 +65,73 @@ impl Metadata { | ||||
|         Ok(metadata) | ||||
|     } | ||||
|  | ||||
|     /* | ||||
|     /// Write Metadata in `metadata.json` file at provided `dir_path` | ||||
|     fn to_path(&self, dir_path: &Path) -> anyhow::Result<()> { | ||||
|     pub async fn to_path(&self, dir_path: &Path) -> anyhow::Result<()> { | ||||
|         let path = dir_path.join("metadata.json"); | ||||
|         let file = File::create(path)?; | ||||
|  | ||||
|         serde_json::to_writer(file, &self)?; | ||||
|  | ||||
|         Ok(()) | ||||
|     } | ||||
|     */ | ||||
| } | ||||
|  | ||||
| /* | ||||
| pub struct DumpService<U, R> { | ||||
|     uuid_resolver_handle: R, | ||||
|     update_handle: U, | ||||
|     dump_path: PathBuf, | ||||
|     db_name: String, | ||||
| } | ||||
|  | ||||
| impl<U, R> DumpService<U, R> | ||||
| where | ||||
|     U: UpdateActorHandle, | ||||
|     R: UuidResolverHandle, | ||||
| { | ||||
|     pub fn new( | ||||
|         uuid_resolver_handle: R, | ||||
|         update_handle: U, | ||||
|         dump_path: PathBuf, | ||||
|         db_name: String, | ||||
|     ) -> Self { | ||||
|         Self { | ||||
|             uuid_resolver_handle, | ||||
|             update_handle, | ||||
|             dump_path, | ||||
|             db_name, | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     pub async fn run(self) { | ||||
|         if let Err(e) = self.perform_dump().await { | ||||
|             error!("{}", e); | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     async fn perform_dump(&self) -> anyhow::Result<()> { | ||||
|         /* | ||||
|         info!("Performing dump."); | ||||
|  | ||||
|         let dump_dir = self.dump_path.clone(); | ||||
|         fs::create_dir_all(&dump_dir).await?; | ||||
|         let temp_dump_dir = spawn_blocking(move || tempfile::tempdir_in(dump_dir)).await??; | ||||
|         let temp_dump_path = temp_dump_dir.path().to_owned(); | ||||
|  | ||||
|         let uuids = self | ||||
|             .uuid_resolver_handle | ||||
|             .dump(temp_dump_path.clone()) | ||||
|             .await?; | ||||
|  | ||||
|         if uuids.is_empty() { | ||||
|             return Ok(()); | ||||
|         } | ||||
|  | ||||
|         let tasks = uuids | ||||
|             .iter() | ||||
|             .map(|&uuid| self.update_handle.dump(uuid, temp_dump_path.clone())) | ||||
|             .collect::<Vec<_>>(); | ||||
|  | ||||
|         futures::future::try_join_all(tasks).await?; | ||||
|  | ||||
|         let dump_dir = self.dump_path.clone(); | ||||
|         let dump_path = self.dump_path.join(format!("{}.dump", self.db_name)); | ||||
|         let dump_path = spawn_blocking(move || -> anyhow::Result<PathBuf> { | ||||
|             let temp_dump_file = tempfile::NamedTempFile::new_in(dump_dir)?; | ||||
|             let temp_dump_file_path = temp_dump_file.path().to_owned(); | ||||
|             compression::to_tar_gz(temp_dump_path, temp_dump_file_path)?; | ||||
|             temp_dump_file.persist(&dump_path)?; | ||||
|             Ok(dump_path) | ||||
|         }) | ||||
|         .await??; | ||||
|  | ||||
|         info!("Created dump in {:?}.", dump_path); | ||||
|         */ | ||||
|         tokio::fs::write(path, serde_json::to_string(self)?).await?; | ||||
|  | ||||
|         Ok(()) | ||||
|     } | ||||
| } | ||||
| */ | ||||
|  | ||||
| /// Generate uid from creation date | ||||
| fn generate_uid() -> String { | ||||
|     Utc::now().format("%Y%m%d-%H%M%S%3f").to_string() | ||||
| } | ||||
|  | ||||
| pub async fn perform_dump(index_controller: &IndexController, dump_path: PathBuf) -> anyhow::Result<String> { | ||||
|     info!("Performing dump."); | ||||
|  | ||||
|     let dump_dir = dump_path.clone(); | ||||
|     let uid = generate_uid(); | ||||
|     fs::create_dir_all(&dump_dir).await?; | ||||
|     let temp_dump_dir = spawn_blocking(move || tempfile::tempdir_in(dump_dir)).await??; | ||||
|     let temp_dump_path = temp_dump_dir.path().to_owned(); | ||||
|  | ||||
|     let uuids = index_controller.uuid_resolver.list().await?; | ||||
|     // maybe we could just keep the vec as-is | ||||
|     let uuids: HashSet<(String, Uuid)> = uuids.into_iter().collect(); | ||||
|  | ||||
|     if uuids.is_empty() { | ||||
|         return Ok(uid); | ||||
|     } | ||||
|  | ||||
|     let indexes = index_controller.list_indexes().await?; | ||||
|  | ||||
|     // we create one directory by index | ||||
|     for meta in indexes.iter() { | ||||
|         tokio::fs::create_dir(temp_dump_path.join(&meta.uid)).await?; | ||||
|     } | ||||
|  | ||||
|     let metadata = Metadata::new(indexes, env!("CARGO_PKG_VERSION").to_string()); | ||||
|     metadata.to_path(&temp_dump_path).await?; | ||||
|  | ||||
|     index_controller.update_handle.dump(uuids, temp_dump_path.clone()).await?; | ||||
|     let dump_dir = dump_path.clone(); | ||||
|     let dump_path = dump_path.join(format!("{}.dump", uid)); | ||||
|     let dump_path = spawn_blocking(move || -> anyhow::Result<PathBuf> { | ||||
|         let temp_dump_file = tempfile::NamedTempFile::new_in(dump_dir)?; | ||||
|         let temp_dump_file_path = temp_dump_file.path().to_owned(); | ||||
|         compression::to_tar_gz(temp_dump_path, temp_dump_file_path)?; | ||||
|         temp_dump_file.persist(&dump_path)?; | ||||
|         Ok(dump_path) | ||||
|     }) | ||||
|     .await??; | ||||
|  | ||||
|     info!("Created dump in {:?}.", dump_path); | ||||
|  | ||||
|     Ok(uid) | ||||
| } | ||||
|  | ||||
| /* | ||||
| /// Write Settings in `settings.json` file at provided `dir_path` | ||||
| fn settings_to_path(settings: &Settings, dir_path: &Path) -> anyhow::Result<()> { | ||||
|     let path = dir_path.join("settings.json"); | ||||
|     let file = File::create(path)?; | ||||
| let path = dir_path.join("settings.json"); | ||||
| let file = File::create(path)?; | ||||
|  | ||||
|     serde_json::to_writer(file, settings)?; | ||||
| serde_json::to_writer(file, settings)?; | ||||
|  | ||||
|     Ok(()) | ||||
| Ok(()) | ||||
| } | ||||
| */ | ||||
|  | ||||
|   | ||||
| @@ -122,8 +122,8 @@ impl<S: IndexStore + Sync + Send> IndexActor<S> { | ||||
|             Snapshot { uuid, path, ret } => { | ||||
|                 let _ = ret.send(self.handle_snapshot(uuid, path).await); | ||||
|             } | ||||
|             Dump { uuid, path, ret } => { | ||||
|                 let _ = ret.send(self.handle_dump(uuid, path).await); | ||||
|             Dump { uid, uuid, path, ret } => { | ||||
|                 let _ = ret.send(self.handle_dump(&uid, uuid, path).await); | ||||
|             } | ||||
|             GetStats { uuid, ret } => { | ||||
|                 let _ = ret.send(self.handle_get_stats(uuid).await); | ||||
| @@ -312,24 +312,52 @@ impl<S: IndexStore + Sync + Send> IndexActor<S> { | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     async fn handle_dump(&self, uuid: Uuid, mut path: PathBuf) -> IndexResult<()> { | ||||
|     /// Create a `documents.jsonl` and a `settings.json` in `path/uid/` with a dump of all the | ||||
|     /// documents and all the settings. | ||||
|     async fn handle_dump(&self, uid: &str, uuid: Uuid, path: PathBuf) -> IndexResult<()> { | ||||
|         use tokio::fs::create_dir_all; | ||||
|         use std::io::prelude::*; | ||||
|  | ||||
|         path.push("indexes"); | ||||
|         create_dir_all(&path) | ||||
|             .await | ||||
|             .map_err(|e| IndexError::Error(e.into()))?; | ||||
|  | ||||
|         if let Some(index) = self.store.get(uuid).await? { | ||||
|             let mut index_path = path.join(format!("index-{}", uuid)); | ||||
|             create_dir_all(&index_path) | ||||
|                 .await | ||||
|                 .map_err(|e| IndexError::Error(e.into()))?; | ||||
|             index_path.push("data.mdb"); | ||||
|             let documents_path = path.join(uid).join("documents.jsonl"); | ||||
|             let settings_path = path.join(uid).join("settings.json"); | ||||
|  | ||||
|             spawn_blocking(move || -> anyhow::Result<()> { | ||||
|                 // first we dump all the documents | ||||
|                 let file = File::create(documents_path)?; | ||||
|                 let mut file = std::io::BufWriter::new(file); | ||||
|  | ||||
|                 // Get write txn to wait for ongoing write transaction before dump. | ||||
|                 let _txn = index.write_txn()?; | ||||
|                 index.env.copy_to_path(index_path, CompactionOption::Enabled)?; | ||||
|                 let txn = index.write_txn()?; | ||||
|                 let documents_ids = index.documents_ids(&txn)?; | ||||
|                 // TODO: TAMO: calling this function here can consume **a lot** of RAM, we should | ||||
|                 // use some kind of iterators -> waiting for a milli release | ||||
|                 let documents = index.documents(&txn, documents_ids)?; | ||||
|  | ||||
|                 let fields_ids_map = index.fields_ids_map(&txn)?; | ||||
|                 // we want to save **all** the fields in the dump. | ||||
|                 let fields_to_dump: Vec<u8> = fields_ids_map.iter().map(|(id, _)| id).collect(); | ||||
|  | ||||
|                 for (_doc_id, document) in documents { | ||||
|                     let json = milli::obkv_to_json(&fields_to_dump, &fields_ids_map, document)?; | ||||
|                     file.write_all(serde_json::to_string(&json)?.as_bytes())?; | ||||
|                     file.write_all(b"\n")?; | ||||
|                 } | ||||
|  | ||||
|  | ||||
|                 // then we dump all the settings | ||||
|                 let file = File::create(settings_path)?; | ||||
|                 let mut file = std::io::BufWriter::new(file); | ||||
|                 let settings = index.settings()?; | ||||
|  | ||||
|                 file.write_all(serde_json::to_string(&settings)?.as_bytes())?; | ||||
|                 file.write_all(b"\n")?; | ||||
|  | ||||
|  | ||||
|                 Ok(()) | ||||
|             }) | ||||
|             .await | ||||
|   | ||||
| @@ -136,9 +136,9 @@ impl IndexActorHandle for IndexActorHandleImpl { | ||||
|         Ok(receiver.await.expect("IndexActor has been killed")?) | ||||
|     } | ||||
|  | ||||
|     async fn dump(&self, uuid: Uuid, path: PathBuf) -> IndexResult<()> { | ||||
|     async fn dump(&self, uid: String, uuid: Uuid, path: PathBuf) -> IndexResult<()> { | ||||
|         let (ret, receiver) = oneshot::channel(); | ||||
|         let msg = IndexMsg::Dump { uuid, path, ret }; | ||||
|         let msg = IndexMsg::Dump { uid, uuid, path, ret }; | ||||
|         let _ = self.sender.send(msg).await; | ||||
|         Ok(receiver.await.expect("IndexActor has been killed")?) | ||||
|     } | ||||
|   | ||||
| @@ -61,6 +61,7 @@ pub enum IndexMsg { | ||||
|         ret: oneshot::Sender<IndexResult<()>>, | ||||
|     }, | ||||
|     Dump { | ||||
|         uid: String, | ||||
|         uuid: Uuid, | ||||
|         path: PathBuf, | ||||
|         ret: oneshot::Sender<IndexResult<()>>, | ||||
|   | ||||
| @@ -97,7 +97,7 @@ pub trait IndexActorHandle { | ||||
|         index_settings: IndexSettings, | ||||
|     ) -> IndexResult<IndexMeta>; | ||||
|     async fn snapshot(&self, uuid: Uuid, path: PathBuf) -> IndexResult<()>; | ||||
|     async fn dump(&self, uuid: Uuid, path: PathBuf) -> IndexResult<()>; | ||||
|     async fn dump(&self, uid: String, uuid: Uuid, path: PathBuf) -> IndexResult<()>; | ||||
|     async fn get_index_stats(&self, uuid: Uuid) -> IndexResult<IndexStats>; | ||||
| } | ||||
|  | ||||
|   | ||||
| @@ -1,4 +1,4 @@ | ||||
| use std::collections::BTreeMap; | ||||
| use std::{collections::BTreeMap, path::PathBuf}; | ||||
| use std::path::Path; | ||||
| use std::sync::Arc; | ||||
| use std::time::Duration; | ||||
| @@ -378,6 +378,13 @@ impl IndexController { | ||||
|         Ok(stats) | ||||
|     } | ||||
|  | ||||
|     pub async fn dump(&self, path: PathBuf) -> anyhow::Result<String> { | ||||
|         eprintln!("index_controller::mod called"); | ||||
|         let res = dump::perform_dump(self, path).await?; | ||||
|         eprintln!("index_controller::mod finished"); | ||||
|         Ok(res) | ||||
|     } | ||||
|  | ||||
|     pub async fn get_all_stats(&self) -> anyhow::Result<Stats> { | ||||
|         let update_infos = self.update_handle.get_info().await?; | ||||
|         let mut database_size = self.get_uuids_size().await? + update_infos.size; | ||||
|   | ||||
| @@ -235,11 +235,11 @@ where | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     async fn handle_dump(&self, uuids: HashSet<Uuid>, path: PathBuf) -> Result<()> { | ||||
|     async fn handle_dump(&self, uuids: HashSet<(String, Uuid)>, path: PathBuf) -> Result<()> { | ||||
|         let index_handle = self.index_handle.clone(); | ||||
|         let update_store = self.store.clone(); | ||||
|         tokio::task::spawn_blocking(move || -> anyhow::Result<()> { | ||||
|             update_store.dump(&uuids, &path)?; | ||||
|             update_store.dump(&uuids, path.to_path_buf())?; | ||||
|  | ||||
|             // Perform the snapshot of each index concurently. Only a third of the capabilities of | ||||
|             // the index actor at a time not to put too much pressure on the index actor | ||||
| @@ -247,7 +247,7 @@ where | ||||
|             let handle = &index_handle; | ||||
|  | ||||
|             let mut stream = futures::stream::iter(uuids.iter()) | ||||
|                 .map(|&uuid| handle.dump(uuid, path.clone())) | ||||
|                 .map(|(uid, uuid)| handle.dump(uid.clone(), *uuid, path.clone())) | ||||
|                 .buffer_unordered(CONCURRENT_INDEX_MSG / 3); | ||||
|  | ||||
|             Handle::current().block_on(async { | ||||
|   | ||||
| @@ -71,16 +71,16 @@ where | ||||
|         receiver.await.expect("update actor killed.") | ||||
|     } | ||||
|  | ||||
|     async fn get_info(&self) -> Result<UpdateStoreInfo> { | ||||
|     async fn dump(&self, uuids: HashSet<(String, Uuid)>, path: PathBuf) -> Result<()> { | ||||
|         let (ret, receiver) = oneshot::channel(); | ||||
|         let msg = UpdateMsg::GetInfo { ret }; | ||||
|         let msg = UpdateMsg::Dump { uuids, path, ret }; | ||||
|         let _ = self.sender.send(msg).await; | ||||
|         receiver.await.expect("update actor killed.") | ||||
|     } | ||||
|  | ||||
|     async fn dump(&self, uuids: HashSet<Uuid>, path: PathBuf) -> Result<()> { | ||||
|     async fn get_info(&self) -> Result<UpdateStoreInfo> { | ||||
|         let (ret, receiver) = oneshot::channel(); | ||||
|         let msg = UpdateMsg::Dump { uuids, path, ret }; | ||||
|         let msg = UpdateMsg::GetInfo { ret }; | ||||
|         let _ = self.sender.send(msg).await; | ||||
|         receiver.await.expect("update actor killed.") | ||||
|     } | ||||
|   | ||||
| @@ -32,7 +32,7 @@ pub enum UpdateMsg<D> { | ||||
|         ret: oneshot::Sender<Result<()>>, | ||||
|     }, | ||||
|     Dump { | ||||
|         uuids: HashSet<Uuid>, | ||||
|         uuids: HashSet<(String, Uuid)>, | ||||
|         path: PathBuf, | ||||
|         ret: oneshot::Sender<Result<()>>, | ||||
|     }, | ||||
|   | ||||
| @@ -40,7 +40,7 @@ pub trait UpdateActorHandle { | ||||
|     async fn update_status(&self, uuid: Uuid, id: u64) -> Result<UpdateStatus>; | ||||
|     async fn delete(&self, uuid: Uuid) -> Result<()>; | ||||
|     async fn snapshot(&self, uuid: HashSet<Uuid>, path: PathBuf) -> Result<()>; | ||||
|     async fn dump(&self, uuid: HashSet<Uuid>, path: PathBuf) -> Result<()>; | ||||
|     async fn dump(&self, uuid: HashSet<(String, Uuid)>, path: PathBuf) -> Result<()>; | ||||
|     async fn get_info(&self) -> Result<UpdateStoreInfo>; | ||||
|     async fn update( | ||||
|         &self, | ||||
|   | ||||
| @@ -1,4 +1,4 @@ | ||||
| use std::borrow::Cow; | ||||
| use std::{borrow::Cow, path::PathBuf}; | ||||
| use std::collections::{BTreeMap, HashSet}; | ||||
| use std::convert::TryInto; | ||||
| use std::fs::{copy, create_dir_all, remove_file, File}; | ||||
| @@ -294,6 +294,7 @@ impl UpdateStore { | ||||
|         result: UpdateStatus, | ||||
|         index_uuid: Uuid, | ||||
|     ) -> heed::Result<()> { | ||||
|         // TODO: TAMO: load already processed updates | ||||
|         let mut wtxn = self.env.write_txn()?; | ||||
|         let (_global_id, update_id) = self.next_update_id(&mut wtxn, index_uuid)?; | ||||
|         self.updates.remap_key_type::<UpdateKeyCodec>().put(&mut wtxn, &(index_uuid, update_id), &result)?; | ||||
| @@ -516,31 +517,34 @@ impl UpdateStore { | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     pub fn dump(&self, uuids: &HashSet<Uuid>, path: impl AsRef<Path>) -> anyhow::Result<()> { | ||||
|     pub fn dump(&self, uuids: &HashSet<(String, Uuid)>, path: PathBuf) -> anyhow::Result<()> { | ||||
|         use std::io::prelude::*; | ||||
|         let state_lock = self.state.write(); | ||||
|         state_lock.swap(State::Snapshoting); // TODO: rename the state | ||||
|         state_lock.swap(State::Snapshoting); // TODO: TAMO rename the state somehow | ||||
|  | ||||
|         let txn = self.env.write_txn()?; | ||||
|  | ||||
|         let update_path = path.as_ref().join("updates"); | ||||
|         create_dir_all(&update_path)?; | ||||
|         for (uid, uuid) in uuids.iter() { | ||||
|             let file = File::create(path.join(uid).join("updates.jsonl"))?; | ||||
|             let mut file = std::io::BufWriter::new(file); | ||||
|  | ||||
|         // acquire write lock to prevent further writes during dump | ||||
|         create_dir_all(&update_path)?; | ||||
|         let db_path = update_path.join("data.mdb"); | ||||
|             for update in &self.list(*uuid)? { | ||||
|                 serde_json::to_writer(&mut file, update)?; | ||||
|                 file.write_all(b"\n")?; | ||||
|             } | ||||
|         } | ||||
|  | ||||
|         // TODO: everything | ||||
|         // create db dump | ||||
|         self.env.copy_to_path(&db_path, CompactionOption::Enabled)?; | ||||
|         // TODO: TAMO: the updates | ||||
|         // already processed updates seems to works, but I've not tried with currently running updates | ||||
|  | ||||
|         let update_files_path = update_path.join("update_files"); | ||||
|         let update_files_path = path.join("update_files"); | ||||
|         create_dir_all(&update_files_path)?; | ||||
|  | ||||
|         let pendings = self.pending_queue.iter(&txn)?.lazily_decode_data(); | ||||
|  | ||||
|         for entry in pendings { | ||||
|             let ((_, uuid, _), pending) = entry?; | ||||
|             if uuids.contains(&uuid) { | ||||
|             if uuids.iter().any(|(_, id)| id == &uuid) { | ||||
|                 if let Some(path) = pending.decode()?.content_path() { | ||||
|                     let name = path.file_name().unwrap(); | ||||
|                     let to = update_files_path.join(name); | ||||
|   | ||||
| @@ -41,9 +41,6 @@ impl<S: UuidStore> UuidResolverActor<S> { | ||||
|                 Some(SnapshotRequest { path, ret }) => { | ||||
|                     let _ = ret.send(self.handle_snapshot(path).await); | ||||
|                 } | ||||
|                 Some(DumpRequest { path, ret }) => { | ||||
|                     let _ = ret.send(self.handle_dump(path).await); | ||||
|                 } | ||||
|                 Some(GetSize { ret }) => { | ||||
|                     let _ = ret.send(self.handle_get_size().await); | ||||
|                 } | ||||
| @@ -85,10 +82,6 @@ impl<S: UuidStore> UuidResolverActor<S> { | ||||
|         self.store.snapshot(path).await | ||||
|     } | ||||
|  | ||||
|     async fn handle_dump(&self, path: PathBuf) -> Result<HashSet<Uuid>> { | ||||
|         self.store.dump(path).await | ||||
|     } | ||||
|  | ||||
|     async fn handle_insert(&self, uid: String, uuid: Uuid) -> Result<()> { | ||||
|         if !is_index_uid_valid(&uid) { | ||||
|             return Err(UuidError::BadlyFormatted(uid)); | ||||
|   | ||||
| @@ -68,7 +68,6 @@ impl UuidResolverHandle for UuidResolverHandleImpl { | ||||
|             .expect("Uuid resolver actor has been killed")?) | ||||
|     } | ||||
|  | ||||
|     /// TODO: we should merge this function with the dump function | ||||
|     async fn snapshot(&self, path: PathBuf) -> Result<HashSet<Uuid>> { | ||||
|         let (ret, receiver) = oneshot::channel(); | ||||
|         let msg = UuidResolveMsg::SnapshotRequest { path, ret }; | ||||
| @@ -78,15 +77,6 @@ impl UuidResolverHandle for UuidResolverHandleImpl { | ||||
|             .expect("Uuid resolver actor has been killed")?) | ||||
|     } | ||||
|  | ||||
|     async fn dump(&self, path: PathBuf) -> Result<HashSet<Uuid>> { | ||||
|         let (ret, receiver) = oneshot::channel(); | ||||
|         let msg = UuidResolveMsg::DumpRequest { path, ret }; | ||||
|         let _ = self.sender.send(msg).await; | ||||
|         Ok(receiver | ||||
|             .await | ||||
|             .expect("Uuid resolver actor has been killed")?) | ||||
|     } | ||||
|  | ||||
|     async fn get_size(&self) -> Result<u64> { | ||||
|         let (ret, receiver) = oneshot::channel(); | ||||
|         let msg = UuidResolveMsg::GetSize { ret }; | ||||
|   | ||||
| @@ -31,10 +31,6 @@ pub enum UuidResolveMsg { | ||||
|         path: PathBuf, | ||||
|         ret: oneshot::Sender<Result<HashSet<Uuid>>>, | ||||
|     }, | ||||
|     DumpRequest { | ||||
|         path: PathBuf, | ||||
|         ret: oneshot::Sender<Result<HashSet<Uuid>>>, | ||||
|     }, | ||||
|     GetSize { | ||||
|         ret: oneshot::Sender<Result<u64>>, | ||||
|     }, | ||||
|   | ||||
| @@ -32,7 +32,6 @@ pub trait UuidResolverHandle { | ||||
|     async fn delete(&self, name: String) -> anyhow::Result<Uuid>; | ||||
|     async fn list(&self) -> anyhow::Result<Vec<(String, Uuid)>>; | ||||
|     async fn snapshot(&self, path: PathBuf) -> Result<HashSet<Uuid>>; | ||||
|     async fn dump(&self, path: PathBuf) -> Result<HashSet<Uuid>>; | ||||
|     async fn get_size(&self) -> Result<u64>; | ||||
| } | ||||
|  | ||||
|   | ||||
| @@ -21,7 +21,6 @@ pub trait UuidStore { | ||||
|     async fn list(&self) -> Result<Vec<(String, Uuid)>>; | ||||
|     async fn insert(&self, name: String, uuid: Uuid) -> Result<()>; | ||||
|     async fn snapshot(&self, path: PathBuf) -> Result<HashSet<Uuid>>; | ||||
|     async fn dump(&self, path: PathBuf) -> Result<HashSet<Uuid>>; | ||||
|     async fn get_size(&self) -> Result<u64>; | ||||
| } | ||||
|  | ||||
| @@ -114,8 +113,6 @@ impl HeedUuidStore { | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     // TODO: we should merge this function and the following function for the dump. it's exactly | ||||
|     // the same code | ||||
|     pub fn snapshot(&self, mut path: PathBuf) -> Result<HashSet<Uuid>> { | ||||
|         let env = self.env.clone(); | ||||
|         let db = self.db; | ||||
| @@ -138,28 +135,6 @@ impl HeedUuidStore { | ||||
|         Ok(entries) | ||||
|     } | ||||
|  | ||||
|     pub fn dump(&self, mut path: PathBuf) -> Result<HashSet<Uuid>> { | ||||
|         let env = self.env.clone(); | ||||
|         let db = self.db; | ||||
|         // Write transaction to acquire a lock on the database. | ||||
|         let txn = env.write_txn()?; | ||||
|         let mut entries = HashSet::new(); | ||||
|         for entry in db.iter(&txn)? { | ||||
|             let (_, uuid) = entry?; | ||||
|             let uuid = Uuid::from_slice(uuid)?; | ||||
|             entries.insert(uuid); | ||||
|         } | ||||
|  | ||||
|         // only perform dump if there are indexes | ||||
|         if !entries.is_empty() { | ||||
|             path.push("index_uuids"); | ||||
|             create_dir_all(&path).unwrap(); | ||||
|             path.push("data.mdb"); | ||||
|             env.copy_to_path(path, CompactionOption::Enabled)?; | ||||
|         } | ||||
|         Ok(entries) | ||||
|     } | ||||
|  | ||||
|     pub fn get_size(&self) -> Result<u64> { | ||||
|         Ok(self.env.size()) | ||||
|     } | ||||
| @@ -197,11 +172,6 @@ impl UuidStore for HeedUuidStore { | ||||
|         tokio::task::spawn_blocking(move || this.snapshot(path)).await? | ||||
|     } | ||||
|  | ||||
|     async fn dump(&self, path: PathBuf) -> Result<HashSet<Uuid>> { | ||||
|         let this = self.clone(); | ||||
|         tokio::task::spawn_blocking(move || this.dump(path)).await? | ||||
|     } | ||||
|  | ||||
|     async fn get_size(&self) -> Result<u64> { | ||||
|         self.get_size() | ||||
|     } | ||||
|   | ||||
| @@ -46,8 +46,8 @@ macro_rules! create_app { | ||||
|             .configure(synonym::services) | ||||
|             .configure(health::services) | ||||
|             .configure(stats::services) | ||||
|             .configure(key::services); | ||||
|         //.configure(routes::dump::services); | ||||
|             .configure(key::services) | ||||
|             .configure(dump::services); | ||||
|         #[cfg(feature = "mini-dashboard")] | ||||
|         let app = if $enable_frontend { | ||||
|             let generated = dashboard::generate(); | ||||
|   | ||||
| @@ -1,14 +1,10 @@ | ||||
| use std::fs::File; | ||||
| use std::path::Path; | ||||
| use actix_web::{post, get, web}; | ||||
| use actix_web::HttpResponse; | ||||
| use serde::{Serialize, Deserialize}; | ||||
|  | ||||
| use actix_web::{get, post}; | ||||
| use actix_web::{HttpResponse, web}; | ||||
| use serde::{Deserialize, Serialize}; | ||||
|  | ||||
| use crate::dump::{DumpInfo, DumpStatus, compressed_dumps_dir, init_dump_process}; | ||||
| use crate::Data; | ||||
| use crate::error::{Error, ResponseError}; | ||||
| use crate::error::ResponseError; | ||||
| use crate::helpers::Authentication; | ||||
| use crate::Data; | ||||
|  | ||||
| pub fn services(cfg: &mut web::ServiceConfig) { | ||||
|     cfg.service(trigger_dump) | ||||
| @@ -19,7 +15,10 @@ pub fn services(cfg: &mut web::ServiceConfig) { | ||||
| async fn trigger_dump( | ||||
|     data: web::Data<Data>, | ||||
| ) -> Result<HttpResponse, ResponseError> { | ||||
|     todo!() | ||||
|     eprintln!("dump started"); | ||||
|     let res = data.dump().await?; | ||||
|  | ||||
|     Ok(HttpResponse::Ok().body(res)) | ||||
| } | ||||
|  | ||||
| #[derive(Debug, Serialize)] | ||||
| @@ -30,13 +29,13 @@ struct DumpStatusResponse { | ||||
|  | ||||
| #[derive(Deserialize)] | ||||
| struct DumpParam { | ||||
|     dump_uid: String, | ||||
|     _dump_uid: String, | ||||
| } | ||||
|  | ||||
| #[get("/dumps/{dump_uid}/status", wrap = "Authentication::Private")] | ||||
| async fn get_dump_status( | ||||
|     data: web::Data<Data>, | ||||
|     path: web::Path<DumpParam>, | ||||
|     _data: web::Data<Data>, | ||||
|     _path: web::Path<DumpParam>, | ||||
| ) -> Result<HttpResponse, ResponseError> { | ||||
|     todo!() | ||||
| } | ||||
|   | ||||
| @@ -9,7 +9,7 @@ pub mod search; | ||||
| pub mod settings; | ||||
| pub mod stats; | ||||
| pub mod synonym; | ||||
| //pub mod dump; | ||||
| pub mod dump; | ||||
|  | ||||
| #[derive(Deserialize)] | ||||
| pub struct IndexParam { | ||||
|   | ||||
		Reference in New Issue
	
	Block a user