mirror of
https://github.com/meilisearch/MeiliSearch
synced 2024-11-22 21:04:27 +01:00
WIP rebase on main
This commit is contained in:
parent
0f94ef8abc
commit
c3552cecdf
@ -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 {
|
||||
|
Loading…
Reference in New Issue
Block a user