Skip to content

feat: avoid building dictionary for massive unique column values #1365

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 6 commits into from
Dec 18, 2023
Merged
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions analytic_engine/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ datafusion = { workspace = true }
future_ext = { workspace = true }
futures = { workspace = true }
generic_error = { workspace = true }
hash_ext = { workspace = true }
hex = { workspace = true }
hyperloglog = { workspace = true }
id_allocator = { workspace = true }
Expand Down
10 changes: 8 additions & 2 deletions analytic_engine/src/sst/factory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ use snafu::{ResultExt, Snafu};
use table_engine::predicate::PredicateRef;
use trace_metric::MetricsCollector;

use super::parquet::writer::WriteOptions;
use crate::{
sst::{
file::Level,
Expand Down Expand Up @@ -200,11 +201,16 @@ impl Factory for FactoryImpl {
store_picker: &'a ObjectStorePickerRef,
level: Level,
) -> Result<Box<dyn SstWriter + Send + 'a>> {
let write_options = WriteOptions {
num_rows_per_row_group: options.num_rows_per_row_group,
max_buffer_size: options.max_buffer_size,
compression: options.compression.into(),
sst_level: level,
};
Ok(Box::new(ParquetSstWriter::new(
path,
level,
write_options,
store_picker,
options,
)))
}
}
54 changes: 31 additions & 23 deletions analytic_engine/src/sst/parquet/encoding.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use std::convert::TryFrom;
use std::{collections::HashMap, convert::TryFrom};

use arrow::{compute, record_batch::RecordBatch as ArrowRecordBatch};
use async_trait::async_trait;
Expand All @@ -26,6 +26,7 @@ use parquet::{
arrow::AsyncArrowWriter,
basic::Compression,
file::{metadata::KeyValue, properties::WriterProperties},
schema::types::ColumnPath,
};
use prost::{bytes, Message};
use snafu::{ensure, Backtrace, OptionExt, ResultExt, Snafu};
Expand Down Expand Up @@ -237,25 +238,40 @@ struct ColumnarRecordEncoder<W> {
arrow_schema: ArrowSchemaRef,
}

#[derive(Debug, Clone)]
pub struct ColumnEncoding {
pub enable_dict: bool,
}

#[derive(Debug, Clone)]
pub struct EncodeOptions {
pub num_rows_per_row_group: usize,
pub max_buffer_size: usize,
pub compression: Compression,
pub column_encodings: HashMap<String, ColumnEncoding>,
}

impl<W: AsyncWrite + Send + Unpin> ColumnarRecordEncoder<W> {
fn try_new(
sink: W,
schema: &Schema,
num_rows_per_row_group: usize,
max_buffer_size: usize,
compression: Compression,
) -> Result<Self> {
fn try_new(sink: W, schema: &Schema, options: &EncodeOptions) -> Result<Self> {
let arrow_schema = schema.to_arrow_schema_ref();

let write_props = WriterProperties::builder()
.set_max_row_group_size(num_rows_per_row_group)
.set_compression(compression)
.build();
let write_props = {
let mut builder = WriterProperties::builder()
.set_max_row_group_size(options.num_rows_per_row_group)
.set_compression(options.compression);

for (col_name, encoding) in &options.column_encodings {
let col_path = ColumnPath::new(vec![col_name.to_string()]);
builder = builder.set_column_dictionary_enabled(col_path, encoding.enable_dict);
}

builder.build()
};

let arrow_writer = AsyncArrowWriter::try_new(
sink,
arrow_schema.clone(),
max_buffer_size,
options.max_buffer_size,
Some(write_props),
)
.box_err()
Expand Down Expand Up @@ -326,18 +342,10 @@ impl ParquetEncoder {
pub fn try_new<W: AsyncWrite + Unpin + Send + 'static>(
sink: W,
schema: &Schema,
num_rows_per_row_group: usize,
max_buffer_size: usize,
compression: Compression,
options: &EncodeOptions,
) -> Result<Self> {
Ok(ParquetEncoder {
record_encoder: Box::new(ColumnarRecordEncoder::try_new(
sink,
schema,
num_rows_per_row_group,
max_buffer_size,
compression,
)?),
record_encoder: Box::new(ColumnarRecordEncoder::try_new(sink, schema, options)?),
})
}

Expand Down
10 changes: 5 additions & 5 deletions analytic_engine/src/sst/parquet/meta_data.rs
Original file line number Diff line number Diff line change
Expand Up @@ -353,14 +353,14 @@ pub struct ParquetMetaData {

pub type ParquetMetaDataRef = Arc<ParquetMetaData>;

impl From<MetaData> for ParquetMetaData {
fn from(meta: MetaData) -> Self {
impl From<&MetaData> for ParquetMetaData {
fn from(meta: &MetaData) -> Self {
Self {
min_key: meta.min_key,
max_key: meta.max_key,
min_key: meta.min_key.clone(),
max_key: meta.max_key.clone(),
time_range: meta.time_range,
max_sequence: meta.max_sequence,
schema: meta.schema,
schema: meta.schema.clone(),
parquet_filter: None,
column_values: None,
}
Expand Down
Loading