-
Notifications
You must be signed in to change notification settings - Fork 265
/
Copy pathmerge_insert.rs
2097 lines (1912 loc) · 81.9 KB
/
merge_insert.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors
//! The merge insert operation merges a batch of new data into an existing batch of old data. This can be
//! used to implement a bulk update-or-insert (upsert) or find-or-create operation. It can also be used to
//! replace a specified region of data with new data (e.g. replace the data for the month of January)
//!
//! The terminology for this operation can be slightly confusing. We try and stick with the terminology from
//! SQL. The "target table" is the OLD data that already exists. The "source table" is the NEW data which is
//! being inserted into the dataset.
//!
//! In order for this operation to work we need to be able to match rows from the source table with rows in the
//! target table. For example, given a row we need to know if this is a brand new row or matches an existing row.
//!
//! This match condition is currently limited to an key-match. This means we consider a row to be a match if the
//! key columns are identical in both the source and the target. This means that you will need some kind of
//! meaningful key column to be able to perform a merge insert.
use std::{
collections::BTreeMap,
sync::{Arc, Mutex},
};
use arrow_array::{
cast::AsArray, types::UInt64Type, BooleanArray, RecordBatch, RecordBatchIterator, StructArray,
UInt64Array,
};
use arrow_schema::{DataType, Field, Schema};
use datafusion::{
execution::{
context::{SessionConfig, SessionContext},
memory_pool::MemoryConsumer,
},
logical_expr::{self, Expr, JoinType},
physical_plan::{
joins::{HashJoinExec, PartitionMode},
projection::ProjectionExec,
repartition::RepartitionExec,
stream::RecordBatchStreamAdapter,
union::UnionExec,
ColumnarValue, ExecutionPlan, PhysicalExpr, SendableRecordBatchStream,
},
prelude::DataFrame,
scalar::ScalarValue,
};
use lance_arrow::{interleave_batches, RecordBatchExt, SchemaExt};
use lance_datafusion::{
chunker::chunk_stream, dataframe::DataFrameExt, exec::get_session_context,
utils::reader_to_stream,
};
use datafusion_physical_expr::expressions::Column;
use futures::{
stream::{self},
Stream, StreamExt, TryStreamExt,
};
use lance_core::{
datatypes::{OnMissing, OnTypeMismatch, SchemaCompareOptions},
error::{box_error, InvalidInputSnafu},
utils::{futures::Capacity, tokio::get_num_compute_intensive_cpus},
Error, Result, ROW_ADDR, ROW_ADDR_FIELD, ROW_ID, ROW_ID_FIELD,
};
use lance_datafusion::{
exec::{execute_plan, LanceExecutionOptions, OneShotExec},
utils::StreamingWriteSource,
};
use lance_file::version::LanceFileVersion;
use lance_index::DatasetIndexExt;
use lance_table::format::{Fragment, Index};
use log::info;
use roaring::RoaringTreemap;
use snafu::{location, ResultExt};
use tokio::task::JoinSet;
use crate::{
datafusion::dataframe::SessionContextExt,
dataset::{
fragment::{FileFragment, FragReadConfig},
transaction::{Operation, Transaction},
write::open_writer,
},
index::DatasetIndexInternalExt,
io::exec::{
project, scalar_index::MapIndexExec, utils::ReplayExec, AddRowAddrExec, Planner, TakeExec,
},
Dataset,
};
use super::{write_fragments_internal, CommitBuilder, WriteParams};
// "update if" expressions typically compare fields from the source table to the target table.
// These tables have the same schema and so filter expressions need to differentiate. To do that
// we wrap the left side and the right side in a struct and make a single "combined schema"
fn combined_schema(schema: &Schema) -> Schema {
let target = Field::new("target", DataType::Struct(schema.fields.clone()), false);
let source = Field::new("source", DataType::Struct(schema.fields.clone()), false);
Schema::new(vec![source, target])
}
// This takes a double-wide table (e.g. the result of the outer join below) and takes the left
// half, puts it into a struct, then takes the right half, and puts that into a struct. This
// makes the table match the "combined schema" so we can apply an "update if" expression
fn unzip_batch(batch: &RecordBatch, schema: &Schema) -> RecordBatch {
// The schema of the combined batches will be:
// target_data_keys, target_data_non_keys, target_data_row_id, source_data_keys, source_data_non_keys
// The keys and non_keys on both sides will be equal
let num_fields = batch.num_columns();
debug_assert_eq!(num_fields % 2, 1);
let half_num_fields = num_fields / 2;
let row_id_col = num_fields - 1;
let source_arrays = batch.columns()[0..half_num_fields].to_vec();
let source = StructArray::new(schema.fields.clone(), source_arrays, None);
let target_arrays = batch.columns()[half_num_fields..row_id_col].to_vec();
let target = StructArray::new(schema.fields.clone(), target_arrays, None);
let combined_schema = combined_schema(schema);
RecordBatch::try_new(
Arc::new(combined_schema),
vec![Arc::new(source), Arc::new(target)],
)
.unwrap()
}
/// Describes how rows should be handled when there is no matching row in the source table
///
/// These are old rows which do not match any new data
#[derive(Debug, Clone, PartialEq)]
pub enum WhenNotMatchedBySource {
/// Do not delete rows from the target table
///
/// This can be used for a find-or-create or an upsert operation
Keep,
/// Delete all rows from target table that don't match a row in the source table
Delete,
/// Delete rows from the target table if there is no match AND the expression evaluates to true
///
/// This can be used to replace a region of data with new data
DeleteIf(Expr),
}
impl WhenNotMatchedBySource {
/// Create an instance of WhenNotMatchedBySource::DeleteIf from
/// an SQL filter string
///
/// This will parse the filter string (using the schema of the provided
/// dataset) and simplify the resulting expression
pub fn delete_if(dataset: &Dataset, expr: &str) -> Result<Self> {
let planner = Planner::new(Arc::new(dataset.schema().into()));
let expr = planner
.parse_filter(expr)
.map_err(box_error)
.context(InvalidInputSnafu {
location: location!(),
})?;
let expr = planner
.optimize_expr(expr)
.map_err(box_error)
.context(InvalidInputSnafu {
location: location!(),
})?;
Ok(Self::DeleteIf(expr))
}
}
/// Describes how rows should be handled when there is a match between the target table and source table
#[derive(Debug, Clone, PartialEq)]
pub enum WhenMatched {
/// The row is deleted from the target table and a new row is inserted based on the source table
///
/// This can be used to achieve upsert behavior
UpdateAll,
/// The row is kept unchanged
///
/// This can be used to achieve find-or-create behavior
DoNothing,
/// The row is updated (similar to UpdateAll) only for rows where the expression evaluates to
/// true
UpdateIf(Expr),
}
impl WhenMatched {
pub fn update_if(dataset: &Dataset, expr: &str) -> Result<Self> {
let dataset_schema: Schema = dataset.schema().into();
let combined_schema = combined_schema(&dataset_schema);
let planner = Planner::new(Arc::new(combined_schema));
let expr = planner
.parse_filter(expr)
.map_err(box_error)
.context(InvalidInputSnafu {
location: location!(),
})?;
let expr = planner
.optimize_expr(expr)
.map_err(box_error)
.context(InvalidInputSnafu {
location: location!(),
})?;
Ok(Self::UpdateIf(expr))
}
}
/// Describes how rows should be handled when there is no matching row in the target table
///
/// These are new rows which do not match any old data
pub enum WhenNotMatched {
/// The new row is inserted into the target table
///
/// This is used in both find-or-create and upsert operations
InsertAll,
/// The new row is ignored
DoNothing,
}
#[derive(Debug, Clone)]
struct MergeInsertParams {
// The column(s) to join on
on: Vec<String>,
// If true, then update all columns of the old data to the new data when there is a match
when_matched: WhenMatched,
// If true, then insert all columns of the new data when there is no match in the old data
insert_not_matched: bool,
// Controls whether data that is not matched by the source is deleted or not
delete_not_matched_by_source: WhenNotMatchedBySource,
}
/// A MergeInsertJob inserts new rows, deletes old rows, and updates existing rows all as
/// part of a single transaction.
pub struct MergeInsertJob {
// The column to merge the new data into
dataset: Arc<Dataset>,
// The parameters controlling how to merge the two streams
params: MergeInsertParams,
}
/// Build a merge insert operation.
///
/// This operation is similar to SQL's MERGE statement. It allows you to merge
/// new data with existing data.
///
/// Use the [MergeInsertBuilder] to construct an merge insert job. For example:
///
/// ```ignore
/// // find-or-create, insert new rows only
/// let builder = MergeInsertBuilder::new(dataset, vec!["my_key"]);
/// let dataset = builder
/// .build()?
/// .execute(new_data)
/// .await?;
///
/// // upsert, insert or update
/// let builder = MergeInsertBuilder::new(dataset, vec!["my_key"]);
/// let dataset = builder
/// .when_not_matched(WhenNotMatched::UpdateAll)
/// .build()?
/// .execute(new_data)
/// .await?;
///
/// // replace data for month=january
/// let builder = MergeInsertBuilder::new(dataset, vec!["my_key"]);
/// let dataset = builder
/// .when_not_matched(WhenNotMatched::UpdateAll)
/// .when_not_matched_by_source(
/// WhenNotMatchedBySource::DeleteIf(month_eq_jan)
/// )
/// .build()?
/// .execute(new_data)
/// .await?;
/// ```
///
#[derive(Debug, Clone)]
pub struct MergeInsertBuilder {
dataset: Arc<Dataset>,
params: MergeInsertParams,
}
impl MergeInsertBuilder {
/// Creates a new builder
///
/// By default this will build a job that has the same semantics as find-or-create
/// - matching rows will be kept as-is
/// - new rows in the new data will be inserted
/// - rows in the old data that do not match will be left as-is
///
/// Use the methods on this builder to customize that behavior
pub fn try_new(dataset: Arc<Dataset>, on: Vec<String>) -> Result<Self> {
if on.is_empty() {
return Err(Error::invalid_input(
"A merge insert operation must specify at least one on key",
location!(),
));
}
Ok(Self {
dataset,
params: MergeInsertParams {
on,
when_matched: WhenMatched::DoNothing,
insert_not_matched: true,
delete_not_matched_by_source: WhenNotMatchedBySource::Keep,
},
})
}
/// Specify what should happen when a target row matches a row in the source
pub fn when_matched(&mut self, behavior: WhenMatched) -> &mut Self {
self.params.when_matched = behavior;
self
}
/// Specify what should happen when a source row has no match in the target
///
/// These are typically "new rows"
pub fn when_not_matched(&mut self, behavior: WhenNotMatched) -> &mut Self {
self.params.insert_not_matched = match behavior {
WhenNotMatched::DoNothing => false,
WhenNotMatched::InsertAll => true,
};
self
}
/// Specify what should happen when a target row has no match in the source
///
/// These are typically "old rows"
pub fn when_not_matched_by_source(&mut self, behavior: WhenNotMatchedBySource) -> &mut Self {
self.params.delete_not_matched_by_source = behavior;
self
}
/// Crate a merge insert job
pub fn try_build(&mut self) -> Result<MergeInsertJob> {
if !self.params.insert_not_matched
&& self.params.when_matched == WhenMatched::DoNothing
&& self.params.delete_not_matched_by_source == WhenNotMatchedBySource::Keep
{
return Err(Error::invalid_input(
"The merge insert job is not configured to change the data in any way",
location!(),
));
}
Ok(MergeInsertJob {
dataset: self.dataset.clone(),
params: self.params.clone(),
})
}
}
enum SchemaComparison {
FullCompatible,
Subschema,
}
impl MergeInsertJob {
pub async fn execute_reader(
self,
source: impl StreamingWriteSource,
) -> Result<(Arc<Dataset>, MergeStats)> {
let stream = source.into_stream();
self.execute(stream).await
}
fn check_compatible_schema(&self, schema: &Schema) -> Result<SchemaComparison> {
let lance_schema: lance_core::datatypes::Schema = schema.try_into()?;
let is_compatible = lance_schema.check_compatible(
self.dataset.schema(),
&SchemaCompareOptions {
compare_dictionary: true,
..Default::default()
},
);
fn is_subschema(schema: &Schema, candidate: &Schema) -> bool {
// Schema::contains() cares about order, but we don't.
for field in candidate.fields() {
if !schema
.field_with_name(field.name())
.map(|f| f.contains(field))
.unwrap_or(false)
{
return false;
}
}
true
}
if let Err(e) = is_compatible {
// It might be a subschema
let dataset_arrow_schema = Schema::from(self.dataset.schema());
if is_subschema(&dataset_arrow_schema, schema) {
Ok(SchemaComparison::Subschema)
} else {
Err(e)
}
} else {
Ok(SchemaComparison::FullCompatible)
}
}
async fn join_key_as_scalar_index(&self) -> Result<Option<Index>> {
if self.params.on.len() != 1 {
// joining on more than one column
Ok(None)
} else {
let col = &self.params.on[0];
self.dataset.load_scalar_index_for_column(col).await
}
}
async fn create_indexed_scan_joined_stream(
&self,
source: SendableRecordBatchStream,
index: Index,
) -> Result<SendableRecordBatchStream> {
// This relies on a few non-standard physical operators and so we cannot use the
// datafusion dataframe API and need to construct the plan manually :'(
let schema = source.schema();
let add_row_addr = match self.check_compatible_schema(&schema)? {
SchemaComparison::FullCompatible => false,
SchemaComparison::Subschema => true,
};
// 1 - Input from user
let input = Arc::new(OneShotExec::new(source));
// 2 - Fork/Replay the input
// Regrettably, this needs to have unbounded capacity, and so we need to fully read
// the new data into memory. In the future, we can do better
let shared_input = Arc::new(ReplayExec::new(Capacity::Unbounded, input));
// 3 - Use the index to map input to row addresses
// First, we need to project to the key column
let field = schema.field_with_name(&self.params.on[0])?;
let index_mapper_input = Arc::new(project(
shared_input.clone(),
// schema for only the key join column
&Schema::new(vec![field.clone()]),
)?);
// Then we pass the key column into the index mapper
let index_column = self.params.on[0].clone();
let mut index_mapper: Arc<dyn ExecutionPlan> = Arc::new(MapIndexExec::new(
// create index from original data and key column
self.dataset.clone(),
index_column.clone(),
index_mapper_input,
));
// If requested, add row addresses to the output
if add_row_addr {
let pos = index_mapper.schema().fields().len(); // Add to end
index_mapper = Arc::new(AddRowAddrExec::try_new(
index_mapper,
self.dataset.clone(),
pos,
)?);
}
// 4 - Take the mapped row ids
let projection = self
.dataset
.empty_projection()
.union_arrow_schema(schema.as_ref(), OnMissing::Error)?;
let mut target = Arc::new(
TakeExec::try_new(
self.dataset.clone(),
index_mapper,
projection,
get_num_compute_intensive_cpus(),
)?
.unwrap(),
) as Arc<dyn ExecutionPlan>;
// 5 - Take puts the row id and row addr at the beginning. A full scan (used when there is
// no scalar index) puts the row id and addr at the end. We need to match these up so
// we reorder those columns at the end.
let schema = target.schema();
let mut columns = schema
.fields()
.iter()
.filter(|f| f.name() != ROW_ID && f.name() != ROW_ADDR)
.cloned()
.collect::<Vec<_>>();
columns.push(Arc::new(ROW_ID_FIELD.clone()));
if add_row_addr {
columns.push(Arc::new(ROW_ADDR_FIELD.clone()));
}
target = Arc::new(project(target, &Schema::new(columns))?);
let column_names = schema
.field_names()
.into_iter()
.filter(|name| name.as_str() != ROW_ID && name.as_str() != ROW_ADDR)
.collect::<Vec<_>>();
// 5a - We also need to scan any new unindexed data and union it in
let unindexed_fragments = self.dataset.unindexed_fragments(&index.name).await?;
if !unindexed_fragments.is_empty() {
let mut builder = self.dataset.scan();
if add_row_addr {
builder.with_row_address();
}
let unindexed_data = builder
.with_row_id()
.with_fragments(unindexed_fragments)
.project(&column_names)
.unwrap()
.create_plan()
.await?;
let unioned = UnionExec::new(vec![target, unindexed_data]);
// Enforce only 1 partition.
target = Arc::new(RepartitionExec::try_new(
Arc::new(unioned),
datafusion::physical_plan::Partitioning::RoundRobinBatch(1),
)?);
}
// We need to prefix the fields in the target with target_ so that we don't have any duplicate
// field names (DF doesn't support this as of version 44)
target = Self::prefix_columns_phys(target, "target_");
// 6 - Finally, join the input (source table) with the taken data (target table)
let source_key = Column::new_with_schema(&index_column, shared_input.schema().as_ref())?;
let target_key = Column::new_with_schema(
&format!("target_{}", index_column),
target.schema().as_ref(),
)?;
let joined = Arc::new(
HashJoinExec::try_new(
shared_input,
target,
vec![(Arc::new(source_key), Arc::new(target_key))],
None,
&JoinType::Full,
None,
PartitionMode::CollectLeft,
true,
)
.unwrap(),
);
execute_plan(
joined,
LanceExecutionOptions {
use_spilling: true,
..Default::default()
},
)
}
fn prefix_columns(df: DataFrame, prefix: &str) -> DataFrame {
let schema = df.schema();
let columns = schema
.fields()
.iter()
.map(|f| {
// Need to "quote" the column name so it gets interpreted case-sensitively
logical_expr::col(format!("\"{}\"", f.name())).alias(format!(
"{}{}",
prefix,
f.name()
))
})
.collect::<Vec<_>>();
df.select(columns).unwrap()
}
fn prefix_columns_phys(inp: Arc<dyn ExecutionPlan>, prefix: &str) -> Arc<dyn ExecutionPlan> {
let schema = inp.schema();
let exprs = schema
.fields()
.iter()
.enumerate()
.map(|(idx, f)| {
let col = Arc::new(Column::new(f.name(), idx)) as Arc<dyn PhysicalExpr>;
let new_name = format!("{}{}", prefix, f.name());
(col, new_name)
})
.collect::<Vec<_>>();
Arc::new(ProjectionExec::try_new(exprs, inp).unwrap())
}
// If the join keys are not indexed then we need to do a full scan of the table
async fn create_full_table_joined_stream(
&self,
source: SendableRecordBatchStream,
) -> Result<SendableRecordBatchStream> {
let session_config = SessionConfig::default().with_target_partitions(1);
let session_ctx = SessionContext::new_with_config(session_config);
let schema = source.schema();
let new_data = session_ctx.read_one_shot(source)?;
let join_cols = self
.params
.on // columns to join on
.iter()
.map(|c| c.as_str())
.collect::<Vec<_>>(); // vector of strings of col names to join
let target_cols = self
.params
.on
.iter()
.map(|c| format!("target_{}", c))
.collect::<Vec<_>>();
let target_cols = target_cols.iter().map(|s| s.as_str()).collect::<Vec<_>>();
match self.check_compatible_schema(&schema)? {
SchemaComparison::FullCompatible => {
let existing = session_ctx.read_lance(self.dataset.clone(), true, false)?;
// We need to rename the columns from the target table so that they don't conflict with the source table
let existing = Self::prefix_columns(existing, "target_");
let joined =
new_data.join(existing, JoinType::Full, &join_cols, &target_cols, None)?; // full join
Ok(joined.execute_stream().await?)
}
SchemaComparison::Subschema => {
let existing = session_ctx.read_lance(self.dataset.clone(), true, true)?;
let columns = schema
.field_names()
.iter()
.map(|s| s.as_str())
.chain([ROW_ID, ROW_ADDR])
.collect::<Vec<_>>();
let projected = existing.select_columns(&columns)?;
// We need to rename the columns from the target table so that they don't conflict with the source table
let projected = Self::prefix_columns(projected, "target_");
// We aren't supporting inserts or deletes right now, so we can use inner join
let join_type = if self.params.insert_not_matched {
JoinType::Left
} else {
JoinType::Inner
};
let joined = new_data.join(projected, join_type, &join_cols, &target_cols, None)?;
Ok(joined.execute_stream().await?)
}
}
}
/// Join the source and target data streams
///
/// If there is a scalar index on the join key, we can use it to do an indexed join. Otherwise we need to do
/// a full outer join.
///
/// Datafusion doesn't allow duplicate column names so during this join we rename the columns from target and
/// prefix them with _target.
async fn create_joined_stream(
&self,
source: SendableRecordBatchStream,
) -> Result<SendableRecordBatchStream> {
// We need to do a full index scan if we're deleting source data
let can_use_scalar_index = matches!(
self.params.delete_not_matched_by_source, // this value marks behavior for rows in target that are not matched by the source. Value assigned earlier.
WhenNotMatchedBySource::Keep
);
if can_use_scalar_index {
// keeping unmatched rows, no deletion
if let Some(index) = self.join_key_as_scalar_index().await? {
self.create_indexed_scan_joined_stream(source, index).await
} else {
self.create_full_table_joined_stream(source).await
}
} else {
info!("The merge insert operation is configured to delete rows from the target table, this requires a potentially costly full table scan");
self.create_full_table_joined_stream(source).await
}
}
async fn update_fragments(
dataset: Arc<Dataset>,
source: SendableRecordBatchStream,
) -> Result<(Vec<Fragment>, Vec<Fragment>)> {
// Expected source schema: _rowaddr, updated_cols*
use datafusion::logical_expr::{col, lit};
let session_ctx = get_session_context(LanceExecutionOptions {
use_spilling: true,
..Default::default()
});
let mut group_stream = session_ctx
.read_one_shot(source)?
.with_column("_fragment_id", col(ROW_ADDR) >> lit(32))?
.sort(vec![col(ROW_ADDR).sort(true, true)])?
.group_by_stream(&["_fragment_id"])
.await?;
// Can update the fragments in parallel.
let updated_fragments = Arc::new(Mutex::new(Vec::new()));
let new_fragments = Arc::new(Mutex::new(Vec::new()));
let mut tasks = JoinSet::new();
let task_limit = dataset.object_store().io_parallelism();
let mut reservation =
MemoryConsumer::new("MergeInsert").register(session_ctx.task_ctx().memory_pool());
while let Some((frag_id, batches)) = group_stream.next().await.transpose()? {
async fn handle_fragment(
dataset: Arc<Dataset>,
fragment: FileFragment,
mut metadata: Fragment,
mut batches: Vec<RecordBatch>,
updated_fragments: Arc<Mutex<Vec<Fragment>>>,
reservation_size: usize,
) -> Result<usize> {
// batches still have _rowaddr
let write_schema = batches[0].schema().as_ref().without_column(ROW_ADDR);
let write_schema = dataset.local_schema().project_by_schema(
&write_schema,
OnMissing::Error,
OnTypeMismatch::Error,
)?;
let updated_rows: usize = batches.iter().map(|batch| batch.num_rows()).sum();
if Some(updated_rows) == metadata.physical_rows {
// All rows have been updated and there are no deletions. So we
// don't need to merge in existing values.
// Also, because we already sorted by row address, the rows
// will be in the correct order.
let data_storage_version = dataset
.manifest()
.data_storage_format
.lance_file_version()?;
let mut writer = open_writer(
dataset.object_store(),
&write_schema,
&dataset.base,
data_storage_version,
None,
)
.await?;
// We need to remove rowaddr before writing.
batches
.iter_mut()
.try_for_each(|batch| match batch.drop_column(ROW_ADDR) {
Ok(b) => {
*batch = b;
Ok(())
}
Err(e) => Err(e),
})?;
if data_storage_version == LanceFileVersion::Legacy {
// Need to match the existing batch size exactly, otherwise
// we'll get errors.
let reader = fragment
.open(
dataset.schema(),
FragReadConfig::default().with_row_address(true),
None,
)
.await?;
let batch_size = reader.legacy_num_rows_in_batch(0).unwrap();
let stream = stream::iter(batches.into_iter().map(Ok));
let stream = Box::pin(RecordBatchStreamAdapter::new(
Arc::new((&write_schema).into()),
stream,
));
let mut stream = chunk_stream(stream, batch_size as usize);
while let Some(chunk) = stream.next().await {
writer.write(&chunk?).await?;
}
} else {
writer.write(batches.as_slice()).await?;
}
let (_num_rows, data_file) = writer.finish().await?;
metadata.files.push(data_file);
updated_fragments.lock().unwrap().push(metadata);
} else {
// TODO: we could skip scanning row addresses we don't need.
let update_schema = batches[0].schema();
let read_columns = update_schema.field_names();
let mut updater = fragment
.updater(
Some(&read_columns),
Some((write_schema, dataset.schema().clone())),
None,
)
.await?;
// We will use interleave to update the rows. The first batch
// will be the original source data, and all subsequent batches
// will be updates.
let mut source_batches = Vec::with_capacity(batches.len() + 1);
source_batches.push(batches[0].clone()); // placeholder for source data
for batch in &batches {
source_batches.push(batch.drop_column(ROW_ADDR)?);
}
// This function is here to help rustc with lifetimes.
fn get_row_addr_iter(
batches: &[RecordBatch],
) -> impl Iterator<Item = (u64, (usize, usize))> + '_ + Send
{
batches.iter().enumerate().flat_map(|(batch_idx, batch)| {
// The index in source batches will be one more.
let batch_idx = batch_idx + 1;
let row_addrs = batch
.column_by_name(ROW_ADDR)
.unwrap()
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
row_addrs
.values()
.iter()
.enumerate()
.map(move |(offset, row_addr)| (*row_addr, (batch_idx, offset)))
})
}
let mut updated_row_addr_iter = get_row_addr_iter(&batches).peekable();
while let Some(batch) = updater.next().await? {
source_batches[0] =
batch.project_by_schema(source_batches[1].schema().as_ref())?;
let original_row_addrs = batch
.column_by_name(ROW_ADDR)
.unwrap()
.as_any()
.downcast_ref::<UInt64Array>()
.unwrap();
let indices = original_row_addrs
.values()
.into_iter()
.enumerate()
.map(|(original_offset, row_addr)| {
match updated_row_addr_iter.peek() {
Some((updated_row_addr, _))
if *updated_row_addr == *row_addr =>
{
updated_row_addr_iter.next().unwrap().1
}
// If we have passed the next updated row address, something went wrong.
Some((updated_row_addr, _)) => {
debug_assert!(
*updated_row_addr > *row_addr,
"Got updated row address that is not in the original batch"
);
(0, original_offset)
}
_ => (0, original_offset),
}
})
.collect::<Vec<_>>();
let updated_batch = interleave_batches(&source_batches, &indices)?;
updater.update(updated_batch).await?;
}
let updated_fragment = updater.finish().await?;
updated_fragments.lock().unwrap().push(updated_fragment);
}
Ok(reservation_size)
}
async fn handle_new_fragments(
dataset: Arc<Dataset>,
batches: Vec<RecordBatch>,
new_fragments: Arc<Mutex<Vec<Fragment>>>,
reservation_size: usize,
) -> Result<usize> {
// Batches still have _rowaddr (used elsewhere to merge with existing data)
// We need to remove it before writing to Lance files.
let num_fields = batches[0].schema().fields().len();
let mut projection = Vec::with_capacity(num_fields - 1);
for (i, field) in batches[0].schema().fields().iter().enumerate() {
if field.name() != ROW_ADDR {
projection.push(i);
}
}
let write_schema = Arc::new(batches[0].schema().project(&projection).unwrap());
let batches = batches
.into_iter()
.map(move |batch| batch.project(&projection));
let reader = RecordBatchIterator::new(batches, write_schema.clone());
let stream = reader_to_stream(Box::new(reader));
let write_schema = dataset.schema().project_by_schema(
write_schema.as_ref(),
OnMissing::Error,
OnTypeMismatch::Error,
)?;
let fragments = write_fragments_internal(
Some(dataset.as_ref()),
dataset.object_store.clone(),
&dataset.base,
write_schema,
stream,
Default::default(), // TODO: support write params.
)
.await?;
new_fragments.lock().unwrap().extend(fragments.default.0);
Ok(reservation_size)
}
// We shouldn't need much more memory beyond what is already in the batches.
let mut memory_size = batches
.iter()
.map(|batch| batch.get_array_memory_size())
.sum();
loop {
let have_additional_cpus = tasks.len() < task_limit;
if have_additional_cpus {
if reservation.try_grow(memory_size).is_ok() {
break;
} else if tasks.is_empty() {
// If there are no tasks running, we can bypass the pool limits.
// This lets us handle the case where we have a single large batch.
memory_size = 0;
break;
}
// If we can't grow the reservation, we will wait for a task to finish
}
if let Some(res) = tasks.join_next().await {
let size = res??;
reservation.shrink(size);
}
}
match frag_id.first() {
Some(ScalarValue::UInt64(Some(frag_id))) => {
let frag_id = *frag_id;
let fragment =
dataset
.get_fragment(frag_id as usize)
.ok_or_else(|| Error::Internal {
message: format!(
"Got non-existent fragment id from merge result: {}",
frag_id
),
location: location!(),
})?;
let metadata = fragment.metadata.clone();
let fut = handle_fragment(
dataset.clone(),
fragment,
metadata,
batches,
updated_fragments.clone(),
memory_size,
);
tasks.spawn(fut);
}
Some(ScalarValue::Null | ScalarValue::UInt64(None)) => {
let fut = handle_new_fragments(
dataset.clone(),
batches,
new_fragments.clone(),
memory_size,
);
tasks.spawn(fut);
}
_ => {
return Err(Error::Internal {
message: format!("Got non-fragment id from merge result: {:?}", frag_id),
location: location!(),
});
}
};
}
while let Some(res) = tasks.join_next().await {
let size = res??;
reservation.shrink(size);
}
let mut updated_fragments = Arc::try_unwrap(updated_fragments)
.unwrap()
.into_inner()
.unwrap();
// Collect the updated fragments, and map the field ids. Tombstone old ones
// as needed.
for fragment in &mut updated_fragments {
let updated_fields = fragment.files.last().unwrap().fields.clone();
for data_file in &mut fragment.files.iter_mut().rev().skip(1) {
for field in &mut data_file.fields {
if updated_fields.contains(field) {
// Tombstone these fields
*field = -2;
}
}
}
}
let new_fragments = Arc::try_unwrap(new_fragments)
.unwrap()
.into_inner()
.unwrap();
Ok((updated_fragments, new_fragments))
}
/// Executes the merge insert job
///
/// This will take in the source, merge it with the existing target data, and insert new