From d898e5743c933c2ee2bbef5f0570fd8d3873c597 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 7 Jan 2026 01:46:49 +0000 Subject: [PATCH 01/25] feat: support attribute insertion in OTAP transform --- .../crates/pdata/src/otap/transform.rs | 493 +++++++++++++++--- 1 file changed, 434 insertions(+), 59 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 5d881150bb..0cf196908e 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -480,6 +480,25 @@ fn create_next_eq_array_for_array(arr: T) -> BooleanArray { eq(&lhs, &rhs).expect("should be able to compare slice with offset of 1") } +#[derive(Debug, Clone, PartialEq)] +pub enum LiteralValue { + Str(String), + Int(i64), + Double(f64), + Bool(bool), +} + +pub struct InsertTransform { + pub(super) entries: Vec<(String, LiteralValue)>, +} + +impl InsertTransform { + #[must_use] + pub fn new(entries: Vec<(String, LiteralValue)>) -> Self { + Self { entries } + } +} + pub struct RenameTransform { pub(super) map: BTreeMap, pub(super) target_bytes: Vec>, @@ -530,6 +549,9 @@ pub struct AttributesTransform { // rows with attribute names in this set will be deleted from the attribute record batch pub delete: Option, + + // rows that will be inserted into the attribute record batch + pub insert: Option, } impl AttributesTransform { @@ -547,6 +569,13 @@ impl AttributesTransform { self } + /// Set the insert transform + #[must_use] + pub fn with_insert(mut self, insert: InsertTransform) -> Self { + self.insert = Some(insert); + self + } + /// Validates the attribute transform operation. The current rule is that no key can be /// duplicated in any of the passed values. This is done to avoid any ambiguity about how /// to apply the transformation. For example, if the following passed @@ -585,6 +614,17 @@ impl AttributesTransform { } } + if let Some(insert) = &self.insert { + let mut insert_keys = BTreeSet::new(); + for (key, _) in &insert.entries { + if !insert_keys.insert(key) { + return Err(Error::InvalidAttributeTransform { + reason: format!("Duplicate key in insert: {key}"), + }); + } + } + } + Ok(()) } } @@ -609,6 +649,8 @@ pub struct TransformStats { pub deleted_entries: u64, /// Exact number of attribute entries whose key was renamed pub renamed_entries: u64, + /// Exact number of attribute entries added by insert rules + pub inserted_entries: u64, } /// Apply an [`AttributesTransform`] to an attributes [`RecordBatch`] and return both the @@ -634,14 +676,27 @@ pub fn transform_attributes_with_stats( transform.validate()?; let schema = attrs_record_batch.schema(); - let key_column_idx = - schema - .index_of(consts::ATTRIBUTE_KEY) - .map_err(|_| Error::ColumnNotFound { - name: consts::ATTRIBUTE_KEY.into(), - })?; - - match schema.field(key_column_idx).data_type() { + let key_column_idx = schema + .index_of(consts::ATTRIBUTE_KEY) + .map_err(|_| Error::ColumnNotFound { + name: consts::ATTRIBUTE_KEY.into(), + })?; + + // Check if we need early materialization for insert + // We only need to materialize if insert is present AND we have parent_id column + // This allows us to get the full list of parents before any deletes are applied + let insert_needed = + transform.insert.is_some() && schema.column_with_name(consts::PARENT_ID).is_some(); + let attrs_record_batch_cow = if insert_needed { + let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; + Cow::Owned(rb) + } else { + Cow::Borrowed(attrs_record_batch) + }; + let attrs_record_batch = attrs_record_batch_cow.as_ref(); + let schema = attrs_record_batch.schema(); + + let (rb, mut stats) = match schema.field(key_column_idx).data_type() { DataType::Utf8 => { let keys_arr = attrs_record_batch .column(key_column_idx) @@ -653,6 +708,7 @@ pub fn transform_attributes_with_stats( let stats = TransformStats { renamed_entries: keys_transform_result.replaced_rows as u64, deleted_entries: keys_transform_result.deleted_rows as u64, + inserted_entries: 0, }; let new_keys = Arc::new(keys_transform_result.new_keys); @@ -660,9 +716,10 @@ pub fn transform_attributes_with_stats( // deletes, it could cause issues if the parent_ids are using the transport optimized // quasi-delta encoding. This is because subsequent runs of key-value pairs may be // joined deleted segments, meaning the delta encoding will change. + // Note: if insert_needed was true, parent IDs are already materialized so this is noop. let any_rows_deleted = keys_transform_result.keep_ranges.is_some(); let should_materialize_parent_ids = - any_rows_deleted && schema.column_with_name(consts::PARENT_ID).is_some(); + (any_rows_deleted || insert_needed) && schema.column_with_name(consts::PARENT_ID).is_some(); let (attrs_record_batch, schema) = if should_materialize_parent_ids { let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; let schema = rb.schema(); @@ -693,7 +750,7 @@ pub fn transform_attributes_with_stats( // safety: this should only return an error if our schema, or column lengths don't match let rb = RecordBatch::try_new(schema, columns) .expect("can build record batch with same schema and columns"); - Ok((rb, stats)) + (rb, stats) } DataType::Dictionary(k, _) => { let (new_dict, keep_ranges, stats) = match *k.clone() { @@ -713,6 +770,7 @@ pub fn transform_attributes_with_stats( TransformStats { deleted_entries: dict_imm_result.deleted_rows as u64, renamed_entries: dict_imm_result.renamed_rows as u64, + inserted_entries: 0, }, ) } @@ -732,6 +790,7 @@ pub fn transform_attributes_with_stats( TransformStats { deleted_entries: dict_imm_result.deleted_rows as u64, renamed_entries: dict_imm_result.renamed_rows as u64, + inserted_entries: 0, }, ) } @@ -744,23 +803,26 @@ pub fn transform_attributes_with_stats( }; // if all rows have been removed (all attributes deleted) just return empty RecordBatch - if keep_ranges.as_ref().map(Vec::len) == Some(0) { - return Ok((RecordBatch::new_empty(schema), stats)); - } - - // Possibly remove any delta-encoding on the parent ID column. If there were any - // deletes, it could cause issues if the parent_ids are using the transport optimized - // quasi-delta encoding. This is because subsequent runs of key-value pairs may be - // joined deleted segments, meaning the delta encoding will change. - let any_rows_deleted = keep_ranges.is_some(); - let should_materialize_parent_ids = - any_rows_deleted && schema.column_with_name(consts::PARENT_ID).is_some(); - let (attrs_record_batch, schema) = if should_materialize_parent_ids { - let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; - let schema = rb.schema(); - (rb, schema) + // NOTE: If we have inserts, we might still return rows even if all existing were deleted. + // But here we just produce the "remaining" batch. Inserts are appended later. + let (attrs_record_batch, schema) = if keep_ranges.as_ref().map(Vec::len) == Some(0) { + (RecordBatch::new_empty(schema.clone()), schema.clone()) } else { - (attrs_record_batch.clone(), schema) + // Possibly remove any delta-encoding on the parent ID column. If there were any + // deletes, it could cause issues if the parent_ids are using the transport optimized + // quasi-delta encoding. This is because subsequent runs of key-value pairs may be + // joined deleted segments, meaning the delta encoding will change. + let any_rows_deleted = keep_ranges.is_some(); + let should_materialize_parent_ids = + (any_rows_deleted || insert_needed) && schema.column_with_name(consts::PARENT_ID).is_some(); + let (attrs_record_batch, schema) = if should_materialize_parent_ids { + let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; + let schema = rb.schema(); + (rb, schema) + } else { + (attrs_record_batch.clone(), schema) + }; + (attrs_record_batch, schema) }; // TODO if there are any optional columns that now contain only null or default values, @@ -785,9 +847,9 @@ pub fn transform_attributes_with_stats( // safety: this should only return an error if our schema, or column lengths don't match let rb = RecordBatch::try_new(schema, columns) .expect("can build record batch with same schema and columns"); - Ok((rb, stats)) + (rb, stats) } - data_type => Err(Error::InvalidListArray { + data_type => return Err(Error::InvalidListArray { expect_oneof: vec![ DataType::Utf8, DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)), @@ -795,7 +857,21 @@ pub fn transform_attributes_with_stats( ], actual: data_type.clone(), }), + }; + + // Handle inserts + if let Some(insert) = &transform.insert { + if let Some(parent_ids) = attrs_record_batch.column_by_name(consts::PARENT_ID) { + let (new_rows, count) = create_inserted_batch(parent_ids, insert, rb.schema().as_ref())?; + if count > 0 { + let combined = arrow::compute::concat_batches(&rb.schema(), &[rb, new_rows]).map_err(|e| Error::Format { error: e.to_string() })?; + stats.inserted_entries = count as u64; + return Ok((combined, stats)); + } + } } + + Ok((rb, stats)) } /// This function is used to perform bulk transformations on OTel attributes. @@ -2367,7 +2443,7 @@ mod test { let test_cases = vec![ ( // most basic transform - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "b".into(), "B".into(), @@ -2381,7 +2457,7 @@ mod test { ), ( // test replacements at array boundaries - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "A".into(), @@ -2393,7 +2469,7 @@ mod test { ), ( // test replacements where replacements longer than target - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "AAA".into(), @@ -2408,7 +2484,7 @@ mod test { ), ( // test replacements where replacements shorter than target - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "aaa".into(), "a".into(), @@ -2423,7 +2499,7 @@ mod test { ), ( // test replacing single contiguous block of keys - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "AA".into(), @@ -2441,7 +2517,7 @@ mod test { ), ( // test multiple replacements - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![ ("a".into(), "AA".into()), ("dd".into(), "D".into()), @@ -2459,7 +2535,7 @@ mod test { ), ( // test multiple replacements interleaved - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![ ("a".into(), "AA".into()), ("dd".into(), "D".into()), @@ -2477,7 +2553,7 @@ mod test { ), ( // test deletion at array boundaries without replaces - AttributesTransform { + AttributesTransform { insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["a".into()]))), }, @@ -2486,7 +2562,7 @@ mod test { ), ( // test delete contiguous segment - AttributesTransform { + AttributesTransform { insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["a".into()]))), }, @@ -2498,7 +2574,7 @@ mod test { ), ( // test multiple deletes - AttributesTransform { + AttributesTransform { insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec![ "a".into(), @@ -2513,7 +2589,7 @@ mod test { ), ( // test adjacent replacement and delete - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "AAA".into(), @@ -2525,7 +2601,7 @@ mod test { ), ( // test we handle an empty rename - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![]))), delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["b".into()]))), }, @@ -2534,7 +2610,7 @@ mod test { ), ( // test we handle an empty delete - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "AAAA".into(), @@ -2706,7 +2782,7 @@ mod test { let result = transform_attributes( &record_batch, - &AttributesTransform { + &AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "k2".into(), "K2".into(), @@ -2863,7 +2939,7 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { + &AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "b".into(), "B".into(), @@ -2899,7 +2975,7 @@ mod test { let test_cases = vec![ ( // basic dict transform - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "a".into(), "AA".into(), @@ -2940,7 +3016,7 @@ mod test { ), ( // test with some nulls - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "a".into(), "AA".into(), @@ -3007,7 +3083,7 @@ mod test { ( // test if there's nulls in the dict keys. This would be unusual // but technically it's possible - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "a".into(), "AA".into(), @@ -3106,7 +3182,7 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { + &AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "c".into(), "CCCCC".into(), @@ -3149,7 +3225,7 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { + &AttributesTransform { insert: None, rename: None, delete: Some(DeleteTransform::new(["a".into()].into_iter().collect())), }, @@ -3184,7 +3260,7 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { + &AttributesTransform { insert: None, rename: None, delete: Some(DeleteTransform::new(["a".into()].into_iter().collect())), }, @@ -3267,7 +3343,7 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { + &AttributesTransform { insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(["b".into()]))), }, @@ -3344,7 +3420,7 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { + &AttributesTransform { insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(["b".into()]))), }, @@ -3405,7 +3481,7 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { + &AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "BBB".into(), @@ -3473,7 +3549,7 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { + &AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "BBB".into(), @@ -3489,28 +3565,28 @@ mod test { #[test] fn test_invalid_attributes_transforms() { let test_cases = vec![ - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "b".into(), )]))), delete: None, }, - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([ ("b".into(), "b".into()), ("a".into(), "b".into()), ]))), delete: None, }, - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "a".into(), )]))), delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["b".into()]))), }, - AttributesTransform { + AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "a".into(), @@ -3550,7 +3626,7 @@ mod test { ) .unwrap(); - let tx = AttributesTransform { + let tx = AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( String::from("a"), String::from("A"), @@ -3592,7 +3668,7 @@ mod test { ) .unwrap(); - let tx = AttributesTransform { + let tx = AttributesTransform { insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( String::from("a"), String::from("A"), @@ -3610,3 +3686,302 @@ mod test { assert_eq!(with_stats, plain); } } + +fn create_inserted_batch( + parent_ids: &ArrayRef, + insert: &InsertTransform, + schema: &arrow::datatypes::Schema, +) -> Result<(RecordBatch, usize)> { + let parent_ids = parent_ids + .as_any() + .downcast_ref::>() + .ok_or_else(|| Error::ColumnDataTypeMismatch { + name: consts::PARENT_ID.into(), + expect: DataType::UInt16, + actual: parent_ids.data_type().clone(), + })?; + + let mut unique_parents = BTreeSet::new(); + for i in 0..parent_ids.len() { + if !parent_ids.is_null(i) { + let _ = unique_parents.insert(parent_ids.value(i)); + } + } + + if unique_parents.is_empty() { + return Ok((RecordBatch::new_empty(Arc::new(schema.clone())), 0)); + } + + let num_parents = unique_parents.len(); + let num_inserts = insert.entries.len(); + let total_rows = num_parents * num_inserts; + + // Build Parent ID column + let mut new_parent_ids = PrimitiveBuilder::::with_capacity(total_rows); + for _parent in &unique_parents { + for _ in 0..num_inserts { + new_parent_ids.append_value(*_parent); + } + } + let new_parent_ids = Arc::new(new_parent_ids.finish()) as ArrayRef; + + // Build Attribute Type column + let mut new_types = PrimitiveBuilder::::with_capacity(total_rows); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + let type_val = match val { + LiteralValue::Str(_) => AttributeValueType::Str, + LiteralValue::Int(_) => AttributeValueType::Int, + LiteralValue::Double(_) => AttributeValueType::Double, + LiteralValue::Bool(_) => AttributeValueType::Bool, + }; + new_types.append_value(type_val as u8); + } + } + let new_types = Arc::new(new_types.finish()) as ArrayRef; + + // Build Key column + let key_col_idx = schema.index_of(consts::ATTRIBUTE_KEY).map_err(|_| Error::ColumnNotFound { name: consts::ATTRIBUTE_KEY.into() })?; + let key_type = schema.field(key_col_idx).data_type(); + + let new_keys: ArrayRef = match key_type { + DataType::Utf8 => { + let mut builder = arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); + for _parent in &unique_parents { + for (key, _) in &insert.entries { + builder.append_value(key); + } + } + Arc::new(builder.finish()) + } + DataType::Dictionary(k, _v) => { + // For simplicity, we can create a StringArray and cast it to Dictionary? + // Or build dictionary properly. building properly is better. + // But keys might be u8 or u16. + // Since we have a small set of keys repeated many times, proper dictionary encoding is good. + // But existing keys are not available here easily (to share dict). + // Merging dictionaries later (concat) handles remapping. + match **k { + DataType::UInt8 => { + // TODO: Optimize by building dictionary + let mut builder = arrow::array::StringDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (key, _) in &insert.entries { + builder.append_value(key); + } + } + Arc::new(builder.finish()) + } + DataType::UInt16 => { + let mut builder = arrow::array::StringDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (key, _) in &insert.entries { + builder.append_value(key); + } + } + Arc::new(builder.finish()) + } + _ => return Err(Error::UnsupportedDictionaryKeyType { expect_oneof: vec![DataType::UInt8, DataType::UInt16], actual: *k.clone() }), + } + } + _ => return Err(Error::InvalidListArray { expect_oneof: vec![DataType::Utf8], actual: key_type.clone() }), + }; + + // Build Value columns + // We need to build columns for STR, INT, DOUBLE, BOOL, BYTES (if present) + // We iterate over inputs and fill. + + // Helper to build column + let _build_col = |name: &str, builder_func: &dyn Fn() -> ArrayRef| -> Result { + if schema.field_with_name(name).is_ok() { + Ok(builder_func()) + } else { + // If column missing in schema, we can't include it. + // But wait, RecordBatch::try_new requires matching columns. + // If schema has the column, we must provide it. + // If schema DOES NOT have the column, but we have values for it? + // Then we would fail to create batch if we try to include it. + // But if we omit it, where does the data go? + // Standard OTAP schema has all columns. + // If some are missing (e.g. projection?), we can't insert that data. + // We'll proceed assuming schema has needed columns or we fill nulls. + Err(Error::ColumnNotFound{ name: name.into() }) + } + }; + + // We collect columns into a map or vec matching schema order. + let mut columns = Vec::with_capacity(schema.fields().len()); + + for field in schema.fields() { + let name = field.name(); + let col: ArrayRef = if name == consts::PARENT_ID { + new_parent_ids.clone() + } else if name == consts::ATTRIBUTE_TYPE { + new_types.clone() + } else if name == consts::ATTRIBUTE_KEY { + new_keys.clone() + } else if name == consts::ATTRIBUTE_STR { + let mut builder = arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Str(s) = val { + builder.append_value(s); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } else if name == consts::ATTRIBUTE_INT { + let mut builder = PrimitiveBuilder::::with_capacity(total_rows); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Int(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } else if name == consts::ATTRIBUTE_DOUBLE { + let mut builder = PrimitiveBuilder::::with_capacity(total_rows); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Double(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } else if name == consts::ATTRIBUTE_BOOL { + let mut builder = arrow::array::BooleanBuilder::with_capacity(total_rows); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Bool(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } else { + // Fill with nulls + arrow::array::new_null_array(field.data_type(), total_rows) + }; + columns.push(col); + } + + Ok((RecordBatch::try_new(Arc::new(schema.clone()), columns).expect("schema check"), total_rows)) +} + +#[cfg(test)] +mod insert_tests { + use super::*; + use std::sync::Arc; + use arrow::datatypes::*; + use arrow::array::*; + use crate::schema::consts; + + #[test] + fn test_insert_attributes_simple() { + // Schema + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0, 1])), + Arc::new(UInt8Array::from_iter_values(vec![AttributeValueType::Str as u8, AttributeValueType::Str as u8])), + Arc::new(StringArray::from_iter_values(vec!["k1", "k2"])), + Arc::new(StringArray::from_iter_values(vec!["v1", "v2"])), + ], + ).unwrap(); + + // Transform: insert "env"="prod" + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "env".into(), + LiteralValue::Str("prod".into()), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).expect("transform failed"); + + assert_eq!(stats.inserted_entries, 2); + + // Expected: 4 rows + assert_eq!(result.num_rows(), 4); + + let keys = result.column_by_name(consts::ATTRIBUTE_KEY).unwrap().as_any().downcast_ref::().unwrap(); + // The original rows come first, then inserted rows. + // Original: k1, k2. Inserted: env, env. + let k0 = keys.value(0); + let k1 = keys.value(1); + let k2 = keys.value(2); + let k3 = keys.value(3); + + assert!(k0 == "k1" || k0 == "k2"); + assert!(k1 == "k1" || k1 == "k2"); + assert_eq!(k2, "env"); + assert_eq!(k3, "env"); + + // Check values + let vals = result.column_by_name(consts::ATTRIBUTE_STR).unwrap().as_any().downcast_ref::().unwrap(); + let v2 = vals.value(2); + let v3 = vals.value(3); + assert_eq!(v2, "prod"); + assert_eq!(v3, "prod"); + } + + #[test] + fn test_insert_attributes_with_delete() { + // Schema + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + // Input: parent 0 has "del_me". + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![AttributeValueType::Str as u8])), + Arc::new(StringArray::from_iter_values(vec!["del_me"])), + Arc::new(StringArray::from_iter_values(vec!["val"])), + ], + ).unwrap(); + + let tx = AttributesTransform { + rename: None, + delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["del_me".into()]))), + insert: Some(InsertTransform::new(vec![( + "new".into(), + LiteralValue::Str("val".into()), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + assert_eq!(stats.deleted_entries, 1); + assert_eq!(stats.inserted_entries, 1); + + // Result should contain 1 row: "new"="val" for parent 0. + assert_eq!(result.num_rows(), 1); + let keys = result.column_by_name(consts::ATTRIBUTE_KEY).unwrap().as_any().downcast_ref::().unwrap(); + assert_eq!(keys.value(0), "new"); + } +} From cd1158cfa9436810ff5954476329539959dea284 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Thu, 8 Jan 2026 00:07:14 +0000 Subject: [PATCH 02/25] Adding missing definition --- rust/otap-dataflow/crates/otap/src/attributes_processor.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index f36e2896fd..6259d53d90 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -182,6 +182,7 @@ impl AttributesProcessor { } else { Some(DeleteTransform::new(deletes)) }, + insert: None, }; transform From fc259acde77c40751abb4ac76cb0aff42423d5d6 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Fri, 9 Jan 2026 09:18:47 +0000 Subject: [PATCH 03/25] Format --- .../benches/attribute_transform/main.rs | 6 + .../crates/pdata/src/otap/transform.rs | 337 +++++++++++------- 2 files changed, 217 insertions(+), 126 deletions(-) diff --git a/rust/otap-dataflow/benchmarks/benches/attribute_transform/main.rs b/rust/otap-dataflow/benchmarks/benches/attribute_transform/main.rs index 790a655954..b389888152 100644 --- a/rust/otap-dataflow/benchmarks/benches/attribute_transform/main.rs +++ b/rust/otap-dataflow/benchmarks/benches/attribute_transform/main.rs @@ -72,6 +72,7 @@ fn generate_dict_keys_attribute_batch( fn bench_transform_attributes(c: &mut Criterion) { // Pre-create AttributesTransform instances to avoid measuring their creation cost let single_replace_no_delete = AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "attr24".into(), "attr_24".into(), @@ -80,6 +81,7 @@ fn bench_transform_attributes(c: &mut Criterion) { }; let single_replace_single_delete = AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "attr24".into(), "attr_24".into(), @@ -88,11 +90,13 @@ fn bench_transform_attributes(c: &mut Criterion) { }; let no_replace_single_delete = AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(["attr15".into()]))), }; let attr3_replace_no_delete = AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "attr3".into(), "attr_3".into(), @@ -101,11 +105,13 @@ fn bench_transform_attributes(c: &mut Criterion) { }; let no_replace_attr9_delete = AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(["attr9".into()]))), }; let attr3_replace_attr9_delete = AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "attr3".into(), "attr_3".into(), diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 0cf196908e..3e4e952e64 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -676,11 +676,12 @@ pub fn transform_attributes_with_stats( transform.validate()?; let schema = attrs_record_batch.schema(); - let key_column_idx = schema - .index_of(consts::ATTRIBUTE_KEY) - .map_err(|_| Error::ColumnNotFound { - name: consts::ATTRIBUTE_KEY.into(), - })?; + let key_column_idx = + schema + .index_of(consts::ATTRIBUTE_KEY) + .map_err(|_| Error::ColumnNotFound { + name: consts::ATTRIBUTE_KEY.into(), + })?; // Check if we need early materialization for insert // We only need to materialize if insert is present AND we have parent_id column @@ -718,8 +719,8 @@ pub fn transform_attributes_with_stats( // joined deleted segments, meaning the delta encoding will change. // Note: if insert_needed was true, parent IDs are already materialized so this is noop. let any_rows_deleted = keys_transform_result.keep_ranges.is_some(); - let should_materialize_parent_ids = - (any_rows_deleted || insert_needed) && schema.column_with_name(consts::PARENT_ID).is_some(); + let should_materialize_parent_ids = (any_rows_deleted || insert_needed) + && schema.column_with_name(consts::PARENT_ID).is_some(); let (attrs_record_batch, schema) = if should_materialize_parent_ids { let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; let schema = rb.schema(); @@ -806,15 +807,15 @@ pub fn transform_attributes_with_stats( // NOTE: If we have inserts, we might still return rows even if all existing were deleted. // But here we just produce the "remaining" batch. Inserts are appended later. let (attrs_record_batch, schema) = if keep_ranges.as_ref().map(Vec::len) == Some(0) { - (RecordBatch::new_empty(schema.clone()), schema.clone()) + (RecordBatch::new_empty(schema.clone()), schema.clone()) } else { // Possibly remove any delta-encoding on the parent ID column. If there were any // deletes, it could cause issues if the parent_ids are using the transport optimized // quasi-delta encoding. This is because subsequent runs of key-value pairs may be // joined deleted segments, meaning the delta encoding will change. let any_rows_deleted = keep_ranges.is_some(); - let should_materialize_parent_ids = - (any_rows_deleted || insert_needed) && schema.column_with_name(consts::PARENT_ID).is_some(); + let should_materialize_parent_ids = (any_rows_deleted || insert_needed) + && schema.column_with_name(consts::PARENT_ID).is_some(); let (attrs_record_batch, schema) = if should_materialize_parent_ids { let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; let schema = rb.schema(); @@ -849,22 +850,28 @@ pub fn transform_attributes_with_stats( .expect("can build record batch with same schema and columns"); (rb, stats) } - data_type => return Err(Error::InvalidListArray { - expect_oneof: vec![ - DataType::Utf8, - DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)), - DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)), - ], - actual: data_type.clone(), - }), + data_type => { + return Err(Error::InvalidListArray { + expect_oneof: vec![ + DataType::Utf8, + DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)), + DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)), + ], + actual: data_type.clone(), + }); + } }; // Handle inserts if let Some(insert) = &transform.insert { if let Some(parent_ids) = attrs_record_batch.column_by_name(consts::PARENT_ID) { - let (new_rows, count) = create_inserted_batch(parent_ids, insert, rb.schema().as_ref())?; + let (new_rows, count) = + create_inserted_batch(parent_ids, insert, rb.schema().as_ref())?; if count > 0 { - let combined = arrow::compute::concat_batches(&rb.schema(), &[rb, new_rows]).map_err(|e| Error::Format { error: e.to_string() })?; + let combined = arrow::compute::concat_batches(&rb.schema(), &[rb, new_rows]) + .map_err(|e| Error::Format { + error: e.to_string(), + })?; stats.inserted_entries = count as u64; return Ok((combined, stats)); } @@ -2443,7 +2450,8 @@ mod test { let test_cases = vec![ ( // most basic transform - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "b".into(), "B".into(), @@ -2457,7 +2465,8 @@ mod test { ), ( // test replacements at array boundaries - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "A".into(), @@ -2469,7 +2478,8 @@ mod test { ), ( // test replacements where replacements longer than target - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "AAA".into(), @@ -2484,7 +2494,8 @@ mod test { ), ( // test replacements where replacements shorter than target - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "aaa".into(), "a".into(), @@ -2499,7 +2510,8 @@ mod test { ), ( // test replacing single contiguous block of keys - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "AA".into(), @@ -2517,7 +2529,8 @@ mod test { ), ( // test multiple replacements - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![ ("a".into(), "AA".into()), ("dd".into(), "D".into()), @@ -2535,7 +2548,8 @@ mod test { ), ( // test multiple replacements interleaved - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![ ("a".into(), "AA".into()), ("dd".into(), "D".into()), @@ -2553,7 +2567,8 @@ mod test { ), ( // test deletion at array boundaries without replaces - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["a".into()]))), }, @@ -2562,7 +2577,8 @@ mod test { ), ( // test delete contiguous segment - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["a".into()]))), }, @@ -2574,7 +2590,8 @@ mod test { ), ( // test multiple deletes - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec![ "a".into(), @@ -2589,7 +2606,8 @@ mod test { ), ( // test adjacent replacement and delete - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "AAA".into(), @@ -2601,7 +2619,8 @@ mod test { ), ( // test we handle an empty rename - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![]))), delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["b".into()]))), }, @@ -2610,7 +2629,8 @@ mod test { ), ( // test we handle an empty delete - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "a".into(), "AAAA".into(), @@ -2782,7 +2802,8 @@ mod test { let result = transform_attributes( &record_batch, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "k2".into(), "K2".into(), @@ -2939,7 +2960,8 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter(vec![( "b".into(), "B".into(), @@ -2975,7 +2997,8 @@ mod test { let test_cases = vec![ ( // basic dict transform - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "a".into(), "AA".into(), @@ -3016,7 +3039,8 @@ mod test { ), ( // test with some nulls - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "a".into(), "AA".into(), @@ -3083,7 +3107,8 @@ mod test { ( // test if there's nulls in the dict keys. This would be unusual // but technically it's possible - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "a".into(), "AA".into(), @@ -3182,7 +3207,8 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "c".into(), "CCCCC".into(), @@ -3225,7 +3251,8 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(["a".into()].into_iter().collect())), }, @@ -3260,7 +3287,8 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(["a".into()].into_iter().collect())), }, @@ -3343,7 +3371,8 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(["b".into()]))), }, @@ -3420,7 +3449,8 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: None, delete: Some(DeleteTransform::new(BTreeSet::from_iter(["b".into()]))), }, @@ -3481,7 +3511,8 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "BBB".into(), @@ -3549,7 +3580,8 @@ mod test { let result = transform_attributes( &input, - &AttributesTransform { insert: None, + &AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "BBB".into(), @@ -3565,28 +3597,32 @@ mod test { #[test] fn test_invalid_attributes_transforms() { let test_cases = vec![ - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "b".into(), )]))), delete: None, }, - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([ ("b".into(), "b".into()), ("a".into(), "b".into()), ]))), delete: None, }, - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "a".into(), )]))), delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["b".into()]))), }, - AttributesTransform { insert: None, + AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( "b".into(), "a".into(), @@ -3626,7 +3662,8 @@ mod test { ) .unwrap(); - let tx = AttributesTransform { insert: None, + let tx = AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( String::from("a"), String::from("A"), @@ -3668,7 +3705,8 @@ mod test { ) .unwrap(); - let tx = AttributesTransform { insert: None, + let tx = AttributesTransform { + insert: None, rename: Some(RenameTransform::new(BTreeMap::from_iter([( String::from("a"), String::from("A"), @@ -3741,12 +3779,18 @@ fn create_inserted_batch( let new_types = Arc::new(new_types.finish()) as ArrayRef; // Build Key column - let key_col_idx = schema.index_of(consts::ATTRIBUTE_KEY).map_err(|_| Error::ColumnNotFound { name: consts::ATTRIBUTE_KEY.into() })?; + let key_col_idx = + schema + .index_of(consts::ATTRIBUTE_KEY) + .map_err(|_| Error::ColumnNotFound { + name: consts::ATTRIBUTE_KEY.into(), + })?; let key_type = schema.field(key_col_idx).data_type(); - + let new_keys: ArrayRef = match key_type { DataType::Utf8 => { - let mut builder = arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); + let mut builder = + arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); for _parent in &unique_parents { for (key, _) in &insert.entries { builder.append_value(key); @@ -3755,63 +3799,73 @@ fn create_inserted_batch( Arc::new(builder.finish()) } DataType::Dictionary(k, _v) => { - // For simplicity, we can create a StringArray and cast it to Dictionary? - // Or build dictionary properly. building properly is better. - // But keys might be u8 or u16. - // Since we have a small set of keys repeated many times, proper dictionary encoding is good. - // But existing keys are not available here easily (to share dict). - // Merging dictionaries later (concat) handles remapping. - match **k { - DataType::UInt8 => { - // TODO: Optimize by building dictionary - let mut builder = arrow::array::StringDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (key, _) in &insert.entries { - builder.append_value(key); - } - } - Arc::new(builder.finish()) - } - DataType::UInt16 => { - let mut builder = arrow::array::StringDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (key, _) in &insert.entries { - builder.append_value(key); - } - } - Arc::new(builder.finish()) - } - _ => return Err(Error::UnsupportedDictionaryKeyType { expect_oneof: vec![DataType::UInt8, DataType::UInt16], actual: *k.clone() }), - } + // For simplicity, we can create a StringArray and cast it to Dictionary? + // Or build dictionary properly. building properly is better. + // But keys might be u8 or u16. + // Since we have a small set of keys repeated many times, proper dictionary encoding is good. + // But existing keys are not available here easily (to share dict). + // Merging dictionaries later (concat) handles remapping. + match **k { + DataType::UInt8 => { + // TODO: Optimize by building dictionary + let mut builder = arrow::array::StringDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (key, _) in &insert.entries { + builder.append_value(key); + } + } + Arc::new(builder.finish()) + } + DataType::UInt16 => { + let mut builder = arrow::array::StringDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (key, _) in &insert.entries { + builder.append_value(key); + } + } + Arc::new(builder.finish()) + } + _ => { + return Err(Error::UnsupportedDictionaryKeyType { + expect_oneof: vec![DataType::UInt8, DataType::UInt16], + actual: *k.clone(), + }); + } + } + } + _ => { + return Err(Error::InvalidListArray { + expect_oneof: vec![DataType::Utf8], + actual: key_type.clone(), + }); } - _ => return Err(Error::InvalidListArray { expect_oneof: vec![DataType::Utf8], actual: key_type.clone() }), }; // Build Value columns // We need to build columns for STR, INT, DOUBLE, BOOL, BYTES (if present) // We iterate over inputs and fill. - + // Helper to build column let _build_col = |name: &str, builder_func: &dyn Fn() -> ArrayRef| -> Result { if schema.field_with_name(name).is_ok() { - Ok(builder_func()) + Ok(builder_func()) } else { - // If column missing in schema, we can't include it. - // But wait, RecordBatch::try_new requires matching columns. - // If schema has the column, we must provide it. - // If schema DOES NOT have the column, but we have values for it? - // Then we would fail to create batch if we try to include it. - // But if we omit it, where does the data go? - // Standard OTAP schema has all columns. - // If some are missing (e.g. projection?), we can't insert that data. - // We'll proceed assuming schema has needed columns or we fill nulls. - Err(Error::ColumnNotFound{ name: name.into() }) + // If column missing in schema, we can't include it. + // But wait, RecordBatch::try_new requires matching columns. + // If schema has the column, we must provide it. + // If schema DOES NOT have the column, but we have values for it? + // Then we would fail to create batch if we try to include it. + // But if we omit it, where does the data go? + // Standard OTAP schema has all columns. + // If some are missing (e.g. projection?), we can't insert that data. + // We'll proceed assuming schema has needed columns or we fill nulls. + Err(Error::ColumnNotFound { name: name.into() }) } }; // We collect columns into a map or vec matching schema order. let mut columns = Vec::with_capacity(schema.fields().len()); - + for field in schema.fields() { let name = field.name(); let col: ArrayRef = if name == consts::PARENT_ID { @@ -3821,7 +3875,8 @@ fn create_inserted_batch( } else if name == consts::ATTRIBUTE_KEY { new_keys.clone() } else if name == consts::ATTRIBUTE_STR { - let mut builder = arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); + let mut builder = + arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); for _parent in &unique_parents { for (_, val) in &insert.entries { if let LiteralValue::Str(s) = val { @@ -3833,8 +3888,9 @@ fn create_inserted_batch( } Arc::new(builder.finish()) } else if name == consts::ATTRIBUTE_INT { - let mut builder = PrimitiveBuilder::::with_capacity(total_rows); - for _parent in &unique_parents { + let mut builder = + PrimitiveBuilder::::with_capacity(total_rows); + for _parent in &unique_parents { for (_, val) in &insert.entries { if let LiteralValue::Int(v) = val { builder.append_value(*v); @@ -3845,8 +3901,9 @@ fn create_inserted_batch( } Arc::new(builder.finish()) } else if name == consts::ATTRIBUTE_DOUBLE { - let mut builder = PrimitiveBuilder::::with_capacity(total_rows); - for _parent in &unique_parents { + let mut builder = + PrimitiveBuilder::::with_capacity(total_rows); + for _parent in &unique_parents { for (_, val) in &insert.entries { if let LiteralValue::Double(v) = val { builder.append_value(*v); @@ -3857,8 +3914,8 @@ fn create_inserted_batch( } Arc::new(builder.finish()) } else if name == consts::ATTRIBUTE_BOOL { - let mut builder = arrow::array::BooleanBuilder::with_capacity(total_rows); - for _parent in &unique_parents { + let mut builder = arrow::array::BooleanBuilder::with_capacity(total_rows); + for _parent in &unique_parents { for (_, val) in &insert.entries { if let LiteralValue::Bool(v) = val { builder.append_value(*v); @@ -3869,22 +3926,25 @@ fn create_inserted_batch( } Arc::new(builder.finish()) } else { - // Fill with nulls - arrow::array::new_null_array(field.data_type(), total_rows) + // Fill with nulls + arrow::array::new_null_array(field.data_type(), total_rows) }; columns.push(col); } - Ok((RecordBatch::try_new(Arc::new(schema.clone()), columns).expect("schema check"), total_rows)) + Ok(( + RecordBatch::try_new(Arc::new(schema.clone()), columns).expect("schema check"), + total_rows, + )) } #[cfg(test)] mod insert_tests { use super::*; - use std::sync::Arc; - use arrow::datatypes::*; - use arrow::array::*; use crate::schema::consts; + use arrow::array::*; + use arrow::datatypes::*; + use std::sync::Arc; #[test] fn test_insert_attributes_simple() { @@ -3899,12 +3959,16 @@ mod insert_tests { let input = RecordBatch::try_new( schema.clone(), vec![ - Arc::new(UInt16Array::from_iter_values(vec![0, 1])), - Arc::new(UInt8Array::from_iter_values(vec![AttributeValueType::Str as u8, AttributeValueType::Str as u8])), + Arc::new(UInt16Array::from_iter_values(vec![0, 1])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + AttributeValueType::Str as u8, + ])), Arc::new(StringArray::from_iter_values(vec!["k1", "k2"])), Arc::new(StringArray::from_iter_values(vec!["v1", "v2"])), ], - ).unwrap(); + ) + .unwrap(); // Transform: insert "env"="prod" let tx = AttributesTransform { @@ -3916,28 +3980,39 @@ mod insert_tests { )])), }; - let (result, stats) = transform_attributes_with_stats(&input, &tx).expect("transform failed"); + let (result, stats) = + transform_attributes_with_stats(&input, &tx).expect("transform failed"); assert_eq!(stats.inserted_entries, 2); - + // Expected: 4 rows assert_eq!(result.num_rows(), 4); - - let keys = result.column_by_name(consts::ATTRIBUTE_KEY).unwrap().as_any().downcast_ref::().unwrap(); + + let keys = result + .column_by_name(consts::ATTRIBUTE_KEY) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); // The original rows come first, then inserted rows. // Original: k1, k2. Inserted: env, env. let k0 = keys.value(0); let k1 = keys.value(1); let k2 = keys.value(2); let k3 = keys.value(3); - + assert!(k0 == "k1" || k0 == "k2"); assert!(k1 == "k1" || k1 == "k2"); assert_eq!(k2, "env"); assert_eq!(k3, "env"); // Check values - let vals = result.column_by_name(consts::ATTRIBUTE_STR).unwrap().as_any().downcast_ref::().unwrap(); + let vals = result + .column_by_name(consts::ATTRIBUTE_STR) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); let v2 = vals.value(2); let v3 = vals.value(3); assert_eq!(v2, "prod"); @@ -3946,7 +4021,7 @@ mod insert_tests { #[test] fn test_insert_attributes_with_delete() { - // Schema + // Schema let schema = Arc::new(Schema::new(vec![ Field::new(consts::PARENT_ID, DataType::UInt16, false), Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), @@ -3959,29 +4034,39 @@ mod insert_tests { schema.clone(), vec![ Arc::new(UInt16Array::from_iter_values(vec![0])), - Arc::new(UInt8Array::from_iter_values(vec![AttributeValueType::Str as u8])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), Arc::new(StringArray::from_iter_values(vec!["del_me"])), Arc::new(StringArray::from_iter_values(vec!["val"])), ], - ).unwrap(); - + ) + .unwrap(); + let tx = AttributesTransform { rename: None, - delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec!["del_me".into()]))), + delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec![ + "del_me".into(), + ]))), insert: Some(InsertTransform::new(vec![( "new".into(), LiteralValue::Str("val".into()), )])), }; - + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); assert_eq!(stats.deleted_entries, 1); assert_eq!(stats.inserted_entries, 1); - + // Result should contain 1 row: "new"="val" for parent 0. assert_eq!(result.num_rows(), 1); - let keys = result.column_by_name(consts::ATTRIBUTE_KEY).unwrap().as_any().downcast_ref::().unwrap(); + let keys = result + .column_by_name(consts::ATTRIBUTE_KEY) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); assert_eq!(keys.value(0), "new"); } } From ad9e92e573868271b5c76c2832c41bd828c834c0 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Fri, 9 Jan 2026 19:03:14 +0000 Subject: [PATCH 04/25] Check empty slice --- rust/otap-dataflow/crates/pdata/src/otap/transform.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 3e4e952e64..7258fdd540 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -1708,6 +1708,9 @@ fn take_ranges_slice(array: T, ranges: &[(usize, usize)]) -> Result where T: Array, { + if ranges.is_empty() { + return Ok(array.slice(0, 0)); + } let slices: Vec = ranges .iter() .map(|&(start, end)| array.slice(start, end - start)) From e79db3f47874651bb74865e360c4dea0527a2f14 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 14 Jan 2026 01:32:18 +0000 Subject: [PATCH 05/25] Add Insert action to AttributeProcessor --- .../crates/otap/src/attributes_processor.rs | 132 +++++++++- .../crates/pdata/src/otap/transform.rs | 235 +++++++++++++----- 2 files changed, 299 insertions(+), 68 deletions(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 925521ac5f..355d7d853f 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -10,9 +10,10 @@ //! Supported actions (current subset): //! - `rename`: Renames an attribute key (non-standard deviation from the Go collector). //! - `delete`: Removes an attribute by key. +//! - `insert`: Inserts a new attribute. //! //! Unsupported actions are ignored if present in the config: -//! `insert`, `upsert`, `update` (value update), `hash`, `extract`, `convert`. +//! `upsert`, `update` (value update), `hash`, `extract`, `convert`. //! We may add support for them later. //! //! Example configuration (YAML): @@ -47,7 +48,8 @@ use otap_df_engine::processor::ProcessorWrapper; use otap_df_pdata::otap::{ OtapArrowRecords, transform::{ - AttributesTransform, DeleteTransform, RenameTransform, transform_attributes_with_stats, + AttributesTransform, DeleteTransform, InsertTransform, LiteralValue, RenameTransform, + transform_attributes_with_stats, }, }; use otap_df_pdata::proto::opentelemetry::arrow::v1::ArrowPayloadType; @@ -81,6 +83,14 @@ pub enum Action { key: String, }, + /// Insert a new attribute. + Insert { + /// The attribute key to insert. + key: String, + /// The value to insert. + value: Value, + }, + /// Other actions are accepted for forward-compatibility but ignored. /// These variants allow deserialization of Go-style configs without effect. #[serde(other)] @@ -91,7 +101,7 @@ pub enum Action { /// Configuration for the AttributesProcessor. /// /// Accepts configuration in the same format as the OpenTelemetry Collector's attributes processor. -/// Supported actions: rename (deviation), delete. Others are ignored. +/// Supported actions: rename (deviation), delete, insert. Others are ignored. /// /// You can control which attribute domains are transformed via `apply_to`. /// Valid values: "signal" (default), "resource", "scope". @@ -140,6 +150,7 @@ impl AttributesProcessor { fn new(config: Config) -> Result { let mut renames = BTreeMap::new(); let mut deletes = BTreeSet::new(); + let mut inserts = Vec::new(); for action in config.actions { match action { @@ -152,6 +163,27 @@ impl AttributesProcessor { } => { let _ = renames.insert(source_key, destination_key); } + Action::Insert { key, value } => { + let lit_val = match value { + Value::String(s) => Some(LiteralValue::Str(s)), + Value::Number(n) => { + if let Some(i) = n.as_i64() { + Some(LiteralValue::Int(i)) + } else if let Some(f) = n.as_f64() { + Some(LiteralValue::Double(f)) + } else { + None + } + } + Value::Bool(b) => Some(LiteralValue::Bool(b)), + // Ignore null, object, array as not supported by LiteralValue + _ => None, + }; + + if let Some(val) = lit_val { + inserts.push((key, val)); + } + } // Unsupported actions are ignored for now Action::Unsupported => {} } @@ -182,7 +214,11 @@ impl AttributesProcessor { } else { Some(DeleteTransform::new(deletes)) }, - insert: None, + insert: if inserts.is_empty() { + None + } else { + Some(InsertTransform::new(inserts)) + }, }; transform @@ -202,7 +238,7 @@ impl AttributesProcessor { #[inline] const fn is_noop(&self) -> bool { - self.transform.rename.is_none() && self.transform.delete.is_none() + self.transform.rename.is_none() && self.transform.delete.is_none() && self.transform.insert.is_none() } #[inline] @@ -747,6 +783,92 @@ mod tests { .validate(|_| async move {}); } + #[test] + fn test_insert_scoped_to_resource_only_logs() { + // Resource has 'a', scope has 'a', log has 'a' and another key to keep batch non-empty + let input = build_logs_with_attrs( + vec![ + KeyValue::new("a", AnyValue::new_string("rv")), + KeyValue::new("r", AnyValue::new_string("keep")), + ], + vec![KeyValue::new("a", AnyValue::new_string("sv"))], + vec![ + KeyValue::new("a", AnyValue::new_string("lv")), + KeyValue::new("b", AnyValue::new_string("keep")), + ], + ); + + let cfg = json!({ + "actions": [ + {"action": "insert", "key": "c", "value": "val"}, + ], + "apply_to": ["resource"] + }); + + // Create a proper pipeline context for the test + let metrics_registry_handle = MetricsRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = + controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let node = test_node("attributes-processor-insert-resource"); + let rt: TestRuntime = TestRuntime::new(); + let mut node_config = NodeUserConfig::new_processor_config(ATTRIBUTES_PROCESSOR_URN); + node_config.config = cfg; + let proc = + create_attributes_processor(pipeline_ctx, node, Arc::new(node_config), rt.config()) + .expect("create processor"); + let phase = rt.set_processor(proc); + + phase + .run_test(|mut ctx| async move { + let mut bytes = BytesMut::new(); + input.encode(&mut bytes).expect("encode"); + let bytes = bytes.freeze(); + let pdata_in = + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + ctx.process(Message::PData(pdata_in)) + .await + .expect("process"); + + let out = ctx.drain_pdata().await; + let first = out.into_iter().next().expect("one output").payload(); + + let otlp_bytes: OtlpProtoBytes = first.try_into().expect("convert to otlp"); + let bytes = match otlp_bytes { + OtlpProtoBytes::ExportLogsRequest(b) => b, + _ => panic!("unexpected otlp variant"), + }; + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); + + let res_attrs = &decoded.resource_logs[0] + .resource + .as_ref() + .unwrap() + .attributes; + + println!("Resource attrs: {:?}", res_attrs); + + assert!(res_attrs.iter().any(|kv| kv.key == "c")); + assert!(res_attrs.iter().any(|kv| kv.key == "r")); + + // Scope 'c' should remain + let scope_attrs = &decoded.resource_logs[0].scope_logs[0] + .scope + .as_ref() + .unwrap() + .attributes; + assert!(!scope_attrs.iter().any(|kv| kv.key == "c")); + assert!(scope_attrs.iter().any(|kv| kv.key == "a")); + + // Log 'c' should be deleted; 'b' should remain + let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; + assert!(!log_attrs.iter().any(|kv| kv.key == "c")); + assert!(log_attrs.iter().any(|kv| kv.key == "b")); + }) + .validate(|_| async move {}); + } + #[test] fn test_delete_scoped_to_resource_only_logs() { // Resource has 'a', scope has 'a', log has 'a' and another key to keep batch non-empty diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 7258fdd540..6754fa6c60 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -8,12 +8,16 @@ use std::sync::Arc; use arrow::array::{ Array, ArrayRef, ArrowPrimitiveType, BooleanArray, DictionaryArray, NullBufferBuilder, - PrimitiveArray, PrimitiveBuilder, RecordBatch, StringArray, UInt32Array, + PrimitiveArray, PrimitiveBuilder, PrimitiveDictionaryBuilder, RecordBatch, StringArray, + StringDictionaryBuilder, UInt32Array, }; use arrow::buffer::{Buffer, MutableBuffer, NullBuffer, OffsetBuffer, ScalarBuffer}; use arrow::compute::kernels::cmp::eq; use arrow::compute::{SortColumn, and, concat}; -use arrow::datatypes::{ArrowDictionaryKeyType, ArrowNativeType, DataType, UInt8Type, UInt16Type}; +use arrow::datatypes::{ + ArrowDictionaryKeyType, ArrowNativeType, DataType, Float64Type, Int64Type, UInt16Type, + UInt8Type, +}; use arrow::row::{RowConverter, SortField}; use crate::arrays::{ @@ -889,9 +893,7 @@ pub fn transform_attributes_with_stats( /// Currently the operations supported are: /// - rename which replaces a given attribute key /// - delete which removes all rows from the record batch for a given key -/// -/// Support for insert will be added in the future (see -/// https://github.com/open-telemetry/otel-arrow/issues/813) +/// - insert which adds new attributes to the record batch /// /// Note that to avoid any ambiguity in how the transformation is applied, this method will /// validate the transform. The caller must ensure the supplied transform is valid. See @@ -3802,16 +3804,9 @@ fn create_inserted_batch( Arc::new(builder.finish()) } DataType::Dictionary(k, _v) => { - // For simplicity, we can create a StringArray and cast it to Dictionary? - // Or build dictionary properly. building properly is better. - // But keys might be u8 or u16. - // Since we have a small set of keys repeated many times, proper dictionary encoding is good. - // But existing keys are not available here easily (to share dict). - // Merging dictionaries later (concat) handles remapping. match **k { DataType::UInt8 => { - // TODO: Optimize by building dictionary - let mut builder = arrow::array::StringDictionaryBuilder::::new(); + let mut builder = StringDictionaryBuilder::::new(); for _parent in &unique_parents { for (key, _) in &insert.entries { builder.append_value(key); @@ -3820,7 +3815,7 @@ fn create_inserted_batch( Arc::new(builder.finish()) } DataType::UInt16 => { - let mut builder = arrow::array::StringDictionaryBuilder::::new(); + let mut builder = StringDictionaryBuilder::::new(); for _parent in &unique_parents { for (key, _) in &insert.entries { builder.append_value(key); @@ -3844,28 +3839,6 @@ fn create_inserted_batch( } }; - // Build Value columns - // We need to build columns for STR, INT, DOUBLE, BOOL, BYTES (if present) - // We iterate over inputs and fill. - - // Helper to build column - let _build_col = |name: &str, builder_func: &dyn Fn() -> ArrayRef| -> Result { - if schema.field_with_name(name).is_ok() { - Ok(builder_func()) - } else { - // If column missing in schema, we can't include it. - // But wait, RecordBatch::try_new requires matching columns. - // If schema has the column, we must provide it. - // If schema DOES NOT have the column, but we have values for it? - // Then we would fail to create batch if we try to include it. - // But if we omit it, where does the data go? - // Standard OTAP schema has all columns. - // If some are missing (e.g. projection?), we can't insert that data. - // We'll proceed assuming schema has needed columns or we fill nulls. - Err(Error::ColumnNotFound { name: name.into() }) - } - }; - // We collect columns into a map or vec matching schema order. let mut columns = Vec::with_capacity(schema.fields().len()); @@ -3878,45 +3851,181 @@ fn create_inserted_batch( } else if name == consts::ATTRIBUTE_KEY { new_keys.clone() } else if name == consts::ATTRIBUTE_STR { - let mut builder = - arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Str(s) = val { - builder.append_value(s); - } else { - builder.append_null(); + match field.data_type() { + DataType::Utf8 => { + let mut builder = + arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Str(s) = val { + builder.append_value(s); + } else { + builder.append_null(); + } + } } + Arc::new(builder.finish()) + } + DataType::Dictionary(k, v) if **v == DataType::Utf8 => match **k { + DataType::UInt8 => { + let mut builder = StringDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Str(s) = val { + builder.append_value(s); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } + DataType::UInt16 => { + let mut builder = StringDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Str(s) = val { + builder.append_value(s); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } + _ => { + return Err(Error::UnsupportedDictionaryKeyType { + expect_oneof: vec![DataType::UInt8, DataType::UInt16], + actual: *k.clone(), + }); + } + }, + dt => { + return Err(Error::ColumnDataTypeMismatch { + name: name.into(), + expect: DataType::Utf8, + actual: dt.clone(), + }); } } - Arc::new(builder.finish()) } else if name == consts::ATTRIBUTE_INT { - let mut builder = - PrimitiveBuilder::::with_capacity(total_rows); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Int(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); + match field.data_type() { + DataType::Int64 => { + let mut builder = PrimitiveBuilder::::with_capacity(total_rows); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Int(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } + DataType::Dictionary(k, v) if **v == DataType::Int64 => match **k { + DataType::UInt8 => { + let mut builder = PrimitiveDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Int(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } + DataType::UInt16 => { + let mut builder = PrimitiveDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Int(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } + _ => { + return Err(Error::UnsupportedDictionaryKeyType { + expect_oneof: vec![DataType::UInt8], + actual: *k.clone(), + }); } + }, + dt => { + return Err(Error::ColumnDataTypeMismatch { + name: name.into(), + expect: DataType::Int64, + actual: dt.clone(), + }); } } - Arc::new(builder.finish()) } else if name == consts::ATTRIBUTE_DOUBLE { - let mut builder = - PrimitiveBuilder::::with_capacity(total_rows); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Double(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); + match field.data_type() { + DataType::Float64 => { + let mut builder = PrimitiveBuilder::::with_capacity(total_rows); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Double(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } } + Arc::new(builder.finish()) + } + DataType::Dictionary(k, v) if **v == DataType::Float64 => match **k { + DataType::UInt8 => { + let mut builder = + PrimitiveDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Double(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } + DataType::UInt16 => { + let mut builder = + PrimitiveDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (_, val) in &insert.entries { + if let LiteralValue::Double(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); + } + } + } + Arc::new(builder.finish()) + } + _ => { + return Err(Error::UnsupportedDictionaryKeyType { + expect_oneof: vec![DataType::UInt8], + actual: *k.clone(), + }); + } + }, + dt => { + return Err(Error::ColumnDataTypeMismatch { + name: name.into(), + expect: DataType::Float64, + actual: dt.clone(), + }); } } - Arc::new(builder.finish()) } else if name == consts::ATTRIBUTE_BOOL { + // Note: Boolean Dictionaries are not standard/supported by simple builders let mut builder = arrow::array::BooleanBuilder::with_capacity(total_rows); for _parent in &unique_parents { for (_, val) in &insert.entries { From 23978001fa38642aa01380bbc13ffdb3968e8495 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 14 Jan 2026 01:42:06 +0000 Subject: [PATCH 06/25] Format --- .../crates/otap/src/attributes_processor.rs | 6 +- .../crates/pdata/src/otap/transform.rs | 55 +++++++++---------- 2 files changed, 31 insertions(+), 30 deletions(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 355d7d853f..8bb4dabade 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -238,7 +238,9 @@ impl AttributesProcessor { #[inline] const fn is_noop(&self) -> bool { - self.transform.rename.is_none() && self.transform.delete.is_none() && self.transform.insert.is_none() + self.transform.rename.is_none() + && self.transform.delete.is_none() + && self.transform.insert.is_none() } #[inline] @@ -848,7 +850,7 @@ mod tests { .attributes; println!("Resource attrs: {:?}", res_attrs); - + assert!(res_attrs.iter().any(|kv| kv.key == "c")); assert!(res_attrs.iter().any(|kv| kv.key == "r")); diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 6754fa6c60..b4a5cddb91 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -15,8 +15,8 @@ use arrow::buffer::{Buffer, MutableBuffer, NullBuffer, OffsetBuffer, ScalarBuffe use arrow::compute::kernels::cmp::eq; use arrow::compute::{SortColumn, and, concat}; use arrow::datatypes::{ - ArrowDictionaryKeyType, ArrowNativeType, DataType, Float64Type, Int64Type, UInt16Type, - UInt8Type, + ArrowDictionaryKeyType, ArrowNativeType, DataType, Float64Type, Int64Type, UInt8Type, + UInt16Type, }; use arrow::row::{RowConverter, SortField}; @@ -3803,34 +3803,32 @@ fn create_inserted_batch( } Arc::new(builder.finish()) } - DataType::Dictionary(k, _v) => { - match **k { - DataType::UInt8 => { - let mut builder = StringDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (key, _) in &insert.entries { - builder.append_value(key); - } + DataType::Dictionary(k, _v) => match **k { + DataType::UInt8 => { + let mut builder = StringDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (key, _) in &insert.entries { + builder.append_value(key); } - Arc::new(builder.finish()) } - DataType::UInt16 => { - let mut builder = StringDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (key, _) in &insert.entries { - builder.append_value(key); - } + Arc::new(builder.finish()) + } + DataType::UInt16 => { + let mut builder = StringDictionaryBuilder::::new(); + for _parent in &unique_parents { + for (key, _) in &insert.entries { + builder.append_value(key); } - Arc::new(builder.finish()) - } - _ => { - return Err(Error::UnsupportedDictionaryKeyType { - expect_oneof: vec![DataType::UInt8, DataType::UInt16], - actual: *k.clone(), - }); } + Arc::new(builder.finish()) } - } + _ => { + return Err(Error::UnsupportedDictionaryKeyType { + expect_oneof: vec![DataType::UInt8, DataType::UInt16], + actual: *k.clone(), + }); + } + }, _ => { return Err(Error::InvalidListArray { expect_oneof: vec![DataType::Utf8], @@ -3937,8 +3935,9 @@ fn create_inserted_batch( } Arc::new(builder.finish()) } - DataType::UInt16 => { - let mut builder = PrimitiveDictionaryBuilder::::new(); + DataType::UInt16 => { + let mut builder = + PrimitiveDictionaryBuilder::::new(); for _parent in &unique_parents { for (_, val) in &insert.entries { if let LiteralValue::Int(v) = val { @@ -4025,7 +4024,7 @@ fn create_inserted_batch( } } } else if name == consts::ATTRIBUTE_BOOL { - // Note: Boolean Dictionaries are not standard/supported by simple builders + // Note: Boolean Dictionaries are not standard/supported by simple builders let mut builder = arrow::array::BooleanBuilder::with_capacity(total_rows); for _parent in &unique_parents { for (_, val) in &insert.entries { From e8beb1cae04f578053e1e39cf59bb3b06719823b Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 14 Jan 2026 02:22:28 +0000 Subject: [PATCH 07/25] Fix MetricsRegistryHandle --- rust/otap-dataflow/crates/otap/src/attributes_processor.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 8bb4dabade..999f352ec6 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -169,10 +169,8 @@ impl AttributesProcessor { Value::Number(n) => { if let Some(i) = n.as_i64() { Some(LiteralValue::Int(i)) - } else if let Some(f) = n.as_f64() { - Some(LiteralValue::Double(f)) } else { - None + n.as_f64().map(LiteralValue::Double) } } Value::Bool(b) => Some(LiteralValue::Bool(b)), @@ -808,7 +806,7 @@ mod tests { }); // Create a proper pipeline context for the test - let metrics_registry_handle = MetricsRegistryHandle::new(); + let metrics_registry_handle = TelemetryRegistryHandle::new(); let controller_ctx = ControllerContext::new(metrics_registry_handle); let pipeline_ctx = controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); From 1d6bbd4adade532df0ddc5d1008853d15a705ea6 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Fri, 16 Jan 2026 00:10:30 +0000 Subject: [PATCH 08/25] Make sure insert doesn't overwrite existing key --- .../crates/pdata/src/otap/transform.rs | 372 +++++++++++++----- 1 file changed, 267 insertions(+), 105 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index b4a5cddb91..fc7f664640 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -867,10 +867,14 @@ pub fn transform_attributes_with_stats( }; // Handle inserts + // According to OTel collector spec, `insert` only inserts if the key does not already exist. + // We need: + // - Original parent IDs from attrs_record_batch to know which parents exist + // - The transformed batch (rb) to check which (parent_id, key) pairs already exist if let Some(insert) = &transform.insert { - if let Some(parent_ids) = attrs_record_batch.column_by_name(consts::PARENT_ID) { + if let Some(original_parent_ids) = attrs_record_batch.column_by_name(consts::PARENT_ID) { let (new_rows, count) = - create_inserted_batch(parent_ids, insert, rb.schema().as_ref())?; + create_inserted_batch(&rb, original_parent_ids, insert, rb.schema().as_ref())?; if count > 0 { let combined = arrow::compute::concat_batches(&rb.schema(), &[rb, new_rows]) .map_err(|e| Error::Format { @@ -3730,12 +3734,62 @@ mod test { } } +/// Helper to extract string key from a record batch key column at a given index. +/// Handles both plain Utf8 and Dictionary-encoded columns. +fn get_key_at_index(key_col: &ArrayRef, idx: usize) -> Option { + match key_col.data_type() { + DataType::Utf8 => key_col + .as_any() + .downcast_ref::() + .and_then(|arr| { + if arr.is_null(idx) { + None + } else { + Some(arr.value(idx).to_string()) + } + }), + DataType::Dictionary(k, _) => match **k { + DataType::UInt8 => key_col + .as_any() + .downcast_ref::>() + .and_then(|dict| { + if dict.is_null(idx) { + None + } else { + let values = dict.values().as_any().downcast_ref::()?; + let key = dict.keys().value(idx); + Some(values.value(key as usize).to_string()) + } + }), + DataType::UInt16 => key_col + .as_any() + .downcast_ref::>() + .and_then(|dict| { + if dict.is_null(idx) { + None + } else { + let values = dict.values().as_any().downcast_ref::()?; + let key = dict.keys().value(idx); + Some(values.value(key as usize).to_string()) + } + }), + _ => None, + }, + _ => None, + } +} + +/// Create a batch of inserted attributes. +/// According to the OTel collector spec, `insert` only inserts if the key does NOT already exist. +/// This function checks existing (parent_id, key) pairs in the current record batch and only +/// inserts new keys. fn create_inserted_batch( + current_batch: &RecordBatch, parent_ids: &ArrayRef, insert: &InsertTransform, schema: &arrow::datatypes::Schema, ) -> Result<(RecordBatch, usize)> { - let parent_ids = parent_ids + let parent_ids_arr = parent_ids .as_any() .downcast_ref::>() .ok_or_else(|| Error::ColumnDataTypeMismatch { @@ -3744,10 +3798,24 @@ fn create_inserted_batch( actual: parent_ids.data_type().clone(), })?; + // Build a set of (parent_id, key) pairs that already exist + let mut existing_keys: BTreeMap> = BTreeMap::new(); + if let Some(key_col) = current_batch.column_by_name(consts::ATTRIBUTE_KEY) { + for i in 0..current_batch.num_rows() { + if !parent_ids_arr.is_null(i) { + let parent = parent_ids_arr.value(i); + if let Some(key) = get_key_at_index(key_col, i) { + let _ = existing_keys.entry(parent).or_default().insert(key); + } + } + } + } + + // Get unique parents let mut unique_parents = BTreeSet::new(); - for i in 0..parent_ids.len() { - if !parent_ids.is_null(i) { - let _ = unique_parents.insert(parent_ids.value(i)); + for i in 0..parent_ids_arr.len() { + if !parent_ids_arr.is_null(i) { + let _ = unique_parents.insert(parent_ids_arr.value(i)); } } @@ -3755,31 +3823,43 @@ fn create_inserted_batch( return Ok((RecordBatch::new_empty(Arc::new(schema.clone())), 0)); } - let num_parents = unique_parents.len(); - let num_inserts = insert.entries.len(); - let total_rows = num_parents * num_inserts; + // Compute which (parent, key, value) tuples to actually insert + // Only insert if the key doesn't already exist for that parent + let mut to_insert: Vec<(u16, &str, &LiteralValue)> = Vec::new(); + for &parent in &unique_parents { + let parent_existing = existing_keys.get(&parent); + for (key, val) in &insert.entries { + let key_exists = parent_existing + .map(|keys| keys.contains(key)) + .unwrap_or(false); + if !key_exists { + to_insert.push((parent, key.as_str(), val)); + } + } + } + + let total_rows = to_insert.len(); + if total_rows == 0 { + return Ok((RecordBatch::new_empty(Arc::new(schema.clone())), 0)); + } // Build Parent ID column let mut new_parent_ids = PrimitiveBuilder::::with_capacity(total_rows); - for _parent in &unique_parents { - for _ in 0..num_inserts { - new_parent_ids.append_value(*_parent); - } + for (parent, _, _) in &to_insert { + new_parent_ids.append_value(*parent); } let new_parent_ids = Arc::new(new_parent_ids.finish()) as ArrayRef; // Build Attribute Type column let mut new_types = PrimitiveBuilder::::with_capacity(total_rows); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - let type_val = match val { - LiteralValue::Str(_) => AttributeValueType::Str, - LiteralValue::Int(_) => AttributeValueType::Int, - LiteralValue::Double(_) => AttributeValueType::Double, - LiteralValue::Bool(_) => AttributeValueType::Bool, - }; - new_types.append_value(type_val as u8); - } + for (_, _, val) in &to_insert { + let type_val = match val { + LiteralValue::Str(_) => AttributeValueType::Str, + LiteralValue::Int(_) => AttributeValueType::Int, + LiteralValue::Double(_) => AttributeValueType::Double, + LiteralValue::Bool(_) => AttributeValueType::Bool, + }; + new_types.append_value(type_val as u8); } let new_types = Arc::new(new_types.finish()) as ArrayRef; @@ -3796,29 +3876,23 @@ fn create_inserted_batch( DataType::Utf8 => { let mut builder = arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); - for _parent in &unique_parents { - for (key, _) in &insert.entries { - builder.append_value(key); - } + for (_, key, _) in &to_insert { + builder.append_value(*key); } Arc::new(builder.finish()) } DataType::Dictionary(k, _v) => match **k { DataType::UInt8 => { let mut builder = StringDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (key, _) in &insert.entries { - builder.append_value(key); - } + for (_, key, _) in &to_insert { + builder.append_value(*key); } Arc::new(builder.finish()) } DataType::UInt16 => { let mut builder = StringDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (key, _) in &insert.entries { - builder.append_value(key); - } + for (_, key, _) in &to_insert { + builder.append_value(*key); } Arc::new(builder.finish()) } @@ -3853,13 +3927,11 @@ fn create_inserted_batch( DataType::Utf8 => { let mut builder = arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Str(s) = val { - builder.append_value(s); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Str(s) = val { + builder.append_value(s); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -3867,26 +3939,22 @@ fn create_inserted_batch( DataType::Dictionary(k, v) if **v == DataType::Utf8 => match **k { DataType::UInt8 => { let mut builder = StringDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Str(s) = val { - builder.append_value(s); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Str(s) = val { + builder.append_value(s); + } else { + builder.append_null(); } } Arc::new(builder.finish()) } DataType::UInt16 => { let mut builder = StringDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Str(s) = val { - builder.append_value(s); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Str(s) = val { + builder.append_value(s); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -3910,13 +3978,11 @@ fn create_inserted_batch( match field.data_type() { DataType::Int64 => { let mut builder = PrimitiveBuilder::::with_capacity(total_rows); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Int(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Int(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -3924,13 +3990,11 @@ fn create_inserted_batch( DataType::Dictionary(k, v) if **v == DataType::Int64 => match **k { DataType::UInt8 => { let mut builder = PrimitiveDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Int(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Int(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -3938,13 +4002,11 @@ fn create_inserted_batch( DataType::UInt16 => { let mut builder = PrimitiveDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Int(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Int(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -3968,13 +4030,11 @@ fn create_inserted_batch( match field.data_type() { DataType::Float64 => { let mut builder = PrimitiveBuilder::::with_capacity(total_rows); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Double(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Double(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -3983,13 +4043,11 @@ fn create_inserted_batch( DataType::UInt8 => { let mut builder = PrimitiveDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Double(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Double(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -3997,13 +4055,11 @@ fn create_inserted_batch( DataType::UInt16 => { let mut builder = PrimitiveDictionaryBuilder::::new(); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Double(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Double(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -4026,13 +4082,11 @@ fn create_inserted_batch( } else if name == consts::ATTRIBUTE_BOOL { // Note: Boolean Dictionaries are not standard/supported by simple builders let mut builder = arrow::array::BooleanBuilder::with_capacity(total_rows); - for _parent in &unique_parents { - for (_, val) in &insert.entries { - if let LiteralValue::Bool(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } + for (_, _, val) in &to_insert { + if let LiteralValue::Bool(v) = val { + builder.append_value(*v); + } else { + builder.append_null(); } } Arc::new(builder.finish()) @@ -4180,4 +4234,112 @@ mod insert_tests { .unwrap(); assert_eq!(keys.value(0), "new"); } + + #[test] + fn test_insert_does_not_overwrite_existing_key() { + // According to OTel collector spec, `insert` only inserts if the key does not already exist. + // This test verifies that existing keys are NOT overwritten. + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + // Input: parent 0 has "existing_key"="original_value" + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec!["existing_key"])), + Arc::new(StringArray::from_iter_values(vec!["original_value"])), + ], + ) + .unwrap(); + + // Try to insert "existing_key"="new_value" - should be skipped because key exists + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "existing_key".into(), + LiteralValue::Str("new_value".into()), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + // No inserts should happen because the key already exists + assert_eq!(stats.inserted_entries, 0); + + // Result should still contain 1 row with the original value + assert_eq!(result.num_rows(), 1); + let keys = result + .column_by_name(consts::ATTRIBUTE_KEY) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(keys.value(0), "existing_key"); + + let vals = result + .column_by_name(consts::ATTRIBUTE_STR) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(vals.value(0), "original_value"); + } + + #[test] + fn test_insert_mixed_existing_and_new_keys() { + // Test: insert multiple keys where some exist and some don't + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + // Input: parent 0 has "a"="av", parent 1 has "b"="bv" + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0, 1])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec!["a", "b"])), + Arc::new(StringArray::from_iter_values(vec!["av", "bv"])), + ], + ) + .unwrap(); + + // Try to insert: + // - "a"="new_a" - should be skipped for parent 0 (exists), inserted for parent 1 + // - "c"="cv" - should be inserted for both parents + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![ + ("a".into(), LiteralValue::Str("new_a".into())), + ("c".into(), LiteralValue::Str("cv".into())), + ])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + // Should insert: + // - parent 0: "c" (not "a" because it exists) + // - parent 1: "a" and "c" (neither exists) + // Total: 3 inserts + assert_eq!(stats.inserted_entries, 3); + + // Result should have: 2 original + 3 new = 5 rows + assert_eq!(result.num_rows(), 5); + } } From 726dbf9915dab44e9e5c088c35c6eb70ae2a89e6 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Fri, 16 Jan 2026 00:57:05 +0000 Subject: [PATCH 09/25] Add test --- .../crates/otap/src/attributes_processor.rs | 320 +++++++++++ .../crates/pdata/src/otap/transform.rs | 496 ++++++++++++++++++ 2 files changed, 816 insertions(+) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 999f352ec6..8affd191dd 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -869,6 +869,326 @@ mod tests { .validate(|_| async move {}); } + #[test] + fn test_insert_int_value_via_config() { + // Test inserting an integer value via JSON configuration + let input = build_logs_with_attrs( + vec![], + vec![], + vec![KeyValue::new("existing", AnyValue::new_string("val"))], + ); + + let cfg = json!({ + "actions": [ + {"action": "insert", "key": "count", "value": 42}, + ], + "apply_to": ["signal"] + }); + + let metrics_registry_handle = TelemetryRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = + controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let node = test_node("attributes-processor-insert-int"); + let rt: TestRuntime = TestRuntime::new(); + let mut node_config = NodeUserConfig::new_processor_config(ATTRIBUTES_PROCESSOR_URN); + node_config.config = cfg; + let proc = + create_attributes_processor(pipeline_ctx, node, Arc::new(node_config), rt.config()) + .expect("create processor"); + let phase = rt.set_processor(proc); + + phase + .run_test(|mut ctx| async move { + let mut bytes = BytesMut::new(); + input.encode(&mut bytes).expect("encode"); + let bytes = bytes.freeze(); + let pdata_in = + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + ctx.process(Message::PData(pdata_in)) + .await + .expect("process"); + + let out = ctx.drain_pdata().await; + let first = out.into_iter().next().expect("one output").payload(); + + let otlp_bytes: OtlpProtoBytes = first.try_into().expect("convert to otlp"); + let bytes = match otlp_bytes { + OtlpProtoBytes::ExportLogsRequest(b) => b, + _ => panic!("unexpected otlp variant"), + }; + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); + + let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; + assert!(log_attrs.iter().any(|kv| kv.key == "count")); + assert!(log_attrs.iter().any(|kv| kv.key == "existing")); + }) + .validate(|_| async move {}); + } + + #[test] + fn test_insert_double_value_via_config() { + // Test inserting a double value via JSON configuration + let input = build_logs_with_attrs( + vec![], + vec![], + vec![KeyValue::new("existing", AnyValue::new_string("val"))], + ); + + let cfg = json!({ + "actions": [ + {"action": "insert", "key": "ratio", "value": 3.14159}, + ], + "apply_to": ["signal"] + }); + + let metrics_registry_handle = TelemetryRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = + controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let node = test_node("attributes-processor-insert-double"); + let rt: TestRuntime = TestRuntime::new(); + let mut node_config = NodeUserConfig::new_processor_config(ATTRIBUTES_PROCESSOR_URN); + node_config.config = cfg; + let proc = + create_attributes_processor(pipeline_ctx, node, Arc::new(node_config), rt.config()) + .expect("create processor"); + let phase = rt.set_processor(proc); + + phase + .run_test(|mut ctx| async move { + let mut bytes = BytesMut::new(); + input.encode(&mut bytes).expect("encode"); + let bytes = bytes.freeze(); + let pdata_in = + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + ctx.process(Message::PData(pdata_in)) + .await + .expect("process"); + + let out = ctx.drain_pdata().await; + let first = out.into_iter().next().expect("one output").payload(); + + let otlp_bytes: OtlpProtoBytes = first.try_into().expect("convert to otlp"); + let bytes = match otlp_bytes { + OtlpProtoBytes::ExportLogsRequest(b) => b, + _ => panic!("unexpected otlp variant"), + }; + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); + + let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; + assert!(log_attrs.iter().any(|kv| kv.key == "ratio")); + assert!(log_attrs.iter().any(|kv| kv.key == "existing")); + }) + .validate(|_| async move {}); + } + + #[test] + fn test_insert_bool_value_via_config() { + // Test inserting a boolean value via JSON configuration + let input = build_logs_with_attrs( + vec![], + vec![], + vec![KeyValue::new("existing", AnyValue::new_string("val"))], + ); + + let cfg = json!({ + "actions": [ + {"action": "insert", "key": "enabled", "value": true}, + ], + "apply_to": ["signal"] + }); + + let metrics_registry_handle = TelemetryRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = + controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let node = test_node("attributes-processor-insert-bool"); + let rt: TestRuntime = TestRuntime::new(); + let mut node_config = NodeUserConfig::new_processor_config(ATTRIBUTES_PROCESSOR_URN); + node_config.config = cfg; + let proc = + create_attributes_processor(pipeline_ctx, node, Arc::new(node_config), rt.config()) + .expect("create processor"); + let phase = rt.set_processor(proc); + + phase + .run_test(|mut ctx| async move { + let mut bytes = BytesMut::new(); + input.encode(&mut bytes).expect("encode"); + let bytes = bytes.freeze(); + let pdata_in = + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + ctx.process(Message::PData(pdata_in)) + .await + .expect("process"); + + let out = ctx.drain_pdata().await; + let first = out.into_iter().next().expect("one output").payload(); + + let otlp_bytes: OtlpProtoBytes = first.try_into().expect("convert to otlp"); + let bytes = match otlp_bytes { + OtlpProtoBytes::ExportLogsRequest(b) => b, + _ => panic!("unexpected otlp variant"), + }; + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); + + let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; + assert!(log_attrs.iter().any(|kv| kv.key == "enabled")); + assert!(log_attrs.iter().any(|kv| kv.key == "existing")); + }) + .validate(|_| async move {}); + } + + #[test] + fn test_insert_does_not_overwrite_existing() { + // Verify that insert action does NOT overwrite existing key (per OTel spec) + let input = build_logs_with_attrs( + vec![], + vec![], + vec![KeyValue::new("target_key", AnyValue::new_string("original_value"))], + ); + + let cfg = json!({ + "actions": [ + {"action": "insert", "key": "target_key", "value": "new_value"}, + ], + "apply_to": ["signal"] + }); + + let metrics_registry_handle = TelemetryRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = + controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let node = test_node("attributes-processor-insert-no-overwrite"); + let rt: TestRuntime = TestRuntime::new(); + let mut node_config = NodeUserConfig::new_processor_config(ATTRIBUTES_PROCESSOR_URN); + node_config.config = cfg; + let proc = + create_attributes_processor(pipeline_ctx, node, Arc::new(node_config), rt.config()) + .expect("create processor"); + let phase = rt.set_processor(proc); + + phase + .run_test(|mut ctx| async move { + let mut bytes = BytesMut::new(); + input.encode(&mut bytes).expect("encode"); + let bytes = bytes.freeze(); + let pdata_in = + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + ctx.process(Message::PData(pdata_in)) + .await + .expect("process"); + + let out = ctx.drain_pdata().await; + let first = out.into_iter().next().expect("one output").payload(); + + let otlp_bytes: OtlpProtoBytes = first.try_into().expect("convert to otlp"); + let bytes = match otlp_bytes { + OtlpProtoBytes::ExportLogsRequest(b) => b, + _ => panic!("unexpected otlp variant"), + }; + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); + + let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; + + // Should have exactly one target_key with original value + let matching_attrs: Vec<_> = log_attrs + .iter() + .filter(|kv| kv.key == "target_key") + .collect(); + assert_eq!(matching_attrs.len(), 1); + + // Value should be original, not overwritten + let value = matching_attrs[0] + .value + .as_ref() + .expect("value") + .value + .as_ref() + .expect("inner value"); + match value { + otap_df_pdata::proto::opentelemetry::common::v1::any_value::Value::StringValue( + s, + ) => { + assert_eq!(s, "original_value"); + } + _ => panic!("expected string value"), + } + }) + .validate(|_| async move {}); + } + + #[test] + fn test_insert_multiple_keys_mixed_existing() { + // Test inserting multiple keys where some exist and some don't + let input = build_logs_with_attrs( + vec![], + vec![], + vec![ + KeyValue::new("existing_a", AnyValue::new_string("val_a")), + KeyValue::new("existing_b", AnyValue::new_string("val_b")), + ], + ); + + let cfg = json!({ + "actions": [ + {"action": "insert", "key": "existing_a", "value": "should_not_replace"}, + {"action": "insert", "key": "new_key", "value": "new_value"}, + ], + "apply_to": ["signal"] + }); + + let metrics_registry_handle = TelemetryRegistryHandle::new(); + let controller_ctx = ControllerContext::new(metrics_registry_handle); + let pipeline_ctx = + controller_ctx.pipeline_context_with("grp".into(), "pipeline".into(), 0, 0); + + let node = test_node("attributes-processor-insert-mixed"); + let rt: TestRuntime = TestRuntime::new(); + let mut node_config = NodeUserConfig::new_processor_config(ATTRIBUTES_PROCESSOR_URN); + node_config.config = cfg; + let proc = + create_attributes_processor(pipeline_ctx, node, Arc::new(node_config), rt.config()) + .expect("create processor"); + let phase = rt.set_processor(proc); + + phase + .run_test(|mut ctx| async move { + let mut bytes = BytesMut::new(); + input.encode(&mut bytes).expect("encode"); + let bytes = bytes.freeze(); + let pdata_in = + OtapPdata::new_default(OtlpProtoBytes::ExportLogsRequest(bytes).into()); + ctx.process(Message::PData(pdata_in)) + .await + .expect("process"); + + let out = ctx.drain_pdata().await; + let first = out.into_iter().next().expect("one output").payload(); + + let otlp_bytes: OtlpProtoBytes = first.try_into().expect("convert to otlp"); + let bytes = match otlp_bytes { + OtlpProtoBytes::ExportLogsRequest(b) => b, + _ => panic!("unexpected otlp variant"), + }; + let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); + + let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; + + // Should have 3 keys: existing_a (unchanged), existing_b, new_key + assert_eq!(log_attrs.len(), 3); + assert!(log_attrs.iter().any(|kv| kv.key == "existing_a")); + assert!(log_attrs.iter().any(|kv| kv.key == "existing_b")); + assert!(log_attrs.iter().any(|kv| kv.key == "new_key")); + }) + .validate(|_| async move {}); + } + #[test] fn test_delete_scoped_to_resource_only_logs() { // Resource has 'a', scope has 'a', log has 'a' and another key to keep batch non-empty diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index fc7f664640..b8026d455c 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -4342,4 +4342,500 @@ mod insert_tests { // Result should have: 2 original + 3 new = 5 rows assert_eq!(result.num_rows(), 5); } + + #[test] + fn test_insert_int_value() { + // Test inserting Int values + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + Field::new(consts::ATTRIBUTE_INT, DataType::Int64, true), + Field::new(consts::ATTRIBUTE_DOUBLE, DataType::Float64, true), + Field::new(consts::ATTRIBUTE_BOOL, DataType::Boolean, true), + ])); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec!["existing"])), + Arc::new(StringArray::from_iter_values(vec!["value"])), + Arc::new(Int64Array::from(vec![None])), + Arc::new(Float64Array::from(vec![None])), + Arc::new(BooleanArray::from(vec![None])), + ], + ) + .unwrap(); + + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "count".into(), + LiteralValue::Int(42), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + assert_eq!(stats.inserted_entries, 1); + assert_eq!(result.num_rows(), 2); + + // Check the int value was inserted + let int_col = result + .column_by_name(consts::ATTRIBUTE_INT) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + // First row (existing) should be null, second row (inserted) should be 42 + assert!(int_col.is_null(0)); + assert_eq!(int_col.value(1), 42); + } + + #[test] + fn test_insert_double_value() { + // Test inserting Double values + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + Field::new(consts::ATTRIBUTE_INT, DataType::Int64, true), + Field::new(consts::ATTRIBUTE_DOUBLE, DataType::Float64, true), + Field::new(consts::ATTRIBUTE_BOOL, DataType::Boolean, true), + ])); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec!["existing"])), + Arc::new(StringArray::from_iter_values(vec!["value"])), + Arc::new(Int64Array::from(vec![None])), + Arc::new(Float64Array::from(vec![None])), + Arc::new(BooleanArray::from(vec![None])), + ], + ) + .unwrap(); + + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "ratio".into(), + LiteralValue::Double(3.14159), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + assert_eq!(stats.inserted_entries, 1); + assert_eq!(result.num_rows(), 2); + + let double_col = result + .column_by_name(consts::ATTRIBUTE_DOUBLE) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert!(double_col.is_null(0)); + assert!((double_col.value(1) - 3.14159).abs() < f64::EPSILON); + } + + #[test] + fn test_insert_bool_value() { + // Test inserting Bool values + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + Field::new(consts::ATTRIBUTE_INT, DataType::Int64, true), + Field::new(consts::ATTRIBUTE_DOUBLE, DataType::Float64, true), + Field::new(consts::ATTRIBUTE_BOOL, DataType::Boolean, true), + ])); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec!["existing"])), + Arc::new(StringArray::from_iter_values(vec!["value"])), + Arc::new(Int64Array::from(vec![None])), + Arc::new(Float64Array::from(vec![None])), + Arc::new(BooleanArray::from(vec![None])), + ], + ) + .unwrap(); + + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "enabled".into(), + LiteralValue::Bool(true), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + assert_eq!(stats.inserted_entries, 1); + assert_eq!(result.num_rows(), 2); + + let bool_col = result + .column_by_name(consts::ATTRIBUTE_BOOL) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert!(bool_col.is_null(0)); + assert!(bool_col.value(1)); + } + + #[test] + fn test_insert_multiple_value_types() { + // Test inserting multiple values of different types at once + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + Field::new(consts::ATTRIBUTE_INT, DataType::Int64, true), + Field::new(consts::ATTRIBUTE_DOUBLE, DataType::Float64, true), + Field::new(consts::ATTRIBUTE_BOOL, DataType::Boolean, true), + ])); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec!["existing"])), + Arc::new(StringArray::from_iter_values(vec!["value"])), + Arc::new(Int64Array::from(vec![None])), + Arc::new(Float64Array::from(vec![None])), + Arc::new(BooleanArray::from(vec![None])), + ], + ) + .unwrap(); + + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![ + ("str_key".into(), LiteralValue::Str("str_val".into())), + ("int_key".into(), LiteralValue::Int(100)), + ("double_key".into(), LiteralValue::Double(2.5)), + ("bool_key".into(), LiteralValue::Bool(false)), + ])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + assert_eq!(stats.inserted_entries, 4); + assert_eq!(result.num_rows(), 5); // 1 original + 4 inserted + } + + #[test] + fn test_insert_with_dictionary_encoded_keys() { + // Test inserting with dictionary-encoded key columns + let key_type = DataType::Dictionary( + Box::new(DataType::UInt8), + Box::new(DataType::Utf8), + ); + let str_type = DataType::Dictionary( + Box::new(DataType::UInt8), + Box::new(DataType::Utf8), + ); + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, key_type.clone(), false), + Field::new(consts::ATTRIBUTE_STR, str_type.clone(), true), + ])); + + let mut key_builder = StringDictionaryBuilder::::new(); + key_builder.append_value("existing_key"); + let keys = Arc::new(key_builder.finish()); + + let mut val_builder = StringDictionaryBuilder::::new(); + val_builder.append_value("existing_value"); + let vals = Arc::new(val_builder.finish()); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + keys, + vals, + ], + ) + .unwrap(); + + // Insert a new key - should work with dictionary-encoded schema + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "new_key".into(), + LiteralValue::Str("new_value".into()), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + assert_eq!(stats.inserted_entries, 1); + assert_eq!(result.num_rows(), 2); + } + + #[test] + fn test_insert_with_dictionary_encoded_keys_respects_existing() { + // Test that insert respects existing keys when using dictionary-encoded columns + let key_type = DataType::Dictionary( + Box::new(DataType::UInt8), + Box::new(DataType::Utf8), + ); + let str_type = DataType::Dictionary( + Box::new(DataType::UInt8), + Box::new(DataType::Utf8), + ); + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, key_type.clone(), false), + Field::new(consts::ATTRIBUTE_STR, str_type.clone(), true), + ])); + + let mut key_builder = StringDictionaryBuilder::::new(); + key_builder.append_value("existing_key"); + let keys = Arc::new(key_builder.finish()); + + let mut val_builder = StringDictionaryBuilder::::new(); + val_builder.append_value("original_value"); + let vals = Arc::new(val_builder.finish()); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + keys, + vals, + ], + ) + .unwrap(); + + // Try to insert existing_key - should be skipped + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "existing_key".into(), + LiteralValue::Str("new_value".into()), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + // No inserts because key already exists + assert_eq!(stats.inserted_entries, 0); + assert_eq!(result.num_rows(), 1); + } + + #[test] + fn test_insert_with_uint16_dictionary_keys() { + // Test inserting with UInt16 dictionary-encoded key columns + let key_type = DataType::Dictionary( + Box::new(DataType::UInt16), + Box::new(DataType::Utf8), + ); + let str_type = DataType::Dictionary( + Box::new(DataType::UInt16), + Box::new(DataType::Utf8), + ); + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, key_type.clone(), false), + Field::new(consts::ATTRIBUTE_STR, str_type.clone(), true), + ])); + + let mut key_builder = StringDictionaryBuilder::::new(); + key_builder.append_value("existing_key"); + let keys = Arc::new(key_builder.finish()); + + let mut val_builder = StringDictionaryBuilder::::new(); + val_builder.append_value("existing_value"); + let vals = Arc::new(val_builder.finish()); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + keys, + vals, + ], + ) + .unwrap(); + + // Insert new key and try to insert existing key + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![ + ("existing_key".into(), LiteralValue::Str("should_skip".into())), + ("new_key".into(), LiteralValue::Str("new_value".into())), + ])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + // Only 1 insert (new_key), existing_key should be skipped + assert_eq!(stats.inserted_entries, 1); + assert_eq!(result.num_rows(), 2); + } + + #[test] + fn test_insert_with_empty_batch() { + // Test inserting when input batch is empty + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + let input = RecordBatch::new_empty(schema.clone()); + + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "key".into(), + LiteralValue::Str("value".into()), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + // No parents to insert into + assert_eq!(stats.inserted_entries, 0); + assert_eq!(result.num_rows(), 0); + } + + #[test] + fn test_insert_with_multiple_parents() { + // Test insert correctly handles multiple different parent IDs + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + // 3 parents: 0, 1, 2 with different existing keys + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0, 0, 1, 2, 2, 2])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + AttributeValueType::Str as u8, + AttributeValueType::Str as u8, + AttributeValueType::Str as u8, + AttributeValueType::Str as u8, + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec![ + "a", "b", "a", "x", "y", "z", + ])), + Arc::new(StringArray::from_iter_values(vec![ + "v0", "v1", "v2", "v3", "v4", "v5", + ])), + ], + ) + .unwrap(); + + // Insert "a" and "new_key" + // Parent 0: has "a" -> skip "a", insert "new_key" + // Parent 1: has "a" -> skip "a", insert "new_key" + // Parent 2: no "a" -> insert both "a" and "new_key" + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![ + ("a".into(), LiteralValue::Str("inserted_a".into())), + ("new_key".into(), LiteralValue::Str("new_val".into())), + ])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + // 1 + 1 + 2 = 4 inserts + assert_eq!(stats.inserted_entries, 4); + // 6 original + 4 inserted = 10 + assert_eq!(result.num_rows(), 10); + } + + #[test] + fn test_insert_with_null_in_key_column() { + // Test that null keys in original data are handled properly + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt16, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, true), // nullable + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + // Parent 0 has a null key + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt16Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from(vec![None::<&str>])), + Arc::new(StringArray::from_iter_values(vec!["value"])), + ], + ) + .unwrap(); + + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(vec![( + "new_key".into(), + LiteralValue::Str("new_value".into()), + )])), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + // Should insert new_key for parent 0 + assert_eq!(stats.inserted_entries, 1); + assert_eq!(result.num_rows(), 2); + } } From 41751dbafbdb6dc3dc0b0d5f116a30f2b2fd0e71 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Fri, 16 Jan 2026 01:02:25 +0000 Subject: [PATCH 10/25] More format --- .../crates/otap/src/attributes_processor.rs | 5 ++- .../crates/pdata/src/otap/transform.rs | 35 ++++++------------- 2 files changed, 14 insertions(+), 26 deletions(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 8affd191dd..3ee0b216f5 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -1049,7 +1049,10 @@ mod tests { let input = build_logs_with_attrs( vec![], vec![], - vec![KeyValue::new("target_key", AnyValue::new_string("original_value"))], + vec![KeyValue::new( + "target_key", + AnyValue::new_string("original_value"), + )], ); let cfg = json!({ diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index b8026d455c..aab53daa2b 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -4553,14 +4553,8 @@ mod insert_tests { #[test] fn test_insert_with_dictionary_encoded_keys() { // Test inserting with dictionary-encoded key columns - let key_type = DataType::Dictionary( - Box::new(DataType::UInt8), - Box::new(DataType::Utf8), - ); - let str_type = DataType::Dictionary( - Box::new(DataType::UInt8), - Box::new(DataType::Utf8), - ); + let key_type = DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)); + let str_type = DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)); let schema = Arc::new(Schema::new(vec![ Field::new(consts::PARENT_ID, DataType::UInt16, false), Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), @@ -4608,14 +4602,8 @@ mod insert_tests { #[test] fn test_insert_with_dictionary_encoded_keys_respects_existing() { // Test that insert respects existing keys when using dictionary-encoded columns - let key_type = DataType::Dictionary( - Box::new(DataType::UInt8), - Box::new(DataType::Utf8), - ); - let str_type = DataType::Dictionary( - Box::new(DataType::UInt8), - Box::new(DataType::Utf8), - ); + let key_type = DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)); + let str_type = DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)); let schema = Arc::new(Schema::new(vec![ Field::new(consts::PARENT_ID, DataType::UInt16, false), Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), @@ -4664,14 +4652,8 @@ mod insert_tests { #[test] fn test_insert_with_uint16_dictionary_keys() { // Test inserting with UInt16 dictionary-encoded key columns - let key_type = DataType::Dictionary( - Box::new(DataType::UInt16), - Box::new(DataType::Utf8), - ); - let str_type = DataType::Dictionary( - Box::new(DataType::UInt16), - Box::new(DataType::Utf8), - ); + let key_type = DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)); + let str_type = DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)); let schema = Arc::new(Schema::new(vec![ Field::new(consts::PARENT_ID, DataType::UInt16, false), Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), @@ -4705,7 +4687,10 @@ mod insert_tests { rename: None, delete: None, insert: Some(InsertTransform::new(vec![ - ("existing_key".into(), LiteralValue::Str("should_skip".into())), + ( + "existing_key".into(), + LiteralValue::Str("should_skip".into()), + ), ("new_key".into(), LiteralValue::Str("new_value".into())), ])), }; From 59f54bc22132fde68fbfdf940e102970658253bb Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Fri, 16 Jan 2026 01:16:55 +0000 Subject: [PATCH 11/25] Fix liternal constant --- rust/otap-dataflow/crates/pdata/src/otap/transform.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index aab53daa2b..363c70408b 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -4432,7 +4432,7 @@ mod insert_tests { delete: None, insert: Some(InsertTransform::new(vec![( "ratio".into(), - LiteralValue::Double(3.14159), + LiteralValue::Double(2.71828), )])), }; @@ -4448,7 +4448,7 @@ mod insert_tests { .downcast_ref::() .unwrap(); assert!(double_col.is_null(0)); - assert!((double_col.value(1) - 3.14159).abs() < f64::EPSILON); + assert!((double_col.value(1) - 2.71828).abs() < f64::EPSILON); } #[test] From 86f8eece725ce2cab7f016f51f2693d66afc1dc5 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Fri, 16 Jan 2026 03:55:49 +0000 Subject: [PATCH 12/25] Rename literal --- rust/otap-dataflow/crates/pdata/src/otap/transform.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 363c70408b..1dc8867b77 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -4432,7 +4432,7 @@ mod insert_tests { delete: None, insert: Some(InsertTransform::new(vec![( "ratio".into(), - LiteralValue::Double(2.71828), + LiteralValue::Double(1.2345), )])), }; @@ -4448,7 +4448,7 @@ mod insert_tests { .downcast_ref::() .unwrap(); assert!(double_col.is_null(0)); - assert!((double_col.value(1) - 2.71828).abs() < f64::EPSILON); + assert!((double_col.value(1) - 1.2345).abs() < f64::EPSILON); } #[test] From 4dd108e37e6a2ec2b6d32e229e8364a46ec4a9cb Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Fri, 16 Jan 2026 05:39:16 +0000 Subject: [PATCH 13/25] Fix literal constant --- rust/otap-dataflow/crates/otap/src/attributes_processor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 3ee0b216f5..2a61dd3e24 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -938,7 +938,7 @@ mod tests { let cfg = json!({ "actions": [ - {"action": "insert", "key": "ratio", "value": 3.14159}, + {"action": "insert", "key": "ratio", "value": 1.2345}, ], "apply_to": ["signal"] }); From 2dcacd034b53ca3d705f9140bc11d48337fab1cd Mon Sep 17 00:00:00 2001 From: Tom Tan Date: Tue, 20 Jan 2026 14:20:43 -0800 Subject: [PATCH 14/25] Update rust/otap-dataflow/crates/otap/src/attributes_processor.rs Co-authored-by: albertlockett --- rust/otap-dataflow/crates/otap/src/attributes_processor.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 2a61dd3e24..44b399713e 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -847,7 +847,6 @@ mod tests { .unwrap() .attributes; - println!("Resource attrs: {:?}", res_attrs); assert!(res_attrs.iter().any(|kv| kv.key == "c")); assert!(res_attrs.iter().any(|kv| kv.key == "r")); From 7ea7ec8d2cd4c747c0524f33e40265cdc1248124 Mon Sep 17 00:00:00 2001 From: Tom Tan Date: Tue, 20 Jan 2026 14:51:34 -0800 Subject: [PATCH 15/25] Update rust/otap-dataflow/crates/otap/src/attributes_processor.rs Co-authored-by: albertlockett --- rust/otap-dataflow/crates/otap/src/attributes_processor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 44b399713e..8ba31f5ce2 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -851,7 +851,7 @@ mod tests { assert!(res_attrs.iter().any(|kv| kv.key == "c")); assert!(res_attrs.iter().any(|kv| kv.key == "r")); - // Scope 'c' should remain + // Scope 'c' should not be inserted; 'a' should remain let scope_attrs = &decoded.resource_logs[0].scope_logs[0] .scope .as_ref() From 6d70540d066fb0afadbcd0ed12d3aa59b9746686 Mon Sep 17 00:00:00 2001 From: Tom Tan Date: Tue, 20 Jan 2026 14:52:24 -0800 Subject: [PATCH 16/25] Update rust/otap-dataflow/crates/otap/src/attributes_processor.rs Co-authored-by: albertlockett --- rust/otap-dataflow/crates/otap/src/attributes_processor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 8ba31f5ce2..3ced38d198 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -860,7 +860,7 @@ mod tests { assert!(!scope_attrs.iter().any(|kv| kv.key == "c")); assert!(scope_attrs.iter().any(|kv| kv.key == "a")); - // Log 'c' should be deleted; 'b' should remain + // Log 'c' should not be inserted; 'b' should remain let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; assert!(!log_attrs.iter().any(|kv| kv.key == "c")); assert!(log_attrs.iter().any(|kv| kv.key == "b")); From cd1afb35734e91320ca9632fbc8ddaad0e2f4859 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Tue, 20 Jan 2026 23:06:30 +0000 Subject: [PATCH 17/25] Let unsupported value types cause a config error when insert --- .../crates/otap/src/attributes_processor.rs | 21 ++----------------- .../crates/pdata/src/otap/transform.rs | 7 ++++--- 2 files changed, 6 insertions(+), 22 deletions(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 3ced38d198..b39f6c95ed 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -88,7 +88,7 @@ pub enum Action { /// The attribute key to insert. key: String, /// The value to insert. - value: Value, + value: LiteralValue, }, /// Other actions are accepted for forward-compatibility but ignored. @@ -164,23 +164,7 @@ impl AttributesProcessor { let _ = renames.insert(source_key, destination_key); } Action::Insert { key, value } => { - let lit_val = match value { - Value::String(s) => Some(LiteralValue::Str(s)), - Value::Number(n) => { - if let Some(i) = n.as_i64() { - Some(LiteralValue::Int(i)) - } else { - n.as_f64().map(LiteralValue::Double) - } - } - Value::Bool(b) => Some(LiteralValue::Bool(b)), - // Ignore null, object, array as not supported by LiteralValue - _ => None, - }; - - if let Some(val) = lit_val { - inserts.push((key, val)); - } + inserts.push((key, value)); } // Unsupported actions are ignored for now Action::Unsupported => {} @@ -847,7 +831,6 @@ mod tests { .unwrap() .attributes; - assert!(res_attrs.iter().any(|kv| kv.key == "c")); assert!(res_attrs.iter().any(|kv| kv.key == "r")); diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 1dc8867b77..9b41be99aa 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -484,12 +484,13 @@ fn create_next_eq_array_for_array(arr: T) -> BooleanArray { eq(&lhs, &rhs).expect("should be able to compare slice with offset of 1") } -#[derive(Debug, Clone, PartialEq)] +#[derive(Debug, Clone, PartialEq, serde::Serialize, serde::Deserialize)] +#[serde(untagged)] pub enum LiteralValue { - Str(String), + Bool(bool), Int(i64), Double(f64), - Bool(bool), + Str(String), } pub struct InsertTransform { From 579799a9fd0310c9ad19edd56535f87491fe0bdf Mon Sep 17 00:00:00 2001 From: Tom Tan Date: Tue, 20 Jan 2026 15:09:01 -0800 Subject: [PATCH 18/25] Update rust/otap-dataflow/crates/otap/src/attributes_processor.rs Co-authored-by: albertlockett --- rust/otap-dataflow/crates/otap/src/attributes_processor.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index b39f6c95ed..0a60b29bde 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -903,7 +903,10 @@ mod tests { let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; - assert!(log_attrs.iter().any(|kv| kv.key == "count")); + assert!( + log_attrs + .iter() + .any(|kv| kv.key == "count" && kv.value == Some(AnyValue::new_int(42))); assert!(log_attrs.iter().any(|kv| kv.key == "existing")); }) .validate(|_| async move {}); From 92027a81b6b8343c8d2bb544660ef1adabbc5c88 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Tue, 20 Jan 2026 23:47:20 +0000 Subject: [PATCH 19/25] Extend schema before insert attribute to log batch --- .../crates/otap/src/attributes_processor.rs | 48 +++++++- .../crates/pdata/src/otap/transform.rs | 114 ++++++++++++++++-- 2 files changed, 149 insertions(+), 13 deletions(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 0a60b29bde..2890268e14 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -831,7 +831,11 @@ mod tests { .unwrap() .attributes; - assert!(res_attrs.iter().any(|kv| kv.key == "c")); + assert!( + res_attrs + .iter() + .any(|kv| kv.key == "c" && kv.value == Some(AnyValue::new_string("val"))) + ); assert!(res_attrs.iter().any(|kv| kv.key == "r")); // Scope 'c' should not be inserted; 'a' should remain @@ -903,10 +907,19 @@ mod tests { let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; - assert!( - log_attrs - .iter() - .any(|kv| kv.key == "count" && kv.value == Some(AnyValue::new_int(42))); + + // Verify the int value was inserted correctly + assert!(log_attrs.iter().any(|kv| kv.key == "count")); + let count_kv = log_attrs.iter().find(|kv| kv.key == "count").unwrap(); + let inner_val = count_kv.value.as_ref().unwrap().value.as_ref().unwrap(); + match inner_val { + otap_df_pdata::proto::opentelemetry::common::v1::any_value::Value::IntValue( + i, + ) => { + assert_eq!(*i, 42); + } + _ => panic!("expected IntValue, got {:?}", inner_val), + } assert!(log_attrs.iter().any(|kv| kv.key == "existing")); }) .validate(|_| async move {}); @@ -964,7 +977,17 @@ mod tests { let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; + + // Verify the double value was inserted correctly assert!(log_attrs.iter().any(|kv| kv.key == "ratio")); + let ratio_kv = log_attrs.iter().find(|kv| kv.key == "ratio").unwrap(); + let inner_val = ratio_kv.value.as_ref().unwrap().value.as_ref().unwrap(); + match inner_val { + otap_df_pdata::proto::opentelemetry::common::v1::any_value::Value::DoubleValue(d) => { + assert!((d - 1.2345).abs() < f64::EPSILON); + } + _ => panic!("expected DoubleValue, got {:?}", inner_val), + } assert!(log_attrs.iter().any(|kv| kv.key == "existing")); }) .validate(|_| async move {}); @@ -1022,7 +1045,17 @@ mod tests { let decoded = ExportLogsServiceRequest::decode(bytes.as_ref()).expect("decode"); let log_attrs = &decoded.resource_logs[0].scope_logs[0].log_records[0].attributes; + + // Verify the bool value was inserted correctly assert!(log_attrs.iter().any(|kv| kv.key == "enabled")); + let enabled_kv = log_attrs.iter().find(|kv| kv.key == "enabled").unwrap(); + let inner_val = enabled_kv.value.as_ref().unwrap().value.as_ref().unwrap(); + match inner_val { + otap_df_pdata::proto::opentelemetry::common::v1::any_value::Value::BoolValue(b) => { + assert!(*b); + } + _ => panic!("expected BoolValue, got {:?}", inner_val), + } assert!(log_attrs.iter().any(|kv| kv.key == "existing")); }) .validate(|_| async move {}); @@ -1172,7 +1205,10 @@ mod tests { assert_eq!(log_attrs.len(), 3); assert!(log_attrs.iter().any(|kv| kv.key == "existing_a")); assert!(log_attrs.iter().any(|kv| kv.key == "existing_b")); - assert!(log_attrs.iter().any(|kv| kv.key == "new_key")); + assert!( + log_attrs.iter().any(|kv| kv.key == "new_key" + && kv.value == Some(AnyValue::new_string("new_value"))) + ); }) .validate(|_| async move {}); } diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 9b41be99aa..bc214925ca 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -15,7 +15,7 @@ use arrow::buffer::{Buffer, MutableBuffer, NullBuffer, OffsetBuffer, ScalarBuffe use arrow::compute::kernels::cmp::eq; use arrow::compute::{SortColumn, and, concat}; use arrow::datatypes::{ - ArrowDictionaryKeyType, ArrowNativeType, DataType, Float64Type, Int64Type, UInt8Type, + ArrowDictionaryKeyType, ArrowNativeType, DataType, Field, Float64Type, Int64Type, UInt8Type, UInt16Type, }; use arrow::row::{RowConverter, SortField}; @@ -874,13 +874,40 @@ pub fn transform_attributes_with_stats( // - The transformed batch (rb) to check which (parent_id, key) pairs already exist if let Some(insert) = &transform.insert { if let Some(original_parent_ids) = attrs_record_batch.column_by_name(consts::PARENT_ID) { - let (new_rows, count) = - create_inserted_batch(&rb, original_parent_ids, insert, rb.schema().as_ref())?; + // Determine which value type columns we need based on what's being inserted + let needs_int = insert + .entries + .iter() + .any(|(_, v)| matches!(v, LiteralValue::Int(_))); + let needs_double = insert + .entries + .iter() + .any(|(_, v)| matches!(v, LiteralValue::Double(_))); + let needs_bool = insert + .entries + .iter() + .any(|(_, v)| matches!(v, LiteralValue::Bool(_))); + let needs_str = insert + .entries + .iter() + .any(|(_, v)| matches!(v, LiteralValue::Str(_))); + + // Extend schema and batch to include missing value columns + let (rb_extended, extended_schema) = + extend_schema_for_inserts(&rb, needs_str, needs_int, needs_double, needs_bool)?; + + let (new_rows, count) = create_inserted_batch( + &rb_extended, + original_parent_ids, + insert, + extended_schema.as_ref(), + )?; if count > 0 { - let combined = arrow::compute::concat_batches(&rb.schema(), &[rb, new_rows]) - .map_err(|e| Error::Format { - error: e.to_string(), - })?; + let combined = + arrow::compute::concat_batches(&extended_schema, &[rb_extended, new_rows]) + .map_err(|e| Error::Format { + error: e.to_string(), + })?; stats.inserted_entries = count as u64; return Ok((combined, stats)); } @@ -3735,6 +3762,79 @@ mod test { } } +/// Extend a RecordBatch's schema to include missing value columns needed for inserts. +/// Returns the extended batch and schema. If no columns need to be added, returns a clone of +/// the original batch with its schema. +fn extend_schema_for_inserts( + batch: &RecordBatch, + needs_str: bool, + needs_int: bool, + needs_double: bool, + needs_bool: bool, +) -> Result<(RecordBatch, Arc)> { + let schema = batch.schema(); + let num_rows = batch.num_rows(); + + // Check which columns already exist + let has_str = schema.column_with_name(consts::ATTRIBUTE_STR).is_some(); + let has_int = schema.column_with_name(consts::ATTRIBUTE_INT).is_some(); + let has_double = schema.column_with_name(consts::ATTRIBUTE_DOUBLE).is_some(); + let has_bool = schema.column_with_name(consts::ATTRIBUTE_BOOL).is_some(); + + // If all needed columns exist, return unchanged + if (!needs_str || has_str) + && (!needs_int || has_int) + && (!needs_double || has_double) + && (!needs_bool || has_bool) + { + return Ok((batch.clone(), schema)); + } + + // Build new schema with missing columns + let mut new_fields: Vec = schema.fields().iter().map(|f| f.as_ref().clone()).collect(); + let mut new_columns: Vec = batch.columns().to_vec(); + + // Add missing columns with null arrays + if needs_str && !has_str { + new_fields.push(Field::new( + consts::ATTRIBUTE_STR, + DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)), + true, + )); + new_columns.push(arrow::array::new_null_array( + &DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)), + num_rows, + )); + } + + if needs_int && !has_int { + new_fields.push(Field::new(consts::ATTRIBUTE_INT, DataType::Int64, true)); + new_columns.push(arrow::array::new_null_array(&DataType::Int64, num_rows)); + } + + if needs_double && !has_double { + new_fields.push(Field::new( + consts::ATTRIBUTE_DOUBLE, + DataType::Float64, + true, + )); + new_columns.push(arrow::array::new_null_array(&DataType::Float64, num_rows)); + } + + if needs_bool && !has_bool { + new_fields.push(Field::new(consts::ATTRIBUTE_BOOL, DataType::Boolean, true)); + new_columns.push(arrow::array::new_null_array(&DataType::Boolean, num_rows)); + } + + let new_schema = Arc::new(arrow::datatypes::Schema::new(new_fields)); + let new_batch = + RecordBatch::try_new(new_schema.clone(), new_columns).map_err(|e| Error::Format { + error: e.to_string(), + })?; + + Ok((new_batch, new_schema)) +} + /// Helper to extract string key from a record batch key column at a given index. /// Handles both plain Utf8 and Dictionary-encoded columns. fn get_key_at_index(key_col: &ArrayRef, idx: usize) -> Option { From 314bba51f38657cebe4ae39e7c7dcd47bd8f908a Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 21 Jan 2026 00:26:30 +0000 Subject: [PATCH 20/25] Switch to BTreeMap store insert entries --- .../crates/otap/src/attributes_processor.rs | 4 +- .../crates/pdata/src/otap/transform.rs | 68 +++++++++---------- 2 files changed, 36 insertions(+), 36 deletions(-) diff --git a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs index 2890268e14..6d832551c8 100644 --- a/rust/otap-dataflow/crates/otap/src/attributes_processor.rs +++ b/rust/otap-dataflow/crates/otap/src/attributes_processor.rs @@ -150,7 +150,7 @@ impl AttributesProcessor { fn new(config: Config) -> Result { let mut renames = BTreeMap::new(); let mut deletes = BTreeSet::new(); - let mut inserts = Vec::new(); + let mut inserts = BTreeMap::new(); for action in config.actions { match action { @@ -164,7 +164,7 @@ impl AttributesProcessor { let _ = renames.insert(source_key, destination_key); } Action::Insert { key, value } => { - inserts.push((key, value)); + let _ = inserts.insert(key, value); } // Unsupported actions are ignored for now Action::Unsupported => {} diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index aaec3a9ee4..fd251be1fe 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -494,12 +494,12 @@ pub enum LiteralValue { } pub struct InsertTransform { - pub(super) entries: Vec<(String, LiteralValue)>, + pub(super) entries: BTreeMap, } impl InsertTransform { #[must_use] - pub fn new(entries: Vec<(String, LiteralValue)>) -> Self { + pub fn new(entries: BTreeMap) -> Self { Self { entries } } } @@ -620,9 +620,9 @@ impl AttributesTransform { } if let Some(insert) = &self.insert { - let mut insert_keys = BTreeSet::new(); - for (key, _) in &insert.entries { - if !insert_keys.insert(key) { + // Check that insert keys don't overlap with rename or delete keys + for key in insert.entries.keys() { + if !all_keys.insert(key) { return Err(Error::InvalidAttributeTransform { reason: format!("Duplicate key in insert: {key}"), }); @@ -3940,7 +3940,7 @@ fn create_inserted_batch( let mut to_insert: Vec<(u16, &str, &LiteralValue)> = Vec::new(); for &parent in &unique_parents { let parent_existing = existing_keys.get(&parent); - for (key, val) in &insert.entries { + for (key, val) in insert.entries.iter() { let key_exists = parent_existing .map(|keys| keys.contains(key)) .unwrap_or(false); @@ -4251,10 +4251,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "env".into(), LiteralValue::Str("prod".into()), - )])), + )]))), }; let (result, stats) = @@ -4325,10 +4325,10 @@ mod insert_tests { delete: Some(DeleteTransform::new(BTreeSet::from_iter(vec![ "del_me".into(), ]))), - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "new".into(), LiteralValue::Str("val".into()), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4376,10 +4376,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "existing_key".into(), LiteralValue::Str("new_value".into()), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4437,10 +4437,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![ + insert: Some(InsertTransform::new(BTreeMap::from([ ("a".into(), LiteralValue::Str("new_a".into())), ("c".into(), LiteralValue::Str("cv".into())), - ])), + ]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4487,10 +4487,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "count".into(), LiteralValue::Int(42), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4542,10 +4542,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "ratio".into(), LiteralValue::Double(1.2345), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4595,10 +4595,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "enabled".into(), LiteralValue::Bool(true), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4648,12 +4648,12 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![ + insert: Some(InsertTransform::new(BTreeMap::from([ ("str_key".into(), LiteralValue::Str("str_val".into())), ("int_key".into(), LiteralValue::Int(100)), ("double_key".into(), LiteralValue::Double(2.5)), ("bool_key".into(), LiteralValue::Bool(false)), - ])), + ]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4699,10 +4699,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "new_key".into(), LiteralValue::Str("new_value".into()), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4748,10 +4748,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "existing_key".into(), LiteralValue::Str("new_value".into()), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4798,13 +4798,13 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![ + insert: Some(InsertTransform::new(BTreeMap::from([ ( "existing_key".into(), LiteralValue::Str("should_skip".into()), ), ("new_key".into(), LiteralValue::Str("new_value".into())), - ])), + ]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4829,10 +4829,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "key".into(), LiteralValue::Str("value".into()), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4882,10 +4882,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![ + insert: Some(InsertTransform::new(BTreeMap::from([ ("a".into(), LiteralValue::Str("inserted_a".into())), ("new_key".into(), LiteralValue::Str("new_val".into())), - ])), + ]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); @@ -4923,10 +4923,10 @@ mod insert_tests { let tx = AttributesTransform { rename: None, delete: None, - insert: Some(InsertTransform::new(vec![( + insert: Some(InsertTransform::new(BTreeMap::from([( "new_key".into(), LiteralValue::Str("new_value".into()), - )])), + )]))), }; let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); From 775243db2701c49d61c1dbc8676021bbea67f600 Mon Sep 17 00:00:00 2001 From: Tom Tan Date: Tue, 20 Jan 2026 16:34:02 -0800 Subject: [PATCH 21/25] Apply suggestion from @albertlockett Co-authored-by: albertlockett --- rust/otap-dataflow/crates/pdata/src/otap/transform.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index fd251be1fe..6803d848e1 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -620,9 +620,8 @@ impl AttributesTransform { } if let Some(insert) = &self.insert { - // Check that insert keys don't overlap with rename or delete keys - for key in insert.entries.keys() { - if !all_keys.insert(key) { + for (key, _) in &insert.entries { + if ! all_keys.insert(key) { return Err(Error::InvalidAttributeTransform { reason: format!("Duplicate key in insert: {key}"), }); From 71fe3d240c60701bd35dcce1da080ccc4ff517c3 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 21 Jan 2026 00:51:35 +0000 Subject: [PATCH 22/25] Use append_str_n for build appends --- .../crates/pdata/src/otap/transform.rs | 274 +++++------------- 1 file changed, 80 insertions(+), 194 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 6803d848e1..fe43c58089 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -8,21 +8,23 @@ use std::sync::Arc; use arrow::array::{ Array, ArrayRef, ArrowPrimitiveType, BooleanArray, DictionaryArray, NullBufferBuilder, - PrimitiveArray, PrimitiveBuilder, PrimitiveDictionaryBuilder, RecordBatch, StringArray, - StringDictionaryBuilder, UInt32Array, + PrimitiveArray, PrimitiveBuilder, RecordBatch, StringArray, UInt32Array, }; use arrow::buffer::{Buffer, MutableBuffer, NullBuffer, OffsetBuffer, ScalarBuffer}; use arrow::compute::kernels::cmp::eq; use arrow::compute::{SortColumn, and, concat}; use arrow::datatypes::{ - ArrowDictionaryKeyType, ArrowNativeType, DataType, Field, Float64Type, Int64Type, UInt8Type, - UInt16Type, + ArrowDictionaryKeyType, ArrowNativeType, DataType, Field, UInt8Type, UInt16Type, }; use arrow::row::{RowConverter, SortField}; use crate::arrays::{ MaybeDictArrayAccessor, NullableArrayAccessor, get_required_array, get_u8_array, }; +use crate::encode::record::array::{ + ArrayAppend, ArrayAppendNulls, ArrayAppendStr, ArrayOptions, Float64ArrayBuilder, + Int64ArrayBuilder, StringArrayBuilder, dictionary::DictionaryOptions, +}; use crate::error::{Error, Result}; use crate::otlp::attributes::{AttributeValueType, parent_id::ParentId}; use crate::schema::consts::{self, metadata}; @@ -620,8 +622,8 @@ impl AttributesTransform { } if let Some(insert) = &self.insert { - for (key, _) in &insert.entries { - if ! all_keys.insert(key) { + for key in insert.entries.keys() { + if !all_keys.insert(key) { return Err(Error::InvalidAttributeTransform { reason: format!("Duplicate key in insert: {key}"), }); @@ -3890,6 +3892,38 @@ fn get_key_at_index(key_col: &ArrayRef, idx: usize) -> Option { } } +/// Returns `ArrayOptions` configured to match the given `DataType`. +/// For dictionary types, configures the appropriate dictionary options. +/// For native types (Utf8, Int64, Float64), returns options with no dictionary. +fn array_options_for_type(data_type: &DataType) -> ArrayOptions { + match data_type { + DataType::Dictionary(k, _) => match **k { + DataType::UInt8 => ArrayOptions { + dictionary_options: Some(DictionaryOptions::dict8()), + optional: false, + default_values_optional: false, + }, + DataType::UInt16 => ArrayOptions { + dictionary_options: Some(DictionaryOptions::dict16()), + optional: false, + default_values_optional: false, + }, + // Default to dict16 for other key types + _ => ArrayOptions { + dictionary_options: Some(DictionaryOptions::dict16()), + optional: false, + default_values_optional: false, + }, + }, + // Native types - no dictionary + _ => ArrayOptions { + dictionary_options: None, + optional: false, + default_values_optional: false, + }, + } +} + /// Create a batch of inserted attributes. /// According to the OTel collector spec, `insert` only inserts if the key does NOT already exist. /// This function checks existing (parent_id, key) pairs in the current record batch and only @@ -3974,7 +4008,7 @@ fn create_inserted_batch( } let new_types = Arc::new(new_types.finish()) as ArrayRef; - // Build Key column + // Build Key column using StringArrayBuilder let key_col_idx = schema .index_of(consts::ATTRIBUTE_KEY) @@ -3982,47 +4016,17 @@ fn create_inserted_batch( name: consts::ATTRIBUTE_KEY.into(), })?; let key_type = schema.field(key_col_idx).data_type(); + let key_options = array_options_for_type(key_type); - let new_keys: ArrayRef = match key_type { - DataType::Utf8 => { - let mut builder = - arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); - for (_, key, _) in &to_insert { - builder.append_value(*key); - } - Arc::new(builder.finish()) - } - DataType::Dictionary(k, _v) => match **k { - DataType::UInt8 => { - let mut builder = StringDictionaryBuilder::::new(); - for (_, key, _) in &to_insert { - builder.append_value(*key); - } - Arc::new(builder.finish()) - } - DataType::UInt16 => { - let mut builder = StringDictionaryBuilder::::new(); - for (_, key, _) in &to_insert { - builder.append_value(*key); - } - Arc::new(builder.finish()) - } - _ => { - return Err(Error::UnsupportedDictionaryKeyType { - expect_oneof: vec![DataType::UInt8, DataType::UInt16], - actual: *k.clone(), - }); - } - }, - _ => { - return Err(Error::InvalidListArray { - expect_oneof: vec![DataType::Utf8], - actual: key_type.clone(), - }); - } - }; + let mut key_builder = StringArrayBuilder::new(key_options); + for (_, key, _) in &to_insert { + key_builder.append_str(key); + } + let new_keys = key_builder + .finish() + .expect("key builder should produce array since optional=false"); - // We collect columns into a map or vec matching schema order. + // We collect columns into a vec matching schema order. let mut columns = Vec::with_capacity(schema.fields().len()); for field in schema.fields() { @@ -4034,162 +4038,44 @@ fn create_inserted_batch( } else if name == consts::ATTRIBUTE_KEY { new_keys.clone() } else if name == consts::ATTRIBUTE_STR { - match field.data_type() { - DataType::Utf8 => { - let mut builder = - arrow::array::StringBuilder::with_capacity(total_rows, total_rows * 10); - for (_, _, val) in &to_insert { - if let LiteralValue::Str(s) = val { - builder.append_value(s); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - DataType::Dictionary(k, v) if **v == DataType::Utf8 => match **k { - DataType::UInt8 => { - let mut builder = StringDictionaryBuilder::::new(); - for (_, _, val) in &to_insert { - if let LiteralValue::Str(s) = val { - builder.append_value(s); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - DataType::UInt16 => { - let mut builder = StringDictionaryBuilder::::new(); - for (_, _, val) in &to_insert { - if let LiteralValue::Str(s) = val { - builder.append_value(s); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - _ => { - return Err(Error::UnsupportedDictionaryKeyType { - expect_oneof: vec![DataType::UInt8, DataType::UInt16], - actual: *k.clone(), - }); - } - }, - dt => { - return Err(Error::ColumnDataTypeMismatch { - name: name.into(), - expect: DataType::Utf8, - actual: dt.clone(), - }); + let options = array_options_for_type(field.data_type()); + let mut builder = StringArrayBuilder::new(options); + for (_, _, val) in &to_insert { + if let LiteralValue::Str(s) = val { + builder.append_str(s); + } else { + builder.append_null(); } } + builder + .finish() + .expect("str builder should produce array since optional=false") } else if name == consts::ATTRIBUTE_INT { - match field.data_type() { - DataType::Int64 => { - let mut builder = PrimitiveBuilder::::with_capacity(total_rows); - for (_, _, val) in &to_insert { - if let LiteralValue::Int(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - DataType::Dictionary(k, v) if **v == DataType::Int64 => match **k { - DataType::UInt8 => { - let mut builder = PrimitiveDictionaryBuilder::::new(); - for (_, _, val) in &to_insert { - if let LiteralValue::Int(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - DataType::UInt16 => { - let mut builder = - PrimitiveDictionaryBuilder::::new(); - for (_, _, val) in &to_insert { - if let LiteralValue::Int(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - _ => { - return Err(Error::UnsupportedDictionaryKeyType { - expect_oneof: vec![DataType::UInt8], - actual: *k.clone(), - }); - } - }, - dt => { - return Err(Error::ColumnDataTypeMismatch { - name: name.into(), - expect: DataType::Int64, - actual: dt.clone(), - }); + let options = array_options_for_type(field.data_type()); + let mut builder = Int64ArrayBuilder::new(options); + for (_, _, val) in &to_insert { + if let LiteralValue::Int(v) = val { + builder.append_value(v); + } else { + builder.append_null(); } } + builder + .finish() + .expect("int builder should produce array since optional=false") } else if name == consts::ATTRIBUTE_DOUBLE { - match field.data_type() { - DataType::Float64 => { - let mut builder = PrimitiveBuilder::::with_capacity(total_rows); - for (_, _, val) in &to_insert { - if let LiteralValue::Double(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - DataType::Dictionary(k, v) if **v == DataType::Float64 => match **k { - DataType::UInt8 => { - let mut builder = - PrimitiveDictionaryBuilder::::new(); - for (_, _, val) in &to_insert { - if let LiteralValue::Double(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - DataType::UInt16 => { - let mut builder = - PrimitiveDictionaryBuilder::::new(); - for (_, _, val) in &to_insert { - if let LiteralValue::Double(v) = val { - builder.append_value(*v); - } else { - builder.append_null(); - } - } - Arc::new(builder.finish()) - } - _ => { - return Err(Error::UnsupportedDictionaryKeyType { - expect_oneof: vec![DataType::UInt8], - actual: *k.clone(), - }); - } - }, - dt => { - return Err(Error::ColumnDataTypeMismatch { - name: name.into(), - expect: DataType::Float64, - actual: dt.clone(), - }); + let options = array_options_for_type(field.data_type()); + let mut builder = Float64ArrayBuilder::new(options); + for (_, _, val) in &to_insert { + if let LiteralValue::Double(v) = val { + builder.append_value(v); + } else { + builder.append_null(); } } + builder + .finish() + .expect("double builder should produce array since optional=false") } else if name == consts::ATTRIBUTE_BOOL { // Note: Boolean Dictionaries are not standard/supported by simple builders let mut builder = arrow::array::BooleanBuilder::with_capacity(total_rows); From c949a48fc9c1dec517092f517db878b584a1f105 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 21 Jan 2026 01:01:58 +0000 Subject: [PATCH 23/25] use StringArrayAccessor and adaptive array builders in create_inserted_batch --- .../crates/pdata/src/otap/transform.rs | 55 +++---------------- 1 file changed, 7 insertions(+), 48 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index fe43c58089..3563e30412 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -3847,51 +3847,6 @@ fn extend_schema_for_inserts( Ok((new_batch, new_schema)) } -/// Helper to extract string key from a record batch key column at a given index. -/// Handles both plain Utf8 and Dictionary-encoded columns. -fn get_key_at_index(key_col: &ArrayRef, idx: usize) -> Option { - match key_col.data_type() { - DataType::Utf8 => key_col - .as_any() - .downcast_ref::() - .and_then(|arr| { - if arr.is_null(idx) { - None - } else { - Some(arr.value(idx).to_string()) - } - }), - DataType::Dictionary(k, _) => match **k { - DataType::UInt8 => key_col - .as_any() - .downcast_ref::>() - .and_then(|dict| { - if dict.is_null(idx) { - None - } else { - let values = dict.values().as_any().downcast_ref::()?; - let key = dict.keys().value(idx); - Some(values.value(key as usize).to_string()) - } - }), - DataType::UInt16 => key_col - .as_any() - .downcast_ref::>() - .and_then(|dict| { - if dict.is_null(idx) { - None - } else { - let values = dict.values().as_any().downcast_ref::()?; - let key = dict.keys().value(idx); - Some(values.value(key as usize).to_string()) - } - }), - _ => None, - }, - _ => None, - } -} - /// Returns `ArrayOptions` configured to match the given `DataType`. /// For dictionary types, configures the appropriate dictionary options. /// For native types (Utf8, Int64, Float64), returns options with no dictionary. @@ -3943,14 +3898,18 @@ fn create_inserted_batch( actual: parent_ids.data_type().clone(), })?; - // Build a set of (parent_id, key) pairs that already exist + // Build a set of (parent_id, key) pairs that already exist using StringArrayAccessor let mut existing_keys: BTreeMap> = BTreeMap::new(); if let Some(key_col) = current_batch.column_by_name(consts::ATTRIBUTE_KEY) { + let key_accessor = MaybeDictArrayAccessor::::try_new(key_col)?; for i in 0..current_batch.num_rows() { if !parent_ids_arr.is_null(i) { let parent = parent_ids_arr.value(i); - if let Some(key) = get_key_at_index(key_col, i) { - let _ = existing_keys.entry(parent).or_default().insert(key); + if let Some(key) = key_accessor.str_at(i) { + let _ = existing_keys + .entry(parent) + .or_default() + .insert(key.to_string()); } } } From 72ccdf6d8ec9dd6c470075df201faedc8d7788f5 Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 21 Jan 2026 01:08:50 +0000 Subject: [PATCH 24/25] Address feedback --- rust/otap-dataflow/crates/pdata/src/otap/transform.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 3563e30412..02399d8b90 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -3899,13 +3899,17 @@ fn create_inserted_batch( })?; // Build a set of (parent_id, key) pairs that already exist using StringArrayAccessor + let key_accessor = current_batch + .column_by_name(consts::ATTRIBUTE_KEY) + .map(MaybeDictArrayAccessor::::try_new) + .transpose()?; + let mut existing_keys: BTreeMap> = BTreeMap::new(); - if let Some(key_col) = current_batch.column_by_name(consts::ATTRIBUTE_KEY) { - let key_accessor = MaybeDictArrayAccessor::::try_new(key_col)?; + if let Some(ref accessor) = key_accessor { for i in 0..current_batch.num_rows() { if !parent_ids_arr.is_null(i) { let parent = parent_ids_arr.value(i); - if let Some(key) = key_accessor.str_at(i) { + if let Some(key) = accessor.str_at(i) { let _ = existing_keys .entry(parent) .or_default() From c19621a3f5df358ee192d022eda3798c141a8f7a Mon Sep 17 00:00:00 2001 From: Thomson Tan Date: Wed, 21 Jan 2026 01:27:21 +0000 Subject: [PATCH 25/25] Make create_inserted_batch generic --- .../crates/pdata/src/otap/transform.rs | 238 +++++++++++++++--- 1 file changed, 210 insertions(+), 28 deletions(-) diff --git a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs index 02399d8b90..73436fe5f1 100644 --- a/rust/otap-dataflow/crates/pdata/src/otap/transform.rs +++ b/rust/otap-dataflow/crates/pdata/src/otap/transform.rs @@ -695,7 +695,7 @@ pub fn transform_attributes_with_stats( let insert_needed = transform.insert.is_some() && schema.column_with_name(consts::PARENT_ID).is_some(); let attrs_record_batch_cow = if insert_needed { - let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; + let rb = materialize_parent_id_for_attributes_auto(attrs_record_batch)?; Cow::Owned(rb) } else { Cow::Borrowed(attrs_record_batch) @@ -725,7 +725,7 @@ pub fn transform_attributes_with_stats( let should_materialize_parent_ids = schema.column_with_name(consts::PARENT_ID).is_some(); let (attrs_record_batch, schema) = if should_materialize_parent_ids { - let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; + let rb = materialize_parent_id_for_attributes_auto(attrs_record_batch)?; let schema = rb.schema(); (rb, schema) } else { @@ -818,7 +818,7 @@ pub fn transform_attributes_with_stats( let should_materialize_parent_ids = schema.column_with_name(consts::PARENT_ID).is_some(); if should_materialize_parent_ids { - let rb = materialize_parent_id_for_attributes::(attrs_record_batch)?; + let rb = materialize_parent_id_for_attributes_auto(attrs_record_batch)?; let schema = rb.schema(); (rb, schema) } else { @@ -891,12 +891,27 @@ pub fn transform_attributes_with_stats( let (rb_extended, extended_schema) = extend_schema_for_inserts(&rb, needs_str, needs_int, needs_double, needs_bool)?; - let (new_rows, count) = create_inserted_batch( - &rb_extended, - original_parent_ids, - insert, - extended_schema.as_ref(), - )?; + let (new_rows, count) = match get_parent_id_value_type(original_parent_ids)? { + DataType::UInt16 => create_inserted_batch::( + &rb_extended, + original_parent_ids, + insert, + extended_schema.as_ref(), + )?, + DataType::UInt32 => create_inserted_batch::( + &rb_extended, + original_parent_ids, + insert, + extended_schema.as_ref(), + )?, + data_type => { + return Err(Error::ColumnDataTypeMismatch { + name: consts::PARENT_ID.into(), + expect: DataType::UInt16, // or UInt32 + actual: data_type, + }); + } + }; if count > 0 { let combined = arrow::compute::concat_batches(&extended_schema, &[rb_extended, new_rows]) @@ -3847,6 +3862,68 @@ fn extend_schema_for_inserts( Ok((new_batch, new_schema)) } +/// Get the value type for a parent ID column, handling both primitive and dictionary-encoded arrays. +/// Returns the underlying primitive type (UInt16 or UInt32). +fn get_parent_id_value_type(arr: &ArrayRef) -> Result { + match arr.data_type() { + DataType::UInt16 | DataType::UInt32 => Ok(arr.data_type().clone()), + DataType::Dictionary(_, v) => match **v { + DataType::UInt16 | DataType::UInt32 => Ok((**v).clone()), + _ => Err(Error::UnsupportedDictionaryValueType { + expect_oneof: vec![DataType::UInt16, DataType::UInt32], + actual: (**v).clone(), + }), + }, + _ => Err(Error::ColumnDataTypeMismatch { + name: consts::PARENT_ID.into(), + expect: DataType::UInt16, // or UInt32 + actual: arr.data_type().clone(), + }), + } +} + +/// Get the value type for a parent ID column from a schema field, handling both primitive and +/// dictionary-encoded types. +/// Returns the underlying primitive type (UInt16 or UInt32). +fn get_parent_id_value_type_from_schema( + schema: &arrow::datatypes::Schema, +) -> Result> { + let Some((_, field)) = schema.column_with_name(consts::PARENT_ID) else { + return Ok(None); + }; + match field.data_type() { + DataType::UInt16 | DataType::UInt32 => Ok(Some(field.data_type().clone())), + DataType::Dictionary(_, v) => match **v { + DataType::UInt16 | DataType::UInt32 => Ok(Some((**v).clone())), + _ => Err(Error::UnsupportedDictionaryValueType { + expect_oneof: vec![DataType::UInt16, DataType::UInt32], + actual: (**v).clone(), + }), + }, + _ => Err(Error::ColumnDataTypeMismatch { + name: consts::PARENT_ID.into(), + expect: DataType::UInt16, // or UInt32 + actual: field.data_type().clone(), + }), + } +} + +/// Materialize parent IDs with automatic type dispatch based on schema. +/// Returns the materialized batch or the original batch if no parent_id column exists. +fn materialize_parent_id_for_attributes_auto(record_batch: &RecordBatch) -> Result { + let schema = record_batch.schema(); + match get_parent_id_value_type_from_schema(&schema)? { + Some(DataType::UInt16) => materialize_parent_id_for_attributes::(record_batch), + Some(DataType::UInt32) => materialize_parent_id_for_attributes::(record_batch), + Some(other) => Err(Error::ColumnDataTypeMismatch { + name: consts::PARENT_ID.into(), + expect: DataType::UInt16, // or UInt32 + actual: other, + }), + None => Ok(record_batch.clone()), + } +} + /// Returns `ArrayOptions` configured to match the given `DataType`. /// For dictionary types, configures the appropriate dictionary options. /// For native types (Utf8, Int64, Float64), returns options with no dictionary. @@ -3883,20 +3960,24 @@ fn array_options_for_type(data_type: &DataType) -> ArrayOptions { /// According to the OTel collector spec, `insert` only inserts if the key does NOT already exist. /// This function checks existing (parent_id, key) pairs in the current record batch and only /// inserts new keys. -fn create_inserted_batch( +/// +/// This function is generic over `T: ParentId` to handle different parent ID types (u16, u32) +/// as well as dictionary-encoded parent IDs. +fn create_inserted_batch( current_batch: &RecordBatch, parent_ids: &ArrayRef, insert: &InsertTransform, schema: &arrow::datatypes::Schema, -) -> Result<(RecordBatch, usize)> { - let parent_ids_arr = parent_ids - .as_any() - .downcast_ref::>() - .ok_or_else(|| Error::ColumnDataTypeMismatch { - name: consts::PARENT_ID.into(), - expect: DataType::UInt16, - actual: parent_ids.data_type().clone(), - })?; +) -> Result<(RecordBatch, usize)> +where + T: ParentId, + ::ArrayType: ArrowPrimitiveType, + <::ArrayType as ArrowPrimitiveType>::Native: + Ord + std::hash::Hash + Copy + Default, +{ + // Use MaybeDictArrayAccessor to handle both primitive and dictionary-encoded parent IDs + let parent_ids_accessor = + MaybeDictArrayAccessor::>::try_new(parent_ids)?; // Build a set of (parent_id, key) pairs that already exist using StringArrayAccessor let key_accessor = current_batch @@ -3904,11 +3985,13 @@ fn create_inserted_batch( .map(MaybeDictArrayAccessor::::try_new) .transpose()?; - let mut existing_keys: BTreeMap> = BTreeMap::new(); + let mut existing_keys: BTreeMap< + <::ArrayType as ArrowPrimitiveType>::Native, + BTreeSet, + > = BTreeMap::new(); if let Some(ref accessor) = key_accessor { for i in 0..current_batch.num_rows() { - if !parent_ids_arr.is_null(i) { - let parent = parent_ids_arr.value(i); + if let Some(parent) = parent_ids_accessor.value_at(i) { if let Some(key) = accessor.str_at(i) { let _ = existing_keys .entry(parent) @@ -3921,9 +4004,9 @@ fn create_inserted_batch( // Get unique parents let mut unique_parents = BTreeSet::new(); - for i in 0..parent_ids_arr.len() { - if !parent_ids_arr.is_null(i) { - let _ = unique_parents.insert(parent_ids_arr.value(i)); + for i in 0..parent_ids_accessor.len() { + if let Some(parent) = parent_ids_accessor.value_at(i) { + let _ = unique_parents.insert(parent); } } @@ -3933,7 +4016,11 @@ fn create_inserted_batch( // Compute which (parent, key, value) tuples to actually insert // Only insert if the key doesn't already exist for that parent - let mut to_insert: Vec<(u16, &str, &LiteralValue)> = Vec::new(); + let mut to_insert: Vec<( + <::ArrayType as ArrowPrimitiveType>::Native, + &str, + &LiteralValue, + )> = Vec::new(); for &parent in &unique_parents { let parent_existing = existing_keys.get(&parent); for (key, val) in insert.entries.iter() { @@ -3951,8 +4038,8 @@ fn create_inserted_batch( return Ok((RecordBatch::new_empty(Arc::new(schema.clone())), 0)); } - // Build Parent ID column - let mut new_parent_ids = PrimitiveBuilder::::with_capacity(total_rows); + // Build Parent ID column using the same primitive type + let mut new_parent_ids = PrimitiveBuilder::::with_capacity(total_rows); for (parent, _, _) in &to_insert { new_parent_ids.append_value(*parent); } @@ -4783,4 +4870,99 @@ mod insert_tests { assert_eq!(stats.inserted_entries, 1); assert_eq!(result.num_rows(), 2); } + + #[test] + fn test_insert_with_u32_parent_ids() { + // Test that insert works with u32 parent IDs (used for metrics datapoint attributes, + // span link attributes, span event attributes). + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt32, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt32Array::from_iter_values(vec![0, 1])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec!["k1", "k2"])), + Arc::new(StringArray::from_iter_values(vec!["v1", "v2"])), + ], + ) + .unwrap(); + + // Insert "env"="prod" - should be inserted for both parents + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(BTreeMap::from([( + "env".into(), + LiteralValue::Str("prod".into()), + )]))), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + assert_eq!(stats.inserted_entries, 2); + assert_eq!(result.num_rows(), 4); + + // Verify parent ID column is still u32 + let parent_col = result.column_by_name(consts::PARENT_ID).unwrap(); + assert_eq!(parent_col.data_type(), &DataType::UInt32); + } + + #[test] + fn test_insert_with_u32_parent_ids_respects_existing() { + // Test that insert with u32 parent IDs doesn't overwrite existing keys + let schema = Arc::new(Schema::new(vec![ + Field::new(consts::PARENT_ID, DataType::UInt32, false), + Field::new(consts::ATTRIBUTE_TYPE, DataType::UInt8, false), + Field::new(consts::ATTRIBUTE_KEY, DataType::Utf8, false), + Field::new(consts::ATTRIBUTE_STR, DataType::Utf8, true), + ])); + + // Parent 0 has "existing" + let input = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(UInt32Array::from_iter_values(vec![0])), + Arc::new(UInt8Array::from_iter_values(vec![ + AttributeValueType::Str as u8, + ])), + Arc::new(StringArray::from_iter_values(vec!["existing"])), + Arc::new(StringArray::from_iter_values(vec!["original"])), + ], + ) + .unwrap(); + + // Try to insert "existing" with a different value - should be skipped + let tx = AttributesTransform { + rename: None, + delete: None, + insert: Some(InsertTransform::new(BTreeMap::from([( + "existing".into(), + LiteralValue::Str("should_not_overwrite".into()), + )]))), + }; + + let (result, stats) = transform_attributes_with_stats(&input, &tx).unwrap(); + + // No inserts because key already exists + assert_eq!(stats.inserted_entries, 0); + assert_eq!(result.num_rows(), 1); + + // Verify original value is preserved + let vals = result + .column_by_name(consts::ATTRIBUTE_STR) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(vals.value(0), "original"); + } }