feat: tests ffor loading data to object store and make sure twe still query read buffer

pull/24376/head
Nga Tran 2021-04-27 18:01:21 -04:00
parent 3b3ac33575
commit 6b25bc3de6
12 changed files with 379 additions and 110 deletions

View File

@ -252,7 +252,8 @@ impl File {
}
}
fn path(&self, location: &FilePath) -> PathBuf {
/// Return full path of the given location
pub fn path(&self, location: &FilePath) -> PathBuf {
let mut path = self.root.clone();
path.push_path(location);
path.to_raw()

View File

@ -1,11 +1,11 @@
use snafu::{OptionExt, ResultExt, Snafu};
use std::collections::BTreeSet;
use std::{collections::BTreeSet, sync::Arc};
use crate::table::Table;
use arrow_deps::datafusion::physical_plan::SendableRecordBatchStream;
use data_types::{partition_metadata::TableSummary, timestamp::TimestampRange};
use internal_types::{schema::Schema, selection::Selection};
use object_store::path::Path;
use object_store::{ObjectStore, path::Path};
use query::predicate::Predicate;
use tracker::{MemRegistry, MemTracker};
@ -94,11 +94,12 @@ impl Chunk {
&mut self,
table_summary: TableSummary,
file_location: Path,
store: Arc<ObjectStore>,
schema: Schema,
range: Option<TimestampRange>,
) {
self.tables
.push(Table::new(table_summary, file_location, schema, range));
.push(Table::new(table_summary, file_location, store, schema, range));
}
/// Return true if this chunk includes the given table

View File

@ -1,42 +1,24 @@
/// This module responsible to write given data to specify object store and
/// read them back
use arrow_deps::{
arrow::{
use arrow_deps::{arrow::{
datatypes::{Schema, SchemaRef},
error::ArrowError,
error::Result as ArrowResult,
record_batch::RecordBatch,
},
datafusion::{
error::DataFusionError,
physical_plan::{
parquet::RowGroupPredicateBuilder, RecordBatchStream, SendableRecordBatchStream,
},
},
parquet::{
}, datafusion::{error::DataFusionError, physical_plan::{RecordBatchStream, SendableRecordBatchStream, common::SizedRecordBatchStream, parquet::RowGroupPredicateBuilder}}, parquet::{
self,
arrow::{arrow_reader::ParquetFileArrowReader, ArrowReader, ArrowWriter},
file::{reader::FileReader, serialized_reader::SerializedFileReader, writer::TryClone},
},
};
}};
use internal_types::selection::Selection;
use object_store::{
path::{ObjectStorePath, Path},
ObjectStore, ObjectStoreApi,
};
use object_store::{ObjectStore, ObjectStoreApi, ObjectStoreIntegration, path::{ObjectStorePath, Path}};
use query::predicate::Predicate;
use bytes::Bytes;
use futures::{Stream, StreamExt};
use parking_lot::Mutex;
use snafu::{OptionExt, ResultExt, Snafu};
use std::{
fs::File,
io::{Cursor, Seek, SeekFrom, Write},
num::NonZeroU32,
sync::Arc,
task::{Context, Poll},
};
use std::{fs::File, io::{Cursor, Seek, SeekFrom, Write}, num::NonZeroU32, sync::Arc, task::{Context, Poll}};
use tokio::sync::mpsc::{channel, Receiver, Sender};
use tokio::task;
use tokio_stream::wrappers::ReceiverStream;
@ -256,27 +238,18 @@ impl Storage {
predicate: &Predicate,
selection: Selection<'_>,
schema: SchemaRef,
path: &Path,
path: Path,
store: Arc<ObjectStore>
) -> Result<SendableRecordBatchStream> {
// TODO: support non local file object store
if !path.local_file() {
panic!("Non local file object store not supported")
}
// The below code is based on
// datafusion::physical_plan::parquet::ParquetExec::execute
// Will be improved as we go
let (response_tx, response_rx): (
Sender<ArrowResult<RecordBatch>>,
Receiver<ArrowResult<RecordBatch>>,
) = channel(2);
// Full path of the parquet file
// Might need different function if this display function does not show the full
// path
let filename = path.display();
// println!("Parquet filename: {}", filename); // TODO: remove after tests
// let (response_tx, response_rx): (
// Sender<ArrowResult<RecordBatch>>,
// Receiver<ArrowResult<RecordBatch>>,
// ) = channel(2);
// Indices of columns in the schema needed to read
let projection: Vec<usize> = Self::column_indices(selection, Arc::clone(&schema));
@ -293,55 +266,164 @@ impl Storage {
// Todo: Convert this tokio into using thread pool Andrew has implemented
// recently
task::spawn_blocking(move || {
if let Err(e) = Self::read_file(
filename,
projection.as_slice(),
predicate_builder.as_ref(),
batch_size,
response_tx,
limit,
) {
println!("Parquet reader thread terminated due to error: {:?}", e);
}
});
// TODO: Until this read_filter is an async, we cannot mate this mutlthread yet
// because it returns wrong results if other thread rerun before full results are returned
// task::spawn_blocking(move || {
// if let Err(e) = Self::read_file(
// path,
// Arc::clone(&store),
// projection.as_slice(),
// predicate_builder.as_ref(),
// batch_size,
// response_tx,
// limit,
// ) {
// println!("Parquet reader thread terminated due to error: {:?}", e);
// }
// });
Ok(Box::pin(ParquetStream {
// Ok(Box::pin(ParquetStream {
// schema,
// inner: ReceiverStream::new(response_rx),
// }))
let mut batches: Vec<Arc<RecordBatch>> = vec![];
if let Err(e) = Self::read_file(
path,
Arc::clone(&store),
projection.as_slice(),
predicate_builder.as_ref(),
batch_size,
&mut batches,
limit,
) {
println!("Parquet reader thread terminated due to error: {:?}", e);
}
println!("Record batches from read_file: {:#?}", batches);
//let batches = vec![Arc::new(batch)];
Ok(Box::pin(SizedRecordBatchStream::new(
schema,
inner: ReceiverStream::new(response_rx),
}))
batches,
)))
}
fn send_result(
response_tx: &Sender<ArrowResult<RecordBatch>>,
result: ArrowResult<RecordBatch>,
) -> Result<()> {
// Note this function is running on its own blocking tokio thread so blocking
// here is ok.
response_tx
.blocking_send(result)
.map_err(|e| DataFusionError::Execution(e.to_string()))
.context(SendResult)?;
Ok(())
}
// fn send_result(
// response_tx: &Sender<ArrowResult<RecordBatch>>,
// result: ArrowResult<RecordBatch>,
// ) -> Result<()> {
// // Note this function is running on its own blocking tokio thread so blocking
// // here is ok.
// response_tx
// .blocking_send(result)
// .map_err(|e| DataFusionError::Execution(e.to_string()))
// .context(SendResult)?;
// Ok(())
// }
// fn read_file(
// path: Path,
// store: Arc<ObjectStore>,
// projection: &[usize],
// predicate_builder: Option<&RowGroupPredicateBuilder>,
// batch_size: usize,
// response_tx: Sender<ArrowResult<RecordBatch>>,
// limit: Option<usize>,
// ) -> Result<()> {
// // TODO: support non local file object store
// let (file_root, file_path) = match (&store.0, path) {
// (ObjectStoreIntegration::File(file), Path::File(location)) => (file, location),
// (_, _) => {
// panic!("Non local file object store not supported")
// }
// };
// // Get full string path
// let full_path = format!("{:?}", file_root.path(&file_path));
// let full_path = full_path.trim_matches('"');
// println!("Full path filename: {}", full_path);
// let mut total_rows = 0;
// let file = File::open(&full_path).context(OpenFile)?;
// let mut file_reader = SerializedFileReader::new(file).context(SerializedFileReaderError)?;
// if let Some(predicate_builder) = predicate_builder {
// let row_group_predicate =
// predicate_builder.build_row_group_predicate(file_reader.metadata().row_groups());
// file_reader.filter_row_groups(&row_group_predicate); //filter out
// // row group based
// // on the predicate
// }
// let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(file_reader));
// let mut batch_reader = arrow_reader
// .get_record_reader_by_columns(projection.to_owned(), batch_size)
// .context(ParquetArrowReaderError)?;
// loop {
// match batch_reader.next() {
// Some(Ok(batch)) => {
// //println!("ParquetExec got new batch from {}", filename);
// total_rows += batch.num_rows();
// Self::send_result(&response_tx, Ok(batch))?;
// if limit.map(|l| total_rows >= l).unwrap_or(false) {
// break;
// }
// }
// None => {
// break;
// }
// Some(Err(e)) => {
// let err_msg =
// //format!("Error reading batch from {}: {}", filename, e.to_string());
// format!("Error reading batch: {}", e.to_string());
// // send error to operator
// Self::send_result(&response_tx, Err(ArrowError::ParquetError(err_msg)))?;
// // terminate thread with error
// return Err(e).context(ReadingFile);
// }
// }
// }
// // finished reading files (dropping response_tx will close
// // channel)
// Ok(())
// }
fn read_file(
filename: String,
path: Path,
store: Arc<ObjectStore>,
projection: &[usize],
predicate_builder: Option<&RowGroupPredicateBuilder>,
batch_size: usize,
response_tx: Sender<ArrowResult<RecordBatch>>,
batches : &mut Vec<Arc<RecordBatch>>,
limit: Option<usize>,
) -> Result<()> {
// TODO: support non local file object store
let (file_root, file_path) = match (&store.0, path) {
(ObjectStoreIntegration::File(file), Path::File(location)) => (file, location),
(_, _) => {
panic!("Non local file object store not supported")
}
};
// Get full string path
let full_path = format!("{:?}", file_root.path(&file_path));
let full_path = full_path.trim_matches('"');
println!("Full path filename: {}", full_path);
let mut total_rows = 0;
let file = File::open(&filename).context(OpenFile)?;
let file = File::open(&full_path).context(OpenFile)?;
let mut file_reader = SerializedFileReader::new(file).context(SerializedFileReaderError)?;
if let Some(predicate_builder) = predicate_builder {
let row_group_predicate =
predicate_builder.build_row_group_predicate(file_reader.metadata().row_groups());
file_reader.filter_row_groups(&row_group_predicate); //filter out
// row group based
// on the predicate
// row group based
// on the predicate
}
let mut arrow_reader = ParquetFileArrowReader::new(Arc::new(file_reader));
let mut batch_reader = arrow_reader
@ -352,7 +434,7 @@ impl Storage {
Some(Ok(batch)) => {
//println!("ParquetExec got new batch from {}", filename);
total_rows += batch.num_rows();
Self::send_result(&response_tx, Ok(batch))?;
batches.push(Arc::new(batch));
if limit.map(|l| total_rows >= l).unwrap_or(false) {
break;
}
@ -362,15 +444,16 @@ impl Storage {
}
Some(Err(e)) => {
let err_msg =
format!("Error reading batch from {}: {}", filename, e.to_string());
// send error to operator
Self::send_result(&response_tx, Err(ArrowError::ParquetError(err_msg)))?;
//format!("Error reading batch from {}: {}", filename, e.to_string());
format!("Error reading batch: {}", e.to_string());
// terminate thread with error
return Err(e).context(ReadingFile);
}
}
}
println!("Record batches right after reading from file: {:#?}", batches);
// finished reading files (dropping response_tx will close
// channel)
Ok(())

View File

@ -5,7 +5,7 @@ use crate::storage::{self, Storage};
use arrow_deps::datafusion::physical_plan::SendableRecordBatchStream;
use data_types::{partition_metadata::TableSummary, timestamp::TimestampRange};
use internal_types::{schema::Schema, selection::Selection};
use object_store::path::Path;
use object_store::{ObjectStore, path::Path};
use query::predicate::Predicate;
#[derive(Debug, Snafu)]
@ -32,6 +32,10 @@ pub struct Table {
/// id>/<tablename>.parquet
object_store_path: Path,
/// Object store of the above relative path to get
/// full path
object_store: Arc<ObjectStore>,
/// Schema that goes with this table's parquet file
table_schema: Schema,
@ -43,12 +47,14 @@ impl Table {
pub fn new(
meta: TableSummary,
path: Path,
store: Arc<ObjectStore>,
schema: Schema,
range: Option<TimestampRange>,
) -> Self {
Self {
table_summary: meta,
object_store_path: path,
object_store: store,
table_schema: schema,
timestamp_range: range,
}
@ -126,12 +132,26 @@ impl Table {
predicate: &Predicate,
selection: Selection<'_>,
) -> Result<SendableRecordBatchStream> {
Storage::read_filter(
let data: Result<SendableRecordBatchStream> = Storage::read_filter(
predicate,
selection,
Arc::clone(&self.table_schema.as_arrow()),
&self.object_store_path,
)
.context(ReadParquet)
self.object_store_path.clone(),
Arc::clone(&self.object_store),
)
.context(ReadParquet);
// let reader: SendableRecordBatchStream = data.unwrap();
// let mut batches = Vec::new();
// while let Some(record_batch) = reader.next().await
// .next().transpose().expect("reading next batch")
// {
// batches.push(record_batch)
// }
// println!("Record Batches in parquet: {:#?}", batches);
data
}
}

View File

@ -34,6 +34,7 @@ serde = "1.0"
serde_json = "1.0"
snafu = "0.6"
snap = "1.0.0"
tempfile = "3.1.0"
tokio = { version = "1.0", features = ["macros", "time"] }
tokio-util = { version = "0.6.3" }
tracker = { path = "../tracker" }
@ -42,7 +43,6 @@ uuid = { version = "0.8", features = ["serde", "v4"] }
[dev-dependencies] # In alphabetical order
criterion = { version = "0.3.4", features = ["async_tokio"] }
flate2 = "1.0.20"
tempfile = "3.1.0"
test_helpers = { path = "../test_helpers" }
[features]

View File

@ -683,7 +683,7 @@ impl Db {
.try_into()
.context(SchemaConversion)?;
let table_time_range = time_range.map(|(start, end)| TimestampRange::new(start, end));
parquet_chunk.add_table(stats, path, schema, table_time_range);
parquet_chunk.add_table(stats, path, Arc::clone(&self.store), schema, table_time_range);
}
// Relock the chunk again (nothing else should have been able
@ -702,7 +702,7 @@ impl Db {
self.metrics.update_chunk_state(chunk.state());
debug!(%partition_key, %table_name, %chunk_id, "chunk marked MOVED. Persisting to object store complete");
Ok(DbChunk::snapshot(&chunk))
Ok(DbChunk::parquet_file_snapshot(&chunk))
}
/// Spawns a task to perform
@ -1686,6 +1686,10 @@ mod tests {
.await
.unwrap();
db.write_chunk_to_object_store("1970-01-01T00", "cpu", 0)
.await
.unwrap();
print!("Partitions2: {:?}", db.partition_keys().unwrap());
db.rollover_partition("1970-01-05T15", "cpu").await.unwrap();
@ -1703,8 +1707,8 @@ mod tests {
to_arc("1970-01-01T00"),
to_arc("cpu"),
0,
ChunkStorage::ReadBuffer,
1213,
ChunkStorage::ReadBufferAndObjectStore,
1213 + 675, // size of RB and OB chunks
),
ChunkSummary::new_without_timestamps(
to_arc("1970-01-01T00"),
@ -1737,6 +1741,7 @@ mod tests {
assert_eq!(db.memory_registries.mutable_buffer.bytes(), 121 + 157 + 159);
assert_eq!(db.memory_registries.read_buffer.bytes(), 1213);
assert_eq!(db.memory_registries.parquet.bytes(), 89); // TODO: This 89 must be replaced with 675. Ticket #1311
}
#[tokio::test]
@ -1758,6 +1763,11 @@ mod tests {
.await
.unwrap();
// write the read buffer chunk to object store
db.write_chunk_to_object_store("1970-01-01T00", "cpu", chunk_id)
.await
.unwrap();
// write into a separate partitiion
write_lp(&db, "cpu bar=1 400000000000000");
write_lp(&db, "mem frob=3 400000000000001");

View File

@ -38,6 +38,8 @@ pub enum ChunkState {
// Chunk has been completely written into object store
WrittenToObjectStore(Arc<ReadBufferChunk>, Arc<ParquetChunk>),
// Todo : There must be another state for chunk that only in object store
}
impl ChunkState {
@ -201,8 +203,8 @@ impl Chunk {
ChunkState::WritingToObjectStore(chunk) => {
(chunk.size(), ChunkStorage::ReadBufferAndObjectStore)
}
ChunkState::WrittenToObjectStore(chunk, _) => {
(chunk.size(), ChunkStorage::ReadBufferAndObjectStore)
ChunkState::WrittenToObjectStore(chunk, parquet_chunk) => {
(chunk.size() + parquet_chunk.size(), ChunkStorage::ReadBufferAndObjectStore)
}
};

View File

@ -107,9 +107,31 @@ impl DbChunk {
chunk: Arc::clone(chunk),
partition_key,
},
ChunkState::WrittenToObjectStore(chunk, _) => Self::ReadBuffer {
chunk: Arc::clone(chunk),
partition_key,
// Since data exists in both read buffer and object store, we should
// return chunk of read buffer
},
// Todo: When we have a state for parquet_file chunk only,
// ChunkState::InObjectStoreOnly(chunk) => {
// let chunk = Arc::clone(chunk);
// Self::ParquetFile { chunk }
// }
};
Arc::new(db_chunk)
}
pub fn parquet_file_snapshot(chunk: &super::catalog::chunk::Chunk) -> Arc<Self> {
use super::catalog::chunk::ChunkState;
let db_chunk = match chunk.state() {
ChunkState::WrittenToObjectStore(_, chunk) => {
let chunk = Arc::clone(chunk);
Self::ParquetFile { chunk }
},
_ => {
panic!("This is not state of a parquet_fil chunk");
}
};
Arc::new(db_chunk)

View File

@ -115,7 +115,7 @@ async fn test_field_columns_with_ts_pred() {
}
#[tokio::test]
async fn test_field_name_plan() {
async fn test_field_name_plan() { // TODO: Test fails
test_helpers::maybe_start_logging();
// Tests that the ordering that comes out is reasonable
let scenarios = OneMeasurementManyFields {}.make().await;

View File

@ -193,6 +193,8 @@ impl DbSetup for MeasurementForWindowAggregateMonths {
db,
};
// TODO: Add 2 more scenarios: one for RUB and one for RUB+OS
vec![scenario1, scenario2, scenario3]
}
}

View File

@ -5,9 +5,9 @@ use query::PartitionChunk;
use async_trait::async_trait;
use crate::db::{test_helpers::write_lp, Db};
use crate::{db::test_helpers::write_lp, Db};
use super::utils::{count_mutable_buffer_chunks, count_read_buffer_chunks, make_db};
use super::utils::{count_mutable_buffer_chunks, count_read_buffer_chunks, count_object_store_chunks, make_db};
/// Holds a database and a description of how its data was configured
#[derive(Debug)]
@ -31,24 +31,29 @@ impl DbSetup for NoData {
async fn make(&self) -> Vec<DbScenario> {
let partition_key = "1970-01-01T00";
let table_name = "cpu";
// Scenario 1: No data in the DB yet
//
let db = make_db().db;
let scenario1 = DbScenario {
scenario_name: "New, Empty Database".into(),
db,
};
// listing partitions (which may create an entry in a map)
// Scenario 2: listing partitions (which may create an entry in a map)
// in an empty database
//
let db = make_db().db;
assert_eq!(count_mutable_buffer_chunks(&db), 0);
assert_eq!(count_read_buffer_chunks(&db), 0);
assert_eq!(count_object_store_chunks(&db), 0);
let scenario2 = DbScenario {
scenario_name: "New, Empty Database after partitions are listed".into(),
db,
};
// a scenario where the database has had data loaded and then deleted
// Scenario 3: the database has had data loaded into RB and then deleted
//
let db = make_db().db;
let data = "cpu,region=west user=23.2 100";
write_lp(&db, data);
@ -60,28 +65,77 @@ impl DbSetup for NoData {
.id(),
0
);
assert_eq!(count_mutable_buffer_chunks(&db), 2); // 2 chunks open and closed
assert_eq!(count_read_buffer_chunks(&db), 0); // nothing yet
assert_eq!(count_object_store_chunks(&db), 0); // nothing yet
assert_eq!(count_mutable_buffer_chunks(&db), 2);
assert_eq!(count_read_buffer_chunks(&db), 0); // only open chunk
// Now load the closed chunk into the RB
db.load_chunk_to_read_buffer(partition_key, table_name, 0)
.await
.unwrap();
assert_eq!(count_mutable_buffer_chunks(&db), 1); // open chunk only
assert_eq!(count_read_buffer_chunks(&db), 1); // close chunk only
assert_eq!(count_object_store_chunks(&db), 0); // nothing yet
assert_eq!(count_mutable_buffer_chunks(&db), 1);
assert_eq!(count_read_buffer_chunks(&db), 1); // only open chunk
// drop chunk 0
db.drop_chunk(partition_key, table_name, 0).unwrap();
assert_eq!(count_mutable_buffer_chunks(&db), 1);
assert_eq!(count_read_buffer_chunks(&db), 0);
assert_eq!(count_mutable_buffer_chunks(&db), 1); // open chunk only
assert_eq!(count_read_buffer_chunks(&db), 0); // nothing after dropping chunk 0
assert_eq!(count_object_store_chunks(&db), 0); // still nothing
let scenario3 = DbScenario {
scenario_name: "Empty Database after drop chunk".into(),
db,
};
vec![scenario1, scenario2, scenario3]
// Scenario 4: the database has had data loaded into RB & Object Store and then deleted
//
let db = make_db().db;
let data = "cpu,region=west user=23.2 100";
write_lp(&db, data);
// move data out of open chunk
assert_eq!(
db.rollover_partition(partition_key, table_name)
.await
.unwrap()
.id(),
0
);
assert_eq!(count_mutable_buffer_chunks(&db), 2); // 2 chunks open and closed
assert_eq!(count_read_buffer_chunks(&db), 0); // nothing yet
assert_eq!(count_object_store_chunks(&db), 0); // nothing yet
// Now load the closed chunk into the RB
db.load_chunk_to_read_buffer(partition_key, table_name, 0)
.await
.unwrap();
assert_eq!(count_mutable_buffer_chunks(&db), 1); // open chunk only
assert_eq!(count_read_buffer_chunks(&db), 1); // close chunk only
assert_eq!(count_object_store_chunks(&db), 0); // nothing yet
// Now write the data in RB to object store but keep it in RB
db.write_chunk_to_object_store(partition_key, "cpu", 0)
.await
.unwrap();
// it should be the same chunk!
assert_eq!(count_mutable_buffer_chunks(&db), 1); // open chunk only
assert_eq!(count_read_buffer_chunks(&db), 1); // closed chunk only
assert_eq!(count_object_store_chunks(&db), 1); // close chunk only
// drop chunk 0
db.drop_chunk(partition_key, table_name, 0).unwrap();
assert_eq!(count_mutable_buffer_chunks(&db), 1);
assert_eq!(count_read_buffer_chunks(&db), 0);
assert_eq!(count_object_store_chunks(&db), 0);
let scenario4 = DbScenario {
scenario_name: "Empty Database after drop chunk".into(),
db,
};
vec![scenario1, scenario2, scenario3, scenario4]
}
}
@ -302,7 +356,27 @@ pub(crate) async fn make_one_chunk_scenarios(partition_key: &str, data: &str) ->
db,
};
vec![scenario1, scenario2, scenario3]
let db = make_db().db;
let table_names = write_lp(&db, data);
for table_name in &table_names {
db.rollover_partition(partition_key, &table_name)
.await
.unwrap();
db.load_chunk_to_read_buffer(partition_key, &table_name, 0)
.await
.unwrap();
db.write_chunk_to_object_store(partition_key, &table_name, 0)
.await
.unwrap();
}
let scenario4 = DbScenario {
scenario_name: "Data in both read buffer and object store".into(),
db,
};
// TODO: Add scenario 5 where data is in object store only
vec![scenario1, scenario2, scenario3, scenario4]
}
/// This function loads two chunks of lp data into 4 different scenarios
@ -382,7 +456,42 @@ pub async fn make_two_chunk_scenarios(
db,
};
vec![scenario1, scenario2, scenario3, scenario4]
// in 2 read buffer chunks that also loaded into object store
let db = make_db().db;
let table_names = write_lp(&db, data1);
for table_name in &table_names {
db.rollover_partition(partition_key, &table_name)
.await
.unwrap();
}
write_lp(&db, data2);
for table_name in &table_names {
db.rollover_partition(partition_key, &table_name)
.await
.unwrap();
db.load_chunk_to_read_buffer(partition_key, &table_name, 0)
.await
.unwrap();
db.load_chunk_to_read_buffer(partition_key, &table_name, 1)
.await
.unwrap();
db.write_chunk_to_object_store(partition_key, &table_name, 0)
.await
.unwrap();
db.write_chunk_to_object_store(partition_key, &table_name, 1)
.await
.unwrap();
}
let scenario5 = DbScenario {
scenario_name: "Data in two read buffer chunks and two parquet file chunks".into(),
db,
};
vec![scenario1, scenario2, scenario3, scenario4, scenario5]
}
/// Rollover the mutable buffer and load chunk 0 to the read bufer

View File

@ -3,11 +3,12 @@ use data_types::{
database_rules::DatabaseRules,
DatabaseName,
};
use object_store::{memory::InMemory, ObjectStore};
use object_store::{disk::File, ObjectStore};
use query::{exec::Executor, Database};
use crate::{db::Db, JobRegistry};
use std::{num::NonZeroU32, sync::Arc};
use tempfile::TempDir;
// A wrapper around a Db and a metrics registry allowing for isolated testing
// of a Db and its metrics.
@ -20,7 +21,12 @@ pub struct TestDb {
/// Used for testing: create a Database with a local store
pub fn make_db() -> TestDb {
let server_id: NonZeroU32 = NonZeroU32::new(1).unwrap();
let object_store = Arc::new(ObjectStore::new_in_memory(InMemory::new()));
// TODO: When we support parquet file in memory, we will either turn this test back to memory
// or have both tests local disk and memory
//let object_store = Arc::new(ObjectStore::new_in_memory(InMemory::new()));
// Create an object store with a specified location in a local disk
let root = TempDir::new().unwrap();
let object_store = Arc::new(ObjectStore::new_file(File::new(root.path())));
let exec = Arc::new(Executor::new(1));
let metrics_registry = Arc::new(metrics::MetricRegistry::new());
@ -72,6 +78,19 @@ pub fn count_mutable_buffer_chunks(db: &Db) -> usize {
/// Returns the number of read buffer chunks in the specified database
pub fn count_read_buffer_chunks(db: &Db) -> usize {
chunk_summary_iter(db)
.filter(|s| s.storage == ChunkStorage::ReadBuffer)
.filter(|s|
s.storage == ChunkStorage::ReadBuffer ||
s.storage == ChunkStorage::ReadBufferAndObjectStore
)
.count()
}
/// Returns the number of object store chunks in the specified database
pub fn count_object_store_chunks(db: &Db) -> usize {
chunk_summary_iter(db)
.filter(|s|
s.storage == ChunkStorage::ReadBufferAndObjectStore ||
s.storage == ChunkStorage::ObjectStoreOnly
)
.count()
}