From 8424d49c64c8f0e383ef0b572d55eead53d0067d Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 19 Mar 2020 15:29:32 -0400 Subject: [PATCH 01/42] Add Partition Store This adds the Partition trait along with helper functions and structs to combine the results from multiple partitions. It also refactors MemDB to have methods that align with the partition trait so that it can be used later by database.rs. There are obvious places where it is inefficient, but this should be enough to get started with refactoring database.rs to use this rather than rocksdb, series_store, and inverted_index. --- src/storage.rs | 1 + src/storage/memdb.rs | 683 ++++++++++++------------------- src/storage/partitioned_store.rs | 556 +++++++++++++++++++++++++ src/storage/series_store.rs | 2 +- 4 files changed, 809 insertions(+), 433 deletions(-) create mode 100644 src/storage/partitioned_store.rs diff --git a/src/storage.rs b/src/storage.rs index 36a66abb85..d656b49c1e 100644 --- a/src/storage.rs +++ b/src/storage.rs @@ -6,6 +6,7 @@ pub mod config_store; pub mod database; pub mod inverted_index; pub mod memdb; +pub mod partitioned_store; pub mod predicate; pub mod rocksdb; pub mod series_store; diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index 955c6a7254..df1826745b 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -1,31 +1,73 @@ use crate::delorean::{Node, Predicate, TimestampRange}; use crate::line_parser::{ParseError, Point, PointType}; -use crate::storage::inverted_index::{InvertedIndex, SeriesFilter}; use crate::storage::predicate::{Evaluate, EvaluateVisitor}; -use crate::storage::series_store::{ReadPoint, SeriesStore}; -use crate::storage::{SeriesDataType, StorageError}; - -use std::collections::{BTreeMap, HashMap}; -use std::sync::{Arc, Mutex, RwLock}; +use crate::storage::series_store::ReadPoint; +use crate::storage::{ + partitioned_store::ReadBatch, partitioned_store::ReadValues, SeriesDataType, StorageError, +}; use croaring::Treemap; -/// memdb implements an in memory database for the InvertedIndex and SeriesStore traits. It -/// works with a ring buffer. Currently, it does not limit the number of series that can be -/// created. It also assumes that data per series arrives in time acending order. +use futures::stream::{self, BoxStream}; +use futures::StreamExt; +use std::collections::{BTreeMap, HashMap}; + +/// memdb implements an in memory database for the Partition trait. It currently assumes that +/// data arrives in time ascending order per series. It has no limits on the number of series +/// or the amount of data per series. It is up to the higher level database to decide when to +/// stop writing into a given MemDB. // TODO: return errors if trying to insert data out of order in an individual series #[derive(Default)] pub struct MemDB { - default_ring_buffer_size: usize, - bucket_id_to_series_data: Arc>>>, - bucket_id_to_series_map: Arc>>>, + series_data: SeriesData, + series_map: SeriesMap, } +#[derive(Default)] struct SeriesData { - ring_buffer_size: usize, - i64_series: HashMap>, - f64_series: HashMap>, + current_size: u64, + i64_series: HashMap>, + f64_series: HashMap>, +} + +struct SeriesBuffer { + values: Vec>, +} + +impl SeriesBuffer { + fn read(&self, range: &TimestampRange) -> Vec> { + let mut start = self.values.len(); + let mut stop = start; + + for (pos, val) in self.values.iter().enumerate() { + if val.time >= range.start { + start = pos; + break; + } + } + + for (pos, val) in self.values[start..].iter().enumerate() { + if val.time >= range.end { + stop = pos; + break; + } + } + + let size = stop - start; + + if size == 0 { + return vec![]; + } + + let mut values = Vec::with_capacity(size); + + for val in self.values[start..stop].iter() { + values.push(val.clone()); + } + + values + } } trait StoreInSeriesData { @@ -43,11 +85,17 @@ impl StoreInSeriesData for PointType { impl StoreInSeriesData for Point { fn write(&self, series_data: &mut SeriesData) { + let point = ReadPoint { + time: self.time, + value: self.value, + }; + series_data.current_size += std::mem::size_of::>() as u64; + match series_data.i64_series.get_mut(&self.series_id.unwrap()) { - Some(buff) => buff.write(&self), + Some(buff) => buff.values.push(point), None => { - let mut buff = new_i64_ring_buffer(series_data.ring_buffer_size); - buff.write(&self); + let mut buff = SeriesBuffer { values: Vec::new() }; + buff.values.push(point); series_data.i64_series.insert(self.series_id.unwrap(), buff); } } @@ -56,105 +104,26 @@ impl StoreInSeriesData for Point { impl StoreInSeriesData for Point { fn write(&self, series_data: &mut SeriesData) { + let point = ReadPoint { + time: self.time, + value: self.value, + }; + series_data.current_size += std::mem::size_of::>() as u64; + match series_data.f64_series.get_mut(&self.series_id.unwrap()) { - Some(buff) => buff.write(&self), + Some(buff) => buff.values.push(point), None => { - let mut buff = new_f64_ring_buffer(series_data.ring_buffer_size); - buff.write(&self); + let mut buff = SeriesBuffer { values: Vec::new() }; + buff.values.push(point); series_data.f64_series.insert(self.series_id.unwrap(), buff); } } } } -impl SeriesData { - fn write_points(&mut self, points: &[PointType]) { - for p in points { - p.write(self); - } - } -} - -struct SeriesRingBuffer { - next_position: usize, - data: Vec>, -} - -fn new_i64_ring_buffer(size: usize) -> SeriesRingBuffer { - let mut data = Vec::with_capacity(size); - for _ in 0..size { - data.push(ReadPoint { - time: std::i64::MAX, - value: 0 as i64, - }) - } - - SeriesRingBuffer { - next_position: 0, - data, - } -} - -fn new_f64_ring_buffer(size: usize) -> SeriesRingBuffer { - let mut data = Vec::with_capacity(size); - for _ in 0..size { - data.push(ReadPoint { - time: std::i64::MAX, - value: 0 as f64, - }) - } - - SeriesRingBuffer { - next_position: 0, - data, - } -} - -impl SeriesRingBuffer { - fn write(&mut self, point: &Point) { - if self.next_position == self.data.len() { - self.next_position = 0; - } - self.data[self.next_position].time = point.time; - self.data[self.next_position].value = point.value.clone(); - self.next_position += 1; - } - - fn get_range(&self, range: &TimestampRange) -> Vec> { - let (_, pos) = self.oldest_time_and_position(); - - let mut values = Vec::new(); - - for i in pos..self.data.len() { - if self.data[i].time > range.end { - return values; - } else if self.data[i].time >= range.start { - values.push(self.data[i].clone()); - } - } - - for i in 0..self.next_position { - if self.data[i].time > range.end { - return values; - } else if self.data[i].time >= range.start { - values.push(self.data[i].clone()); - } - } - - values - } - - fn oldest_time_and_position(&self) -> (i64, usize) { - let mut pos = self.next_position; - if self.next_position == self.data.len() || self.data[pos].time == std::i64::MAX { - pos = 0; - } - - (self.data[pos].time, pos) - } -} - +#[derive(Default)] struct SeriesMap { + current_size: u64, last_id: u64, series_key_to_id: HashMap, series_id_to_key_and_type: HashMap, @@ -163,16 +132,6 @@ struct SeriesMap { } impl SeriesMap { - fn new() -> SeriesMap { - SeriesMap { - last_id: 0, - series_key_to_id: HashMap::new(), - series_id_to_key_and_type: HashMap::new(), - tag_keys: BTreeMap::new(), - posting_list: HashMap::new(), - } - } - fn insert_series(&mut self, point: &mut PointType) -> Result<(), ParseError> { if let Some(id) = self.series_key_to_id.get(point.series()) { point.set_series_id(*id); @@ -192,10 +151,16 @@ impl SeriesMap { self.series_id_to_key_and_type .insert(self.last_id, (point.series().clone(), series_type)); + // update the estimated size of the map + self.current_size += (point.series().len() * 2 + 16) as u64; + for pair in point.index_pairs()? { // insert this id into the posting list let list_key = list_key(&pair.key, &pair.value); + // update estimated size for the index pairs + self.current_size += (list_key.len() + pair.key.len() + pair.value.len() + 8) as u64; + let posting_list = self .posting_list .entry(list_key) @@ -227,129 +192,56 @@ fn list_key(key: &str, value: &str) -> Vec { } impl MemDB { - pub fn new() -> MemDB { - MemDB { - default_ring_buffer_size: 10000, - bucket_id_to_series_data: Arc::new(RwLock::new(HashMap::new())), - bucket_id_to_series_map: Arc::new(RwLock::new(HashMap::new())), - } + pub fn new() -> Self { + Default::default() } - fn get_or_create_series_ids_for_points( - &self, - bucket_id: u32, - points: &mut [PointType], - ) -> Result<(), StorageError> { - // first try to do everything with just a read lock - if self.get_series_ids_for_points(bucket_id, points) { - return Ok(()); - } - - // if we got this far, we have new series to insert - let buckets = self.bucket_id_to_series_map.read().unwrap(); - let mut series_map = buckets.get(&bucket_id).unwrap().write().unwrap(); + pub fn size(&self) -> u64 { + self.series_data.current_size + self.series_map.current_size + } + fn write(&mut self, points: &mut [PointType]) -> Result<(), StorageError> { for p in points { - match p.series_id() { - Some(_) => (), - None => match series_map.insert_series(p) { - Ok(_) => (), - Err(e) => { - return Err(StorageError { - description: format!("error parsing line protocol metadata {}", e), - }) - } - }, - } + self.series_map.insert_series(p).map_err(|e| StorageError { + description: format!("error parsing line protocol metadata {}", e), + })?; + p.write(&mut self.series_data); } Ok(()) } - // get_series_ids_for_points attempts to fill the series ids for all points in the passed in - // collection using only a read lock. If no SeriesMap exists for the bucket, it will be inserted. - // It will return true if all points have series ids filled in. - fn get_series_ids_for_points(&self, bucket_id: u32, points: &mut [PointType]) -> bool { - let buckets = self.bucket_id_to_series_map.read().unwrap(); - match buckets.get(&bucket_id) { - Some(b) => { - let b = b.read().unwrap(); - let mut all_have_id = true; - for p in points { - match b.series_key_to_id.get(p.series()) { - Some(id) => p.set_series_id(*id), - None => all_have_id = false, - } - } - - if all_have_id { - return true; - } - } - None => { - // we've never even seen this bucket. insert it and then we'll get to inserting the series - drop(buckets); - self.bucket_id_to_series_map - .write() - .unwrap() - .insert(bucket_id, RwLock::new(SeriesMap::new())); - } - } - - false - } - fn get_tag_keys( &self, - bucket_id: u32, - _predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - match self.bucket_id_to_series_map.read().unwrap().get(&bucket_id) { - Some(map) => { - let keys: Vec = map.read().unwrap().tag_keys.keys().cloned().collect(); - Ok(Box::new(keys.into_iter())) - } - None => Err(StorageError { - description: format!("bucket {} not found", bucket_id), - }), - } + _range: &TimestampRange, + _predicate: &Predicate, + ) -> Result, StorageError> { + let keys: Vec = self.series_map.tag_keys.keys().cloned().collect(); + Ok(stream::iter(keys.into_iter()).boxed()) } fn get_tag_values( &self, - bucket_id: u32, tag_key: &str, - _predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - match self.bucket_id_to_series_map.read().unwrap().get(&bucket_id) { - Some(map) => match map.read().unwrap().tag_keys.get(tag_key) { - Some(values) => { - let values: Vec = values.keys().cloned().collect(); - Ok(Box::new(values.into_iter())) - } - None => Ok(Box::new(vec![].into_iter())), - }, - None => Err(StorageError { - description: format!("bucket {} not found", bucket_id), - }), + _range: &TimestampRange, + _predicate: &Predicate, + ) -> Result, StorageError> { + match self.series_map.tag_keys.get(tag_key) { + Some(values) => { + let values: Vec = values.keys().cloned().collect(); + Ok(stream::iter(values.into_iter()).boxed()) + } + None => Ok(stream::iter(vec![].into_iter()).boxed()), } } - fn read_series_matching( + fn read( &self, - bucket_id: u32, - predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - let pred = match predicate { - Some(p) => p, - None => { - return Err(StorageError { - description: "unable to list all series".to_string(), - }) - } - }; - - let root = match &pred.root { + _batch_size: usize, + predicate: &Predicate, + range: &TimestampRange, + ) -> Result, StorageError> { + let root = match &predicate.root { Some(r) => r, None => { return Err(StorageError { @@ -358,136 +250,32 @@ impl MemDB { } }; - let bucket_map = self.bucket_id_to_series_map.read().unwrap(); - let series_map = match bucket_map.get(&bucket_id) { - Some(m) => m.read().unwrap(), - None => { - return Err(StorageError { - description: format!("no series written for bucket {}", bucket_id), - }) - } - }; - - let map = evaluate_node(&series_map, &root)?; - let mut filters = Vec::with_capacity(map.cardinality() as usize); + let map = evaluate_node(&self.series_map, &root)?; + let mut read_batches = Vec::with_capacity(map.cardinality() as usize); for id in map.iter() { - let (key, series_type) = series_map.series_id_to_key_and_type.get(&id).unwrap(); - filters.push(SeriesFilter { - id, - key: key.clone(), - value_predicate: None, - series_type: *series_type, - }); + let (key, series_type) = self.series_map.series_id_to_key_and_type.get(&id).unwrap(); + + let values = match series_type { + SeriesDataType::I64 => { + let buff = self.series_data.i64_series.get(&id).unwrap(); + ReadValues::I64(buff.read(range)) + } + SeriesDataType::F64 => { + let buff = self.series_data.f64_series.get(&id).unwrap(); + ReadValues::F64(buff.read(range)) + } + }; + + let batch = ReadBatch { + key: key.to_string(), + values, + }; + + read_batches.push(batch); } - Ok(Box::new(filters.into_iter())) - } - - fn write_points_with_series_ids( - &self, - bucket_id: u32, - points: &[PointType], - ) -> Result<(), StorageError> { - let bucket_data = self.bucket_id_to_series_data.read().unwrap(); - - // ensure the the bucket has a series data struct to write into - let series_data = match bucket_data.get(&bucket_id) { - Some(d) => d, - None => { - drop(bucket_data); - let mut bucket_data = self.bucket_id_to_series_data.write().unwrap(); - let series_data = SeriesData { - ring_buffer_size: self.default_ring_buffer_size, - i64_series: HashMap::new(), - f64_series: HashMap::new(), - }; - bucket_data.insert(bucket_id, Mutex::new(series_data)); - drop(bucket_data); - return self.write_points_with_series_ids(bucket_id, points); - } - }; - - let mut series_data = series_data.lock().unwrap(); - series_data.write_points(points); - Ok(()) - } - - fn read_range( - &self, - bucket_id: u32, - series_id: u64, - range: &TimestampRange, - batch_size: usize, - ) -> Result>> + Send>, StorageError> { - let buckets = self.bucket_id_to_series_data.read().unwrap(); - let data = match buckets.get(&bucket_id) { - Some(d) => d, - None => { - return Err(StorageError { - description: format!("bucket {} not found", bucket_id), - }) - } - }; - - let data = data.lock().unwrap(); - let buff = match FromSeries::from_series(&data, series_id) { - Some(b) => b, - None => { - return Err(StorageError { - description: format!("series {} not found", series_id), - }) - } - }; - - let values = buff.get_range(&range); - Ok(Box::new(PointsIterator { - values: Some(values), - batch_size, - })) - } -} - -trait FromSeries: Clone { - fn from_series<'a>(data: &'a SeriesData, series_id: u64) -> Option<&'a SeriesRingBuffer>; -} - -impl FromSeries for i64 { - fn from_series<'a>(data: &'a SeriesData, series_id: u64) -> Option<&'a SeriesRingBuffer> { - data.i64_series.get(&series_id) - } -} - -impl FromSeries for f64 { - fn from_series<'a>(data: &'a SeriesData, series_id: u64) -> Option<&'a SeriesRingBuffer> { - data.f64_series.get(&series_id) - } -} - -struct PointsIterator { - values: Option>>, - batch_size: usize, -} - -impl Iterator for PointsIterator { - type Item = Vec>; - - fn next(&mut self) -> Option { - if let Some(mut values) = self.values.take() { - if self.batch_size > values.len() { - return Some(values); - } - - let remaining = values.split_off(self.batch_size); - - if !remaining.is_empty() { - self.values = Some(remaining); - } - - return Some(values); - } - - None + Ok(stream::iter(read_batches.into_iter()).boxed()) } } @@ -503,98 +291,129 @@ fn evaluate_node(series_map: &SeriesMap, n: &Node) -> Result Result<(), StorageError> { - self.get_or_create_series_ids_for_points(bucket_id, points) - } - - fn read_series_matching( - &self, - bucket_id: u32, - predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - self.read_series_matching(bucket_id, predicate) - } - - fn get_tag_keys( - &self, - bucket_id: u32, - predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - self.get_tag_keys(bucket_id, predicate) - } - - fn get_tag_values( - &self, - bucket_id: u32, - tag_key: &str, - predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - self.get_tag_values(bucket_id, tag_key, predicate) - } -} - -impl SeriesStore for MemDB { - fn write_points_with_series_ids( - &self, - bucket_id: u32, - points: &[PointType], - ) -> Result<(), StorageError> { - self.write_points_with_series_ids(bucket_id, points) - } - - fn read_i64_range( - &self, - bucket_id: u32, - series_id: u64, - range: &TimestampRange, - batch_size: usize, - ) -> Result>> + Send>, StorageError> { - self.read_range(bucket_id, series_id, range, batch_size) - } - - fn read_f64_range( - &self, - bucket_id: u32, - series_id: u64, - range: &TimestampRange, - batch_size: usize, - ) -> Result>> + Send>, StorageError> { - self.read_range(bucket_id, series_id, range, batch_size) - } -} - #[cfg(test)] mod tests { use super::*; - use crate::storage::inverted_index; - use crate::storage::series_store; + use crate::storage::predicate::parse_predicate; #[test] - fn write_and_read_i64() { - let db = Box::new(MemDB::new()); - series_store::tests::write_and_read_i64(db); - } + fn write_and_read() { + let p1 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 1, 0); + let p2 = PointType::new_i64("cpu,host=a,region=west\tusage_system".to_string(), 1, 0); + let p3 = PointType::new_i64("cpu,host=a,region=west\tusage_user".to_string(), 1, 0); + let p4 = PointType::new_i64("mem,host=b,region=west\tfree".to_string(), 1, 0); + let p5 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 2, 1); - #[test] - fn write_and_read_f64() { - let db = Box::new(MemDB::new()); - series_store::tests::write_and_read_f64(db); - } + let mut points = vec![p1, p2, p3, p4, p5]; - #[test] - fn series_id_indexing() { - let db = Box::new(MemDB::new()); - inverted_index::tests::series_id_indexing(db) - } + let mut memdb = MemDB::new(); + memdb.write(&mut points).unwrap(); - #[test] - fn series_metadata_indexing() { - let db = Box::new(MemDB::new()); - inverted_index::tests::series_metadata_indexing(db); + // check that we've accounted for the size. + assert_eq!(memdb.size(), 1000); + + let empty_pred = &Predicate { root: None }; + let empty_range = TimestampRange { start: 0, end: 0 }; + + let tag_keys = memdb.get_tag_keys(&empty_range, &empty_pred).unwrap(); + let tag_keys: Vec = futures::executor::block_on_stream(tag_keys).collect(); + + assert_eq!(tag_keys, vec!["_f", "_m", "host", "region"]); + + let tag_values = memdb + .get_tag_values("host", &empty_range, &empty_pred) + .unwrap(); + let tag_values: Vec = futures::executor::block_on_stream(tag_values).collect(); + assert_eq!(tag_values, vec!["a", "b"]); + + // get all series + + // get series with measurement = mem + let pred = parse_predicate(r#"_m = "cpu""#).unwrap(); + let batches = memdb + .read(10, &pred, &TimestampRange { start: 0, end: 5 }) + .unwrap(); + let batches: Vec = futures::executor::block_on_stream(batches).collect(); + + assert_eq!( + batches, + vec![ + ReadBatch { + key: "cpu,host=b,region=west\tusage_system".to_string(), + values: ReadValues::I64(vec![ + ReadPoint { time: 0, value: 1 }, + ReadPoint { time: 1, value: 2 }, + ]), + }, + ReadBatch { + key: "cpu,host=a,region=west\tusage_system".to_string(), + values: ReadValues::I64(vec![ReadPoint { time: 0, value: 1 }]), + }, + ReadBatch { + key: "cpu,host=a,region=west\tusage_user".to_string(), + values: ReadValues::I64(vec![ReadPoint { time: 0, value: 1 }]), + }, + ], + ); + + // get series with host = a + let pred = parse_predicate(r#"host = "a""#).unwrap(); + let batches = memdb + .read(10, &pred, &TimestampRange { start: 0, end: 5 }) + .unwrap(); + let batches: Vec = futures::executor::block_on_stream(batches).collect(); + assert_eq!( + batches, + vec![ + ReadBatch { + key: "cpu,host=a,region=west\tusage_system".to_string(), + values: ReadValues::I64(vec![ReadPoint { time: 0, value: 1 }]), + }, + ReadBatch { + key: "cpu,host=a,region=west\tusage_user".to_string(), + values: ReadValues::I64(vec![ReadPoint { time: 0, value: 1 }]), + }, + ] + ); + + // get series with measurement = cpu and host = b + let pred = parse_predicate(r#"_m = "cpu" and host = "b""#).unwrap(); + let batches = memdb + .read(10, &pred, &TimestampRange { start: 0, end: 5 }) + .unwrap(); + let batches: Vec = futures::executor::block_on_stream(batches).collect(); + assert_eq!( + batches, + vec![ReadBatch { + key: "cpu,host=b,region=west\tusage_system".to_string(), + values: ReadValues::I64(vec![ + ReadPoint { time: 0, value: 1 }, + ReadPoint { time: 1, value: 2 }, + ]), + },] + ); + + let pred = parse_predicate(r#"host = "a" OR _m = "mem""#).unwrap(); + let batches = memdb + .read(10, &pred, &TimestampRange { start: 0, end: 5 }) + .unwrap(); + let batches: Vec = futures::executor::block_on_stream(batches).collect(); + assert_eq!( + batches, + vec![ + ReadBatch { + key: "cpu,host=a,region=west\tusage_system".to_string(), + values: ReadValues::I64(vec![ReadPoint { time: 0, value: 1 },]), + }, + ReadBatch { + key: "cpu,host=a,region=west\tusage_user".to_string(), + values: ReadValues::I64(vec![ReadPoint { time: 0, value: 1 },]), + }, + ReadBatch { + key: "mem,host=b,region=west\tfree".to_string(), + values: ReadValues::I64(vec![ReadPoint { time: 0, value: 1 },]), + }, + ] + ); } } diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs new file mode 100644 index 0000000000..cb0929101a --- /dev/null +++ b/src/storage/partitioned_store.rs @@ -0,0 +1,556 @@ +use crate::delorean::{Predicate, TimestampRange}; +use crate::line_parser::PointType; +use crate::storage::series_store::ReadPoint; +use crate::storage::StorageError; + +use core::pin::Pin; +use core::task::{Context, Poll}; +use failure::_core::cmp::Ordering; +use futures::stream::{BoxStream, Stream}; + +/// partitioned_store is a trait and set of helper functions and structs to define Partitions +/// that store data. The helper funcs and structs merge results from multiple partitions together. + +/// A Partition is a block of data. It has methods for reading the metadata like which measurements, +/// tags, tag values, and fields exist. Along with the raw time series data. It is designed to work +/// as a stream so that it can be used in safely an asynchronous context. A partition is the +/// lowest level organization scheme. Above it you will have a database which keeps track of +/// what organizations and buckets exist. A bucket will have 1-many partitions and a partition +/// will only ever contain data for a single bucket. +pub trait Partition { + fn id(&self) -> String; + + fn size(&self) -> u64; + + fn write(&self, points: &[PointType]) -> Result<(), StorageError>; + + // fn measurements( + // &self, + // ) -> Result>>, StorageError>; + // + // fn tag_keys( + // &self, + // measurement: &str, + // ) -> Result>>, StorageError>; + // + // fn fields( + // &self, + // measurement: &str, + // ) -> Result>>, StorageError>; + // + // fn tag_values( + // &self, + // measurement: &str, + // key: &str, + // ) -> Result>>, StorageError>; + + fn get_tag_keys( + &self, + range: &TimestampRange, + predicate: &Predicate, + ) -> Result, StorageError>; + + fn get_tag_values( + &self, + tag_key: &str, + range: &TimestampRange, + predicate: &Predicate, + ) -> Result, StorageError>; + + fn read( + &self, + batch_size: usize, + predicate: &Predicate, + range: &TimestampRange, + ) -> Result, StorageError>; +} + +/// StringMergeStream will do a merge sort with deduplication of multiple streams of Strings. This +/// is used for combining results from multiple partitions for calls to get measurements, tag keys, +/// tag values, or field keys. It assumes the incoming streams are in sorted order. +pub struct StringMergeStream<'a> { + streams: Vec>, + next_vals: Vec>>, + drained: bool, +} + +impl StringMergeStream<'_> { + fn new(streams: Vec>) -> StringMergeStream<'_> { + let mut next_vals = Vec::with_capacity(streams.len()); + for _ in 0..streams.len() { + next_vals.push(Poll::Pending); + } + + StringMergeStream { + streams, + next_vals, + drained: false, + } + } +} + +impl Stream for StringMergeStream<'_> { + type Item = String; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + if self.drained { + return Poll::Ready(None); + } + + for pos in 0..self.next_vals.len() { + match self.next_vals[pos] { + Poll::Pending => { + let v = self.streams[pos].as_mut().poll_next(cx); + if v.is_pending() { + return Poll::Pending; + } + + self.next_vals[pos] = v; + } + Poll::Ready(None) => { + let v = self.streams[pos].as_mut().poll_next(cx); + if v.is_pending() { + return Poll::Pending; + } + + self.next_vals[pos] = v; + } + Poll::Ready(_) => (), + } + } + + let mut next_pos = 0; + + for pos in 1..self.next_vals.len() { + if let Poll::Ready(Some(s)) = &self.next_vals[pos] { + match &self.next_vals[next_pos] { + Poll::Ready(None) => next_pos = pos, + Poll::Ready(Some(next)) => { + match next.cmp(s) { + Ordering::Greater => next_pos = pos, + Ordering::Equal => self.next_vals[pos] = self.streams[pos].as_mut().poll_next(cx), + Ordering::Less => (), + } + } + Poll::Pending => return Poll::Pending, + } + } + } + + match &mut self.next_vals[next_pos] { + Poll::Ready(v) => { + let next = v.take(); + + if next.is_none() { + self.drained = true; + } + + Poll::Ready(next) + } + Poll::Pending => Poll::Pending, + } + } +} + +/// ReadMergeStream will do a merge sort of the ReadBatches from multiple partitions. When merging +/// it will ensure that batches are sent through in lexographical order by key. In situations +/// where multiple partitions have batches with the same key, they are merged together in time +/// ascending order. For any given key, multiple read batches can come through. +pub struct ReadMergeStream<'a> { + streams: Vec>, + next_vals: Vec>>, + drained: bool, +} + +impl ReadMergeStream<'_> { + fn new(streams: Vec>) -> ReadMergeStream<'_> { + let mut next_vals = Vec::with_capacity(streams.len()); + for _ in 0..streams.len() { + next_vals.push(Poll::Pending); + } + + ReadMergeStream { + streams, + next_vals, + drained: false, + } + } +} + +impl Stream for ReadMergeStream<'_> { + type Item = ReadBatch; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + if self.drained { + return Poll::Ready(None); + } + + let mut min_key: Option = None; + let mut min_time = std::i64::MAX; + let mut min_pos = 0; + + // find the key that we should send next and make sure that things are populated + for pos in 0..self.next_vals.len() { + match &self.next_vals[pos] { + Poll::Pending => match self.streams[pos].as_mut().poll_next(cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(Some(batch)) => { + match &min_key { + Some(k) => { + match batch.key.cmp(k) { + Ordering::Less => { + min_key = Some(batch.key.clone()); + let (_, min) = batch.start_stop_times().unwrap(); + min_pos = pos; + min_time = min; + }, + Ordering::Equal => { + let (_, min) = batch.start_stop_times().unwrap(); + if min < min_time { + min_pos = pos; + min_time = min; + } + }, + Ordering::Greater => (), + } + } + None => { + min_key = Some(batch.key.clone()); + let (_, min) = batch.start_stop_times().unwrap(); + min_pos = pos; + min_time = min; + } + } + self.next_vals[pos] = Poll::Ready(Some(batch)); + } + Poll::Ready(None) => self.next_vals[pos] = Poll::Ready(None), + }, + Poll::Ready(None) => (), + Poll::Ready(Some(batch)) => match &min_key { + Some(k) => { + match batch.key.cmp(k) { + Ordering::Less => { + min_key = Some(batch.key.clone()); + let (_, min) = batch.start_stop_times().unwrap(); + min_pos = pos; + min_time = min; + }, + Ordering::Equal => { + let (_, min) = batch.start_stop_times().unwrap(); + if min < min_time { + min_pos = pos; + min_time = min; + } + }, + Ordering::Greater => (), + } + } + None => { + min_key = Some(batch.key.clone()); + let (_, min) = batch.start_stop_times().unwrap(); + min_pos = pos; + min_time = min; + } + }, + } + } + + if min_key.is_none() { + self.drained = true; + return Poll::Ready(None); + } + let min_key = min_key.unwrap(); + + let mut batch = match &mut self.next_vals[min_pos] { + Poll::Ready(b) => b.take(), + Poll::Pending => Some(ReadBatch { + key: min_key.clone(), + values: ReadValues::I64(vec![]), + }), // shouldn't be possible to hit this + } + .unwrap(); + self.next_vals[min_pos] = Poll::Pending; + + let mut sort = false; + + for pos in 0..self.next_vals.len() { + if let Poll::Ready(Some(b)) = &mut self.next_vals[pos] { + if b.key == min_key { + if batch.append_below_time(b, min_time) { + self.next_vals[pos] = Poll::Pending; + } + sort = true; + } + } + } + + if sort { + batch.sort(); + } + + Poll::Ready(Some(batch)) + } +} + +#[derive(Debug, PartialEq)] +pub enum ReadValues { + I64(Vec>), + F64(Vec>), +} + +impl Ord for ReadPoint { + fn cmp(&self, other: &Self) -> Ordering { + self.time.cmp(&other.time) + } +} + +impl PartialOrd for ReadPoint { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +#[derive(Debug, PartialEq)] +pub struct ReadBatch { + pub key: String, + pub values: ReadValues, +} + +impl ReadBatch { + fn start_stop_times(&self) -> Option<(i64, i64)> { + match &self.values { + ReadValues::I64(vals) => { + if vals.is_empty() { + return None; + } + Some((vals[0].time, vals[vals.len() - 1].time)) + } + ReadValues::F64(vals) => { + if vals.is_empty() { + return None; + } + Some((vals[0].time, vals[vals.len() - 1].time)) + } + } + } + + fn sort(&mut self) { + match &mut self.values { + ReadValues::I64(vals) => vals.sort(), + ReadValues::F64(vals) => vals.sort_by(|a, b| a.time.partial_cmp(&b.time).unwrap()), + } + } + + // append_below_time will append all values from other that have a time < than the one passed in. + // it returns true if other has been cleared of all values + fn append_below_time(&mut self, other: &mut ReadBatch, t: i64) -> bool { + match (&mut self.values, &mut other.values) { + (ReadValues::I64(vals), ReadValues::I64(other_vals)) => { + let pos = other_vals.iter().position(|val| val.time > t); + match pos { + None => { + vals.append(other_vals); + other_vals.clear(); + true + } + Some(pos) => { + let mut rest = other_vals.split_off(pos); + vals.append(other_vals); + other_vals.append(&mut rest); + false + } + } + } + (ReadValues::F64(vals), ReadValues::F64(other_vals)) => { + let pos = other_vals.iter().position(|val| val.time > t); + match pos { + None => { + vals.append(other_vals); + other_vals.clear(); + true + } + Some(pos) => { + let mut rest = other_vals.split_off(pos); + vals.append(other_vals); + other_vals.append(&mut rest); + false + } + } + } + (_, _) => true, // do nothing here + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use futures::{stream, StreamExt}; + + #[test] + fn string_merge_stream() { + let one = stream::iter(vec!["a".to_string(), "c".to_string()].into_iter()); + let two = stream::iter(vec!["b".to_string(), "c".to_string(), "d".to_string()].into_iter()); + let three = + stream::iter(vec!["c".to_string(), "e".to_string(), "f".to_string()].into_iter()); + let four = stream::iter(vec![].into_iter()); + + let merger = + StringMergeStream::new(vec![one.boxed(), two.boxed(), three.boxed(), four.boxed()]); + + let stream = futures::executor::block_on_stream(merger); + let vals: Vec = stream.collect(); + + assert_eq!( + vals, + vec![ + "a".to_string(), + "b".to_string(), + "c".to_string(), + "d".to_string(), + "e".to_string(), + "f".to_string() + ], + ); + } + + #[test] + fn read_merge_stream() { + let one = stream::iter( + vec![ + ReadBatch { + key: "foo".to_string(), + values: ReadValues::I64(vec![ + ReadPoint { time: 3, value: 30 }, + ReadPoint { time: 4, value: 40 }, + ]), + }, + ReadBatch { + key: "test".to_string(), + values: ReadValues::F64(vec![ + ReadPoint { + time: 1, + value: 1.1, + }, + ReadPoint { + time: 2, + value: 2.2, + }, + ]), + }, + ] + .into_iter(), + ); + + let two = stream::iter( + vec![ + ReadBatch { + key: "bar".to_string(), + values: ReadValues::F64(vec![ + ReadPoint { + time: 5, + value: 5.5, + }, + ReadPoint { + time: 6, + value: 6.6, + }, + ]), + }, + ReadBatch { + key: "foo".to_string(), + values: ReadValues::I64(vec![ + ReadPoint { time: 1, value: 10 }, + ReadPoint { time: 2, value: 20 }, + ReadPoint { time: 6, value: 60 }, + ReadPoint { + time: 11, + value: 110, + }, + ]), + }, + ] + .into_iter(), + ); + + let three = stream::iter( + vec![ReadBatch { + key: "foo".to_string(), + values: ReadValues::I64(vec![ + ReadPoint { time: 5, value: 50 }, + ReadPoint { + time: 10, + value: 100, + }, + ]), + }] + .into_iter(), + ); + + let four = stream::iter(vec![].into_iter()); + + let merger = + ReadMergeStream::new(vec![one.boxed(), two.boxed(), three.boxed(), four.boxed()]); + let stream = futures::executor::block_on_stream(merger); + let vals: Vec = stream.collect(); + + assert_eq!( + vals, + vec![ + ReadBatch { + key: "bar".to_string(), + values: ReadValues::F64(vec![ + ReadPoint { + time: 5, + value: 5.5 + }, + ReadPoint { + time: 6, + value: 6.6 + }, + ]), + }, + ReadBatch { + key: "foo".to_string(), + values: ReadValues::I64(vec![ + ReadPoint { time: 1, value: 10 }, + ReadPoint { time: 2, value: 20 }, + ReadPoint { time: 3, value: 30 }, + ReadPoint { time: 4, value: 40 }, + ]), + }, + ReadBatch { + key: "foo".to_string(), + values: ReadValues::I64(vec![ + ReadPoint { time: 5, value: 50 }, + ReadPoint { time: 6, value: 60 }, + ReadPoint { + time: 10, + value: 100 + }, + ]), + }, + ReadBatch { + key: "foo".to_string(), + values: ReadValues::I64(vec![ReadPoint { + time: 11, + value: 110 + },]), + }, + ReadBatch { + key: "test".to_string(), + values: ReadValues::F64(vec![ + ReadPoint { + time: 1, + value: 1.1 + }, + ReadPoint { + time: 2, + value: 2.2 + } + ]), + }, + ], + ) + } +} diff --git a/src/storage/series_store.rs b/src/storage/series_store.rs index 3d18fc2bb6..41dafe391c 100644 --- a/src/storage/series_store.rs +++ b/src/storage/series_store.rs @@ -26,7 +26,7 @@ pub trait SeriesStore: Sync + Send { ) -> Result>> + Send>, StorageError>; } -#[derive(Debug, PartialEq, Clone)] +#[derive(Debug, Eq, PartialEq, Clone)] pub struct ReadPoint { pub time: i64, pub value: T, From 2663d9d7567c4f1f7d7aeaf09b23acfc6036db3a Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 22 Mar 2020 17:59:25 -0400 Subject: [PATCH 02/42] Update memdb and partition_store Clean up based on PR feedback. --- src/storage/memdb.rs | 163 ++++++++++++++++--------------- src/storage/partitioned_store.rs | 89 ++++++++--------- 2 files changed, 124 insertions(+), 128 deletions(-) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index df1826745b..835a9dc80f 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -1,10 +1,9 @@ use crate::delorean::{Node, Predicate, TimestampRange}; use crate::line_parser::{ParseError, Point, PointType}; use crate::storage::predicate::{Evaluate, EvaluateVisitor}; +use crate::storage::{SeriesDataType, StorageError}; +use crate::storage::partitioned_store::{ReadBatch, ReadValues}; use crate::storage::series_store::ReadPoint; -use crate::storage::{ - partitioned_store::ReadBatch, partitioned_store::ReadValues, SeriesDataType, StorageError, -}; use croaring::Treemap; use futures::stream::{self, BoxStream}; @@ -26,7 +25,7 @@ pub struct MemDB { #[derive(Default)] struct SeriesData { - current_size: u64, + current_size: usize, i64_series: HashMap>, f64_series: HashMap>, } @@ -37,36 +36,15 @@ struct SeriesBuffer { impl SeriesBuffer { fn read(&self, range: &TimestampRange) -> Vec> { - let mut start = self.values.len(); - let mut stop = start; + let start = match self.values.iter().position(|val| val.time >= range.start) { + Some(pos) => pos, + None => return vec![], + }; - for (pos, val) in self.values.iter().enumerate() { - if val.time >= range.start { - start = pos; - break; - } - } + let stop = self.values[start..].iter().position(|val| val.time >= range.end); + let stop = stop.unwrap_or_else(|| self.values.len()); - for (pos, val) in self.values[start..].iter().enumerate() { - if val.time >= range.end { - stop = pos; - break; - } - } - - let size = stop - start; - - if size == 0 { - return vec![]; - } - - let mut values = Vec::with_capacity(size); - - for val in self.values[start..stop].iter() { - values.push(val.clone()); - } - - values + self.values[start..stop].to_vec() } } @@ -89,13 +67,12 @@ impl StoreInSeriesData for Point { time: self.time, value: self.value, }; - series_data.current_size += std::mem::size_of::>() as u64; + series_data.current_size += std::mem::size_of::>(); match series_data.i64_series.get_mut(&self.series_id.unwrap()) { Some(buff) => buff.values.push(point), None => { - let mut buff = SeriesBuffer { values: Vec::new() }; - buff.values.push(point); + let buff = SeriesBuffer { values: vec![point] }; series_data.i64_series.insert(self.series_id.unwrap(), buff); } } @@ -108,13 +85,12 @@ impl StoreInSeriesData for Point { time: self.time, value: self.value, }; - series_data.current_size += std::mem::size_of::>() as u64; + series_data.current_size += std::mem::size_of::>(); match series_data.f64_series.get_mut(&self.series_id.unwrap()) { Some(buff) => buff.values.push(point), None => { - let mut buff = SeriesBuffer { values: Vec::new() }; - buff.values.push(point); + let buff = SeriesBuffer { values: vec![point] }; series_data.f64_series.insert(self.series_id.unwrap(), buff); } } @@ -123,7 +99,7 @@ impl StoreInSeriesData for Point { #[derive(Default)] struct SeriesMap { - current_size: u64, + current_size: usize, last_id: u64, series_key_to_id: HashMap, series_id_to_key_and_type: HashMap, @@ -132,6 +108,13 @@ struct SeriesMap { } impl SeriesMap { + // SERIES_KEY_COPIES are the number of copies of the key this map contains. This is + // used to provide a rough estimate of the memory size. + const SERIES_KEY_COPIES: usize = 2; + // SERIES_ID_BYTES is the number of bytes the different copies of the series ID in + // this map represents. It's used to provide a rough estimate of the memory size. + const SERIES_ID_BYTES: usize = 24; + fn insert_series(&mut self, point: &mut PointType) -> Result<(), ParseError> { if let Some(id) = self.series_key_to_id.get(point.series()) { point.set_series_id(*id); @@ -151,15 +134,17 @@ impl SeriesMap { self.series_id_to_key_and_type .insert(self.last_id, (point.series().clone(), series_type)); - // update the estimated size of the map - self.current_size += (point.series().len() * 2 + 16) as u64; + // update the estimated size of the map. This is a rough estimate based on + // having the series key twice (once in the map to ID and again in the ID + // to map. And then adding another 16 bytes for the two IDs + self.current_size += point.series().len() * SeriesMap::SERIES_KEY_COPIES + SeriesMap::SERIES_ID_BYTES; for pair in point.index_pairs()? { // insert this id into the posting list let list_key = list_key(&pair.key, &pair.value); // update estimated size for the index pairs - self.current_size += (list_key.len() + pair.key.len() + pair.value.len() + 8) as u64; + self.current_size += list_key.len() + pair.key.len() + pair.value.len(); let posting_list = self .posting_list @@ -196,7 +181,7 @@ impl MemDB { Default::default() } - pub fn size(&self) -> u64 { + pub fn size(&self) -> usize { self.series_data.current_size + self.series_map.current_size } @@ -216,8 +201,8 @@ impl MemDB { _range: &TimestampRange, _predicate: &Predicate, ) -> Result, StorageError> { - let keys: Vec = self.series_map.tag_keys.keys().cloned().collect(); - Ok(stream::iter(keys.into_iter()).boxed()) + let keys = self.series_map.tag_keys.keys().cloned(); + Ok(stream::iter(keys).boxed()) } fn get_tag_values( @@ -228,10 +213,10 @@ impl MemDB { ) -> Result, StorageError> { match self.series_map.tag_keys.get(tag_key) { Some(values) => { - let values: Vec = values.keys().cloned().collect(); - Ok(stream::iter(values.into_iter()).boxed()) + let values = values.keys().cloned(); + Ok(stream::iter(values).boxed()) } - None => Ok(stream::iter(vec![].into_iter()).boxed()), + None => Ok(stream::empty().boxed()), } } @@ -297,43 +282,41 @@ mod tests { use crate::storage::predicate::parse_predicate; #[test] - fn write_and_read() { - let p1 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 1, 0); - let p2 = PointType::new_i64("cpu,host=a,region=west\tusage_system".to_string(), 1, 0); - let p3 = PointType::new_i64("cpu,host=a,region=west\tusage_user".to_string(), 1, 0); - let p4 = PointType::new_i64("mem,host=b,region=west\tfree".to_string(), 1, 0); - let p5 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 2, 1); - - let mut points = vec![p1, p2, p3, p4, p5]; - - let mut memdb = MemDB::new(); - memdb.write(&mut points).unwrap(); - - // check that we've accounted for the size. - assert_eq!(memdb.size(), 1000); - - let empty_pred = &Predicate { root: None }; - let empty_range = TimestampRange { start: 0, end: 0 }; - - let tag_keys = memdb.get_tag_keys(&empty_range, &empty_pred).unwrap(); - let tag_keys: Vec = futures::executor::block_on_stream(tag_keys).collect(); + fn write_and_read_tag_keys() { + let memdb = setup_db(); + let tag_keys = memdb + .get_tag_keys(&TimestampRange { start: 0, end: 0 }, &Predicate{root: None}) + .unwrap(); + let tag_keys: Vec<_> = futures::executor::block_on_stream(tag_keys).collect(); assert_eq!(tag_keys, vec!["_f", "_m", "host", "region"]); + } + + #[test] + fn write_and_read_tag_values() { + let memdb = setup_db(); let tag_values = memdb - .get_tag_values("host", &empty_range, &empty_pred) + .get_tag_values("host", &TimestampRange { start: 0, end: 0 }, &Predicate{root: None}) .unwrap(); - let tag_values: Vec = futures::executor::block_on_stream(tag_values).collect(); + let tag_values: Vec<_> = futures::executor::block_on_stream(tag_values).collect(); assert_eq!(tag_values, vec!["a", "b"]); + } - // get all series + #[test] + fn write_and_check_size() { + let memdb = setup_db(); + assert_eq!(memdb.size(), 1000); + } - // get series with measurement = mem + #[test] + fn write_and_get_measurement_series() { + let memdb = setup_db(); let pred = parse_predicate(r#"_m = "cpu""#).unwrap(); let batches = memdb .read(10, &pred, &TimestampRange { start: 0, end: 5 }) .unwrap(); - let batches: Vec = futures::executor::block_on_stream(batches).collect(); + let batches: Vec<_> = futures::executor::block_on_stream(batches).collect(); assert_eq!( batches, @@ -355,13 +338,16 @@ mod tests { }, ], ); + } - // get series with host = a + #[test] + fn write_and_get_tag_match_series() { + let memdb = setup_db(); let pred = parse_predicate(r#"host = "a""#).unwrap(); let batches = memdb .read(10, &pred, &TimestampRange { start: 0, end: 5 }) .unwrap(); - let batches: Vec = futures::executor::block_on_stream(batches).collect(); + let batches: Vec<_> = futures::executor::block_on_stream(batches).collect(); assert_eq!( batches, vec![ @@ -375,13 +361,16 @@ mod tests { }, ] ); + } - // get series with measurement = cpu and host = b + #[test] + fn write_and_measurement_and_tag_match_series() { + let memdb = setup_db(); let pred = parse_predicate(r#"_m = "cpu" and host = "b""#).unwrap(); let batches = memdb .read(10, &pred, &TimestampRange { start: 0, end: 5 }) .unwrap(); - let batches: Vec = futures::executor::block_on_stream(batches).collect(); + let batches: Vec<_> = futures::executor::block_on_stream(batches).collect(); assert_eq!( batches, vec![ReadBatch { @@ -392,12 +381,16 @@ mod tests { ]), },] ); + } + #[test] + fn write_and_measurement_or_tag_match() { + let memdb = setup_db(); let pred = parse_predicate(r#"host = "a" OR _m = "mem""#).unwrap(); let batches = memdb .read(10, &pred, &TimestampRange { start: 0, end: 5 }) .unwrap(); - let batches: Vec = futures::executor::block_on_stream(batches).collect(); + let batches: Vec<_> = futures::executor::block_on_stream(batches).collect(); assert_eq!( batches, vec![ @@ -416,4 +409,18 @@ mod tests { ] ); } + + fn setup_db() -> MemDB { + let p1 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 1, 0); + let p2 = PointType::new_i64("cpu,host=a,region=west\tusage_system".to_string(), 1, 0); + let p3 = PointType::new_i64("cpu,host=a,region=west\tusage_user".to_string(), 1, 0); + let p4 = PointType::new_i64("mem,host=b,region=west\tfree".to_string(), 1, 0); + let p5 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 2, 1); + + let mut points = vec![p1, p2, p3, p4, p5]; + + let mut memdb = MemDB::new(); + memdb.write(&mut points).unwrap(); + memdb + } } diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index cb0929101a..13f5f6ced3 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -1,16 +1,16 @@ +//! partitioned_store is a trait and set of helper functions and structs to define Partitions +//! that store data. The helper funcs and structs merge results from multiple partitions together. use crate::delorean::{Predicate, TimestampRange}; use crate::line_parser::PointType; use crate::storage::series_store::ReadPoint; use crate::storage::StorageError; -use core::pin::Pin; -use core::task::{Context, Poll}; -use failure::_core::cmp::Ordering; +use std::pin::Pin; +use std::mem; +use std::task::{Context, Poll}; +use std::cmp::Ordering; use futures::stream::{BoxStream, Stream}; -/// partitioned_store is a trait and set of helper functions and structs to define Partitions -/// that store data. The helper funcs and structs merge results from multiple partitions together. - /// A Partition is a block of data. It has methods for reading the metadata like which measurements, /// tags, tag values, and fields exist. Along with the raw time series data. It is designed to work /// as a stream so that it can be used in safely an asynchronous context. A partition is the @@ -76,14 +76,11 @@ pub struct StringMergeStream<'a> { impl StringMergeStream<'_> { fn new(streams: Vec>) -> StringMergeStream<'_> { - let mut next_vals = Vec::with_capacity(streams.len()); - for _ in 0..streams.len() { - next_vals.push(Poll::Pending); - } + let len = streams.len(); StringMergeStream { streams, - next_vals, + next_vals: vec![Poll::Pending; len], drained: false, } } @@ -97,20 +94,14 @@ impl Stream for StringMergeStream<'_> { return Poll::Ready(None); } + let mut one_pending = false; + for pos in 0..self.next_vals.len() { match self.next_vals[pos] { Poll::Pending => { let v = self.streams[pos].as_mut().poll_next(cx); if v.is_pending() { - return Poll::Pending; - } - - self.next_vals[pos] = v; - } - Poll::Ready(None) => { - let v = self.streams[pos].as_mut().poll_next(cx); - if v.is_pending() { - return Poll::Pending; + one_pending = true; } self.next_vals[pos] = v; @@ -119,6 +110,10 @@ impl Stream for StringMergeStream<'_> { } } + if one_pending { + return Poll::Pending; + } + let mut next_pos = 0; for pos in 1..self.next_vals.len() { @@ -137,18 +132,13 @@ impl Stream for StringMergeStream<'_> { } } - match &mut self.next_vals[next_pos] { - Poll::Ready(v) => { - let next = v.take(); + let val = mem::replace(&mut self.next_vals[next_pos], Poll::Pending); - if next.is_none() { - self.drained = true; - } - - Poll::Ready(next) - } - Poll::Pending => Poll::Pending, + if let Poll::Ready(None) = val { + self.drained = true; } + + val } } @@ -164,14 +154,11 @@ pub struct ReadMergeStream<'a> { impl ReadMergeStream<'_> { fn new(streams: Vec>) -> ReadMergeStream<'_> { - let mut next_vals = Vec::with_capacity(streams.len()); - for _ in 0..streams.len() { - next_vals.push(Poll::Pending); - } + let len = streams.len(); ReadMergeStream { streams, - next_vals, + next_vals: vec![Poll::Pending; len], drained: false, } } @@ -189,11 +176,13 @@ impl Stream for ReadMergeStream<'_> { let mut min_time = std::i64::MAX; let mut min_pos = 0; + let mut one_pending = false; + // find the key that we should send next and make sure that things are populated for pos in 0..self.next_vals.len() { match &self.next_vals[pos] { Poll::Pending => match self.streams[pos].as_mut().poll_next(cx) { - Poll::Pending => return Poll::Pending, + Poll::Pending => one_pending = true, Poll::Ready(Some(batch)) => { match &min_key { Some(k) => { @@ -255,21 +244,22 @@ impl Stream for ReadMergeStream<'_> { } } + if one_pending { + return Poll::Pending; + } + if min_key.is_none() { self.drained = true; return Poll::Ready(None); } let min_key = min_key.unwrap(); - let mut batch = match &mut self.next_vals[min_pos] { - Poll::Ready(b) => b.take(), - Poll::Pending => Some(ReadBatch { - key: min_key.clone(), - values: ReadValues::I64(vec![]), - }), // shouldn't be possible to hit this - } - .unwrap(); - self.next_vals[min_pos] = Poll::Pending; + let batch = mem::replace(&mut self.next_vals[min_pos], Poll::Pending); + + let mut batch = match batch { + Poll::Ready(Some(b)) => b, + _ => unreachable!(), + }; let mut sort = false; @@ -292,7 +282,7 @@ impl Stream for ReadMergeStream<'_> { } } -#[derive(Debug, PartialEq)] +#[derive(Debug, PartialEq, Clone)] pub enum ReadValues { I64(Vec>), F64(Vec>), @@ -310,7 +300,7 @@ impl PartialOrd for ReadPoint { } } -#[derive(Debug, PartialEq)] +#[derive(Debug, PartialEq, Clone)] pub struct ReadBatch { pub key: String, pub values: ReadValues, @@ -323,13 +313,13 @@ impl ReadBatch { if vals.is_empty() { return None; } - Some((vals[0].time, vals[vals.len() - 1].time)) + Some((vals.first()?.time, vals.last()?.time)) } ReadValues::F64(vals) => { if vals.is_empty() { return None; } - Some((vals[0].time, vals[vals.len() - 1].time)) + Some((vals.first()?.time, vals.last()?.time)) } } } @@ -350,7 +340,6 @@ impl ReadBatch { match pos { None => { vals.append(other_vals); - other_vals.clear(); true } Some(pos) => { From 215c1138ea60f5c38503110a36fd802015f1e510 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 22 Mar 2020 18:08:04 -0400 Subject: [PATCH 03/42] Update memdb and partitioned_store Run fmt and fix test failure for size check on memdb. --- src/storage/memdb.rs | 33 +++++++++----- src/storage/partitioned_store.rs | 76 +++++++++++++++----------------- 2 files changed, 59 insertions(+), 50 deletions(-) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index 835a9dc80f..8ccd154b77 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -1,9 +1,9 @@ use crate::delorean::{Node, Predicate, TimestampRange}; use crate::line_parser::{ParseError, Point, PointType}; -use crate::storage::predicate::{Evaluate, EvaluateVisitor}; -use crate::storage::{SeriesDataType, StorageError}; use crate::storage::partitioned_store::{ReadBatch, ReadValues}; +use crate::storage::predicate::{Evaluate, EvaluateVisitor}; use crate::storage::series_store::ReadPoint; +use crate::storage::{SeriesDataType, StorageError}; use croaring::Treemap; use futures::stream::{self, BoxStream}; @@ -41,7 +41,9 @@ impl SeriesBuffer { None => return vec![], }; - let stop = self.values[start..].iter().position(|val| val.time >= range.end); + let stop = self.values[start..] + .iter() + .position(|val| val.time >= range.end); let stop = stop.unwrap_or_else(|| self.values.len()); self.values[start..stop].to_vec() @@ -72,7 +74,9 @@ impl StoreInSeriesData for Point { match series_data.i64_series.get_mut(&self.series_id.unwrap()) { Some(buff) => buff.values.push(point), None => { - let buff = SeriesBuffer { values: vec![point] }; + let buff = SeriesBuffer { + values: vec![point], + }; series_data.i64_series.insert(self.series_id.unwrap(), buff); } } @@ -90,7 +94,9 @@ impl StoreInSeriesData for Point { match series_data.f64_series.get_mut(&self.series_id.unwrap()) { Some(buff) => buff.values.push(point), None => { - let buff = SeriesBuffer { values: vec![point] }; + let buff = SeriesBuffer { + values: vec![point], + }; series_data.f64_series.insert(self.series_id.unwrap(), buff); } } @@ -137,7 +143,8 @@ impl SeriesMap { // update the estimated size of the map. This is a rough estimate based on // having the series key twice (once in the map to ID and again in the ID // to map. And then adding another 16 bytes for the two IDs - self.current_size += point.series().len() * SeriesMap::SERIES_KEY_COPIES + SeriesMap::SERIES_ID_BYTES; + self.current_size += + point.series().len() * SeriesMap::SERIES_KEY_COPIES + SeriesMap::SERIES_ID_BYTES; for pair in point.index_pairs()? { // insert this id into the posting list @@ -285,19 +292,25 @@ mod tests { fn write_and_read_tag_keys() { let memdb = setup_db(); let tag_keys = memdb - .get_tag_keys(&TimestampRange { start: 0, end: 0 }, &Predicate{root: None}) + .get_tag_keys( + &TimestampRange { start: 0, end: 0 }, + &Predicate { root: None }, + ) .unwrap(); let tag_keys: Vec<_> = futures::executor::block_on_stream(tag_keys).collect(); assert_eq!(tag_keys, vec!["_f", "_m", "host", "region"]); - } #[test] fn write_and_read_tag_values() { let memdb = setup_db(); let tag_values = memdb - .get_tag_values("host", &TimestampRange { start: 0, end: 0 }, &Predicate{root: None}) + .get_tag_values( + "host", + &TimestampRange { start: 0, end: 0 }, + &Predicate { root: None }, + ) .unwrap(); let tag_values: Vec<_> = futures::executor::block_on_stream(tag_values).collect(); assert_eq!(tag_values, vec!["a", "b"]); @@ -306,7 +319,7 @@ mod tests { #[test] fn write_and_check_size() { let memdb = setup_db(); - assert_eq!(memdb.size(), 1000); + assert_eq!(memdb.size(), 904); } #[test] diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 13f5f6ced3..ab3db8c708 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -5,11 +5,11 @@ use crate::line_parser::PointType; use crate::storage::series_store::ReadPoint; use crate::storage::StorageError; -use std::pin::Pin; -use std::mem; -use std::task::{Context, Poll}; -use std::cmp::Ordering; use futures::stream::{BoxStream, Stream}; +use std::cmp::Ordering; +use std::mem; +use std::pin::Pin; +use std::task::{Context, Poll}; /// A Partition is a block of data. It has methods for reading the metadata like which measurements, /// tags, tag values, and fields exist. Along with the raw time series data. It is designed to work @@ -120,13 +120,13 @@ impl Stream for StringMergeStream<'_> { if let Poll::Ready(Some(s)) = &self.next_vals[pos] { match &self.next_vals[next_pos] { Poll::Ready(None) => next_pos = pos, - Poll::Ready(Some(next)) => { - match next.cmp(s) { - Ordering::Greater => next_pos = pos, - Ordering::Equal => self.next_vals[pos] = self.streams[pos].as_mut().poll_next(cx), - Ordering::Less => (), + Poll::Ready(Some(next)) => match next.cmp(s) { + Ordering::Greater => next_pos = pos, + Ordering::Equal => { + self.next_vals[pos] = self.streams[pos].as_mut().poll_next(cx) } - } + Ordering::Less => (), + }, Poll::Pending => return Poll::Pending, } } @@ -185,24 +185,22 @@ impl Stream for ReadMergeStream<'_> { Poll::Pending => one_pending = true, Poll::Ready(Some(batch)) => { match &min_key { - Some(k) => { - match batch.key.cmp(k) { - Ordering::Less => { - min_key = Some(batch.key.clone()); - let (_, min) = batch.start_stop_times().unwrap(); + Some(k) => match batch.key.cmp(k) { + Ordering::Less => { + min_key = Some(batch.key.clone()); + let (_, min) = batch.start_stop_times().unwrap(); + min_pos = pos; + min_time = min; + } + Ordering::Equal => { + let (_, min) = batch.start_stop_times().unwrap(); + if min < min_time { min_pos = pos; min_time = min; - }, - Ordering::Equal => { - let (_, min) = batch.start_stop_times().unwrap(); - if min < min_time { - min_pos = pos; - min_time = min; - } - }, - Ordering::Greater => (), + } } - } + Ordering::Greater => (), + }, None => { min_key = Some(batch.key.clone()); let (_, min) = batch.start_stop_times().unwrap(); @@ -216,24 +214,22 @@ impl Stream for ReadMergeStream<'_> { }, Poll::Ready(None) => (), Poll::Ready(Some(batch)) => match &min_key { - Some(k) => { - match batch.key.cmp(k) { - Ordering::Less => { - min_key = Some(batch.key.clone()); - let (_, min) = batch.start_stop_times().unwrap(); + Some(k) => match batch.key.cmp(k) { + Ordering::Less => { + min_key = Some(batch.key.clone()); + let (_, min) = batch.start_stop_times().unwrap(); + min_pos = pos; + min_time = min; + } + Ordering::Equal => { + let (_, min) = batch.start_stop_times().unwrap(); + if min < min_time { min_pos = pos; min_time = min; - }, - Ordering::Equal => { - let (_, min) = batch.start_stop_times().unwrap(); - if min < min_time { - min_pos = pos; - min_time = min; - } - }, - Ordering::Greater => (), + } } - } + Ordering::Greater => (), + }, None => { min_key = Some(batch.key.clone()); let (_, min) = batch.start_stop_times().unwrap(); From 23e0b097ca65fbd806f041f20c4692d56a341deb Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 25 Mar 2020 10:00:44 -0400 Subject: [PATCH 04/42] fix: Use sort_by_key on time rather than implementing traits Needed because the PartialOrd/Ord trait implementations would then disagree with the PartialEq/Eq trait implementations, which isn't really what's desired. Change the method name to make it clearer that this isn't using the Ord trait's sort. --- src/storage/partitioned_store.rs | 20 ++++---------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index ab3db8c708..579aafe1ea 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -271,7 +271,7 @@ impl Stream for ReadMergeStream<'_> { } if sort { - batch.sort(); + batch.sort_by_time(); } Poll::Ready(Some(batch)) @@ -284,18 +284,6 @@ pub enum ReadValues { F64(Vec>), } -impl Ord for ReadPoint { - fn cmp(&self, other: &Self) -> Ordering { - self.time.cmp(&other.time) - } -} - -impl PartialOrd for ReadPoint { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - #[derive(Debug, PartialEq, Clone)] pub struct ReadBatch { pub key: String, @@ -320,10 +308,10 @@ impl ReadBatch { } } - fn sort(&mut self) { + fn sort_by_time(&mut self) { match &mut self.values { - ReadValues::I64(vals) => vals.sort(), - ReadValues::F64(vals) => vals.sort_by(|a, b| a.time.partial_cmp(&b.time).unwrap()), + ReadValues::I64(vals) => vals.sort_by_key(|v| v.time), + ReadValues::F64(vals) => vals.sort_by_key(|v| v.time), } } From 1b4b3fe7ad99c36e2f12edeebb81c2fd64b52eb5 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Wed, 25 Mar 2020 10:03:08 -0400 Subject: [PATCH 05/42] refactor: Rename a variable to make its purpose clearer I was reading this as "sortED" which is the opposite of what this means. --- src/storage/partitioned_store.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 579aafe1ea..a517b26d0d 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -257,7 +257,7 @@ impl Stream for ReadMergeStream<'_> { _ => unreachable!(), }; - let mut sort = false; + let mut sort_needed = false; for pos in 0..self.next_vals.len() { if let Poll::Ready(Some(b)) = &mut self.next_vals[pos] { @@ -265,12 +265,12 @@ impl Stream for ReadMergeStream<'_> { if batch.append_below_time(b, min_time) { self.next_vals[pos] = Poll::Pending; } - sort = true; + sort_needed = true; } } } - if sort { + if sort_needed { batch.sort_by_time(); } From 97aedbd167dd1d70d79ffa2330304b23c9353dc2 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 26 Mar 2020 11:15:10 -0400 Subject: [PATCH 06/42] fix: Remove redundant empty checks --- src/storage/partitioned_store.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index a517b26d0d..cbda539807 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -294,15 +294,9 @@ impl ReadBatch { fn start_stop_times(&self) -> Option<(i64, i64)> { match &self.values { ReadValues::I64(vals) => { - if vals.is_empty() { - return None; - } Some((vals.first()?.time, vals.last()?.time)) } ReadValues::F64(vals) => { - if vals.is_empty() { - return None; - } Some((vals.first()?.time, vals.last()?.time)) } } From f223fe408b191de491b40804ae2d4b14315d911e Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 26 Mar 2020 11:19:05 -0400 Subject: [PATCH 07/42] fix: Remove redundant clear --- src/storage/partitioned_store.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index cbda539807..20f3487c38 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -333,7 +333,6 @@ impl ReadBatch { match pos { None => { vals.append(other_vals); - other_vals.clear(); true } Some(pos) => { From 4375afd85cb4e405f2e24518f56bf0f03571b3d0 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 26 Mar 2020 13:28:32 -0400 Subject: [PATCH 08/42] Update partition_store Updated StreamState to use a single vector to track the streams and the next value. --- src/storage/partitioned_store.rs | 66 ++++++++++++++++---------------- 1 file changed, 34 insertions(+), 32 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index a517b26d0d..7df5315e0d 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -69,18 +69,21 @@ pub trait Partition { /// is used for combining results from multiple partitions for calls to get measurements, tag keys, /// tag values, or field keys. It assumes the incoming streams are in sorted order. pub struct StringMergeStream<'a> { - streams: Vec>, - next_vals: Vec>>, + streams: Vec>, drained: bool, } +struct StreamState<'a, T> { + stream: BoxStream<'a, T>, + next: Poll>, +} + impl StringMergeStream<'_> { fn new(streams: Vec>) -> StringMergeStream<'_> { - let len = streams.len(); + let streams = streams.into_iter().map(|s| StreamState{stream: s, next: Poll::Pending}).collect(); StringMergeStream { streams, - next_vals: vec![Poll::Pending; len], drained: false, } } @@ -96,17 +99,10 @@ impl Stream for StringMergeStream<'_> { let mut one_pending = false; - for pos in 0..self.next_vals.len() { - match self.next_vals[pos] { - Poll::Pending => { - let v = self.streams[pos].as_mut().poll_next(cx); - if v.is_pending() { - one_pending = true; - } - - self.next_vals[pos] = v; - } - Poll::Ready(_) => (), + for state in self.streams.iter_mut() { + if state.next.is_pending() { + state.next = state.stream.as_mut().poll_next(cx); + one_pending = one_pending || state.next.is_pending(); } } @@ -114,30 +110,37 @@ impl Stream for StringMergeStream<'_> { return Poll::Pending; } + let mut next_val: Option = None; let mut next_pos = 0; - for pos in 1..self.next_vals.len() { - if let Poll::Ready(Some(s)) = &self.next_vals[pos] { - match &self.next_vals[next_pos] { - Poll::Ready(None) => next_pos = pos, - Poll::Ready(Some(next)) => match next.cmp(s) { - Ordering::Greater => next_pos = pos, - Ordering::Equal => { - self.next_vals[pos] = self.streams[pos].as_mut().poll_next(cx) - } - Ordering::Less => (), - }, - Poll::Pending => return Poll::Pending, - } + for (pos, state) in self.streams.iter_mut().enumerate() { + match (&next_val, &state.next) { + (None, Poll::Ready(Some(ref val))) => { + next_val = Some(val.clone()); + next_pos = pos; + }, + (Some(next), Poll::Ready(Some(ref val))) => { + if next > val { + next_val = Some(val.clone()); + next_pos = pos; + + } else if next == val { + state.next = state.stream.as_mut().poll_next(cx); + } + }, + (Some(_), Poll::Ready(None)) => (), + _ => unreachable!(), } } - let val = mem::replace(&mut self.next_vals[next_pos], Poll::Pending); - - if let Poll::Ready(None) = val { + if next_val.is_none() { self.drained = true; + return Poll::Ready(None); } + let next_state: &mut StreamState<'_, String> = &mut self.streams[next_pos]; + + let val = mem::replace(&mut next_state.next, next_state.stream.as_mut().poll_next(cx)); val } } @@ -339,7 +342,6 @@ impl ReadBatch { match pos { None => { vals.append(other_vals); - other_vals.clear(); true } Some(pos) => { From aa18015c29ee12f3caef27285f98bc3ed51be719 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 26 Mar 2020 16:23:58 -0400 Subject: [PATCH 09/42] Update partition_store and memdb Update ReadMergeStream to use a single vec to store stream state. Update ReadBatch to always assume values in the batch. Update memdb to not put batches with no values in the result set. --- src/storage/memdb.rs | 4 + src/storage/partitioned_store.rs | 197 ++++++++++++++----------------- 2 files changed, 94 insertions(+), 107 deletions(-) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index 8ccd154b77..096d6f77df 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -259,6 +259,10 @@ impl MemDB { } }; + if values.is_empty() { + continue; + } + let batch = ReadBatch { key: key.to_string(), values, diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 7df5315e0d..9698844c5c 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -10,6 +10,7 @@ use std::cmp::Ordering; use std::mem; use std::pin::Pin; use std::task::{Context, Poll}; +use futures::StreamExt; /// A Partition is a block of data. It has methods for reading the metadata like which measurements, /// tags, tag values, and fields exist. Along with the raw time series data. It is designed to work @@ -69,7 +70,7 @@ pub trait Partition { /// is used for combining results from multiple partitions for calls to get measurements, tag keys, /// tag values, or field keys. It assumes the incoming streams are in sorted order. pub struct StringMergeStream<'a> { - streams: Vec>, + states: Vec>, drained: bool, } @@ -80,10 +81,10 @@ struct StreamState<'a, T> { impl StringMergeStream<'_> { fn new(streams: Vec>) -> StringMergeStream<'_> { - let streams = streams.into_iter().map(|s| StreamState{stream: s, next: Poll::Pending}).collect(); + let states = streams.into_iter().map(|s| StreamState{stream: s, next: Poll::Pending}).collect(); StringMergeStream { - streams, + states, drained: false, } } @@ -99,7 +100,7 @@ impl Stream for StringMergeStream<'_> { let mut one_pending = false; - for state in self.streams.iter_mut() { + for state in self.states.iter_mut() { if state.next.is_pending() { state.next = state.stream.as_mut().poll_next(cx); one_pending = one_pending || state.next.is_pending(); @@ -113,7 +114,7 @@ impl Stream for StringMergeStream<'_> { let mut next_val: Option = None; let mut next_pos = 0; - for (pos, state) in self.streams.iter_mut().enumerate() { + for (pos, state) in self.states.iter_mut().enumerate() { match (&next_val, &state.next) { (None, Poll::Ready(Some(ref val))) => { next_val = Some(val.clone()); @@ -129,6 +130,7 @@ impl Stream for StringMergeStream<'_> { } }, (Some(_), Poll::Ready(None)) => (), + (None, Poll::Ready(None)) => (), _ => unreachable!(), } } @@ -138,7 +140,7 @@ impl Stream for StringMergeStream<'_> { return Poll::Ready(None); } - let next_state: &mut StreamState<'_, String> = &mut self.streams[next_pos]; + let next_state: &mut StreamState<'_, String> = &mut self.states[next_pos]; let val = mem::replace(&mut next_state.next, next_state.stream.as_mut().poll_next(cx)); val @@ -150,18 +152,16 @@ impl Stream for StringMergeStream<'_> { /// where multiple partitions have batches with the same key, they are merged together in time /// ascending order. For any given key, multiple read batches can come through. pub struct ReadMergeStream<'a> { - streams: Vec>, - next_vals: Vec>>, + states: Vec>, drained: bool, } impl ReadMergeStream<'_> { fn new(streams: Vec>) -> ReadMergeStream<'_> { - let len = streams.len(); + let states = streams.into_iter().map(|s| StreamState{stream: s, next: Poll::Pending}).collect(); ReadMergeStream { - streams, - next_vals: vec![Poll::Pending; len], + states, drained: false, } } @@ -175,71 +175,13 @@ impl Stream for ReadMergeStream<'_> { return Poll::Ready(None); } - let mut min_key: Option = None; - let mut min_time = std::i64::MAX; - let mut min_pos = 0; - + // ensure that every stream in pending state is called next and return if any are still pending let mut one_pending = false; - // find the key that we should send next and make sure that things are populated - for pos in 0..self.next_vals.len() { - match &self.next_vals[pos] { - Poll::Pending => match self.streams[pos].as_mut().poll_next(cx) { - Poll::Pending => one_pending = true, - Poll::Ready(Some(batch)) => { - match &min_key { - Some(k) => match batch.key.cmp(k) { - Ordering::Less => { - min_key = Some(batch.key.clone()); - let (_, min) = batch.start_stop_times().unwrap(); - min_pos = pos; - min_time = min; - } - Ordering::Equal => { - let (_, min) = batch.start_stop_times().unwrap(); - if min < min_time { - min_pos = pos; - min_time = min; - } - } - Ordering::Greater => (), - }, - None => { - min_key = Some(batch.key.clone()); - let (_, min) = batch.start_stop_times().unwrap(); - min_pos = pos; - min_time = min; - } - } - self.next_vals[pos] = Poll::Ready(Some(batch)); - } - Poll::Ready(None) => self.next_vals[pos] = Poll::Ready(None), - }, - Poll::Ready(None) => (), - Poll::Ready(Some(batch)) => match &min_key { - Some(k) => match batch.key.cmp(k) { - Ordering::Less => { - min_key = Some(batch.key.clone()); - let (_, min) = batch.start_stop_times().unwrap(); - min_pos = pos; - min_time = min; - } - Ordering::Equal => { - let (_, min) = batch.start_stop_times().unwrap(); - if min < min_time { - min_pos = pos; - min_time = min; - } - } - Ordering::Greater => (), - }, - None => { - min_key = Some(batch.key.clone()); - let (_, min) = batch.start_stop_times().unwrap(); - min_pos = pos; - min_time = min; - } - }, + for state in self.states.iter_mut() { + if state.next.is_pending() { + state.next = state.stream.as_mut().poll_next(cx); + one_pending = one_pending || state.next.is_pending(); } } @@ -247,37 +189,75 @@ impl Stream for ReadMergeStream<'_> { return Poll::Pending; } - if min_key.is_none() { - self.drained = true; - return Poll::Ready(None); - } - let min_key = min_key.unwrap(); + // find the minimum key for the next batch and keep track of the other batches that have + // the same key + let mut next_min_key: Option = None; + let mut min_time = std::i64::MAX; + let mut min_pos = 0; + let mut positions = Vec::with_capacity(self.states.len()); - let batch = mem::replace(&mut self.next_vals[min_pos], Poll::Pending); - - let mut batch = match batch { - Poll::Ready(Some(b)) => b, - _ => unreachable!(), - }; - - let mut sort_needed = false; - - for pos in 0..self.next_vals.len() { - if let Poll::Ready(Some(b)) = &mut self.next_vals[pos] { - if b.key == min_key { - if batch.append_below_time(b, min_time) { - self.next_vals[pos] = Poll::Pending; + for (pos, state) in self.states.iter().enumerate() { + match (&next_min_key, &state.next) { + (None, Poll::Ready(Some(batch))) => { + next_min_key = Some(batch.key.clone()); + min_pos = pos; + let (_, t) = batch.start_stop_times(); + min_time = t; + }, + (Some(min_key), Poll::Ready(Some(batch))) => { + if min_key > &batch.key { + next_min_key = Some(batch.key.clone()); + min_pos = pos; + positions = Vec::with_capacity(self.states.len()); + let (_, t) = batch.start_stop_times(); + min_time = t; + } else if min_key == &batch.key { + // if this batch has an end time less than the existing min time, make this + // the batch that we want to pull out first + let (_, t) = batch.start_stop_times(); + if t < min_time { + min_time = t; + positions.push(min_pos); + min_pos = pos; + } else { + positions.push(pos); + } } - sort_needed = true; - } + }, + (Some(_), Poll::Ready(None)) => (), + (None, Poll::Ready(None)) => (), + _ => unreachable!(), } } - if sort_needed { - batch.sort_by_time(); + if next_min_key.is_none() { + self.drained = true; + return Poll::Ready(None); } - Poll::Ready(Some(batch)) + let mut val = mem::replace(&mut self.states[min_pos].next, Poll::Pending); + + if positions.is_empty() { + return val; + } + + // pull out all the values with times less than the end time from the val batch + match &mut val { + Poll::Ready(Some(batch)) => { + for pos in positions { + if let Poll::Ready(Some(b)) = &mut self.states[pos].next { + if batch.append_below_time(b, min_time) { + let _ = mem::replace(&mut self.states[pos].next, Poll::Pending); + } + } + } + + batch.sort_by_time(); + }, + _ => unreachable!(), + } + + val } } @@ -287,6 +267,15 @@ pub enum ReadValues { F64(Vec>), } +impl ReadValues { + pub fn is_empty(&self) -> bool { + match self { + ReadValues::I64(vals) => vals.is_empty(), + ReadValues::F64(vals) => vals.is_empty(), + } + } +} + #[derive(Debug, PartialEq, Clone)] pub struct ReadBatch { pub key: String, @@ -294,19 +283,13 @@ pub struct ReadBatch { } impl ReadBatch { - fn start_stop_times(&self) -> Option<(i64, i64)> { + fn start_stop_times(&self) -> (i64, i64) { match &self.values { ReadValues::I64(vals) => { - if vals.is_empty() { - return None; - } - Some((vals.first()?.time, vals.last()?.time)) + (vals.first().unwrap().time, vals.last().unwrap().time) } ReadValues::F64(vals) => { - if vals.is_empty() { - return None; - } - Some((vals.first()?.time, vals.last()?.time)) + (vals.first().unwrap().time, vals.last().unwrap().time) } } } From fa8f04ddb355014700d227f7f791bcb243389e88 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 09:13:48 -0400 Subject: [PATCH 10/42] style: Cargo fmt --- src/storage/partitioned_store.rs | 42 ++++++++++++++++++++------------ 1 file changed, 26 insertions(+), 16 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 7092207597..7e1a67301f 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -6,11 +6,11 @@ use crate::storage::series_store::ReadPoint; use crate::storage::StorageError; use futures::stream::{BoxStream, Stream}; +use futures::StreamExt; use std::cmp::Ordering; use std::mem; use std::pin::Pin; use std::task::{Context, Poll}; -use futures::StreamExt; /// A Partition is a block of data. It has methods for reading the metadata like which measurements, /// tags, tag values, and fields exist. Along with the raw time series data. It is designed to work @@ -81,7 +81,13 @@ struct StreamState<'a, T> { impl StringMergeStream<'_> { fn new(streams: Vec>) -> StringMergeStream<'_> { - let states = streams.into_iter().map(|s| StreamState{stream: s, next: Poll::Pending}).collect(); + let states = streams + .into_iter() + .map(|s| StreamState { + stream: s, + next: Poll::Pending, + }) + .collect(); StringMergeStream { states, @@ -119,16 +125,15 @@ impl Stream for StringMergeStream<'_> { (None, Poll::Ready(Some(ref val))) => { next_val = Some(val.clone()); next_pos = pos; - }, + } (Some(next), Poll::Ready(Some(ref val))) => { if next > val { next_val = Some(val.clone()); next_pos = pos; - } else if next == val { state.next = state.stream.as_mut().poll_next(cx); } - }, + } (Some(_), Poll::Ready(None)) => (), (None, Poll::Ready(None)) => (), _ => unreachable!(), @@ -142,7 +147,10 @@ impl Stream for StringMergeStream<'_> { let next_state: &mut StreamState<'_, String> = &mut self.states[next_pos]; - let val = mem::replace(&mut next_state.next, next_state.stream.as_mut().poll_next(cx)); + let val = mem::replace( + &mut next_state.next, + next_state.stream.as_mut().poll_next(cx), + ); val } } @@ -158,7 +166,13 @@ pub struct ReadMergeStream<'a> { impl ReadMergeStream<'_> { fn new(streams: Vec>) -> ReadMergeStream<'_> { - let states = streams.into_iter().map(|s| StreamState{stream: s, next: Poll::Pending}).collect(); + let states = streams + .into_iter() + .map(|s| StreamState { + stream: s, + next: Poll::Pending, + }) + .collect(); ReadMergeStream { states, @@ -203,7 +217,7 @@ impl Stream for ReadMergeStream<'_> { min_pos = pos; let (_, t) = batch.start_stop_times(); min_time = t; - }, + } (Some(min_key), Poll::Ready(Some(batch))) => { if min_key > &batch.key { next_min_key = Some(batch.key.clone()); @@ -223,7 +237,7 @@ impl Stream for ReadMergeStream<'_> { positions.push(pos); } } - }, + } (Some(_), Poll::Ready(None)) => (), (None, Poll::Ready(None)) => (), _ => unreachable!(), @@ -253,7 +267,7 @@ impl Stream for ReadMergeStream<'_> { } batch.sort_by_time(); - }, + } _ => unreachable!(), } @@ -285,12 +299,8 @@ pub struct ReadBatch { impl ReadBatch { fn start_stop_times(&self) -> (i64, i64) { match &self.values { - ReadValues::I64(vals) => { - (vals.first().unwrap().time, vals.last().unwrap().time) - }, - ReadValues::F64(vals) => { - (vals.first().unwrap().time, vals.last().unwrap().time) - }, + ReadValues::I64(vals) => (vals.first().unwrap().time, vals.last().unwrap().time), + ReadValues::F64(vals) => (vals.first().unwrap().time, vals.last().unwrap().time), } } From 48954424e18e813738346fd4dd58811cdfce45f4 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 09:14:09 -0400 Subject: [PATCH 11/42] docs: Add some documentation and TODO comments --- src/storage/memdb.rs | 2 ++ src/storage/partitioned_store.rs | 6 ++++++ 2 files changed, 8 insertions(+) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index 096d6f77df..b31fd6d3cc 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -259,6 +259,8 @@ impl MemDB { } }; + // TODO: Encode in the type system that `ReadBatch`es will never be created with an + // empty vector, as we're doing here. if values.is_empty() { continue; } diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 7e1a67301f..48e6ac1c42 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -275,6 +275,7 @@ impl Stream for ReadMergeStream<'_> { } } +// TODO: Make a constructor function that fails if given an empty `Vec` of `ReadPoint`s. #[derive(Debug, PartialEq, Clone)] pub enum ReadValues { I64(Vec>), @@ -297,6 +298,11 @@ pub struct ReadBatch { } impl ReadBatch { + /// Returns the first time and the last time in the batch. + /// + /// # Panics + /// + /// Will panic if there are no values in the `ReadValues`. fn start_stop_times(&self) -> (i64, i64) { match &self.values { ReadValues::I64(vals) => (vals.first().unwrap().time, vals.last().unwrap().time), From fed0a5e8b02fdea9fd1cfce268f80aa56fd9a3bf Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 09:23:51 -0400 Subject: [PATCH 12/42] fix: Allow dead code temporarily while partition is being refactored --- src/storage/memdb.rs | 2 ++ src/storage/partitioned_store.rs | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index b31fd6d3cc..93a8664f6f 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -1,3 +1,5 @@ +#![allow(dead_code)] + use crate::delorean::{Node, Predicate, TimestampRange}; use crate::line_parser::{ParseError, Point, PointType}; use crate::storage::partitioned_store::{ReadBatch, ReadValues}; diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 48e6ac1c42..144d6f2487 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -80,6 +80,7 @@ struct StreamState<'a, T> { } impl StringMergeStream<'_> { + #[allow(dead_code)] fn new(streams: Vec>) -> StringMergeStream<'_> { let states = streams .into_iter() @@ -165,6 +166,7 @@ pub struct ReadMergeStream<'a> { } impl ReadMergeStream<'_> { + #[allow(dead_code)] fn new(streams: Vec>) -> ReadMergeStream<'_> { let states = streams .into_iter() From 9411391c4c6604da5f1492be57a490f71122ff72 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 09:24:02 -0400 Subject: [PATCH 13/42] fix: Remove unused imports --- src/storage/partitioned_store.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 144d6f2487..03870bbdd5 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -6,8 +6,6 @@ use crate::storage::series_store::ReadPoint; use crate::storage::StorageError; use futures::stream::{BoxStream, Stream}; -use futures::StreamExt; -use std::cmp::Ordering; use std::mem; use std::pin::Pin; use std::task::{Context, Poll}; From 431f4c56339648664e6a21ba0969e89d754d076e Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 09:30:18 -0400 Subject: [PATCH 14/42] refactor: Remove unneeded let binding --- src/storage/partitioned_store.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 03870bbdd5..d34a7ee600 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -146,11 +146,10 @@ impl Stream for StringMergeStream<'_> { let next_state: &mut StreamState<'_, String> = &mut self.states[next_pos]; - let val = mem::replace( + mem::replace( &mut next_state.next, next_state.stream.as_mut().poll_next(cx), - ); - val + ) } } From 533deedbdde78981a80835f47622476c70faee3d Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 10:41:44 -0400 Subject: [PATCH 15/42] fix: Allow if-elses rather than matches --- src/lib.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/lib.rs b/src/lib.rs index 3163074bb5..fac57debfa 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -1,5 +1,6 @@ #![deny(rust_2018_idioms)] #![warn(clippy::explicit_iter_loop)] +#![allow(clippy::comparison_chain)] use std::{error, fmt}; From ace1060388338564bc25f2350b0abd5bbffc3376 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 10:42:44 -0400 Subject: [PATCH 16/42] refactor: Take the advice of the clippy::explicit_iter_loop lint --- src/storage/partitioned_store.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index d34a7ee600..5e020f23a2 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -105,7 +105,7 @@ impl Stream for StringMergeStream<'_> { let mut one_pending = false; - for state in self.states.iter_mut() { + for state in &mut self.states { if state.next.is_pending() { state.next = state.stream.as_mut().poll_next(cx); one_pending = one_pending || state.next.is_pending(); @@ -191,7 +191,7 @@ impl Stream for ReadMergeStream<'_> { // ensure that every stream in pending state is called next and return if any are still pending let mut one_pending = false; - for state in self.states.iter_mut() { + for state in &mut self.states { if state.next.is_pending() { state.next = state.stream.as_mut().poll_next(cx); one_pending = one_pending || state.next.is_pending(); From 37e1526933a43f2cb170f2f3d17302c58b42214c Mon Sep 17 00:00:00 2001 From: Jake Goulding Date: Fri, 27 Mar 2020 13:48:46 -0400 Subject: [PATCH 17/42] refactor: DRY up the Point -> ReadPoint conversion code --- src/storage/memdb.rs | 10 ++-------- src/storage/series_store.rs | 7 +++++++ 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index 93a8664f6f..7fb05cc36f 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -67,10 +67,7 @@ impl StoreInSeriesData for PointType { impl StoreInSeriesData for Point { fn write(&self, series_data: &mut SeriesData) { - let point = ReadPoint { - time: self.time, - value: self.value, - }; + let point: ReadPoint<_> = self.into(); series_data.current_size += std::mem::size_of::>(); match series_data.i64_series.get_mut(&self.series_id.unwrap()) { @@ -87,10 +84,7 @@ impl StoreInSeriesData for Point { impl StoreInSeriesData for Point { fn write(&self, series_data: &mut SeriesData) { - let point = ReadPoint { - time: self.time, - value: self.value, - }; + let point: ReadPoint<_> = self.into(); series_data.current_size += std::mem::size_of::>(); match series_data.f64_series.get_mut(&self.series_id.unwrap()) { diff --git a/src/storage/series_store.rs b/src/storage/series_store.rs index 41dafe391c..772fe206cc 100644 --- a/src/storage/series_store.rs +++ b/src/storage/series_store.rs @@ -32,6 +32,13 @@ pub struct ReadPoint { pub value: T, } +impl From<&'_ crate::line_parser::Point> for ReadPoint { + fn from(other: &'_ crate::line_parser::Point) -> Self { + let crate::line_parser::Point { time, value, .. } = other; + Self { time: *time, value: *value } + } +} + // Test helpers for other implementations to run #[cfg(test)] pub mod tests { From 2ab44b26037663503c355d6383a3ff2afbae52be Mon Sep 17 00:00:00 2001 From: Jake Goulding Date: Fri, 27 Mar 2020 13:51:16 -0400 Subject: [PATCH 18/42] doc: use doc comments instead of code comments --- src/storage/memdb.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index 7fb05cc36f..f8578c37e5 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -110,11 +110,12 @@ struct SeriesMap { } impl SeriesMap { - // SERIES_KEY_COPIES are the number of copies of the key this map contains. This is - // used to provide a rough estimate of the memory size. + /// The number of copies of the key this map contains. This is + /// used to provide a rough estimate of the memory size. const SERIES_KEY_COPIES: usize = 2; - // SERIES_ID_BYTES is the number of bytes the different copies of the series ID in - // this map represents. It's used to provide a rough estimate of the memory size. + /// The number of bytes the different copies of the series ID in + /// this map represents. This is used to provide a rough estimate + /// of the memory size. const SERIES_ID_BYTES: usize = 24; fn insert_series(&mut self, point: &mut PointType) -> Result<(), ParseError> { From 90dbac2df52ab6537d267152c961cd5440cea2d4 Mon Sep 17 00:00:00 2001 From: Jake Goulding Date: Fri, 27 Mar 2020 13:56:05 -0400 Subject: [PATCH 19/42] doc: Moving details about size estimates to the doc comment --- src/storage/memdb.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index f8578c37e5..2bc68936ba 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -112,6 +112,11 @@ struct SeriesMap { impl SeriesMap { /// The number of copies of the key this map contains. This is /// used to provide a rough estimate of the memory size. + /// + /// It occurs: + /// + /// 1. in the map to ID + /// 2. in the ID to map const SERIES_KEY_COPIES: usize = 2; /// The number of bytes the different copies of the series ID in /// this map represents. This is used to provide a rough estimate @@ -137,9 +142,7 @@ impl SeriesMap { self.series_id_to_key_and_type .insert(self.last_id, (point.series().clone(), series_type)); - // update the estimated size of the map. This is a rough estimate based on - // having the series key twice (once in the map to ID and again in the ID - // to map. And then adding another 16 bytes for the two IDs + // update the estimated size of the map. self.current_size += point.series().len() * SeriesMap::SERIES_KEY_COPIES + SeriesMap::SERIES_ID_BYTES; From 1247544debe77b00b850521a9e7d4bbf38039f83 Mon Sep 17 00:00:00 2001 From: Jake Goulding Date: Fri, 27 Mar 2020 14:28:38 -0400 Subject: [PATCH 20/42] refactor: remove unneeded explicit types --- src/storage/partitioned_store.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 5e020f23a2..12c0eb3e24 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -372,7 +372,7 @@ mod tests { StringMergeStream::new(vec![one.boxed(), two.boxed(), three.boxed(), four.boxed()]); let stream = futures::executor::block_on_stream(merger); - let vals: Vec = stream.collect(); + let vals: Vec<_> = stream.collect(); assert_eq!( vals, @@ -465,7 +465,7 @@ mod tests { let merger = ReadMergeStream::new(vec![one.boxed(), two.boxed(), three.boxed(), four.boxed()]); let stream = futures::executor::block_on_stream(merger); - let vals: Vec = stream.collect(); + let vals: Vec<_> = stream.collect(); assert_eq!( vals, From 2eb09f29ef55732050f56c5eb65bab10b36424e1 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 15:52:51 -0400 Subject: [PATCH 21/42] refactor: Prefer match on the result of cmp to if/else if/else chains --- src/lib.rs | 1 - src/storage/partitioned_store.rs | 45 ++++++++++++++++++-------------- 2 files changed, 26 insertions(+), 20 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index fac57debfa..3163074bb5 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -1,6 +1,5 @@ #![deny(rust_2018_idioms)] #![warn(clippy::explicit_iter_loop)] -#![allow(clippy::comparison_chain)] use std::{error, fmt}; diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 12c0eb3e24..767f6554d1 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -6,6 +6,7 @@ use crate::storage::series_store::ReadPoint; use crate::storage::StorageError; use futures::stream::{BoxStream, Stream}; +use std::cmp::Ordering; use std::mem; use std::pin::Pin; use std::task::{Context, Poll}; @@ -125,14 +126,16 @@ impl Stream for StringMergeStream<'_> { next_val = Some(val.clone()); next_pos = pos; } - (Some(next), Poll::Ready(Some(ref val))) => { - if next > val { + (Some(next), Poll::Ready(Some(ref val))) => match next.cmp(val) { + Ordering::Greater => { next_val = Some(val.clone()); next_pos = pos; - } else if next == val { + } + Ordering::Equal => { state.next = state.stream.as_mut().poll_next(cx); } - } + _ => (), + }, (Some(_), Poll::Ready(None)) => (), (None, Poll::Ready(None)) => (), _ => unreachable!(), @@ -218,23 +221,27 @@ impl Stream for ReadMergeStream<'_> { min_time = t; } (Some(min_key), Poll::Ready(Some(batch))) => { - if min_key > &batch.key { - next_min_key = Some(batch.key.clone()); - min_pos = pos; - positions = Vec::with_capacity(self.states.len()); - let (_, t) = batch.start_stop_times(); - min_time = t; - } else if min_key == &batch.key { - // if this batch has an end time less than the existing min time, make this - // the batch that we want to pull out first - let (_, t) = batch.start_stop_times(); - if t < min_time { - min_time = t; - positions.push(min_pos); + match min_key.cmp(&batch.key) { + Ordering::Greater => { + next_min_key = Some(batch.key.clone()); min_pos = pos; - } else { - positions.push(pos); + positions = Vec::with_capacity(self.states.len()); + let (_, t) = batch.start_stop_times(); + min_time = t; } + Ordering::Equal => { + // if this batch has an end time less than the existing min time, make this + // the batch that we want to pull out first + let (_, t) = batch.start_stop_times(); + if t < min_time { + min_time = t; + positions.push(min_pos); + min_pos = pos; + } else { + positions.push(pos); + } + } + _ => (), } } (Some(_), Poll::Ready(None)) => (), From fa88416dff121134594cc21bfcf9a65b13c22330 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 27 Mar 2020 15:53:37 -0400 Subject: [PATCH 22/42] style: Cargo fmt --- src/storage/series_store.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/storage/series_store.rs b/src/storage/series_store.rs index 772fe206cc..24d2701494 100644 --- a/src/storage/series_store.rs +++ b/src/storage/series_store.rs @@ -35,7 +35,10 @@ pub struct ReadPoint { impl From<&'_ crate::line_parser::Point> for ReadPoint { fn from(other: &'_ crate::line_parser::Point) -> Self { let crate::line_parser::Point { time, value, .. } = other; - Self { time: *time, value: *value } + Self { + time: *time, + value: *value, + } } } From 834d5b84a6585ae826b810c52a75f5ebe0f0ef9b Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 27 Mar 2020 16:24:02 -0400 Subject: [PATCH 23/42] Update memdb and partitioned_store Fix bug in memdb with pulling out a range of data. Simplify code in partition store to address PR feedback. --- src/storage/memdb.rs | 10 +++--- src/storage/partitioned_store.rs | 55 ++++++++------------------------ 2 files changed, 19 insertions(+), 46 deletions(-) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index 2bc68936ba..ac7830fb94 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -38,12 +38,14 @@ struct SeriesBuffer { impl SeriesBuffer { fn read(&self, range: &TimestampRange) -> Vec> { - let start = match self.values.iter().position(|val| val.time >= range.start) { + let start = match self.values + .iter() + .position(|val| val.time >= range.start) { Some(pos) => pos, None => return vec![], }; - let stop = self.values[start..] + let stop = self.values .iter() .position(|val| val.time >= range.end); let stop = stop.unwrap_or_else(|| self.values.len()); @@ -68,7 +70,7 @@ impl StoreInSeriesData for PointType { impl StoreInSeriesData for Point { fn write(&self, series_data: &mut SeriesData) { let point: ReadPoint<_> = self.into(); - series_data.current_size += std::mem::size_of::>(); + series_data.current_size += std::mem::size_of::>(); match series_data.i64_series.get_mut(&self.series_id.unwrap()) { Some(buff) => buff.values.push(point), @@ -325,7 +327,7 @@ mod tests { #[test] fn write_and_check_size() { let memdb = setup_db(); - assert_eq!(memdb.size(), 904); + assert_eq!(memdb.size(), 704); } #[test] diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 767f6554d1..3bb48d6715 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -24,26 +24,6 @@ pub trait Partition { fn write(&self, points: &[PointType]) -> Result<(), StorageError>; - // fn measurements( - // &self, - // ) -> Result>>, StorageError>; - // - // fn tag_keys( - // &self, - // measurement: &str, - // ) -> Result>>, StorageError>; - // - // fn fields( - // &self, - // measurement: &str, - // ) -> Result>>, StorageError>; - // - // fn tag_values( - // &self, - // measurement: &str, - // key: &str, - // ) -> Result>>, StorageError>; - fn get_tag_keys( &self, range: &TimestampRange, @@ -67,7 +47,7 @@ pub trait Partition { /// StringMergeStream will do a merge sort with deduplication of multiple streams of Strings. This /// is used for combining results from multiple partitions for calls to get measurements, tag keys, -/// tag values, or field keys. It assumes the incoming streams are in sorted order. +/// tag values, or field keys. It assumes the incoming streams are in sorted order with no duplicates. pub struct StringMergeStream<'a> { states: Vec>, drained: bool, @@ -160,6 +140,11 @@ impl Stream for StringMergeStream<'_> { /// it will ensure that batches are sent through in lexographical order by key. In situations /// where multiple partitions have batches with the same key, they are merged together in time /// ascending order. For any given key, multiple read batches can come through. +/// +/// It assume that the input streams send batches in key lexographical order and that values are +/// always of the same type for a given key, and that those values are in time sorted order. A +/// stream can have multiple batches with the same key, as long as the values across those batches +/// are in time sorted order (ascending). pub struct ReadMergeStream<'a> { states: Vec>, drained: bool, @@ -267,7 +252,7 @@ impl Stream for ReadMergeStream<'_> { for pos in positions { if let Poll::Ready(Some(b)) = &mut self.states[pos].next { if batch.append_below_time(b, min_time) { - let _ = mem::replace(&mut self.states[pos].next, Poll::Pending); + self.states[pos].next = Poll::Pending; } } } @@ -330,32 +315,18 @@ impl ReadBatch { (ReadValues::I64(vals), ReadValues::I64(other_vals)) => { let pos = other_vals.iter().position(|val| val.time > t); match pos { - None => { - vals.append(other_vals); - true - } - Some(pos) => { - let mut rest = other_vals.split_off(pos); - vals.append(other_vals); - other_vals.append(&mut rest); - false - } + None => vals.append(other_vals), + Some(pos) => vals.extend(other_vals.drain(..pos)), } + other_vals.is_empty() } (ReadValues::F64(vals), ReadValues::F64(other_vals)) => { let pos = other_vals.iter().position(|val| val.time > t); match pos { - None => { - vals.append(other_vals); - true - } - Some(pos) => { - let mut rest = other_vals.split_off(pos); - vals.append(other_vals); - other_vals.append(&mut rest); - false - } + None => vals.append(other_vals), + Some(pos) => vals.extend(other_vals.drain(..pos)), } + other_vals.is_empty() } (_, _) => true, // do nothing here } From 38ec343b33f0b9ce96df87a20dc20dc5e526d58a Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 1 Apr 2020 16:46:07 -0400 Subject: [PATCH 24/42] Refactor partition_store, database, main, and rpc Refactored Partition to be a concrete type enum with three different types of partitions: memory, s3, or remote server. Had to move to this rather than a trait because the methods on partition are async. Refactored database to use the new partition structure and for now only use a memory based database. Persistence will be reintroduced later with the S3 partition implementation. Refactored main and rpc to use the new database. --- src/line_parser.rs | 2 +- src/main.rs | 97 +++----- src/rpc.rs | 182 +++++++------- src/storage.rs | 2 + src/storage/database.rs | 393 +++++++++++++++++++++++++------ src/storage/memdb.rs | 72 +++--- src/storage/partitioned_store.rs | 77 ++++-- src/storage/remote_partition.rs | 4 + src/storage/s3_partition.rs | 4 + 9 files changed, 552 insertions(+), 281 deletions(-) create mode 100644 src/storage/remote_partition.rs create mode 100644 src/storage/s3_partition.rs diff --git a/src/line_parser.rs b/src/line_parser.rs index c89d902b27..518e9c53c2 100644 --- a/src/line_parser.rs +++ b/src/line_parser.rs @@ -155,7 +155,7 @@ pub struct Pair { #[derive(Debug, Clone)] pub struct ParseError { - description: String, + pub description: String, } impl fmt::Display for ParseError { diff --git a/src/main.rs b/src/main.rs index 35e7fedd18..e5a2c3f8e0 100644 --- a/src/main.rs +++ b/src/main.rs @@ -10,8 +10,8 @@ use delorean::delorean::{ use delorean::line_parser; use delorean::line_parser::index_pairs; use delorean::storage::database::Database; +use delorean::storage::partitioned_store::ReadValues; use delorean::storage::predicate::parse_predicate; -use delorean::storage::SeriesDataType; use delorean::time::{parse_duration, time_as_i64_nanos}; use std::env::VarError; @@ -48,13 +48,14 @@ async fn write(req: hyper::Request, app: Arc) -> Result b, + let bucket_id = match bucket_id { + Some(id) => id, None => { // create this as the default bucket let b = Bucket { @@ -67,12 +68,9 @@ async fn write(req: hyper::Request, app: Arc) -> Result, app: Arc) -> Result, app: Arc) -> Result b, - None => { - return Err(ApplicationError::new( + .get_bucket_id_by_name(read_info.org_id, &read_info.bucket_name) + .await + .map_err(|_| StatusCode::INTERNAL_SERVER_ERROR)? + .ok_or_else(|| { + ApplicationError::new( StatusCode::NOT_FOUND, &format!("bucket {} not found", read_info.bucket_name), - )); - } - }; + ) + })?; - let series = app + let batches = app .db - .read_series_matching_predicate_and_range(&bucket, Some(&predicate), Some(&range)) + .read_points(read_info.org_id, bucket_id, &predicate, &range) + .await .map_err(|_| StatusCode::INTERNAL_SERVER_ERROR)?; - let db = &app.db; - let mut response_body = vec![]; - for s in series { + for batch in batches { let mut wtr = Writer::from_writer(vec![]); - let pairs = index_pairs(&s.key).map_err(|_| StatusCode::INTERNAL_SERVER_ERROR)?; + let pairs = index_pairs(&batch.key).map_err(|_| StatusCode::INTERNAL_SERVER_ERROR)?; let mut cols = Vec::with_capacity(pairs.len() + 2); let mut vals = Vec::with_capacity(pairs.len() + 2); @@ -200,40 +195,26 @@ async fn read(req: hyper::Request, app: Arc) -> Result { - let points = db - .read_i64_range(&bucket, &s, &range, 10) - .map_err(|_| StatusCode::INTERNAL_SERVER_ERROR)?; - - for batch in points { - for p in batch { - let t = p.time.to_string(); - let v = p.value.to_string(); - vals[vcol] = v; - vals[tcol] = t; - - wtr.write_record(&vals).unwrap(); - } + match batch.values { + ReadValues::I64(values) => { + for val in values { + let t = val.time.to_string(); + let v = val.time.to_string(); + vals[vcol] = v; + vals[tcol] = t; + wtr.write_record(&vals).unwrap(); } } - SeriesDataType::F64 => { - let points = db - .read_f64_range(&bucket, &s, &range, 10) - .map_err(|_| StatusCode::INTERNAL_SERVER_ERROR)?; - - for batch in points { - for p in batch { - let t = p.time.to_string(); - let v = p.value.to_string(); - vals[vcol] = v; - vals[tcol] = t; - - wtr.write_record(&vals).unwrap(); - } + ReadValues::F64(values) => { + for val in values { + let t = val.time.to_string(); + let v = val.time.to_string(); + vals[vcol] = v; + vals[tcol] = t; + wtr.write_record(&vals).unwrap(); } } - }; + } let mut data = wtr .into_inner() diff --git a/src/rpc.rs b/src/rpc.rs index 9c1337a076..12529210c0 100644 --- a/src/rpc.rs +++ b/src/rpc.rs @@ -1,4 +1,3 @@ -use delorean::delorean::Bucket; use delorean::delorean::{ delorean_server::Delorean, read_response::{ @@ -7,11 +6,11 @@ use delorean::delorean::{ storage_server::Storage, CapabilitiesResponse, CreateBucketRequest, CreateBucketResponse, DeleteBucketRequest, DeleteBucketResponse, GetBucketsResponse, Organization, Predicate, ReadFilterRequest, - ReadGroupRequest, ReadResponse, ReadSource, StringValuesResponse, TagKeysRequest, + ReadGroupRequest, ReadResponse, ReadSource, StringValuesResponse, Tag, TagKeysRequest, TagValuesRequest, TimestampRange, }; -use delorean::storage::database::Database; -use delorean::storage::SeriesDataType; +use delorean::line_parser::index_pairs; +use delorean::storage::partitioned_store::ReadValues; use std::convert::TryInto; use std::sync::Arc; @@ -77,19 +76,12 @@ trait GrpcInputs { .map_err(|_| Status::invalid_argument("org_id did not fit in a u32"))?) } - fn bucket(&self, db: &Database) -> Result, Status> { - let bucket_id = self + fn bucket_id(&self) -> Result { + Ok(self .read_source()? .bucket_id .try_into() - .map_err(|_| Status::invalid_argument("bucket_id did not fit in a u32"))?; - - let maybe_bucket = db - .get_bucket_by_id(bucket_id) - .map_err(|_| Status::internal("could not query for bucket"))?; - - Ok(maybe_bucket - .ok_or_else(|| Status::not_found(&format!("bucket {} not found", bucket_id)))?) + .map_err(|_| Status::invalid_argument("bucket_id did not fit in a u32"))?) } } @@ -129,8 +121,8 @@ impl Storage for GrpcServer { let read_filter_request = req.into_inner(); - let _org_id = read_filter_request.org_id()?; - let bucket = read_filter_request.bucket(&self.app.db)?; + let org_id = read_filter_request.org_id()?; + let bucket_id = read_filter_request.bucket_id()?; let predicate = read_filter_request.predicate; let range = read_filter_request.range; @@ -138,13 +130,15 @@ impl Storage for GrpcServer { // TODO: is this blocking because of the blocking calls to the database...? tokio::spawn(async move { - let predicate = predicate.as_ref(); + let predicate = predicate.as_ref().expect("TODO: must have a predicate"); // TODO: The call to read_series_matching_predicate_and_range takes an optional range, // but read_f64_range requires a range-- should this route require a range or use a // default or something else? let range = range.as_ref().expect("TODO: Must have a range?"); - if let Err(e) = send_series_filters(tx.clone(), app, &bucket, predicate, &range).await { + if let Err(e) = + send_series_filters(tx.clone(), app, org_id, bucket_id, predicate, &range).await + { tx.send(Err(e)).await.unwrap(); } }); @@ -169,24 +163,28 @@ impl Storage for GrpcServer { ) -> Result, Status> { let (mut tx, rx) = mpsc::channel(4); - let tag_keys_request = req.get_ref(); + let tag_keys_request = req.into_inner(); - let _org_id = tag_keys_request.org_id()?; - let bucket = tag_keys_request.bucket(&self.app.db)?; - let predicate = tag_keys_request.predicate.clone(); - let _range = tag_keys_request.range.as_ref(); + let org_id = tag_keys_request.org_id()?; + let bucket_id = tag_keys_request.bucket_id()?; + let predicate = tag_keys_request.predicate; + let range = tag_keys_request.range; let app = self.app.clone(); tokio::spawn(async move { - match app.db.get_tag_keys(&bucket, predicate.as_ref()) { + match app + .db + .get_tag_keys(org_id, bucket_id, predicate.as_ref(), range.as_ref()) + .await + { Err(_) => tx .send(Err(Status::internal("could not query for tag keys"))) .await .unwrap(), - Ok(tag_keys_iter) => { + Ok(tag_keys) => { // TODO: Should these be batched? If so, how? - let tag_keys: Vec<_> = tag_keys_iter.map(|s| s.into_bytes()).collect(); + let tag_keys: Vec<_> = tag_keys.into_iter().map(|s| s.into_bytes()).collect(); tx.send(Ok(StringValuesResponse { values: tag_keys })) .await .unwrap(); @@ -205,26 +203,37 @@ impl Storage for GrpcServer { ) -> Result, Status> { let (mut tx, rx) = mpsc::channel(4); - let tag_values_request = req.get_ref(); + let tag_values_request = req.into_inner(); - let _org_id = tag_values_request.org_id()?; - let bucket = tag_values_request.bucket(&self.app.db)?; - let predicate = tag_values_request.predicate.clone(); - let _range = tag_values_request.range.as_ref(); + let org_id = tag_values_request.org_id()?; + let bucket_id = tag_values_request.bucket_id()?; + let predicate = tag_values_request.predicate; + let range = tag_values_request.range; let tag_key = tag_values_request.tag_key.clone(); let app = self.app.clone(); tokio::spawn(async move { - match app.db.get_tag_values(&bucket, &tag_key, predicate.as_ref()) { + match app + .db + .get_tag_values( + org_id, + bucket_id, + &tag_key, + predicate.as_ref(), + range.as_ref(), + ) + .await + { Err(_) => tx .send(Err(Status::internal("could not query for tag values"))) .await .unwrap(), - Ok(tag_values_iter) => { + Ok(tag_values) => { // TODO: Should these be batched? If so, how? - let tag_values: Vec<_> = tag_values_iter.map(|s| s.into_bytes()).collect(); + let tag_values: Vec<_> = + tag_values.into_iter().map(|s| s.into_bytes()).collect(); tx.send(Ok(StringValuesResponse { values: tag_values })) .await .unwrap(); @@ -246,76 +255,65 @@ impl Storage for GrpcServer { async fn send_series_filters( mut tx: mpsc::Sender>, app: Arc, - bucket: &Bucket, - predicate: Option<&Predicate>, + org_id: u32, + bucket_id: u32, + predicate: &Predicate, range: &TimestampRange, ) -> Result<(), Status> { - let filter_iter = app + let batches = app .db - .read_series_matching_predicate_and_range(&bucket, predicate, Some(range)) - .map_err(|e| Status::internal(format!("could not query for filters: {}", e)))?; + .read_points(org_id, bucket_id, predicate, range) + .await + .map_err(|err| Status::internal(format!("error reading db: {}", err.description)))?; - for series_filter in filter_iter { - let tags = series_filter.tags(); - let data_type = match series_filter.series_type { - SeriesDataType::F64 => DataType::Float, - SeriesDataType::I64 => DataType::Integer, - } as _; - let series = SeriesFrame { data_type, tags }; - let data = Data::Series(series); - let data = Some(data); - let frame = Frame { data }; - let frames = vec![frame]; - let series_frame_response_header = Ok(ReadResponse { frames }); + let mut last_frame_key = String::new(); - tx.send(series_frame_response_header).await.unwrap(); + for batch in batches { + // only send the series frame header if we haven't sent it for this key. We have to do + // this because a single series key can be spread out over multiple ReadBatches, which + // should each be sent as their own data frames + if last_frame_key != batch.key { + last_frame_key = batch.key.clone(); + let tags = index_pairs(&batch.key) + .map_err(|err| Status::invalid_argument(err.description))? + .into_iter() + .map(|p| Tag { + key: p.key.bytes().collect(), + value: p.value.bytes().collect(), + }) + .collect(); - // TODO: Should this match https://github.com/influxdata/influxdb/blob/d96f3dc5abb6bb187374caa9e7c7a876b4799bd2/storage/reads/response_writer.go#L21 ? - const BATCH_SIZE: usize = 1; + let data_type = match batch.values { + ReadValues::F64(_) => DataType::Float, + ReadValues::I64(_) => DataType::Integer, + } as _; + let series = SeriesFrame { data_type, tags }; + let data = Data::Series(series); + let data = Some(data); + let frame = Frame { data }; + let frames = vec![frame]; + let series_frame_response_header = Ok(ReadResponse { frames }); - match series_filter.series_type { - SeriesDataType::F64 => { - let iter = app - .db - .read_f64_range(&bucket, &series_filter, &range, BATCH_SIZE) - .map_err(|e| { - Status::internal(format!("could not query for SeriesFilter data: {}", e)) - })?; + tx.send(series_frame_response_header).await.unwrap(); + } - let frames = iter - .map(|batch| { - // TODO: Performance hazard; splitting this vector is non-ideal - let (timestamps, values) = - batch.into_iter().map(|p| (p.time, p.value)).unzip(); - let frame = FloatPointsFrame { timestamps, values }; - let data = Data::FloatPoints(frame); - let data = Some(data); - Frame { data } - }) - .collect(); + match batch.values { + ReadValues::F64(values) => { + let (timestamps, values) = values.into_iter().map(|p| (p.time, p.value)).unzip(); + let frame = FloatPointsFrame { timestamps, values }; + let data = Data::FloatPoints(frame); + let data = Some(data); + let frames = vec![Frame { data }]; let data_frame_response = Ok(ReadResponse { frames }); tx.send(data_frame_response).await.unwrap(); } - SeriesDataType::I64 => { - let iter = app - .db - .read_i64_range(&bucket, &series_filter, &range, BATCH_SIZE) - .map_err(|e| { - Status::internal(format!("could not query for SeriesFilter data: {}", e)) - })?; - - let frames = iter - .map(|batch| { - // TODO: Performance hazard; splitting this vector is non-ideal - let (timestamps, values) = - batch.into_iter().map(|p| (p.time, p.value)).unzip(); - let frame = IntegerPointsFrame { timestamps, values }; - let data = Data::IntegerPoints(frame); - let data = Some(data); - Frame { data } - }) - .collect(); + ReadValues::I64(values) => { + let (timestamps, values) = values.into_iter().map(|p| (p.time, p.value)).unzip(); + let frame = IntegerPointsFrame { timestamps, values }; + let data = Data::IntegerPoints(frame); + let data = Some(data); + let frames = vec![Frame { data }]; let data_frame_response = Ok(ReadResponse { frames }); tx.send(data_frame_response).await.unwrap(); diff --git a/src/storage.rs b/src/storage.rs index d656b49c1e..7dbad2c2a2 100644 --- a/src/storage.rs +++ b/src/storage.rs @@ -8,7 +8,9 @@ pub mod inverted_index; pub mod memdb; pub mod partitioned_store; pub mod predicate; +pub mod remote_partition; pub mod rocksdb; +pub mod s3_partition; pub mod series_store; // The values for these enum variants have no real meaning, but they diff --git a/src/storage/database.rs b/src/storage/database.rs index 80fface776..1d0e3c336b 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -1,110 +1,359 @@ use crate::delorean::{Bucket, Predicate, TimestampRange}; use crate::line_parser::PointType; -use crate::storage::config_store::ConfigStore; -use crate::storage::inverted_index::{InvertedIndex, SeriesFilter}; -use crate::storage::rocksdb::RocksDB; -use crate::storage::series_store::{ReadPoint, SeriesStore}; +use crate::storage::memdb::MemDB; +use crate::storage::partitioned_store::{Partition, ReadBatch}; use crate::storage::StorageError; +use futures::StreamExt; +use std::collections::HashMap; use std::sync::Arc; +use tokio::sync::RwLock; pub struct Database { - local_index: Arc, - local_series_store: Arc, - local_config_store: Arc, + organizations: RwLock>>, } -impl Database { - pub fn new(dir: &str) -> Database { - let db = Arc::new(RocksDB::new(dir)); +#[derive(Default)] +struct Organization { + bucket_data: HashMap>, + bucket_name_to_id: HashMap, +} - Database { - local_index: db.clone(), - local_config_store: db.clone(), - local_series_store: db, +impl Organization { + // create_bucket_if_not_exists inserts the bucket into the map and returns its id + fn create_bucket_if_not_exists(&mut self, mut bucket: Bucket) -> u32 { + match self.bucket_name_to_id.get(&bucket.name) { + Some(id) => *id, + None => { + let id = (self.bucket_data.len() + 1) as u32; + bucket.id = id; + self.bucket_name_to_id.insert(bucket.name.clone(), id); + self.bucket_data + .insert(id, Arc::new(BucketData::new(bucket))); + id + } + } + } +} + +struct BucketData { + _config: Bucket, + // TODO: wire up rules for partitioning data and storing and reading from multiple partitions + partition: RwLock, +} + +impl BucketData { + const BATCH_SIZE: usize = 100_000; + + fn new(bucket: Bucket) -> BucketData { + let partition_id = bucket.name.clone(); + let partition = Partition::MemDB(Box::new(MemDB::new(partition_id))); + + BucketData { + _config: bucket, + partition: RwLock::new(partition), } } - pub fn write_points( - &self, - _org_id: u32, - bucket: &Bucket, - points: &mut [PointType], - ) -> Result<(), StorageError> { - self.local_index - .get_or_create_series_ids_for_points(bucket.id, points)?; - self.local_series_store - .write_points_with_series_ids(bucket.id, points) + async fn write_points(&self, points: &mut [PointType]) -> Result<(), StorageError> { + self.partition.write().await.write_points(points).await } - pub fn get_bucket_by_name( + async fn read_points( + &self, + predicate: &Predicate, + range: &TimestampRange, + ) -> Result, StorageError> { + let p = self.partition.read().await; + let stream = p + .read_points(BucketData::BATCH_SIZE, predicate, range) + .await?; + Ok(stream.collect().await) + } + + async fn get_tag_keys( + &self, + predicate: Option<&Predicate>, + range: Option<&TimestampRange>, + ) -> Result, StorageError> { + let p = self.partition.read().await; + let stream = p.get_tag_keys(predicate, range).await?; + Ok(stream.collect().await) + } + + async fn get_tag_values( + &self, + tag_key: &str, + predicate: Option<&Predicate>, + range: Option<&TimestampRange>, + ) -> Result, StorageError> { + let p = self.partition.read().await; + let stream = p.get_tag_values(tag_key, predicate, range).await?; + Ok(stream.collect().await) + } +} + +impl Database { + pub fn new(_dir: &str) -> Database { + Database { + organizations: RwLock::new(HashMap::new()), + } + } + + pub async fn write_points( + &self, + org_id: u32, + bucket_id: u32, + points: &mut [PointType], + ) -> Result<(), StorageError> { + let bucket_data = self.bucket_data(org_id, bucket_id).await?; + + bucket_data.write_points(points).await + } + + pub async fn get_bucket_id_by_name( &self, org_id: u32, bucket_name: &str, - ) -> Result>, StorageError> { - self.local_config_store - .get_bucket_by_name(org_id, bucket_name) + ) -> Result, StorageError> { + let orgs = self.organizations.read().await; + let org = orgs.get(&org_id).ok_or_else(|| StorageError { + description: format!("org {} not found", org_id), + })?; + + let id = match org.read().await.bucket_name_to_id.get(bucket_name) { + Some(id) => Some(*id), + None => None, + }; + + Ok(id) } - pub fn get_bucket_by_id(&self, bucket_id: u32) -> Result>, StorageError> { - self.local_config_store.get_bucket_by_id(bucket_id) - } - - pub fn create_bucket_if_not_exists( + pub async fn create_bucket_if_not_exists( &self, org_id: u32, - bucket: &Bucket, + bucket: Bucket, ) -> Result { - self.local_config_store - .create_bucket_if_not_exists(org_id, bucket) + let mut orgs = self.organizations.write().await; + let org = orgs + .entry(org_id) + .or_insert_with(|| RwLock::new(Organization::default())); + let mut org = org.write().await; + + Ok(org.create_bucket_if_not_exists(bucket)) } - pub fn read_series_matching_predicate_and_range( + pub async fn read_points( &self, - bucket: &Bucket, - predicate: Option<&Predicate>, - _range: Option<&TimestampRange>, - ) -> Result + Send>, StorageError> { - self.local_index.read_series_matching(bucket.id, predicate) - } - - pub fn read_i64_range( - &self, - bucket: &Bucket, - series_filter: &SeriesFilter, + org_id: u32, + bucket_id: u32, + predicate: &Predicate, range: &TimestampRange, - batch_size: usize, - ) -> Result>> + Send>, StorageError> { - self.local_series_store - .read_i64_range(bucket.id, series_filter.id, range, batch_size) + ) -> Result, StorageError> { + let bucket_data = self.bucket_data(org_id, bucket_id).await?; + + bucket_data.read_points(predicate, range).await } - pub fn read_f64_range( + pub async fn get_tag_keys( &self, - bucket: &Bucket, - series_filter: &SeriesFilter, - range: &TimestampRange, - batch_size: usize, - ) -> Result>> + Send>, StorageError> { - self.local_series_store - .read_f64_range(bucket.id, series_filter.id, range, batch_size) - } - - pub fn get_tag_keys( - &self, - bucket: &Bucket, + org_id: u32, + bucket_id: u32, predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - self.local_index.get_tag_keys(bucket.id, predicate) + range: Option<&TimestampRange>, + ) -> Result, StorageError> { + let bucket_data = self.bucket_data(org_id, bucket_id).await?; + + bucket_data.get_tag_keys(predicate, range).await } - pub fn get_tag_values( + pub async fn get_tag_values( &self, - bucket: &Bucket, + org_id: u32, + bucket_id: u32, tag_key: &str, predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - self.local_index - .get_tag_values(bucket.id, tag_key, predicate) + range: Option<&TimestampRange>, + ) -> Result, StorageError> { + let bucket_data = self.bucket_data(org_id, bucket_id).await?; + + bucket_data.get_tag_values(tag_key, predicate, range).await + } + + async fn bucket_data( + &self, + org_id: u32, + bucket_id: u32, + ) -> Result, StorageError> { + let orgs = self.organizations.read().await; + let org = orgs.get(&org_id).ok_or_else(|| StorageError { + description: format!("org {} not found", org_id), + })?; + + let org = org.read().await; + + match org.bucket_data.get(&bucket_id) { + Some(b) => Ok(Arc::clone(b)), + None => Err(StorageError { + description: format!("bucket {} not found", bucket_id), + }), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::line_parser::PointType; + use crate::storage::database::Database; + use crate::storage::partitioned_store::ReadValues; + use crate::storage::predicate::parse_predicate; + use crate::storage::series_store::ReadPoint; + + #[tokio::test] + async fn create_bucket() { + let database = Database::new(""); + let org_id = 2; + let bucket = Bucket { + org_id, + id: 0, + name: "first".to_string(), + retention: "0".to_string(), + posting_list_rollover: 10_000, + index_levels: vec![], + }; + let bucket_id = database + .create_bucket_if_not_exists(org_id, bucket.clone()) + .await + .unwrap(); + assert_eq!(bucket_id, 1); + + let bucket_two = Bucket { + org_id, + id: 0, + name: "second".to_string(), + retention: "0".to_string(), + posting_list_rollover: 10_000, + index_levels: vec![], + }; + + let bucket_id = database + .create_bucket_if_not_exists(org_id, bucket_two) + .await + .unwrap(); + assert_eq!(bucket_id, 2); + + let bucket_id = database + .create_bucket_if_not_exists(org_id, bucket) + .await + .unwrap(); + assert_eq!(bucket_id, 1); + } + + #[tokio::test] + async fn get_tag_keys() { + let (db, org, bucket) = setup_db_and_bucket().await; + db.write_points( + org, + bucket, + &mut [ + PointType::new_i64("cpu,host=a,region=west\tfoo".to_string(), 1, 0), + PointType::new_i64("mem,foo=bar\tasdf".to_string(), 1, 0), + ], + ) + .await + .unwrap(); + + let keys = db.get_tag_keys(org, bucket, None, None).await.unwrap(); + + assert_eq!(keys, vec!["_f", "_m", "foo", "host", "region"]); + } + + #[tokio::test] + async fn get_tag_values() { + let (db, org, bucket) = setup_db_and_bucket().await; + db.write_points( + org, + bucket, + &mut [ + PointType::new_i64("cpu,host=a,region=west\tfoo".to_string(), 1, 0), + PointType::new_i64("mem,host=b\tasdf".to_string(), 1, 0), + ], + ) + .await + .unwrap(); + + let values = db + .get_tag_values(org, bucket, "host", None, None) + .await + .unwrap(); + + assert_eq!(values, vec!["a", "b"]); + + let values = db + .get_tag_values(org, bucket, "region", None, None) + .await + .unwrap(); + + assert_eq!(values, vec!["west"]); + + let values = db + .get_tag_values(org, bucket, "_m", None, None) + .await + .unwrap(); + + assert_eq!(values, vec!["cpu", "mem"]); + } + + #[tokio::test] + async fn read_points() { + let (db, org, bucket) = setup_db_and_bucket().await; + db.write_points( + org, + bucket, + &mut [ + PointType::new_i64("cpu,host=a,region=west\tval".to_string(), 3, 1), + PointType::new_i64("cpu,host=a,region=west\tval".to_string(), 2, 5), + PointType::new_i64("cpu,host=a,region=west\tval".to_string(), 1, 10), + PointType::new_i64("cpu,host=b,region=west\tval".to_string(), 5, 9), + ], + ) + .await + .unwrap(); + + let pred = parse_predicate(r#"host = "a""#).unwrap(); + let range = TimestampRange { start: 0, end: 11 }; + let batches = db.read_points(org, bucket, &pred, &range).await.unwrap(); + + assert_eq!( + batches, + vec![ReadBatch { + key: "cpu,host=a,region=west\tval".to_string(), + values: ReadValues::I64(vec![ + ReadPoint { value: 3, time: 1 }, + ReadPoint { value: 2, time: 5 }, + ReadPoint { value: 1, time: 10 }, + ]) + }] + ); + } + + async fn setup_db_and_bucket() -> (Database, u32, u32) { + let database = Database::new(""); + let org_id = 1; + let bucket = Bucket { + org_id, + id: 0, + name: "foo".to_string(), + retention: "0".to_string(), + posting_list_rollover: 10_000, + index_levels: vec![], + }; + let bucket_id = database + .create_bucket_if_not_exists(org_id, bucket) + .await + .unwrap(); + + (database, org_id, bucket_id) } } diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index ac7830fb94..67051c48fa 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -21,6 +21,7 @@ use std::collections::{BTreeMap, HashMap}; #[derive(Default)] pub struct MemDB { + pub id: String, series_data: SeriesData, series_map: SeriesMap, } @@ -38,16 +39,12 @@ struct SeriesBuffer { impl SeriesBuffer { fn read(&self, range: &TimestampRange) -> Vec> { - let start = match self.values - .iter() - .position(|val| val.time >= range.start) { + let start = match self.values.iter().position(|val| val.time >= range.start) { Some(pos) => pos, None => return vec![], }; - let stop = self.values - .iter() - .position(|val| val.time >= range.end); + let stop = self.values.iter().position(|val| val.time >= range.end); let stop = stop.unwrap_or_else(|| self.values.len()); self.values[start..stop].to_vec() @@ -186,15 +183,17 @@ fn list_key(key: &str, value: &str) -> Vec { } impl MemDB { - pub fn new() -> Self { - Default::default() + pub fn new(id: String) -> Self { + let mut memdb: MemDB = Default::default(); + memdb.id = id; + memdb } pub fn size(&self) -> usize { self.series_data.current_size + self.series_map.current_size } - fn write(&mut self, points: &mut [PointType]) -> Result<(), StorageError> { + pub fn write_points(&mut self, points: &mut [PointType]) -> Result<(), StorageError> { for p in points { self.series_map.insert_series(p).map_err(|e| StorageError { description: format!("error parsing line protocol metadata {}", e), @@ -205,20 +204,20 @@ impl MemDB { Ok(()) } - fn get_tag_keys( + pub fn get_tag_keys( &self, - _range: &TimestampRange, - _predicate: &Predicate, + _predicate: Option<&Predicate>, + _range: Option<&TimestampRange>, ) -> Result, StorageError> { let keys = self.series_map.tag_keys.keys().cloned(); Ok(stream::iter(keys).boxed()) } - fn get_tag_values( + pub fn get_tag_values( &self, tag_key: &str, - _range: &TimestampRange, - _predicate: &Predicate, + _predicate: Option<&Predicate>, + _range: Option<&TimestampRange>, ) -> Result, StorageError> { match self.series_map.tag_keys.get(tag_key) { Some(values) => { @@ -229,7 +228,7 @@ impl MemDB { } } - fn read( + pub fn read_points( &self, _batch_size: usize, predicate: &Predicate, @@ -297,45 +296,34 @@ mod tests { use crate::storage::predicate::parse_predicate; #[test] - fn write_and_read_tag_keys() { + fn get_tag_keys() { let memdb = setup_db(); - let tag_keys = memdb - .get_tag_keys( - &TimestampRange { start: 0, end: 0 }, - &Predicate { root: None }, - ) - .unwrap(); + let tag_keys = memdb.get_tag_keys(None, None).unwrap(); let tag_keys: Vec<_> = futures::executor::block_on_stream(tag_keys).collect(); assert_eq!(tag_keys, vec!["_f", "_m", "host", "region"]); } #[test] - fn write_and_read_tag_values() { + fn get_tag_values() { let memdb = setup_db(); - let tag_values = memdb - .get_tag_values( - "host", - &TimestampRange { start: 0, end: 0 }, - &Predicate { root: None }, - ) - .unwrap(); + let tag_values = memdb.get_tag_values("host", None, None).unwrap(); let tag_values: Vec<_> = futures::executor::block_on_stream(tag_values).collect(); assert_eq!(tag_values, vec!["a", "b"]); } #[test] - fn write_and_check_size() { + fn check_size() { let memdb = setup_db(); assert_eq!(memdb.size(), 704); } #[test] - fn write_and_get_measurement_series() { + fn get_measurement_series() { let memdb = setup_db(); let pred = parse_predicate(r#"_m = "cpu""#).unwrap(); let batches = memdb - .read(10, &pred, &TimestampRange { start: 0, end: 5 }) + .read_points(10, &pred, &TimestampRange { start: 0, end: 5 }) .unwrap(); let batches: Vec<_> = futures::executor::block_on_stream(batches).collect(); @@ -362,11 +350,11 @@ mod tests { } #[test] - fn write_and_get_tag_match_series() { + fn get_tag_match_series() { let memdb = setup_db(); let pred = parse_predicate(r#"host = "a""#).unwrap(); let batches = memdb - .read(10, &pred, &TimestampRange { start: 0, end: 5 }) + .read_points(10, &pred, &TimestampRange { start: 0, end: 5 }) .unwrap(); let batches: Vec<_> = futures::executor::block_on_stream(batches).collect(); assert_eq!( @@ -385,11 +373,11 @@ mod tests { } #[test] - fn write_and_measurement_and_tag_match_series() { + fn measurement_and_tag_match_series() { let memdb = setup_db(); let pred = parse_predicate(r#"_m = "cpu" and host = "b""#).unwrap(); let batches = memdb - .read(10, &pred, &TimestampRange { start: 0, end: 5 }) + .read_points(10, &pred, &TimestampRange { start: 0, end: 5 }) .unwrap(); let batches: Vec<_> = futures::executor::block_on_stream(batches).collect(); assert_eq!( @@ -405,11 +393,11 @@ mod tests { } #[test] - fn write_and_measurement_or_tag_match() { + fn measurement_or_tag_match() { let memdb = setup_db(); let pred = parse_predicate(r#"host = "a" OR _m = "mem""#).unwrap(); let batches = memdb - .read(10, &pred, &TimestampRange { start: 0, end: 5 }) + .read_points(10, &pred, &TimestampRange { start: 0, end: 5 }) .unwrap(); let batches: Vec<_> = futures::executor::block_on_stream(batches).collect(); assert_eq!( @@ -440,8 +428,8 @@ mod tests { let mut points = vec![p1, p2, p3, p4, p5]; - let mut memdb = MemDB::new(); - memdb.write(&mut points).unwrap(); + let mut memdb = MemDB::new("foo".to_string()); + memdb.write_points(&mut points).unwrap(); memdb } } diff --git a/src/storage/partitioned_store.rs b/src/storage/partitioned_store.rs index 3bb48d6715..c82f60ac4a 100644 --- a/src/storage/partitioned_store.rs +++ b/src/storage/partitioned_store.rs @@ -1,7 +1,10 @@ -//! partitioned_store is a trait and set of helper functions and structs to define Partitions +//! partitioned_store is an enum and set of helper functions and structs to define Partitions //! that store data. The helper funcs and structs merge results from multiple partitions together. use crate::delorean::{Predicate, TimestampRange}; use crate::line_parser::PointType; +use crate::storage::memdb::MemDB; +use crate::storage::remote_partition::RemotePartition; +use crate::storage::s3_partition::S3Partition; use crate::storage::series_store::ReadPoint; use crate::storage::StorageError; @@ -15,34 +18,76 @@ use std::task::{Context, Poll}; /// tags, tag values, and fields exist. Along with the raw time series data. It is designed to work /// as a stream so that it can be used in safely an asynchronous context. A partition is the /// lowest level organization scheme. Above it you will have a database which keeps track of -/// what organizations and buckets exist. A bucket will have 1-many partitions and a partition +/// what organizations and buckets exist. A bucket will have 1 to many partitions and a partition /// will only ever contain data for a single bucket. -pub trait Partition { - fn id(&self) -> String; +pub enum Partition { + MemDB(Box), + S3(Box), + Remote(Box), +} - fn size(&self) -> u64; +impl Partition { + pub fn id(&self) -> &str { + match self { + Partition::MemDB(db) => &db.id, + Partition::S3(_) => panic!("s3 partition not implemented!"), + Partition::Remote(_) => panic!("remote partition not implemented!"), + } + } - fn write(&self, points: &[PointType]) -> Result<(), StorageError>; + pub fn size(&self) -> usize { + match self { + Partition::MemDB(db) => db.size(), + Partition::S3(_) => panic!("s3 partition not implemented!"), + Partition::Remote(_) => panic!("remote partition not implemented!"), + } + } - fn get_tag_keys( + pub async fn write_points(&mut self, points: &mut [PointType]) -> Result<(), StorageError> { + match self { + Partition::MemDB(db) => db.write_points(points), + Partition::S3(_) => panic!("s3 partition not implemented!"), + Partition::Remote(_) => panic!("remote partition not implemented!"), + } + } + + pub async fn get_tag_keys( &self, - range: &TimestampRange, - predicate: &Predicate, - ) -> Result, StorageError>; + predicate: Option<&Predicate>, + range: Option<&TimestampRange>, + ) -> Result, StorageError> { + match self { + Partition::MemDB(db) => db.get_tag_keys(predicate, range), + Partition::S3(_) => panic!("s3 partition not implemented!"), + Partition::Remote(_) => panic!("remote partition not implemented!"), + } + } - fn get_tag_values( + pub async fn get_tag_values( &self, tag_key: &str, - range: &TimestampRange, - predicate: &Predicate, - ) -> Result, StorageError>; + predicate: Option<&Predicate>, + range: Option<&TimestampRange>, + ) -> Result, StorageError> { + match self { + Partition::MemDB(db) => db.get_tag_values(tag_key, predicate, range), + Partition::S3(_) => panic!("s3 partition not implemented!"), + Partition::Remote(_) => panic!("remote partition not implemented!"), + } + } - fn read( + pub async fn read_points( &self, batch_size: usize, predicate: &Predicate, range: &TimestampRange, - ) -> Result, StorageError>; + ) -> Result, StorageError> { + match self { + Partition::MemDB(db) => db.read_points(batch_size, predicate, range), + Partition::S3(_) => panic!("s3 partition not implemented!"), + Partition::Remote(_) => panic!("remote partition not implemented!"), + } + } } /// StringMergeStream will do a merge sort with deduplication of multiple streams of Strings. This diff --git a/src/storage/remote_partition.rs b/src/storage/remote_partition.rs new file mode 100644 index 0000000000..d47c4297cc --- /dev/null +++ b/src/storage/remote_partition.rs @@ -0,0 +1,4 @@ +/// RemotePartition represents partitions that are on remote delorean servers. It implements the +/// methods that the Partition enum requires to answer queries. +/// TODO: implement me +pub struct RemotePartition {} diff --git a/src/storage/s3_partition.rs b/src/storage/s3_partition.rs new file mode 100644 index 0000000000..0ca212b779 --- /dev/null +++ b/src/storage/s3_partition.rs @@ -0,0 +1,4 @@ +/// S3Partition represents a partition of data stored on S3. It implements the methods that the +/// Partition enum requires to answer queries. +/// TODO: implment me +pub struct S3Partition {} From ac5ecb01dc91b5b3369e71c8f3664d688e544c71 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 09:02:39 -0400 Subject: [PATCH 25/42] fix: Add some more logging for internal server errors --- src/main.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/main.rs b/src/main.rs index e5a2c3f8e0..12e1b118ca 100644 --- a/src/main.rs +++ b/src/main.rs @@ -52,7 +52,10 @@ async fn write(req: hyper::Request, app: Arc) -> Result id, @@ -70,7 +73,10 @@ async fn write(req: hyper::Request, app: Arc) -> Result, app: Arc) -> Result Date: Thu, 2 Apr 2020 09:07:40 -0400 Subject: [PATCH 26/42] fix: Keep ParseError's description private; use Display instead This gives us more flexibility to refactor ParseError in the future. --- src/line_parser.rs | 2 +- src/rpc.rs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/line_parser.rs b/src/line_parser.rs index 518e9c53c2..c89d902b27 100644 --- a/src/line_parser.rs +++ b/src/line_parser.rs @@ -155,7 +155,7 @@ pub struct Pair { #[derive(Debug, Clone)] pub struct ParseError { - pub description: String, + description: String, } impl fmt::Display for ParseError { diff --git a/src/rpc.rs b/src/rpc.rs index 12529210c0..8684a2f582 100644 --- a/src/rpc.rs +++ b/src/rpc.rs @@ -264,7 +264,7 @@ async fn send_series_filters( .db .read_points(org_id, bucket_id, predicate, range) .await - .map_err(|err| Status::internal(format!("error reading db: {}", err.description)))?; + .map_err(|err| Status::internal(format!("error reading db: {}", err)))?; let mut last_frame_key = String::new(); @@ -275,7 +275,7 @@ async fn send_series_filters( if last_frame_key != batch.key { last_frame_key = batch.key.clone(); let tags = index_pairs(&batch.key) - .map_err(|err| Status::invalid_argument(err.description))? + .map_err(|err| Status::invalid_argument(err.to_string()))? .into_iter() .map(|p| Tag { key: p.key.bytes().collect(), From d9cf5c952a3100ed81e8a065766e1724fbdf13be Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 09:39:13 -0400 Subject: [PATCH 27/42] fix: Remove RocksDB code --- Cargo.lock | 59 -- Cargo.toml | 1 - src/storage.rs | 1 - src/storage/rocksdb.rs | 1649 ---------------------------------------- 4 files changed, 1710 deletions(-) delete mode 100644 src/storage/rocksdb.rs diff --git a/Cargo.lock b/Cargo.lock index 0845312c45..fe6674102e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -136,29 +136,6 @@ dependencies = [ "which 3.1.0 (registry+https://github.com/rust-lang/crates.io-index)", ] -[[package]] -name = "bindgen" -version = "0.53.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -dependencies = [ - "bitflags 1.2.1 (registry+https://github.com/rust-lang/crates.io-index)", - "cexpr 0.3.6 (registry+https://github.com/rust-lang/crates.io-index)", - "cfg-if 0.1.9 (registry+https://github.com/rust-lang/crates.io-index)", - "clang-sys 0.28.1 (registry+https://github.com/rust-lang/crates.io-index)", - "clap 2.33.0 (registry+https://github.com/rust-lang/crates.io-index)", - "env_logger 0.7.1 (registry+https://github.com/rust-lang/crates.io-index)", - "lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)", - "lazycell 1.2.1 (registry+https://github.com/rust-lang/crates.io-index)", - "log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)", - "peeking_take_while 0.1.2 (registry+https://github.com/rust-lang/crates.io-index)", - "proc-macro2 1.0.9 (registry+https://github.com/rust-lang/crates.io-index)", - "quote 1.0.2 (registry+https://github.com/rust-lang/crates.io-index)", - "regex 1.3.4 (registry+https://github.com/rust-lang/crates.io-index)", - "rustc-hash 1.1.0 (registry+https://github.com/rust-lang/crates.io-index)", - "shlex 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", - "which 3.1.0 (registry+https://github.com/rust-lang/crates.io-index)", -] - [[package]] name = "bitflags" version = "1.2.1" @@ -220,9 +197,6 @@ dependencies = [ name = "cc" version = "1.0.50" source = "registry+https://github.com/rust-lang/crates.io-index" -dependencies = [ - "jobserver 0.1.21 (registry+https://github.com/rust-lang/crates.io-index)", -] [[package]] name = "cexpr" @@ -420,7 +394,6 @@ dependencies = [ "prost-types 0.6.1 (registry+https://github.com/rust-lang/crates.io-index)", "rand 0.7.3 (registry+https://github.com/rust-lang/crates.io-index)", "reqwest 0.10.3 (registry+https://github.com/rust-lang/crates.io-index)", - "rocksdb 0.13.0 (registry+https://github.com/rust-lang/crates.io-index)", "serde 1.0.104 (registry+https://github.com/rust-lang/crates.io-index)", "serde_json 1.0.48 (registry+https://github.com/rust-lang/crates.io-index)", "serde_urlencoded 0.6.1 (registry+https://github.com/rust-lang/crates.io-index)", @@ -787,14 +760,6 @@ name = "itoa" version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -[[package]] -name = "jobserver" -version = "0.1.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -dependencies = [ - "libc 0.2.67 (registry+https://github.com/rust-lang/crates.io-index)", -] - [[package]] name = "js-sys" version = "0.3.35" @@ -849,17 +814,6 @@ dependencies = [ "winapi 0.3.8 (registry+https://github.com/rust-lang/crates.io-index)", ] -[[package]] -name = "librocksdb-sys" -version = "6.6.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -dependencies = [ - "bindgen 0.53.1 (registry+https://github.com/rust-lang/crates.io-index)", - "cc 1.0.50 (registry+https://github.com/rust-lang/crates.io-index)", - "glob 0.3.0 (registry+https://github.com/rust-lang/crates.io-index)", - "libc 0.2.67 (registry+https://github.com/rust-lang/crates.io-index)", -] - [[package]] name = "log" version = "0.4.8" @@ -1400,15 +1354,6 @@ dependencies = [ "winreg 0.6.2 (registry+https://github.com/rust-lang/crates.io-index)", ] -[[package]] -name = "rocksdb" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -dependencies = [ - "libc 0.2.67 (registry+https://github.com/rust-lang/crates.io-index)", - "librocksdb-sys 6.6.4 (registry+https://github.com/rust-lang/crates.io-index)", -] - [[package]] name = "rust-argon2" version = "0.7.0" @@ -2227,7 +2172,6 @@ dependencies = [ "checksum base64 0.10.1 (registry+https://github.com/rust-lang/crates.io-index)" = "0b25d992356d2eb0ed82172f5248873db5560c4721f564b13cb5193bda5e668e" "checksum base64 0.11.0 (registry+https://github.com/rust-lang/crates.io-index)" = "b41b7ea54a0c9d92199de89e20e58d49f02f8e699814ef3fdf266f6f748d15c7" "checksum bindgen 0.52.0 (registry+https://github.com/rust-lang/crates.io-index)" = "f1c85344eb535a31b62f0af37be84441ba9e7f0f4111eb0530f43d15e513fe57" -"checksum bindgen 0.53.1 (registry+https://github.com/rust-lang/crates.io-index)" = "99de13bb6361e01e493b3db7928085dcc474b7ba4f5481818e53a89d76b8393f" "checksum bitflags 1.2.1 (registry+https://github.com/rust-lang/crates.io-index)" = "cf1de2fe8c75bc145a2f577add951f8134889b4795d47466a54a5c846d691693" "checksum blake2b_simd 0.5.10 (registry+https://github.com/rust-lang/crates.io-index)" = "d8fb2d74254a3a0b5cac33ac9f8ed0e44aa50378d9dbb2e5d83bd21ed1dc2c8a" "checksum bstr 0.2.11 (registry+https://github.com/rust-lang/crates.io-index)" = "502ae1441a0a5adb8fbd38a5955a6416b9493e92b465de5e4a9bde6a539c2c48" @@ -2296,7 +2240,6 @@ dependencies = [ "checksum iovec 0.1.4 (registry+https://github.com/rust-lang/crates.io-index)" = "b2b3ea6ff95e175473f8ffe6a7eb7c00d054240321b84c57051175fe3c1e075e" "checksum itertools 0.8.2 (registry+https://github.com/rust-lang/crates.io-index)" = "f56a2d0bc861f9165be4eb3442afd3c236d8a98afd426f65d92324ae1091a484" "checksum itoa 0.4.5 (registry+https://github.com/rust-lang/crates.io-index)" = "b8b7a7c0c47db5545ed3fef7468ee7bb5b74691498139e4b3f6a20685dc6dd8e" -"checksum jobserver 0.1.21 (registry+https://github.com/rust-lang/crates.io-index)" = "5c71313ebb9439f74b00d9d2dcec36440beaf57a6aa0623068441dd7cd81a7f2" "checksum js-sys 0.3.35 (registry+https://github.com/rust-lang/crates.io-index)" = "7889c7c36282151f6bf465be4700359318aef36baa951462382eae49e9577cf9" "checksum kernel32-sys 0.2.2 (registry+https://github.com/rust-lang/crates.io-index)" = "7507624b29483431c0ba2d82aece8ca6cdba9382bff4ddd0f7490560c056098d" "checksum lazy_static 1.4.0 (registry+https://github.com/rust-lang/crates.io-index)" = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" @@ -2304,7 +2247,6 @@ dependencies = [ "checksum lexical-core 0.6.7 (registry+https://github.com/rust-lang/crates.io-index)" = "f86d66d380c9c5a685aaac7a11818bdfa1f733198dfd9ec09c70b762cd12ad6f" "checksum libc 0.2.67 (registry+https://github.com/rust-lang/crates.io-index)" = "eb147597cdf94ed43ab7a9038716637d2d1bf2bc571da995d0028dec06bd3018" "checksum libloading 0.5.2 (registry+https://github.com/rust-lang/crates.io-index)" = "f2b111a074963af1d37a139918ac6d49ad1d0d5e47f72fd55388619691a7d753" -"checksum librocksdb-sys 6.6.4 (registry+https://github.com/rust-lang/crates.io-index)" = "4e3b727e2dd20ec2fb7ed93f23d9fd5328a0871185485ebdaff007b47d3e27e4" "checksum log 0.4.8 (registry+https://github.com/rust-lang/crates.io-index)" = "14b6052be84e6b71ab17edffc2eeabf5c2c3ae1fdb464aae35ac50c67a44e1f7" "checksum matches 0.1.8 (registry+https://github.com/rust-lang/crates.io-index)" = "7ffc5c5338469d4d3ea17d269fa8ea3512ad247247c30bd2df69e68309ed0a08" "checksum maybe-uninit 2.0.0 (registry+https://github.com/rust-lang/crates.io-index)" = "60302e4db3a61da70c0cb7991976248362f30319e88850c487b9b95bbf059e00" @@ -2366,7 +2308,6 @@ dependencies = [ "checksum regex-syntax 0.6.14 (registry+https://github.com/rust-lang/crates.io-index)" = "b28dfe3fe9badec5dbf0a79a9cccad2cfc2ab5484bdb3e44cbd1ae8b3ba2be06" "checksum remove_dir_all 0.5.2 (registry+https://github.com/rust-lang/crates.io-index)" = "4a83fa3702a688b9359eccba92d153ac33fd2e8462f9e0e3fdf155239ea7792e" "checksum reqwest 0.10.3 (registry+https://github.com/rust-lang/crates.io-index)" = "a9f62f24514117d09a8fc74b803d3d65faa27cea1c7378fb12b0d002913f3831" -"checksum rocksdb 0.13.0 (registry+https://github.com/rust-lang/crates.io-index)" = "12069b106981c6103d3eab7dd1c86751482d0779a520b7c14954c8b586c1e643" "checksum rust-argon2 0.7.0 (registry+https://github.com/rust-lang/crates.io-index)" = "2bc8af4bda8e1ff4932523b94d3dd20ee30a87232323eda55903ffd71d2fb017" "checksum rustc-hash 1.1.0 (registry+https://github.com/rust-lang/crates.io-index)" = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" "checksum rustc_version 0.2.3 (registry+https://github.com/rust-lang/crates.io-index)" = "138e3e0acb6c9fb258b19b67cb8abd63c00679d2851805ea151465464fe9030a" diff --git a/Cargo.toml b/Cargo.toml index 068b5e7109..1c5309f46a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -22,7 +22,6 @@ futures = "0.3.1" serde_json = "1.0.44" serde = "1.0" csv = "1.1" -rocksdb = "0.13" byteorder = "1.3.4" num_cpus = "1.11.1" diff --git a/src/storage.rs b/src/storage.rs index 7dbad2c2a2..4b57121ea8 100644 --- a/src/storage.rs +++ b/src/storage.rs @@ -9,7 +9,6 @@ pub mod memdb; pub mod partitioned_store; pub mod predicate; pub mod remote_partition; -pub mod rocksdb; pub mod s3_partition; pub mod series_store; diff --git a/src/storage/rocksdb.rs b/src/storage/rocksdb.rs deleted file mode 100644 index 4f299098c3..0000000000 --- a/src/storage/rocksdb.rs +++ /dev/null @@ -1,1649 +0,0 @@ -use crate::delorean::{Bucket, IndexLevel, Node, Predicate, TimestampRange}; -use crate::line_parser::PointType; -use crate::storage::config_store::ConfigStore; -use crate::storage::inverted_index::{InvertedIndex, SeriesFilter}; -use crate::storage::predicate::{Evaluate, EvaluateVisitor}; -use crate::storage::series_store::{ReadPoint, SeriesStore}; -use crate::storage::{SeriesDataType, StorageError}; - -use std::collections::hash_map::Entry; -use std::collections::HashMap; -use std::convert::{TryFrom, TryInto}; -use std::io::Cursor; -use std::sync::{Arc, Mutex, RwLock}; - -use byteorder::{BigEndian, ByteOrder, ReadBytesExt, WriteBytesExt}; -use croaring::treemap::NativeSerializer; -use croaring::Treemap; -use prost::Message; -use rocksdb::{ - ColumnFamilyDescriptor, DBIterator, Direction, IteratorMode, Options, WriteBatch, DB, -}; - -/// Database wraps a RocksDB database for storing the raw series data, an inverted index of the -/// metadata and the metadata about what buckets exist in the system. -/// -/// Each bucket gets a unique u32 ID assigned. This ID is unique across all orgs. Bucket names -/// are unique within an organization. -/// -/// Series (measurement + tagset + field) are identified by a u64 ID that is unique within a bucket. -/// Each bucket keeps an incrementing counter for new series IDs. -pub struct RocksDB { - db: Arc>, - // bucket_map is an in memory map of what buckets exist in the system. the key is the org id and bucket name together as bytes - bucket_map: Arc, Arc>>>, - // `bucket_id_map` is an in-memory map of bucket IDs to buckets that exist in the system. - bucket_id_map: Arc>>>, - // series_insert_lock is a map of mutexes for creating new series in each bucket. Bucket ids are unique across all orgs - series_insert_lock: Arc>>>, -} - -const BUCKET_CF: &str = "buckets"; -const BUCKET_CF_WRITE_BUFFER_SIZE: usize = 1024 * 1024; // 1MB -const INDEX_CF_WRITE_BUFFER_SIZE: usize = 10 * 1024 * 1024; // 10MB - -impl RocksDB { - pub fn new(dir: &str) -> RocksDB { - let mut opts = Options::default(); - - // create the database and missing column families - opts.create_if_missing(true); - opts.create_missing_column_families(true); - - // ensure rocks uses more than one thread for compactions/etc - let core_count = num_cpus::get(); - opts.increase_parallelism(core_count as i32); - - // ensure the buckets CF exists and open up any index CFs - let cf_descriptors: Vec = match DB::list_cf(&opts, dir) { - Ok(names) => names - .into_iter() - .map(|name| { - if name == BUCKET_CF { - bucket_cf_descriptor() - } else { - ColumnFamilyDescriptor::new(&name, index_cf_options()) - } - }) - .collect(), - Err(_) => vec![bucket_cf_descriptor()], - }; - - let db = DB::open_cf_descriptors(&opts, dir, cf_descriptors).unwrap(); - - let mut database = RocksDB { - db: Arc::new(RwLock::new(db)), - bucket_map: Arc::new(RwLock::new(HashMap::new())), - bucket_id_map: Arc::new(RwLock::new(HashMap::new())), - series_insert_lock: Arc::new(RwLock::new(HashMap::new())), - }; - database.load_bucket_map(); - - database - } - - /// write_points will write values into the database under the given org_id and bucket_name. It - /// also inserts the series and their metadata into the index if not already present. - /// It does no validation against the org_id. It will create the bucket with the default - /// single index level of all time if it hasn't been created yet. - /// - /// # Arguments - /// * bucket_id - the globally unique bucket id - /// * points - individual values with their timestamps, series keys, and series IDs - pub fn write_points(&self, bucket_id: u32, points: &[PointType]) -> Result<(), StorageError> { - // TODO: validate bucket exists? - - let mut batch = WriteBatch::default(); - - for p in points { - let id = p.series_id().ok_or_else(|| StorageError { - description: format!("point {:?} had no series id", p), - })?; - - let key = key_for_series_and_time(bucket_id, id, p.time()); - let value = p.to_rocks_db_bytes(); - - batch.put(key, value).unwrap(); - } - - self.db - .read() - .unwrap() - .write(batch) - .expect("unexpected RocksDB error"); - Ok(()) - } - - // TODO: update this so it decompresses at least the first point to verify the data type or return error - fn read_range( - &self, - bucket_id: u32, - series_id: u64, - range: &TimestampRange, - batch_size: usize, - ) -> Result>> + Send>, StorageError> { - let (iter, series_prefix) = self.get_db_points_iter(bucket_id, series_id, range.start); - - Ok(Box::new(PointsIterator { - batch_size, - iter, - stop_time: range.end, - series_prefix, - drained: false, - read: FromBytes::from, - })) - } - - fn get_db_points_iter<'a>( - &self, - bucket_id: u32, - series_id: u64, - start: i64, - ) -> (DBIterator<'a>, Vec) { - let prefix = prefix_for_series(bucket_id, series_id, start); - let mode = IteratorMode::From(&prefix, Direction::Forward); - - let iter = self.db.read().unwrap().iterator(mode); - let prefix = prefix[0..12].to_vec(); - - (iter, prefix) - } - - /// If the bucket name exists within an org, this function returns the ID (ignoring whether the - /// bucket options are different than the one that exists). If it doesn't exist, this function - /// creates the bucket and returns its unique identifier. - /// - /// # Arguments - /// - /// * `org_id` - The organization this bucket is under - /// * `bucket` - The bucket to create along with all of its configuration options. Ignores the ID. - pub fn create_bucket_if_not_exists( - &self, - org_id: u32, - bucket: &Bucket, - ) -> Result { - validate_bucket_fields(bucket)?; - - let key = bucket_key(org_id, &bucket.name); - if let Some(b) = self.bucket_map.read().unwrap().get(&key) { - return Ok(b.id); - } - - let mut map = self.bucket_map.write().unwrap(); - let mut id_map = self.bucket_id_map.write().unwrap(); - if let Some(b) = map.get(&key) { - return Ok(b.id); - } - - let db = self.db.read().unwrap(); - - // assign the ID and insert the bucket - let buckets = db - .cf_handle(BUCKET_CF) - .expect("unexpected rocksdb error while trying to get the buckets column family"); - - let mut buf: Vec = vec![]; - let mut store = bucket.clone(); - - // get the next bucket ID - let next_id = match db - .get_cf(buckets, next_bucket_id_key()) - .expect("unexpected rocksdb error while trying to get the next bucket id") - { - Some(val) => u32_from_bytes(&val), - None => 1, - }; - - store.id = next_id; - store - .encode(&mut buf) - .expect("unexpected error encoding bucket"); - - // write the bucket and the next ID counter atomically - let mut batch = WriteBatch::default(); - batch.put_cf(&buckets, &key, buf).unwrap(); - batch - .put_cf(&buckets, next_bucket_id_key(), (store.id + 1).to_be_bytes()) - .unwrap(); - db.write(batch) - .expect("unexpected rocksdb error writing to DB"); - - let id = store.id; - let arc_bucket = Arc::new(store); - map.insert(key, arc_bucket.clone()); - id_map.insert(id, arc_bucket); - - Ok(id) - } - - /// Looks up the bucket object by org id and name and returns it. - /// - /// # Arguments - /// - /// * `org_id` - The organization this bucket is under - /// * `name` - The name of the bucket (which is unique under an organization) - pub fn get_bucket_by_name( - &self, - org_id: u32, - name: &str, - ) -> Result>, StorageError> { - let buckets = self.bucket_map.read().unwrap(); - let key = bucket_key(org_id, name); - Ok(buckets.get(&key).map(Arc::clone)) - } - - /// Looks up the bucket object by bucket id and returns it. - /// - /// # Arguments - /// - /// * `bucket_id` - The ID of the bucket (which is globally unique) - pub fn get_bucket_by_id(&self, bucket_id: u32) -> Result>, StorageError> { - let buckets = self.bucket_id_map.read().unwrap(); - Ok(buckets.get(&bucket_id).map(Arc::clone)) - } - - // TODO: ensure that points with timestamps older than the first index level get matched against the appropriate index - // TODO: maybe add an LRU for the series ID mappings? - /// get_series_ids consumes the passed in points vector and returns a vector of series, one for - /// each point (which is now in the series struct). The series struct has an option with the ID - /// if the series already exists in the top level index. - /// - /// # Arguments - /// - /// * `org_id` - The organization this bucket is under - /// * `bucket` - The bucket these series are to be inserted into - /// * `points` - The points that will be later written to the DB that need to have series IDs - /// - /// # Returns - /// A vector of series where each point in the passed in vector is contained in a series - pub fn get_series_ids( - &self, - bucket_id: u32, - points: &mut [PointType], - ) -> Result<(), StorageError> { - let cf_name = index_cf_name(bucket_id); - - for point in points { - if let Some(id) = self.get_series_id(&cf_name, &point.series()) { - point.set_series_id(id); - } - } - - Ok(()) - } - - // TODO: create test with different data and predicates loaded to ensure it hits the index properly - // TODO: refactor this to return an iterator so queries with many series don't materialize all at once - // TODO: wire up the time range part of this - /// get_series_filters returns a collection of series and associated value filters that can be used - /// to iterate over raw tsm data. The predicate passed in is the same as that used in the Go based - /// storage layer. - pub fn get_series_filters( - &self, - bucket_id: u32, - predicate: Option<&Predicate>, - ) -> Result, StorageError> { - if let Some(pred) = predicate { - if let Some(root) = &pred.root { - let map = self.evaluate_node(bucket_id, &root)?; - let mut filters = Vec::with_capacity(map.cardinality() as usize); - - for id in map.iter() { - let (key, series_type) = self.get_series_key_and_type_by_id(bucket_id, id)?; - filters.push(SeriesFilter { - id, - key, - value_predicate: None, - series_type, - }); - } - - return Ok(filters); - } - } - - // TODO: return list of all series - Err(StorageError { - description: "get for all series ids not wired up yet".to_string(), - }) - } - - fn get_series_key_and_type_by_id( - &self, - bucket_id: u32, - id: u64, - ) -> Result<(String, SeriesDataType), StorageError> { - let cf_name = index_cf_name(bucket_id); - let db = self.db.read().unwrap(); - - match db.cf_handle(&cf_name) { - Some(cf) => match db.get_cf(cf, index_series_id_from_id(id)).unwrap() { - Some(val) => { - let t = val[0].try_into().unwrap(); - let key = std::str::from_utf8(&val[1..]).unwrap().to_owned(); - Ok((key, t)) - } - None => Err(StorageError { - description: "series id not found".to_string(), - }), - }, - None => Err(StorageError { - description: "unable to find index".to_string(), - }), - } - } - - fn evaluate_node(&self, bucket_id: u32, n: &Node) -> Result { - struct Visitor<'a> { - db: &'a RocksDB, - bucket_id: u32, - }; - - impl EvaluateVisitor for Visitor<'_> { - fn equal(&mut self, left: &str, right: &str) -> Result { - self.db - .get_posting_list_for_tag_key_value(self.bucket_id, left, right) - } - } - - Evaluate::evaluate( - Visitor { - db: self, - bucket_id, - }, - n, - ) - } - - fn get_posting_list_for_tag_key_value( - &self, - bucket_id: u32, - key: &str, - value: &str, - ) -> Result { - // first get the cf for this index - let cf_name = index_cf_name(bucket_id); - let db = self.db.read().unwrap(); - - match db.cf_handle(&cf_name) { - Some(cf) => { - match db - .get_cf(cf, index_key_value_posting_list(bucket_id, key, value)) - .unwrap() - { - Some(val) => { - let map = Treemap::deserialize(&val) - .expect("unexpected error deserializing tree map"); - Ok(map) - } - None => Ok(Treemap::create()), - } - } - None => Err(StorageError { - description: "unable to find index".to_string(), - }), - } - } - - // TODO: handle predicate - pub fn get_tag_keys(&self, bucket_id: u32, _predicate: Option<&Predicate>) -> Vec { - let cf_name = index_cf_name(bucket_id); - let mut keys = vec![]; - - let db = self.db.read().unwrap(); - - if let Some(index) = db.cf_handle(&cf_name) { - let prefix = index_tag_key_prefix(bucket_id); - let mode = IteratorMode::From(&prefix, Direction::Forward); - let iter = db - .iterator_cf(index, mode) - .expect("unexpected rocksdb error getting iterator for index"); - - for (key, _) in iter { - if !key.starts_with(&prefix) { - break; - } - - let k = std::str::from_utf8(&key[prefix.len()..]).unwrap(); // TODO: determine what we want to do with errors - keys.push(k.to_string()); - } - }; - - keys - } - - pub fn get_tag_values( - &self, - bucket_id: u32, - tag: &str, - _predicate: Option<&Predicate>, - ) -> Vec { - let cf_name = index_cf_name(bucket_id); - - let db = self.db.read().unwrap(); - let mut values = vec![]; - - if let Some(index) = db.cf_handle(&cf_name) { - let prefix = index_tag_key_value_prefix(bucket_id, tag); - let mode = IteratorMode::From(&prefix, Direction::Forward); - let iter = db - .iterator_cf(index, mode) - .expect("unexpected rocksdb error getting iterator for index"); - - for (key, _) in iter { - if !key.starts_with(&prefix) { - break; - } - - let v = std::str::from_utf8(&key[prefix.len()..]).unwrap(); // TODO: determine what to do with errors - values.push(v.to_string()); - } - } - - values - } - - // ensure_series_mutex_exists makes sure that the passed in bucket id has a mutex, which is used - // when inserting new series into a bucket - fn ensure_series_mutex_exists(&self, bucket_id: u32) { - let map = self.series_insert_lock.read().expect("mutex poisoned"); - - if let Some(_next_id_mutex) = map.get(&bucket_id) { - return; - } - - // if we got this far we need to create a mutex for this bucket - drop(map); - let mut map = self.series_insert_lock.write().expect("mutex poisoned"); - - // now only insert the new mutex if someone else hasn't done it between dropping read and obtaining write - if map.get(&bucket_id).is_none() { - map.insert(bucket_id, Mutex::new(1)); - } - } - - // TODO: ensure that points with timestamps older than the first index level get inserted only into the higher levels - // TODO: build the index for levels other than the first - // insert_series_without_ids will insert any series into the index and obtain an identifier for it. - // the passed in series vector is modified so that the newly inserted series have their ids - pub fn insert_series_without_ids(&self, bucket_id: u32, points: &mut [PointType]) { - // We want to get a lock on new series only for this bucket - self.ensure_series_mutex_exists(bucket_id); - let map = self.series_insert_lock.read().expect("mutex poisoned"); - let next_id = map - .get(&bucket_id) - .expect("should exist because of call to ensure_series_mutex_exists"); - let mut next_id = next_id.lock().expect("mutex poisoned"); - - let mut batch = WriteBatch::default(); - - // create the column family to store the index if it doesn't exist - let cf_name = index_cf_name(bucket_id); - let index_exists = self.db.read().unwrap().cf_handle(&cf_name).is_some(); - - if !index_exists { - self.db - .write() - .unwrap() - .create_cf(&cf_name, &index_cf_options()) - .unwrap(); - } - - let db = self.db.read().unwrap(); - let index_cf = db - .cf_handle(&cf_name) - .expect("index column family should have already been inserted"); - - // Keep an in memory map for updating multiple index entries at a time - let mut index_map: HashMap, Treemap> = HashMap::new(); - let mut series_id_map: HashMap = HashMap::new(); - - // now loop through the series and insert the index entries into the map - for point in points { - // don't bother with series in the collection that already have IDs - if point.series_id().is_some() { - continue; - } - - // if we've already put this series in the map in this write, skip it - if let Some(id) = series_id_map.get(point.series()) { - point.set_series_id(*id); - continue; - } - - // now that we have the mutex on series, make sure these weren't inserted in some other thread - if let Some(id) = self.get_series_id(&cf_name, &point.series()) { - point.set_series_id(id); - continue; - } - - point.set_series_id(*next_id); - let id = *next_id; - let series_id = next_id.to_be_bytes(); - - batch - .put_cf(index_cf, index_series_key_id(&point.series()), &series_id) - .unwrap(); - batch - .put_cf( - index_cf, - index_series_id(&series_id), - index_series_id_value(series_type_from_point_type(&point), &point.series()), - ) - .unwrap(); - series_id_map.insert(point.series().clone(), *next_id); - *next_id += 1; - - // insert the index entries - // TODO: do the error handling bits, but how to handle? Should all series be validated before - // and fail the whole write if any one is bad, or insert the ones we can and ignore and log the bad? - - let pairs = point.index_pairs().unwrap(); - for pair in pairs { - // insert the tag key index - batch - .put_cf(index_cf, index_tag_key(bucket_id, &pair.key), vec![0 as u8]) - .unwrap(); - - // insert the tag value index - batch - .put_cf( - index_cf, - index_tag_key_value(bucket_id, &pair.key, &pair.value), - vec![0 as u8], - ) - .unwrap(); - - // update the key to id bitmap - let index_key_posting_list_key = index_key_posting_list(bucket_id, &pair.key); - - // put it in the temporary in memory map for a single write update later - let tree = match index_map.entry(index_key_posting_list_key) { - Entry::Occupied(e) => e.into_mut(), - Entry::Vacant(e) => { - let map = match self.db.read().unwrap().get_cf(index_cf, e.key()).unwrap() { - Some(b) => Treemap::deserialize(&b) - .expect("unexpected error deserializing posting list"), - None => Treemap::create(), - }; - e.insert(map) - } - }; - tree.add(id); - - // update the key/value to id bitmap - let index_key_value_posting_list_key = - index_key_value_posting_list(bucket_id, &pair.key, &pair.value); - - let tree = match index_map.entry(index_key_value_posting_list_key) { - Entry::Occupied(e) => e.into_mut(), - Entry::Vacant(e) => { - let map = match self.db.read().unwrap().get_cf(index_cf, e.key()).unwrap() { - Some(b) => Treemap::deserialize(&b) - .expect("unexpected error deserializing posting list"), - None => Treemap::create(), - }; - e.insert(map) - } - }; - tree.add(id); - } - } - - // do the index writes from the in temporary in memory map - for (k, v) in &index_map { - let _ = batch.put_cf(index_cf, k, v.serialize().unwrap()); - } - - // save the next series id - let bucket_cf = db.cf_handle(BUCKET_CF).unwrap(); - let mut next_series_id_val = Vec::with_capacity(8); - next_series_id_val.write_u64::(*next_id).unwrap(); - let _ = batch.put_cf(bucket_cf, next_series_id_key(bucket_id), next_series_id_val); - - db.write(batch).expect("unexpected rocksdb error"); - } - - fn get_series_id(&self, cf_name: &str, series_key: &str) -> Option { - // this column family might not exist if this index hasn't been created yet - if let Some(cf) = self.db.read().unwrap().cf_handle(cf_name) { - if let Some(val) = self - .db - .read() - .unwrap() - .get_cf(cf, index_series_key_id(series_key)) - .expect("unexpected rocksdb error") - { - let mut c = Cursor::new(val); - Some(c.read_u64::().unwrap()) - } else { - None - } - } else { - None - } - } - - fn load_bucket_map(&mut self) { - let db = self.db.read().unwrap(); - - let buckets = db.cf_handle(BUCKET_CF).unwrap(); - let prefix = &[BucketEntryType::Bucket.into()]; - let iter = db - .iterator_cf(&buckets, IteratorMode::From(prefix, Direction::Forward)) - .unwrap(); - - let mut id_mutex_map = HashMap::new(); - let mut bucket_map = self.bucket_map.write().unwrap(); - let mut bucket_id_map = self.bucket_id_map.write().unwrap(); - - for (key, value) in iter { - match key[0].try_into().unwrap() { - BucketEntryType::NextSeriesID => { - // read the bucket id from the key - let mut c = Cursor::new(&key[1..]); - let bucket_id = c.read_u32::().unwrap_or_else(|_| { - panic!("couldn't read the bucket id from the key {:?}", key) - }); - - // and the next series ID - let mut c = Cursor::new(value); - let next_id = c.read_u64::().unwrap_or_else(|_| { - panic!("couldn't read the next series id for bucket {}", bucket_id) - }); - id_mutex_map.insert(bucket_id, Mutex::new(next_id)); - } - BucketEntryType::Bucket => { - let bucket = Bucket::decode(&*value).expect("unexpected error decoding bucket"); - let key = bucket_key(bucket.org_id, &bucket.name); - let arc_bucket = Arc::new(bucket); - bucket_map.insert(key, arc_bucket.clone()); - bucket_id_map.insert(arc_bucket.id, arc_bucket); - } - BucketEntryType::NextBucketID => (), - } - } - self.series_insert_lock = Arc::new(RwLock::new(id_mutex_map)); - } -} - -pub trait ToRocksDBBytes { - fn to_rocks_db_bytes(&self) -> Vec; -} - -impl ToRocksDBBytes for PointType { - fn to_rocks_db_bytes(&self) -> Vec { - match self { - PointType::I64(inner) => inner.value.to_rocks_db_bytes(), - PointType::F64(inner) => inner.value.to_rocks_db_bytes(), - } - } -} - -impl ToRocksDBBytes for i64 { - fn to_rocks_db_bytes(&self) -> Vec { - let mut value = Vec::with_capacity(8); - value.write_i64::(*self).unwrap(); - value - } -} - -impl ToRocksDBBytes for f64 { - fn to_rocks_db_bytes(&self) -> Vec { - let mut value = Vec::with_capacity(8); - value.write_f64::(*self).unwrap(); - value - } -} - -impl InvertedIndex for RocksDB { - fn get_or_create_series_ids_for_points( - &self, - bucket_id: u32, - points: &mut [PointType], - ) -> Result<(), StorageError> { - self.get_series_ids(bucket_id, points)?; - self.insert_series_without_ids(bucket_id, points); - Ok(()) - } - - fn read_series_matching( - &self, - bucket_id: u32, - predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - let filters = self.get_series_filters(bucket_id, predicate)?; - Ok(Box::new(filters.into_iter())) - } - - fn get_tag_keys( - &self, - bucket_id: u32, - predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - let keys = self.get_tag_keys(bucket_id, predicate); - Ok(Box::new(keys.into_iter())) - } - - fn get_tag_values( - &self, - bucket_id: u32, - tag_key: &str, - predicate: Option<&Predicate>, - ) -> Result + Send>, StorageError> { - let values = self.get_tag_values(bucket_id, tag_key, predicate); - Ok(Box::new(values.into_iter())) - } -} - -impl SeriesStore for RocksDB { - fn write_points_with_series_ids( - &self, - bucket_id: u32, - points: &[PointType], - ) -> Result<(), StorageError> { - self.write_points(bucket_id, &points) - } - - fn read_i64_range( - &self, - bucket_id: u32, - series_id: u64, - range: &TimestampRange, - batch_size: usize, - ) -> Result>> + Send>, StorageError> { - self.read_range(bucket_id, series_id, range, batch_size) - } - - fn read_f64_range( - &self, - bucket_id: u32, - series_id: u64, - range: &TimestampRange, - batch_size: usize, - ) -> Result>> + Send>, StorageError> { - self.read_range(bucket_id, series_id, range, batch_size) - } -} - -impl ConfigStore for RocksDB { - fn create_bucket_if_not_exists( - &self, - org_id: u32, - bucket: &Bucket, - ) -> Result { - self.create_bucket_if_not_exists(org_id, bucket) - } - - fn get_bucket_by_name( - &self, - org_id: u32, - bucket_name: &str, - ) -> Result>, StorageError> { - self.get_bucket_by_name(org_id, bucket_name) - } - - fn get_bucket_by_id(&self, bucket_id: u32) -> Result>, StorageError> { - self.get_bucket_by_id(bucket_id) - } -} - -/* -Index entries all have the prefix: - - -Index keeps the following entries (entry type is the first part). So key:value - -series key to ID: : -ID to series key: : - -key posting list: : -key/value posting list: <0x0>: - -this one is for show keys or show values where key = value queries -tag value map: <0x0><0x0><0x0>: -*/ - -/* - -TODO: The index todo list -1. no predicate (return all series) -2. starts with -3. regex match -4. not equal -5. not regex -6. value matches -7. convert series/predicate matcher to return iterator over SeriesFilter -8. index levels - -TODO: other pieces - - API endpoint to delete old series data - - API endpoint to delete old indexes - - API endpoint to run tsm compaction - - Write/read other data types - - Buckets backed by alternate storage - - Meta store abstracted from RocksDB - - Index abstracted to Trait - - Raw data iterator abstracted to Trait - -*/ - -fn prefix_for_series(bucket_id: u32, series_id: u64, start_time: i64) -> Vec { - let mut v = Vec::with_capacity(20); - v.write_u32::(bucket_id).unwrap(); - v.write_u64::(series_id).unwrap(); - v.write_i64::(start_time).unwrap(); - v -} - -pub struct PointsIterator<'a, T: Clone> { - batch_size: usize, - iter: DBIterator<'a>, - stop_time: i64, - series_prefix: Vec, - drained: bool, - read: fn(b: &[u8]) -> T, -} - -impl Iterator for PointsIterator<'_, T> { - type Item = Vec>; - - fn next(&mut self) -> Option { - if self.drained { - return None; - } - - let mut v = Vec::with_capacity(self.batch_size); - let mut n = 0; - - // we have to check if the iterator is still valid. There are some edge cases where - // this function could get called with an invalid iterator because it has gone to - // the end of th rocksdb keyspace. Calling next on it segfaults the program, so check it first. - // Here's the issue: https://github.com/rust-rocksdb/rust-rocksdb/issues/361 - if !self.iter.valid() { - self.drained = true; - return None; - } - while let Some((key, value)) = self.iter.next() { - if !key.starts_with(&self.series_prefix) { - self.drained = true; - break; - } - - let time = BigEndian::read_i64(&key[12..]); - if time > self.stop_time { - self.drained = true; - break; - } - - let point = ReadPoint { - value: (self.read)(&value), - time, - }; - - v.push(point); - - n += 1; - if n >= self.batch_size { - break; - } - } - - if v.is_empty() { - self.drained = true; - None - } else { - Some(v) - } - } -} - -// IndexEntryType is used as a u8 prefix for any key in rocks for these different index entries -enum IndexEntryType { - SeriesKeyToID, - IDToSeriesKey, - KeyPostingList, - KeyValuePostingList, - KeyList, - KeyValueList, -} - -fn index_cf_options() -> Options { - let mut options = Options::default(); - options.set_write_buffer_size(INDEX_CF_WRITE_BUFFER_SIZE); - options -} - -fn index_cf_name(bucket_id: u32) -> String { - format!("index_{}", bucket_id,) -} - -fn index_series_key_id(series_key: &str) -> Vec { - let mut v = Vec::with_capacity(series_key.len() + 1); - v.push(IndexEntryType::SeriesKeyToID as u8); - v.extend_from_slice(series_key.as_bytes()); - v -} - -fn index_series_id(id: &[u8]) -> Vec { - let mut v = Vec::with_capacity(8 + 1); - v.push(IndexEntryType::IDToSeriesKey as u8); - v.extend_from_slice(id); - v -} - -fn index_series_id_value(t: SeriesDataType, key: &str) -> Vec { - let mut v = Vec::with_capacity(1 + key.len()); - v.push(t.into()); - v.extend_from_slice(key.as_bytes()); - v -} - -fn series_type_from_point_type(p: &PointType) -> SeriesDataType { - match p { - PointType::I64(_) => SeriesDataType::I64, - PointType::F64(_) => SeriesDataType::F64, - } -} - -fn index_series_id_from_id(id: u64) -> Vec { - let mut v = Vec::with_capacity(8 + 1); - v.push(IndexEntryType::IDToSeriesKey as u8); - v.write_u64::(id).unwrap(); - v -} - -fn index_tag_key(bucket_id: u32, key: &str) -> Vec { - let mut v = Vec::with_capacity(key.len() + 5); - v.push(IndexEntryType::KeyList as u8); - v.write_u32::(bucket_id).unwrap(); - v.extend_from_slice(key.as_bytes()); - v -} - -fn index_tag_key_prefix(bucket_id: u32) -> Vec { - let mut v = Vec::with_capacity(5); - v.push(IndexEntryType::KeyList as u8); - v.write_u32::(bucket_id).unwrap(); - v -} - -fn index_tag_key_value(bucket_id: u32, key: &str, value: &str) -> Vec { - let mut v = Vec::with_capacity(key.len() + value.len() + 6); - v.push(IndexEntryType::KeyValueList as u8); - v.write_u32::(bucket_id).unwrap(); - v.extend_from_slice(key.as_bytes()); - v.push(0 as u8); - v.extend_from_slice(value.as_bytes()); - v -} - -fn index_tag_key_value_prefix(bucket_id: u32, key: &str) -> Vec { - let mut v = Vec::with_capacity(key.len() + 6); - v.push(IndexEntryType::KeyValueList as u8); - v.write_u32::(bucket_id).unwrap(); - v.extend_from_slice(key.as_bytes()); - v.push(0 as u8); - v -} - -fn index_key_posting_list(bucket_id: u32, key: &str) -> Vec { - let mut v = Vec::with_capacity(key.len() + 6); - v.push(IndexEntryType::KeyPostingList as u8); - v.write_u32::(bucket_id).unwrap(); - v.extend_from_slice(key.as_bytes()); - v -} - -fn index_key_value_posting_list(bucket_id: u32, key: &str, value: &str) -> Vec { - let mut v = Vec::with_capacity(key.len() + value.len() + 6); - v.push(IndexEntryType::KeyValuePostingList as u8); - v.write_u32::(bucket_id).unwrap(); - v.extend_from_slice(key.as_bytes()); - v.push(0 as u8); - v.extend_from_slice(value.as_bytes()); - v -} - -// next_series_id_key gives the key in the buckets CF in rocks that holds the value for the next series ID -fn next_series_id_key(bucket_id: u32) -> Vec { - let mut v = Vec::with_capacity(5); - v.push(BucketEntryType::NextSeriesID.into()); - v.write_u32::(bucket_id).unwrap(); - v -} - -// The values for these enum variants have no real meaning, but they -// are serialized to disk. Revisit these whenever it's time to decide -// on an on-disk format. -enum BucketEntryType { - Bucket = 0, - NextSeriesID = 1, - NextBucketID = 2, -} - -impl From for u8 { - fn from(other: BucketEntryType) -> Self { - other as u8 - } -} - -impl TryFrom for BucketEntryType { - type Error = u8; - - fn try_from(other: u8) -> Result { - use BucketEntryType::*; - - match other { - v if v == Bucket as u8 => Ok(Bucket), - v if v == NextSeriesID as u8 => Ok(NextSeriesID), - v if v == NextBucketID as u8 => Ok(NextBucketID), - _ => Err(other), - } - } -} - -// TODO: ensure required fields are present and write tests -fn validate_bucket_fields(_bucket: &Bucket) -> Result<(), StorageError> { - Ok(()) -} -// returns the byte key to find this bucket in the buckets CF in rocks -fn bucket_key(org_id: u32, bucket_name: &str) -> Vec { - let s = bucket_name.as_bytes(); - let mut key = Vec::with_capacity(3 + s.len()); - key.push(BucketEntryType::Bucket.into()); - key.write_u32::(org_id).unwrap(); - key.extend_from_slice(s); - key -} - -fn next_bucket_id_key() -> Vec { - vec![BucketEntryType::NextBucketID.into()] -} - -fn bucket_cf_descriptor() -> ColumnFamilyDescriptor { - let mut buckets_options = Options::default(); - buckets_options.set_write_buffer_size(BUCKET_CF_WRITE_BUFFER_SIZE); - ColumnFamilyDescriptor::new(BUCKET_CF, buckets_options) -} - -fn u32_from_bytes(b: &[u8]) -> u32 { - let mut c = Cursor::new(b); - c.read_u32::().unwrap() -} - -trait FromBytes { - fn from(b: &[u8]) -> Self; -} - -impl FromBytes for i64 { - fn from(b: &[u8]) -> i64 { - let mut c = Cursor::new(b); - c.read_i64::().unwrap() - } -} - -impl FromBytes for f64 { - fn from(b: &[u8]) -> f64 { - let mut c = Cursor::new(b); - c.read_f64::().unwrap() - } -} - -impl Bucket { - pub fn new(org_id: u32, name: String) -> Bucket { - Bucket { - org_id, - id: 0, - name, - retention: "0".to_string(), - posting_list_rollover: 10_000, - index_levels: vec![IndexLevel { - duration_seconds: 0, - timezone: "EDT".to_string(), - }], - } - } -} - -fn key_for_series_and_time(bucket_id: u32, series_id: u64, timestamp: i64) -> Vec { - let mut v = Vec::with_capacity(20); - v.write_u32::(bucket_id).unwrap(); - v.write_u64::(series_id).unwrap(); - v.write_i64::(timestamp).unwrap(); - v -} - -#[cfg(test)] -mod tests { - use super::*; - - use dotenv::dotenv; - use std::env; - - use crate::storage::predicate::parse_predicate; - - #[test] - fn create_and_get_buckets() { - let bucket: Arc; - let org_id = 1; - let mut bucket2 = Bucket::new(2, "Foo".to_string()); - { - let db = test_database("create_and_get_buckets", true); - let mut b = Bucket::new(org_id, "Foo".to_string()); - - b.id = db.create_bucket_if_not_exists(org_id, &b).unwrap(); - assert_eq!(b.id, 1); - - let stored_bucket = db.get_bucket_by_name(org_id, &b.name).unwrap().unwrap(); - assert_eq!(Arc::new(b.clone()), stored_bucket); - - let bucket_by_id = db.get_bucket_by_id(b.id).unwrap().unwrap(); - assert_eq!(Arc::new(b.clone()), bucket_by_id); - - bucket = stored_bucket; - - // ensure it doesn't insert again - let id = db.create_bucket_if_not_exists(org_id, &b).unwrap(); - assert_eq!(id, 1); - - // ensure second bucket in another org - bucket2.id = db - .create_bucket_if_not_exists(bucket2.org_id, &bucket2) - .unwrap(); - assert_eq!(bucket2.id, 2); - let stored2 = db - .get_bucket_by_name(bucket2.org_id, &bucket2.name) - .unwrap() - .unwrap(); - assert_eq!(Arc::new(bucket2), stored2); - - // ensure second bucket gets new ID - let mut b2 = Bucket::new(org_id, "two".to_string()); - b2.id = db.create_bucket_if_not_exists(org_id, &b2).unwrap(); - assert_eq!(b2.id, 3); - let stored_bucket = db.get_bucket_by_name(org_id, &b2.name).unwrap().unwrap(); - assert_eq!(Arc::new(b2), stored_bucket); - - // TODO: ensure that a bucket orders levels correctly - } - - // ensure it persists across database reload - { - let db = test_database("create_and_get_buckets", false); - let stored_bucket = db - .get_bucket_by_name(org_id, &bucket.name) - .unwrap() - .unwrap(); - assert_eq!(bucket, stored_bucket); - - // ensure a new bucket will get a new ID - let mut b = Bucket::new(org_id, "asdf".to_string()); - b.id = db.create_bucket_if_not_exists(org_id, &b).unwrap(); - assert_eq!(b.id, 4); - } - } - - #[test] - fn series_id_indexing() { - let org_id = 23; - let mut b = Bucket::new(org_id, "series".to_string()); - let mut b2 = Bucket::new(1, "series".to_string()); - let p1 = PointType::new_i64("one".to_string(), 1, 0); - let p2 = PointType::new_i64("two".to_string(), 23, 40); - let p3 = PointType::new_i64("three".to_string(), 33, 86); - let p4 = PointType::new_i64("four".to_string(), 234, 100); - - { - let db = test_database("series_id_indexing", true); - b.id = db.create_bucket_if_not_exists(org_id, &b).unwrap(); - b2.id = db.create_bucket_if_not_exists(b2.org_id, &b2).unwrap(); - - let mut points = vec![p1.clone(), p2.clone()]; - db.get_or_create_series_ids_for_points(b.id, &mut points) - .unwrap(); - assert_eq!(points[0].series_id(), Some(1)); - assert_eq!(points[1].series_id(), Some(2)); - - // now insert a new series and make sure it shows up - let mut points = vec![p1.clone(), p3.clone()]; - db.get_series_ids(b.id, &mut points).unwrap(); - assert_eq!(points[0].series_id(), Some(1)); - assert_eq!(points[1].series_id(), None); - - db.get_or_create_series_ids_for_points(b.id, &mut points) - .unwrap(); - assert_eq!(points[0].series_id(), Some(1)); - assert_eq!(points[1].series_id(), Some(3)); - - let mut points = vec![p1.clone()]; - db.get_series_ids(b2.id, &mut points).unwrap(); - assert_eq!(points[0].series_id(), None); - - // insert a series into the other org bucket - db.get_or_create_series_ids_for_points(b2.id, &mut points) - .unwrap(); - assert_eq!(points[0].series_id(), Some(1)); - } - - // now make sure that a new series gets inserted properly after restart - { - let db = test_database("series_id_indexing", false); - - // check the first org - let mut points = vec![p4.clone()]; - db.insert_series_without_ids(b.id, &mut points); - assert_eq!(points[0].series_id(), Some(4)); - - let mut points = vec![p1.clone(), p2.clone(), p3.clone(), p4]; - db.get_series_ids(b.id, &mut points).unwrap(); - assert_eq!(points[0].series_id(), Some(1)); - assert_eq!(points[1].series_id(), Some(2)); - assert_eq!(points[2].series_id(), Some(3)); - assert_eq!(points[3].series_id(), Some(4)); - - // check the second org - let mut points = vec![p2.clone()]; - db.insert_series_without_ids(b2.id, &mut points); - assert_eq!(points[0].series_id(), Some(2)); - - let mut points = vec![p1, p2, p3]; - db.get_series_ids(b2.id, &mut points).unwrap(); - assert_eq!(points[0].series_id(), Some(1)); - assert_eq!(points[1].series_id(), Some(2)); - assert_eq!(points[2].series_id(), None); - } - } - - #[test] - fn series_metadata_indexing() { - let mut bucket = Bucket::new(1, "foo".to_string()); - let db = test_database("series_metadata_indexing", true); - let p1 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 1, 0); - let p2 = PointType::new_i64("cpu,host=a,region=west\tusage_system".to_string(), 1, 0); - let p3 = PointType::new_i64("cpu,host=a,region=west\tusage_user".to_string(), 1, 0); - let p4 = PointType::new_i64("mem,host=b,region=west\tfree".to_string(), 1, 0); - - bucket.id = db - .create_bucket_if_not_exists(bucket.org_id, &bucket) - .unwrap(); - let mut points = vec![p1, p2, p3, p4]; - db.get_or_create_series_ids_for_points(bucket.id, &mut points) - .unwrap(); - - let tag_keys = db.get_tag_keys(bucket.id, None); - assert_eq!(tag_keys, vec!["_f", "_m", "host", "region"]); - - let tag_values = db.get_tag_values(bucket.id, "host", None); - assert_eq!(tag_values, vec!["a", "b"]); - - // get all series - - // get series with measurement = mem - let pred = parse_predicate(r#"_m = "cpu""#).unwrap(); - let series: Vec = db - .read_series_matching(bucket.id, Some(&pred)) - .unwrap() - .collect(); - assert_eq!( - series, - vec![ - SeriesFilter { - id: 1, - key: "cpu,host=b,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - }, - SeriesFilter { - id: 2, - key: "cpu,host=a,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - }, - SeriesFilter { - id: 3, - key: "cpu,host=a,region=west\tusage_user".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - }, - ] - ); - - // get series with host = a - let pred = parse_predicate(r#"host = "a""#).unwrap(); - let series: Vec = db - .read_series_matching(bucket.id, Some(&pred)) - .unwrap() - .collect(); - assert_eq!( - series, - vec![ - SeriesFilter { - id: 2, - key: "cpu,host=a,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - }, - SeriesFilter { - id: 3, - key: "cpu,host=a,region=west\tusage_user".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - }, - ] - ); - - // get series with measurement = cpu and host = b - let pred = parse_predicate(r#"_m = "cpu" and host = "b""#).unwrap(); - let series: Vec = db - .read_series_matching(bucket.id, Some(&pred)) - .unwrap() - .collect(); - assert_eq!( - series, - vec![SeriesFilter { - id: 1, - key: "cpu,host=b,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - },] - ); - - let pred = parse_predicate(r#"host = "a" OR _m = "mem""#).unwrap(); - let series: Vec = db - .read_series_matching(bucket.id, Some(&pred)) - .unwrap() - .collect(); - assert_eq!( - series, - vec![ - SeriesFilter { - id: 2, - key: "cpu,host=a,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - }, - SeriesFilter { - id: 3, - key: "cpu,host=a,region=west\tusage_user".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - }, - SeriesFilter { - id: 4, - key: "mem,host=b,region=west\tfree".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - }, - ] - ); - } - - #[test] - fn catch_rocksdb_iterator_segfault() { - let mut b1 = Bucket::new(1, "bucket1".to_string()); - let db = test_database("catch_rocksdb_iterator_segfault", true); - - let p1 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 1, 1); - - b1.id = db.create_bucket_if_not_exists(b1.org_id, &b1).unwrap(); - - let mut points = vec![p1]; - db.get_or_create_series_ids_for_points(b1.id, &mut points) - .unwrap(); - db.write_points(b1.id, &points).unwrap(); - - // test that we'll only read from the bucket we wrote points into - let range = TimestampRange { start: 1, end: 4 }; - let pred = parse_predicate(r#"_m = "cpu""#).unwrap(); - let mut iter = db.read_series_matching(b1.id, Some(&pred)).unwrap(); - - let series_filter = iter.next().unwrap(); - assert_eq!( - series_filter, - SeriesFilter { - id: 1, - key: "cpu,host=b,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - } - ); - assert_eq!(iter.next(), None); - let mut points_iter = db - .read_i64_range(b1.id, series_filter.id, &range, 10) - .unwrap(); - let points = points_iter.next().unwrap(); - assert_eq!(points, vec![ReadPoint { time: 1, value: 1 },]); - assert_eq!(points_iter.next(), None); - } - - #[test] - fn write_and_read_points() { - let mut b1 = Bucket::new(1, "bucket1".to_string()); - let mut b2 = Bucket::new(2, "bucket2".to_string()); - let db = test_database("write_and_read_points", true); - - let p1 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 1, 1); - let p2 = PointType::new_i64("cpu,host=b,region=west\tusage_system".to_string(), 1, 2); - let p3 = PointType::new_i64("mem,host=b,region=west\tfree".to_string(), 1, 2); - let p4 = PointType::new_i64("mem,host=b,region=west\tfree".to_string(), 1, 4); - - b1.id = db.create_bucket_if_not_exists(b1.org_id, &b1).unwrap(); - b2.id = db.create_bucket_if_not_exists(b2.org_id, &b2).unwrap(); - - let mut b1_points = vec![p1.clone(), p2.clone()]; - db.get_or_create_series_ids_for_points(b1.id, &mut b1_points) - .unwrap(); - db.write_points(b1.id, &b1_points).unwrap(); - - let mut b2_points = vec![p1, p2, p3, p4]; - db.get_or_create_series_ids_for_points(b2.id, &mut b2_points) - .unwrap(); - db.write_points(b2.id, &b2_points).unwrap(); - - // test that we'll only read from the bucket we wrote points into - let range = TimestampRange { start: 1, end: 4 }; - let pred = parse_predicate(r#"_m = "cpu" OR _m = "mem""#).unwrap(); - let mut iter = db.read_series_matching(b1.id, Some(&pred)).unwrap(); - let series_filter = iter.next().unwrap(); - assert_eq!( - series_filter, - SeriesFilter { - id: 1, - key: "cpu,host=b,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - } - ); - assert_eq!(iter.next(), None); - let mut points_iter = db - .read_i64_range(b1.id, series_filter.id, &range, 10) - .unwrap(); - let points = points_iter.next().unwrap(); - assert_eq!( - points, - vec![ - ReadPoint { time: 1, value: 1 }, - ReadPoint { time: 2, value: 1 }, - ] - ); - assert_eq!(points_iter.next(), None); - - // test that we'll read multiple series - let pred = parse_predicate(r#"_m = "cpu" OR _m = "mem""#).unwrap(); - let mut iter = db.read_series_matching(b2.id, Some(&pred)).unwrap(); - let series_filter = iter.next().unwrap(); - assert_eq!( - series_filter, - SeriesFilter { - id: 1, - key: "cpu,host=b,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - } - ); - let mut points_iter = db - .read_i64_range(b2.id, series_filter.id, &range, 10) - .unwrap(); - let points = points_iter.next().unwrap(); - assert_eq!( - points, - vec![ - ReadPoint { time: 1, value: 1 }, - ReadPoint { time: 2, value: 1 }, - ] - ); - - let series_filter = iter.next().unwrap(); - assert_eq!( - series_filter, - SeriesFilter { - id: 2, - key: "mem,host=b,region=west\tfree".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - } - ); - let mut points_iter = db - .read_i64_range(b2.id, series_filter.id, &range, 10) - .unwrap(); - let points = points_iter.next().unwrap(); - assert_eq!( - points, - vec![ - ReadPoint { time: 2, value: 1 }, - ReadPoint { time: 4, value: 1 }, - ] - ); - - // test that the batch size is honored - let pred = parse_predicate(r#"host = "b""#).unwrap(); - let mut iter = db.read_series_matching(b1.id, Some(&pred)).unwrap(); - let series_filter = iter.next().unwrap(); - assert_eq!( - series_filter, - SeriesFilter { - id: 1, - key: "cpu,host=b,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - } - ); - assert_eq!(iter.next(), None); - let mut points_iter = db - .read_i64_range(b1.id, series_filter.id, &range, 1) - .unwrap(); - let points = points_iter.next().unwrap(); - assert_eq!(points, vec![ReadPoint { time: 1, value: 1 },]); - let points = points_iter.next().unwrap(); - assert_eq!(points, vec![ReadPoint { time: 2, value: 1 },]); - - // test that the time range is properly limiting - let range = TimestampRange { start: 2, end: 3 }; - let pred = parse_predicate(r#"_m = "cpu" OR _m = "mem""#).unwrap(); - let mut iter = db.read_series_matching(b2.id, Some(&pred)).unwrap(); - let series_filter = iter.next().unwrap(); - assert_eq!( - series_filter, - SeriesFilter { - id: 1, - key: "cpu,host=b,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - } - ); - let mut points_iter = db - .read_i64_range(b2.id, series_filter.id, &range, 10) - .unwrap(); - let points = points_iter.next().unwrap(); - assert_eq!(points, vec![ReadPoint { time: 2, value: 1 },]); - - let series_filter = iter.next().unwrap(); - assert_eq!( - series_filter, - SeriesFilter { - id: 2, - key: "mem,host=b,region=west\tfree".to_string(), - value_predicate: None, - series_type: SeriesDataType::I64 - } - ); - let mut points_iter = db - .read_i64_range(b2.id, series_filter.id, &range, 10) - .unwrap(); - let points = points_iter.next().unwrap(); - assert_eq!(points, vec![ReadPoint { time: 2, value: 1 },]); - } - - #[test] - fn write_and_read_float_values() { - let mut b1 = Bucket::new(1, "bucket1".to_string()); - let db = test_database("write_and_read_float_values", true); - - let p1 = PointType::new_f64("cpu,host=b,region=west\tusage_system".to_string(), 1.0, 1); - let p2 = PointType::new_f64("cpu,host=b,region=west\tusage_system".to_string(), 2.2, 2); - - b1.id = db.create_bucket_if_not_exists(b1.org_id, &b1).unwrap(); - - let mut points = vec![p1, p2]; - db.get_or_create_series_ids_for_points(b1.id, &mut points) - .unwrap(); - db.write_points_with_series_ids(b1.id, &points).unwrap(); - - // test that we'll only read from the bucket we wrote points into - let range = TimestampRange { start: 0, end: 4 }; - let pred = parse_predicate(r#"_m = "cpu""#).unwrap(); - let mut iter = db.read_series_matching(b1.id, Some(&pred)).unwrap(); - let series_filter = iter.next().unwrap(); - assert_eq!( - series_filter, - SeriesFilter { - id: 1, - key: "cpu,host=b,region=west\tusage_system".to_string(), - value_predicate: None, - series_type: SeriesDataType::F64 - } - ); - assert_eq!(iter.next(), None); - let mut points_iter = db - .read_f64_range(b1.id, series_filter.id, &range, 10) - .unwrap(); - let points = points_iter.next().unwrap(); - assert_eq!( - points, - vec![ - ReadPoint { - time: 1, - value: 1.0 - }, - ReadPoint { - time: 2, - value: 2.2 - }, - ] - ); - assert_eq!(points_iter.next(), None); - } - - // Test helpers - fn get_test_storage_path() -> String { - dotenv().ok(); // load .env file if present - match env::var("TEST_DELOREAN_DB_DIR") { - Ok(val) => val, - Err(_) => { - // default test asset path is /delorean - let mut path = env::temp_dir(); - path.push("delorean/"); - path.into_os_string().into_string().unwrap() - } - } - } - - fn test_database(name: &str, remove_old: bool) -> RocksDB { - let path = std::path::Path::new(&get_test_storage_path()).join(name); - if remove_old { - let _ = std::fs::remove_dir_all(path.to_str().unwrap()); - } - RocksDB::new(path.to_str().unwrap()) - } -} From 6549bebf2ccbf7ec6b97cbb95053f22bb0b50f46 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 10:05:10 -0400 Subject: [PATCH 28/42] refactor: Make MemDB::new more idiomatic --- src/storage/memdb.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/storage/memdb.rs b/src/storage/memdb.rs index 67051c48fa..5c8d2dca23 100644 --- a/src/storage/memdb.rs +++ b/src/storage/memdb.rs @@ -184,9 +184,10 @@ fn list_key(key: &str, value: &str) -> Vec { impl MemDB { pub fn new(id: String) -> Self { - let mut memdb: MemDB = Default::default(); - memdb.id = id; - memdb + MemDB { + id, + ..Default::default() + } } pub fn size(&self) -> usize { From a7d33b803cdb3e46c3ec11ec7d9d9b9ab8f09b6c Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 10:57:32 -0400 Subject: [PATCH 29/42] fix: Create org if it doesn't exist --- src/storage/database.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/storage/database.rs b/src/storage/database.rs index 1d0e3c336b..f2071de1ef 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -116,10 +116,14 @@ impl Database { org_id: u32, bucket_name: &str, ) -> Result, StorageError> { + if !self.organizations.read().await.contains_key(&org_id) { + let mut orgs = self.organizations.write().await; + orgs.insert(org_id, RwLock::new(Organization::default())); + } + let orgs = self.organizations.read().await; - let org = orgs.get(&org_id).ok_or_else(|| StorageError { - description: format!("org {} not found", org_id), - })?; + + let org = orgs.get(&org_id).expect("Should have found or just inserted org"); let id = match org.read().await.bucket_name_to_id.get(bucket_name) { Some(id) => Some(*id), From a81011336472415221954b20e3c8e4d79783c2d5 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 11:18:46 -0400 Subject: [PATCH 30/42] fix: Return ReadPoint.value instead of time twice --- src/main.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main.rs b/src/main.rs index 22c46d8ab6..addcc3dcc5 100644 --- a/src/main.rs +++ b/src/main.rs @@ -208,7 +208,7 @@ async fn read(req: hyper::Request, app: Arc) -> Result { for val in values { let t = val.time.to_string(); - let v = val.time.to_string(); + let v = val.value.to_string(); vals[vcol] = v; vals[tcol] = t; wtr.write_record(&vals).unwrap(); @@ -217,7 +217,7 @@ async fn read(req: hyper::Request, app: Arc) -> Result { for val in values { let t = val.time.to_string(); - let v = val.time.to_string(); + let v = val.value.to_string(); vals[vcol] = v; vals[tcol] = t; wtr.write_record(&vals).unwrap(); From ee0afb5e022df149fca389a8c5fd31626f0f78e6 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 11:40:45 -0400 Subject: [PATCH 31/42] fix: Rename field to not start with an underscore --- src/storage/database.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/storage/database.rs b/src/storage/database.rs index f2071de1ef..33a1d5c1ed 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -37,7 +37,7 @@ impl Organization { } struct BucketData { - _config: Bucket, + config: Bucket, // TODO: wire up rules for partitioning data and storing and reading from multiple partitions partition: RwLock, } @@ -50,7 +50,7 @@ impl BucketData { let partition = Partition::MemDB(Box::new(MemDB::new(partition_id))); BucketData { - _config: bucket, + config: bucket, partition: RwLock::new(partition), } } From d097af781d6e585c070306fa9ea35c08231973f6 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 11:52:01 -0400 Subject: [PATCH 32/42] feature: Implement the get_buckets rpc to get bucket ids in the test --- src/rpc.rs | 15 +++++++++++++-- src/storage/database.rs | 13 +++++++++++++ tests/end-to-end.rs | 17 +++++++++++++---- 3 files changed, 39 insertions(+), 6 deletions(-) diff --git a/src/rpc.rs b/src/rpc.rs index 8684a2f582..84d87718e4 100644 --- a/src/rpc.rs +++ b/src/rpc.rs @@ -42,9 +42,20 @@ impl Delorean for GrpcServer { async fn get_buckets( &self, - _req: tonic::Request, + req: tonic::Request, ) -> Result, Status> { - Ok(tonic::Response::new(GetBucketsResponse { buckets: vec![] })) + let org = req.into_inner(); + + let org_id = org.id; + + let buckets = self + .app + .db + .buckets(org_id) + .await + .map_err(|err| Status::internal(format!("error reading db: {}", err)))?; + + Ok(tonic::Response::new(GetBucketsResponse { buckets })) } } diff --git a/src/storage/database.rs b/src/storage/database.rs index 33a1d5c1ed..8b7b237584 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -184,6 +184,19 @@ impl Database { bucket_data.get_tag_values(tag_key, predicate, range).await } + pub async fn buckets(&self, org_id: u32) -> Result, StorageError> { + Ok(match self.organizations.read().await.get(&org_id) { + None => vec![], + Some(org) => org + .read() + .await + .bucket_data + .values() + .map(|bd| bd.config.clone()) + .collect(), + }) + } + async fn bucket_data( &self, org_id: u32, diff --git a/tests/end-to-end.rs b/tests/end-to-end.rs index 5412f8b499..ccbcee536d 100644 --- a/tests/end-to-end.rs +++ b/tests/end-to-end.rs @@ -199,12 +199,21 @@ cpu_load_short,server01,us-east,value,{},1234567.891011 let mut storage_client = StorageClient::connect(GRPC_URL_BASE).await?; - let org_id = u64::from(u32::MAX); - let bucket_id = 1; // TODO: how do we know this? + // Get the ID of the bucket that was created with the auto-incrementing in MemDB + let get_buckets_request = tonic::Request::new(Organization { + id: org_id, + name: "test".into(), + buckets: vec![], + }); + let get_buckets_response = grpc_client.get_buckets(get_buckets_request).await?; + let get_buckets_response = get_buckets_response.into_inner(); + let org_buckets = get_buckets_response.buckets; + let bucket_id = org_buckets.first().unwrap().id; + let partition_id = u64::from(u32::MAX); let read_source = ReadSource { - org_id, - bucket_id, + org_id: org_id.into(), + bucket_id: bucket_id.into(), partition_id, }; let mut d = Vec::new(); From 32ed04c75326a89f11524d6237c7e9e205e72f9a Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 11:53:07 -0400 Subject: [PATCH 33/42] style: Cargo fmt --- src/storage/database.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/storage/database.rs b/src/storage/database.rs index 8b7b237584..792f8093ae 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -123,7 +123,9 @@ impl Database { let orgs = self.organizations.read().await; - let org = orgs.get(&org_id).expect("Should have found or just inserted org"); + let org = orgs + .get(&org_id) + .expect("Should have found or just inserted org"); let id = match org.read().await.bucket_name_to_id.get(bucket_name) { Some(id) => Some(*id), From cd470908d744858b35f8d05c97c4b16761af3370 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 11:53:53 -0400 Subject: [PATCH 34/42] fix: Insert org if it's not found --- src/storage/database.rs | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/storage/database.rs b/src/storage/database.rs index 792f8093ae..cb0a1d7c0c 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -204,11 +204,16 @@ impl Database { org_id: u32, bucket_id: u32, ) -> Result, StorageError> { - let orgs = self.organizations.read().await; - let org = orgs.get(&org_id).ok_or_else(|| StorageError { - description: format!("org {} not found", org_id), - })?; + if !self.organizations.read().await.contains_key(&org_id) { + let mut orgs = self.organizations.write().await; + orgs.insert(org_id, RwLock::new(Organization::default())); + } + let orgs = self.organizations.read().await; + + let org = orgs + .get(&org_id) + .expect("Should have found or just inserted org"); let org = org.read().await; match org.bucket_data.get(&bucket_id) { From c83805854d422395a0ab76dafc6acf80642eb93c Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 2 Apr 2020 15:47:43 -0400 Subject: [PATCH 35/42] Fix end-to-end test for new rpc.rs behavior --- tests/end-to-end.rs | 20 ++++++++------------ 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/tests/end-to-end.rs b/tests/end-to-end.rs index ccbcee536d..a01056af98 100644 --- a/tests/end-to-end.rs +++ b/tests/end-to-end.rs @@ -226,7 +226,7 @@ cpu_load_short,server01,us-east,value,{},1234567.891011 let range = TimestampRange { start: ns_since_epoch, - end: ns_since_epoch + 3, + end: ns_since_epoch + 4, }; let range = Some(range); @@ -268,7 +268,7 @@ cpu_load_short,server01,us-east,value,{},1234567.891011 assert_eq!( frames.len(), - 5, + 4, "expected exactly 5 frames, but there were {}", frames.len() ); @@ -277,26 +277,22 @@ cpu_load_short,server01,us-east,value,{},1234567.891011 assert_eq!(f.data_type, DataType::Float as i32, "in frame 0"); assert_eq!( tags_as_strings(&f.tags), - vec![("host", "server01"), ("region", "us-west")] + vec![("_m", "cpu_load_short"), ("host", "server01"), ("region", "us-west"), ("_f", "value")] ); let f = assert_unwrap!(&frames[1], Data::FloatPoints, "in frame 1"); - assert_eq!(f.timestamps, [ns_since_epoch], "in frame 1"); - assert_eq!(f.values, [0.64], "in frame 1"); + assert_eq!(f.timestamps, [ns_since_epoch, ns_since_epoch + 3], "in frame 1"); + assert_eq!(f.values, [0.64, 0.000_003], "in frame 1"); - let f = assert_unwrap!(&frames[2], Data::FloatPoints, "in frame 2"); - assert_eq!(f.timestamps, [ns_since_epoch + 3], "in frame 2"); - assert_eq!(f.values, [0.000_003], "in frame 2"); - - let f = assert_unwrap!(&frames[3], Data::Series, "in frame 3"); + let f = assert_unwrap!(&frames[2], Data::Series, "in frame 3"); assert_eq!(f.data_type, DataType::Float as i32, "in frame 3"); assert_eq!( tags_as_strings(&f.tags), - vec![("host", "server01"), ("region", "us-east")] + vec![("_m", "cpu_load_short"), ("host", "server01"), ("region", "us-east"), ("_f", "value")] ); - let f = assert_unwrap!(&frames[4], Data::FloatPoints, "in frame 4"); + let f = assert_unwrap!(&frames[3], Data::FloatPoints, "in frame 4"); assert_eq!(f.timestamps, [ns_since_epoch + 2], "in frame 4"); assert_eq!(f.values, [1_234_567.891_011], "in frame 4"); From 43a2af9a6057fd92045361bd35bb1bcc676329fa Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 16:01:58 -0400 Subject: [PATCH 36/42] Revert "fix: Insert org if it's not found" This reverts commit cd470908d744858b35f8d05c97c4b16761af3370. --- src/storage/database.rs | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/src/storage/database.rs b/src/storage/database.rs index cb0a1d7c0c..792f8093ae 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -204,16 +204,11 @@ impl Database { org_id: u32, bucket_id: u32, ) -> Result, StorageError> { - if !self.organizations.read().await.contains_key(&org_id) { - let mut orgs = self.organizations.write().await; - orgs.insert(org_id, RwLock::new(Organization::default())); - } - let orgs = self.organizations.read().await; + let org = orgs.get(&org_id).ok_or_else(|| StorageError { + description: format!("org {} not found", org_id), + })?; - let org = orgs - .get(&org_id) - .expect("Should have found or just inserted org"); let org = org.read().await; match org.bucket_data.get(&bucket_id) { From 1334dafc0aec5db2654d3809c406cdab53c646ba Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 16:03:05 -0400 Subject: [PATCH 37/42] Revert "fix: Create org if it doesn't exist" This reverts commit a7d33b803cdb3e46c3ec11ec7d9d9b9ab8f09b6c. --- src/storage/database.rs | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/src/storage/database.rs b/src/storage/database.rs index 792f8093ae..45925d5d3f 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -116,16 +116,10 @@ impl Database { org_id: u32, bucket_name: &str, ) -> Result, StorageError> { - if !self.organizations.read().await.contains_key(&org_id) { - let mut orgs = self.organizations.write().await; - orgs.insert(org_id, RwLock::new(Organization::default())); - } - let orgs = self.organizations.read().await; - - let org = orgs - .get(&org_id) - .expect("Should have found or just inserted org"); + let org = orgs.get(&org_id).ok_or_else(|| StorageError { + description: format!("org {} not found", org_id), + })?; let id = match org.read().await.bucket_name_to_id.get(bucket_name) { Some(id) => Some(*id), From 38e660818e559d35b0b800b34b4c4063ce1c3cc8 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 16:39:57 -0400 Subject: [PATCH 38/42] style: Cargo fmt --- tests/end-to-end.rs | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/tests/end-to-end.rs b/tests/end-to-end.rs index a01056af98..ab810fc69d 100644 --- a/tests/end-to-end.rs +++ b/tests/end-to-end.rs @@ -277,11 +277,20 @@ cpu_load_short,server01,us-east,value,{},1234567.891011 assert_eq!(f.data_type, DataType::Float as i32, "in frame 0"); assert_eq!( tags_as_strings(&f.tags), - vec![("_m", "cpu_load_short"), ("host", "server01"), ("region", "us-west"), ("_f", "value")] + vec![ + ("_m", "cpu_load_short"), + ("host", "server01"), + ("region", "us-west"), + ("_f", "value") + ] ); let f = assert_unwrap!(&frames[1], Data::FloatPoints, "in frame 1"); - assert_eq!(f.timestamps, [ns_since_epoch, ns_since_epoch + 3], "in frame 1"); + assert_eq!( + f.timestamps, + [ns_since_epoch, ns_since_epoch + 3], + "in frame 1" + ); assert_eq!(f.values, [0.64, 0.000_003], "in frame 1"); let f = assert_unwrap!(&frames[2], Data::Series, "in frame 3"); @@ -289,7 +298,12 @@ cpu_load_short,server01,us-east,value,{},1234567.891011 assert_eq!( tags_as_strings(&f.tags), - vec![("_m", "cpu_load_short"), ("host", "server01"), ("region", "us-east"), ("_f", "value")] + vec![ + ("_m", "cpu_load_short"), + ("host", "server01"), + ("region", "us-east"), + ("_f", "value") + ] ); let f = assert_unwrap!(&frames[3], Data::FloatPoints, "in frame 4"); From db950b4aee2907e46589c72267a08ce900d73f31 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Thu, 2 Apr 2020 16:45:21 -0400 Subject: [PATCH 39/42] fix: Return None instead of Err if org doesn't exist on bucket lookup --- src/storage/database.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/storage/database.rs b/src/storage/database.rs index 45925d5d3f..60df11f1d9 100644 --- a/src/storage/database.rs +++ b/src/storage/database.rs @@ -117,9 +117,11 @@ impl Database { bucket_name: &str, ) -> Result, StorageError> { let orgs = self.organizations.read().await; - let org = orgs.get(&org_id).ok_or_else(|| StorageError { - description: format!("org {} not found", org_id), - })?; + + let org = match orgs.get(&org_id) { + Some(org) => org, + None => return Ok(None), + }; let id = match org.read().await.bucket_name_to_id.get(bucket_name) { Some(id) => Some(*id), From db0942c861aabeea3b40fd902764d81b75ed51e5 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 3 Apr 2020 09:11:08 -0400 Subject: [PATCH 40/42] feature: Implement the create bucket gRPC request --- src/rpc.rs | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/rpc.rs b/src/rpc.rs index 84d87718e4..447cf81c3f 100644 --- a/src/rpc.rs +++ b/src/rpc.rs @@ -28,8 +28,21 @@ pub struct GrpcServer { impl Delorean for GrpcServer { async fn create_bucket( &self, - _req: tonic::Request, + req: tonic::Request, ) -> Result, Status> { + let create_bucket_request = req.into_inner(); + + let org_id = create_bucket_request.org_id; + let bucket = create_bucket_request + .bucket + .ok_or_else(|| Status::invalid_argument("missing bucket argument"))?; + + self.app + .db + .create_bucket_if_not_exists(org_id, bucket) + .await + .map_err(|err| Status::internal(format!("error creating bucket: {}", err)))?; + Ok(tonic::Response::new(CreateBucketResponse {})) } From 52b2137ab07cd4ba61c676d089af902224336da1 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 3 Apr 2020 09:11:31 -0400 Subject: [PATCH 41/42] fix: Don't create a bucket+org on write --- src/main.rs | 30 ++++++------------------------ 1 file changed, 6 insertions(+), 24 deletions(-) diff --git a/src/main.rs b/src/main.rs index addcc3dcc5..76e7d00399 100644 --- a/src/main.rs +++ b/src/main.rs @@ -3,7 +3,6 @@ #[macro_use] extern crate log; -use delorean::delorean::Bucket; use delorean::delorean::{ delorean_server::DeloreanServer, storage_server::StorageServer, TimestampRange, }; @@ -55,31 +54,14 @@ async fn write(req: hyper::Request, app: Arc) -> Result id, - None => { - // create this as the default bucket - let b = Bucket { - org_id: write_info.org_id, - id: 0, - name: write_info.bucket_name.clone(), - retention: "0".to_string(), - posting_list_rollover: 10_000, - index_levels: vec![], - }; - - app.db - .create_bucket_if_not_exists(write_info.org_id, b) - .await - .map_err(|e| { - debug!("Error creating bucket: {}", e); - StatusCode::INTERNAL_SERVER_ERROR - })? - } - }; - let mut payload = req.into_body(); let mut body = BytesMut::new(); From 7bdea751d2bade3382cd0602bf94f774fb56cc15 Mon Sep 17 00:00:00 2001 From: "Carol (Nichols || Goulding)" Date: Fri, 3 Apr 2020 09:18:15 -0400 Subject: [PATCH 42/42] test: Explicitly create the bucket in the end-to-end test --- tests/end-to-end.rs | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/tests/end-to-end.rs b/tests/end-to-end.rs index ab810fc69d..effe8fd9dd 100644 --- a/tests/end-to-end.rs +++ b/tests/end-to-end.rs @@ -33,14 +33,13 @@ mod grpc { tonic::include_proto!("delorean"); } -use grpc::delorean_client::DeloreanClient; -use grpc::storage_client::StorageClient; -use grpc::Organization; -use grpc::ReadSource; use grpc::{ + delorean_client::DeloreanClient, node::{Comparison, Value}, read_response::{frame::Data, DataType}, - Node, Predicate, ReadFilterRequest, Tag, TagKeysRequest, TagValuesRequest, TimestampRange, + storage_client::StorageClient, + Bucket, CreateBucketRequest, Node, Organization, Predicate, ReadFilterRequest, ReadSource, Tag, + TagKeysRequest, TagValuesRequest, TimestampRange, }; type Error = Box; @@ -132,6 +131,19 @@ async fn read_and_write_data() -> Result<()> { // This checks that gRPC is functioning and that we're starting from an org without buckets. assert!(org_buckets.is_empty()); + let create_bucket_request = tonic::Request::new(CreateBucketRequest { + org_id, + bucket: Some(Bucket { + org_id, + id: 0, + name: bucket_name.to_string(), + retention: "0".to_string(), + posting_list_rollover: 10_000, + index_levels: vec![], + }), + }); + grpc_client.create_bucket(create_bucket_request).await?; + let start_time = SystemTime::now(); let ns_since_epoch: i64 = start_time .duration_since(SystemTime::UNIX_EPOCH)