feat: Add row count to the parquet_file record in the catalog (#3847)

Fixes #3842.
pull/24376/head
Carol (Nichols || Goulding) 2022-02-24 10:20:50 -05:00 committed by GitHub
parent d62a052394
commit 252ced7adf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 56 additions and 11 deletions

View File

@ -44,4 +44,7 @@ message IoxMetadata {
// The maximum sequence number from a sequencer in this parquet file.
int64 max_sequence_number = 13;
// Number of rows of data in this file.
int64 row_count = 14;
}

View File

@ -48,6 +48,9 @@ pub enum Error {
#[snafu(display("Error while casting Timenanosecond on Time column"))]
TimeCasting,
#[snafu(display("Could not convert row count to i64"))]
RowCountTypeConversion { source: std::num::TryFromIntError },
}
/// A specialized `Error` for Ingester's Compact errors
@ -131,6 +134,9 @@ pub async fn compact_persisting_batch(
.filter(|b| b.num_rows() != 0)
.collect();
let row_count: usize = output_batches.iter().map(|b| b.num_rows()).sum();
let row_count = row_count.try_into().context(RowCountTypeConversionSnafu)?;
// Compute min and max of the `time` column
let (min_time, max_time) = compute_timenanosecond_min_max(&output_batches)?;
@ -151,6 +157,7 @@ pub async fn compact_persisting_batch(
time_of_last_write: Time::from_timestamp_nanos(max_time),
min_sequence_number: min_seq,
max_sequence_number: max_seq,
row_count,
};
Ok(Some((output_batches, meta)))
@ -342,6 +349,7 @@ mod tests {
20000,
seq_num_start,
seq_num_end,
3,
);
assert_eq!(expected_meta, meta);
}

View File

@ -268,6 +268,7 @@ impl Persister for IngesterData {
parquet_file.max_time,
parquet_file.file_size_bytes,
parquet_file.parquet_metadata.clone(),
parquet_file.row_count,
)
.await
})

View File

@ -139,6 +139,7 @@ mod tests {
time_of_last_write: now(),
min_sequence_number: SequenceNumber::new(5),
max_sequence_number: SequenceNumber::new(6),
row_count: 0,
};
let object_store = object_store();
@ -163,6 +164,7 @@ mod tests {
time_of_last_write: now(),
min_sequence_number: SequenceNumber::new(5),
max_sequence_number: SequenceNumber::new(6),
row_count: 3,
};
let chunk1 = Arc::new(
@ -201,6 +203,7 @@ mod tests {
time_of_last_write: now(),
min_sequence_number: SequenceNumber::new(5),
max_sequence_number: SequenceNumber::new(6),
row_count: 0,
};
let path = parquet_file_object_store_path(&metadata, &object_store);

View File

@ -13,10 +13,10 @@ use std::sync::Arc;
use time::{SystemProvider, Time, TimeProvider};
use uuid::Uuid;
/// Create a persting batch, some tombstones and corresponding metadata fot them after compaction
/// Create a persisting batch, some tombstones and corresponding metadata for them after compaction
pub async fn make_persisting_batch_with_meta() -> (Arc<PersistingBatch>, Vec<Tombstone>, IoxMetadata)
{
// record bacthes of input data
// record batches of input data
let batches = create_batches_with_influxtype_different_columns_different_order().await;
// tombstones
@ -49,6 +49,8 @@ pub async fn make_persisting_batch_with_meta() -> (Arc<PersistingBatch>, Vec<Tom
let namespace_id = 1;
let table_id = 1;
let partition_id = 1;
let row_count: usize = batches.iter().map(|b| b.num_rows()).sum();
let row_count = row_count.try_into().unwrap();
// make the persisting batch
let persisting_batch = make_persisting_batch(
@ -78,6 +80,7 @@ pub async fn make_persisting_batch_with_meta() -> (Arc<PersistingBatch>, Vec<Tom
7000,
seq_num_start,
seq_num_end,
row_count,
);
(persisting_batch, tombstones, meta)
@ -120,6 +123,7 @@ pub fn make_meta(
max_time: i64,
min_sequence_number: i64,
max_sequence_number: i64,
row_count: i64,
) -> IoxMetadata {
IoxMetadata {
object_store_id,
@ -135,6 +139,7 @@ pub fn make_meta(
time_of_last_write: Time::from_timestamp_nanos(max_time),
min_sequence_number: SequenceNumber::new(min_sequence_number),
max_sequence_number: SequenceNumber::new(max_sequence_number),
row_count,
}
}

View File

@ -0,0 +1,4 @@
ALTER TABLE
IF EXISTS parquet_file
ADD
COLUMN row_count BIGINT NOT NULL DEFAULT 0;

View File

@ -592,6 +592,7 @@ pub trait ParquetFileRepo: Send + Sync {
max_time: Timestamp,
file_size_bytes: i64,
parquet_metadata: Vec<u8>,
row_count: i64,
) -> Result<ParquetFile>;
/// Flag the parquet file for deletion
@ -1041,6 +1042,8 @@ pub struct ParquetFile {
pub file_size_bytes: i64,
/// thrift-encoded parquet metadata
pub parquet_metadata: Vec<u8>,
/// the number of rows of data in this file
pub row_count: i64,
}
/// Data for a processed tombstone reference in the catalog.
@ -1539,9 +1542,9 @@ pub(crate) mod test_helpers {
let min_time = Timestamp::new(1);
let max_time = Timestamp::new(10);
// Must have no rows
let row_count = repos.parquet_files().count().await.unwrap();
assert_eq!(row_count, 0);
// Must have no parquet file records
let num_parquet_files = repos.parquet_files().count().await.unwrap();
assert_eq!(num_parquet_files, 0);
let parquet_file = repos
.parquet_files()
@ -1556,6 +1559,7 @@ pub(crate) mod test_helpers {
max_time,
1337,
b"md1".to_vec(),
0,
)
.await
.unwrap();
@ -1574,6 +1578,7 @@ pub(crate) mod test_helpers {
max_time,
1338,
b"md2".to_vec(),
0,
)
.await
.unwrap_err();
@ -1592,13 +1597,14 @@ pub(crate) mod test_helpers {
max_time,
1339,
b"md3".to_vec(),
0,
)
.await
.unwrap();
// Must have 2 rows
let row_count = repos.parquet_files().count().await.unwrap();
assert_eq!(row_count, 2);
// Must have 2 parquet files
let num_parquet_files = repos.parquet_files().count().await.unwrap();
assert_eq!(num_parquet_files, 2);
let exist_id = parquet_file.id;
let non_exist_id = ParquetFileId::new(other_file.id.get() + 10);
@ -1719,6 +1725,7 @@ pub(crate) mod test_helpers {
to_delete: false,
file_size_bytes: 1337,
parquet_metadata: b"md1".to_vec(),
row_count: 0,
};
let other_parquet = ParquetFile {
id: ParquetFileId::new(0), //fake id that will never be used
@ -1733,6 +1740,7 @@ pub(crate) mod test_helpers {
to_delete: false,
file_size_bytes: 1338,
parquet_metadata: b"md2".to_vec(),
row_count: 0,
};
let another_parquet = ParquetFile {
id: ParquetFileId::new(0), //fake id that will never be used
@ -1747,6 +1755,7 @@ pub(crate) mod test_helpers {
to_delete: false,
file_size_bytes: 1339,
parquet_metadata: b"md3".to_vec(),
row_count: 0,
};
let parquet_file_count_before = txn.parquet_files().count().await.unwrap();

View File

@ -204,6 +204,7 @@ pub async fn add_parquet_file_with_tombstones(
parquet_file.max_time,
parquet_file.file_size_bytes,
parquet_file.parquet_metadata.clone(),
parquet_file.row_count,
)
.await?;

View File

@ -649,6 +649,7 @@ impl ParquetFileRepo for MemTxn {
max_time: Timestamp,
file_size_bytes: i64,
parquet_metadata: Vec<u8>,
row_count: i64,
) -> Result<ParquetFile> {
let stage = self.stage();
@ -670,6 +671,7 @@ impl ParquetFileRepo for MemTxn {
max_sequence_number,
min_time,
max_time,
row_count,
to_delete: false,
file_size_bytes,
parquet_metadata,

View File

@ -233,7 +233,7 @@ decorate!(
decorate!(
impl_trait = ParquetFileRepo,
methods = [
"parquet_create" = create( &mut self, sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId, object_store_id: Uuid, min_sequence_number: SequenceNumber, max_sequence_number: SequenceNumber, min_time: Timestamp, max_time: Timestamp, file_size_bytes: i64, parquet_metadata: Vec<u8>) -> Result<ParquetFile>;
"parquet_create" = create( &mut self, sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId, object_store_id: Uuid, min_sequence_number: SequenceNumber, max_sequence_number: SequenceNumber, min_time: Timestamp, max_time: Timestamp, file_size_bytes: i64, parquet_metadata: Vec<u8>, row_count: i64) -> Result<ParquetFile>;
"parquet_flag_for_delete" = flag_for_delete(&mut self, id: ParquetFileId) -> Result<()>;
"parquet_list_by_sequencer_greater_than" = list_by_sequencer_greater_than(&mut self, sequencer_id: SequencerId, sequence_number: SequenceNumber) -> Result<Vec<ParquetFile>>;
"parquet_exist" = exist(&mut self, id: ParquetFileId) -> Result<bool>;

View File

@ -962,11 +962,12 @@ impl ParquetFileRepo for PostgresTxn {
max_time: Timestamp,
file_size_bytes: i64,
parquet_metadata: Vec<u8>,
row_count: i64,
) -> Result<ParquetFile> {
let rec = sqlx::query_as::<_, ParquetFile>(
r#"
INSERT INTO parquet_file ( sequencer_id, table_id, partition_id, object_store_id, min_sequence_number, max_sequence_number, min_time, max_time, to_delete, file_size_bytes, parquet_metadata )
VALUES ( $1, $2, $3, $4, $5, $6, $7, $8, false, $9, $10 )
INSERT INTO parquet_file ( sequencer_id, table_id, partition_id, object_store_id, min_sequence_number, max_sequence_number, min_time, max_time, to_delete, file_size_bytes, parquet_metadata, row_count )
VALUES ( $1, $2, $3, $4, $5, $6, $7, $8, false, $9, $10, $11 )
RETURNING *
"#,
)
@ -980,6 +981,7 @@ RETURNING *
.bind(max_time) // $8
.bind(file_size_bytes) // $9
.bind(parquet_metadata) // $10
.bind(row_count) // $11
.fetch_one(&mut self.inner)
.await
.map_err(|e| {

View File

@ -510,6 +510,9 @@ pub struct IoxMetadata {
/// sequence number of the last write
pub max_sequence_number: SequenceNumber,
/// number of rows of data
pub row_count: i64,
}
impl IoxMetadata {
@ -529,6 +532,7 @@ impl IoxMetadata {
time_of_last_write: Some(self.time_of_last_write.date_time().into()),
min_sequence_number: self.min_sequence_number.get(),
max_sequence_number: self.max_sequence_number.get(),
row_count: self.row_count,
};
let mut buf = Vec::new();
@ -584,6 +588,7 @@ impl IoxMetadata {
time_of_last_write,
min_sequence_number: SequenceNumber::new(proto_msg.min_sequence_number),
max_sequence_number: SequenceNumber::new(proto_msg.max_sequence_number),
row_count: proto_msg.row_count,
})
}
@ -615,6 +620,7 @@ impl IoxMetadata {
to_delete: false,
file_size_bytes: file_size_bytes as i64,
parquet_metadata: metadata.thrift_bytes().to_vec(),
row_count: self.row_count,
}
}
}
@ -1223,6 +1229,7 @@ mod tests {
time_of_last_write: Time::from_timestamp(3234, 3456),
min_sequence_number: SequenceNumber::new(5),
max_sequence_number: SequenceNumber::new(6),
row_count: 3,
};
let proto = iox_metadata.to_protobuf().unwrap();