deltacat 1.1.13__py3-none-any.whl → 1.1.15__py3-none-any.whl

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
Files changed (25) hide show
  1. deltacat/__init__.py +1 -1
  2. deltacat/compute/compactor/compaction_session.py +3 -2
  3. deltacat/compute/compactor/model/compact_partition_params.py +11 -1
  4. deltacat/compute/compactor/model/compaction_session_audit_info.py +2 -2
  5. deltacat/compute/compactor/model/delta_annotated.py +2 -4
  6. deltacat/compute/compactor/steps/hash_bucket.py +2 -3
  7. deltacat/compute/compactor_v2/compaction_session.py +26 -27
  8. deltacat/compute/compactor_v2/constants.py +4 -0
  9. deltacat/compute/compactor_v2/private/__init__.py +0 -0
  10. deltacat/compute/compactor_v2/private/compaction_utils.py +753 -0
  11. deltacat/compute/compactor_v2/steps/merge.py +0 -3
  12. deltacat/compute/compactor_v2/utils/delta.py +2 -3
  13. deltacat/compute/compactor_v2/utils/io.py +0 -2
  14. deltacat/compute/compactor_v2/utils/merge.py +0 -1
  15. deltacat/tests/compute/compact_partition_multiple_rounds_test_cases.py +855 -0
  16. deltacat/tests/compute/compactor_v2/test_compaction_session.py +1 -1
  17. deltacat/tests/compute/test_compact_partition_multiple_rounds.py +330 -0
  18. deltacat/tests/compute/test_compact_partition_rebase.py +1 -1
  19. deltacat/tests/compute/test_util_create_table_deltas_repo.py +118 -0
  20. deltacat/tests/local_deltacat_storage/__init__.py +8 -5
  21. {deltacat-1.1.13.dist-info → deltacat-1.1.15.dist-info}/METADATA +1 -1
  22. {deltacat-1.1.13.dist-info → deltacat-1.1.15.dist-info}/RECORD +25 -21
  23. {deltacat-1.1.13.dist-info → deltacat-1.1.15.dist-info}/LICENSE +0 -0
  24. {deltacat-1.1.13.dist-info → deltacat-1.1.15.dist-info}/WHEEL +0 -0
  25. {deltacat-1.1.13.dist-info → deltacat-1.1.15.dist-info}/top_level.txt +0 -0
@@ -0,0 +1,753 @@
1
+ import numpy as np
2
+ import functools
3
+ import logging
4
+ import ray
5
+ import time
6
+ import json
7
+ from math import ceil
8
+
9
+ from deltacat.compute.compactor import (
10
+ PyArrowWriteResult,
11
+ HighWatermark,
12
+ RoundCompletionInfo,
13
+ )
14
+ from deltacat.aws import s3u as s3_utils
15
+ from deltacat.compute.compactor.model.compactor_version import CompactorVersion
16
+ from deltacat.compute.compactor_v2.model.evaluate_compaction_result import (
17
+ ExecutionCompactionResult,
18
+ )
19
+ from deltacat.compute.compactor_v2.model.merge_file_group import (
20
+ RemoteMergeFileGroupsProvider,
21
+ )
22
+ from deltacat.compute.compactor_v2.model.hash_bucket_input import HashBucketInput
23
+
24
+ from deltacat import logs
25
+ from deltacat.compute.compactor_v2.model.merge_input import MergeInput
26
+ from deltacat.compute.compactor_v2.model.merge_result import MergeResult
27
+ from deltacat.compute.compactor_v2.model.hash_bucket_result import HashBucketResult
28
+ from deltacat.compute.compactor.model.materialize_result import MaterializeResult
29
+ from deltacat.compute.compactor_v2.utils.merge import (
30
+ generate_local_merge_input,
31
+ )
32
+ from deltacat.compute.compactor_v2.utils.task_options import (
33
+ hash_bucket_resource_options_provider,
34
+ )
35
+ from deltacat.compute.compactor.utils import round_completion_file as rcf
36
+ from deltacat.compute.compactor import DeltaAnnotated
37
+ from deltacat.compute.compactor_v2.utils.delta import contains_delete_deltas
38
+ from deltacat.compute.compactor_v2.deletes.delete_strategy import (
39
+ DeleteStrategy,
40
+ )
41
+ from deltacat.compute.compactor_v2.deletes.delete_file_envelope import (
42
+ DeleteFileEnvelope,
43
+ )
44
+ from deltacat.compute.compactor_v2.deletes.utils import prepare_deletes
45
+
46
+ from deltacat.storage import (
47
+ Delta,
48
+ DeltaType,
49
+ DeltaLocator,
50
+ Partition,
51
+ Manifest,
52
+ Stream,
53
+ StreamLocator,
54
+ )
55
+ from deltacat.compute.compactor.model.compact_partition_params import (
56
+ CompactPartitionParams,
57
+ )
58
+ from deltacat.utils.ray_utils.concurrency import (
59
+ invoke_parallel,
60
+ task_resource_options_provider,
61
+ )
62
+ from deltacat.compute.compactor_v2.steps import merge as mg
63
+ from deltacat.compute.compactor_v2.steps import hash_bucket as hb
64
+ from deltacat.compute.compactor_v2.utils import io
65
+
66
+ from typing import List, Optional
67
+ from collections import defaultdict
68
+ from deltacat.compute.compactor.model.compaction_session_audit_info import (
69
+ CompactionSessionAuditInfo,
70
+ )
71
+ from deltacat.compute.compactor_v2.utils.task_options import (
72
+ merge_resource_options_provider,
73
+ local_merge_resource_options_provider,
74
+ )
75
+
76
+
77
+ logger = logs.configure_deltacat_logger(logging.getLogger(__name__))
78
+
79
+
80
+ def _fetch_compaction_metadata(
81
+ params: CompactPartitionParams,
82
+ ) -> tuple[Optional[Manifest], Optional[RoundCompletionInfo]]:
83
+
84
+ # read the results from any previously completed compaction round
85
+ round_completion_info: Optional[RoundCompletionInfo] = None
86
+ high_watermark: Optional[HighWatermark] = None
87
+ previous_compacted_delta_manifest: Optional[Manifest] = None
88
+
89
+ if not params.rebase_source_partition_locator:
90
+ round_completion_info = rcf.read_round_completion_file(
91
+ params.compaction_artifact_s3_bucket,
92
+ params.source_partition_locator,
93
+ params.destination_partition_locator,
94
+ **params.s3_client_kwargs,
95
+ )
96
+ if not round_completion_info:
97
+ logger.info(
98
+ "Both rebase partition and round completion file not found. Performing an entire backfill on source."
99
+ )
100
+ else:
101
+ compacted_delta_locator = round_completion_info.compacted_delta_locator
102
+
103
+ previous_compacted_delta_manifest = (
104
+ params.deltacat_storage.get_delta_manifest(
105
+ compacted_delta_locator, **params.deltacat_storage_kwargs
106
+ )
107
+ )
108
+
109
+ high_watermark = round_completion_info.high_watermark
110
+ logger.info(f"Setting round completion high watermark: {high_watermark}")
111
+ assert (
112
+ params.hash_bucket_count == round_completion_info.hash_bucket_count
113
+ ), (
114
+ "The hash bucket count has changed. "
115
+ "Kindly run rebase compaction and trigger incremental again. "
116
+ f"Hash Bucket count in RCF={round_completion_info.hash_bucket_count} "
117
+ f"not equal to Hash bucket count in args={params.hash_bucket_count}."
118
+ )
119
+
120
+ logger.info(f"Round completion file: {round_completion_info}")
121
+ return (
122
+ previous_compacted_delta_manifest,
123
+ round_completion_info,
124
+ )
125
+
126
+
127
+ def _build_uniform_deltas(
128
+ params: CompactPartitionParams,
129
+ mutable_compaction_audit: CompactionSessionAuditInfo,
130
+ input_deltas: List[Delta],
131
+ delta_discovery_start: float,
132
+ ) -> tuple[List[DeltaAnnotated], DeleteStrategy, List[DeleteFileEnvelope], Partition]:
133
+
134
+ delete_strategy: Optional[DeleteStrategy] = None
135
+ delete_file_envelopes: Optional[List[DeleteFileEnvelope]] = None
136
+ delete_file_size_bytes: int = 0
137
+ if contains_delete_deltas(input_deltas):
138
+ input_deltas, delete_file_envelopes, delete_strategy = prepare_deletes(
139
+ params, input_deltas
140
+ )
141
+ for delete_file_envelope in delete_file_envelopes:
142
+ delete_file_size_bytes += delete_file_envelope.table_size_bytes
143
+ logger.info(
144
+ f" Input deltas contain {DeltaType.DELETE}-type deltas. Total delete file size={delete_file_size_bytes}."
145
+ f" Total length of delete file envelopes={len(delete_file_envelopes)}"
146
+ )
147
+ uniform_deltas: List[DeltaAnnotated] = io.create_uniform_input_deltas(
148
+ input_deltas=input_deltas,
149
+ hash_bucket_count=params.hash_bucket_count,
150
+ compaction_audit=mutable_compaction_audit,
151
+ deltacat_storage=params.deltacat_storage,
152
+ previous_inflation=params.previous_inflation,
153
+ min_delta_bytes=params.min_delta_bytes_in_batch,
154
+ min_file_counts=params.min_files_in_batch,
155
+ # disable input split during rebase as the rebase files are already uniform
156
+ enable_input_split=params.rebase_source_partition_locator is None,
157
+ deltacat_storage_kwargs=params.deltacat_storage_kwargs,
158
+ )
159
+ delta_discovery_end: float = time.monotonic()
160
+
161
+ mutable_compaction_audit.set_uniform_deltas_created(len(uniform_deltas))
162
+ mutable_compaction_audit.set_delta_discovery_time_in_seconds(
163
+ delta_discovery_end - delta_discovery_start
164
+ )
165
+
166
+ s3_utils.upload(
167
+ mutable_compaction_audit.audit_url,
168
+ str(json.dumps(mutable_compaction_audit)),
169
+ **params.s3_client_kwargs,
170
+ )
171
+
172
+ return (
173
+ uniform_deltas,
174
+ delete_strategy,
175
+ delete_file_envelopes,
176
+ )
177
+
178
+
179
+ def _group_uniform_deltas(
180
+ params: CompactPartitionParams, uniform_deltas: List[DeltaAnnotated]
181
+ ) -> List[List[DeltaAnnotated]]:
182
+ num_deltas = len(uniform_deltas)
183
+ num_rounds = params.num_rounds
184
+ if num_rounds == 1:
185
+ return [uniform_deltas]
186
+ assert (
187
+ num_rounds > 0
188
+ ), f"num_rounds parameter should be greater than zero but is {params.num_rounds}"
189
+ assert (
190
+ num_rounds <= num_deltas
191
+ ), f"{params.num_rounds} rounds should be less than the number of uniform deltas, which is {len(uniform_deltas)}"
192
+ size = ceil(num_deltas / num_rounds)
193
+ uniform_deltas_grouped = list(
194
+ map(
195
+ lambda x: uniform_deltas[x * size : x * size + size],
196
+ list(range(num_rounds)),
197
+ )
198
+ )
199
+ num_deltas_after_grouping = sum(len(sublist) for sublist in uniform_deltas_grouped)
200
+ assert (
201
+ num_deltas_after_grouping == num_deltas
202
+ ), f"uniform_deltas_grouped expected to have {num_deltas} deltas, but has {num_deltas_after_grouping}"
203
+ return uniform_deltas_grouped
204
+
205
+
206
+ def _stage_new_partition(params: CompactPartitionParams) -> Partition:
207
+ compacted_stream_locator: Optional[
208
+ StreamLocator
209
+ ] = params.destination_partition_locator.stream_locator
210
+ compacted_stream: Stream = params.deltacat_storage.get_stream(
211
+ compacted_stream_locator.namespace,
212
+ compacted_stream_locator.table_name,
213
+ compacted_stream_locator.table_version,
214
+ **params.deltacat_storage_kwargs,
215
+ )
216
+ compacted_partition: Partition = params.deltacat_storage.stage_partition(
217
+ compacted_stream,
218
+ params.destination_partition_locator.partition_values,
219
+ **params.deltacat_storage_kwargs,
220
+ )
221
+ return compacted_partition
222
+
223
+
224
+ def _run_hash_and_merge(
225
+ params: CompactPartitionParams,
226
+ uniform_deltas: List[DeltaAnnotated],
227
+ round_completion_info: RoundCompletionInfo,
228
+ delete_strategy: Optional[DeleteStrategy],
229
+ delete_file_envelopes: Optional[DeleteFileEnvelope],
230
+ mutable_compaction_audit: CompactionSessionAuditInfo,
231
+ previous_compacted_delta_manifest: Optional[Manifest],
232
+ compacted_partition: Partition,
233
+ ) -> List[MergeResult]:
234
+ telemetry_time_hb = 0
235
+ total_input_records_count = np.int64(0)
236
+ total_hb_record_count = np.int64(0)
237
+ if params.hash_bucket_count == 1:
238
+ logger.info("Hash bucket count set to 1. Running local merge")
239
+ merge_start: float = time.monotonic()
240
+ merge_results, total_input_records_count = _run_local_merge(
241
+ params,
242
+ uniform_deltas,
243
+ compacted_partition,
244
+ round_completion_info,
245
+ delete_strategy,
246
+ delete_file_envelopes,
247
+ mutable_compaction_audit,
248
+ previous_compacted_delta_manifest,
249
+ total_input_records_count,
250
+ )
251
+ merge_invoke_end = time.monotonic()
252
+ else:
253
+ # hash bucket
254
+ hb_start = time.monotonic()
255
+ all_hash_group_idx_to_obj_id = defaultdict(list)
256
+ all_hash_group_idx_to_size_bytes = defaultdict(int)
257
+ all_hash_group_idx_to_num_rows = defaultdict(int)
258
+ (hb_results, hb_invoke_end) = _hash_bucket(params, uniform_deltas)
259
+ hb_end = time.monotonic()
260
+
261
+ # we use time.time() here because time.monotonic() has no reference point
262
+ # whereas time.time() measures epoch seconds. Hence, it will be reasonable
263
+ # to compare time.time()s captured in different nodes.
264
+ hb_results_retrieved_at = time.time()
265
+
266
+ telemetry_time_hb = mutable_compaction_audit.save_step_stats(
267
+ CompactionSessionAuditInfo.HASH_BUCKET_STEP_NAME,
268
+ hb_results,
269
+ hb_results_retrieved_at,
270
+ hb_invoke_end - hb_start,
271
+ hb_end - hb_start,
272
+ )
273
+
274
+ s3_utils.upload(
275
+ mutable_compaction_audit.audit_url,
276
+ str(json.dumps(mutable_compaction_audit)),
277
+ **params.s3_client_kwargs,
278
+ )
279
+
280
+ hb_data_processed_size_bytes = np.int64(0)
281
+
282
+ # initialize all hash groups
283
+ for hb_group in range(params.hash_group_count):
284
+ all_hash_group_idx_to_num_rows[hb_group] = 0
285
+ all_hash_group_idx_to_obj_id[hb_group] = []
286
+ all_hash_group_idx_to_size_bytes[hb_group] = 0
287
+
288
+ for hb_result in hb_results:
289
+ hb_data_processed_size_bytes += hb_result.hb_size_bytes
290
+ total_input_records_count += hb_result.hb_record_count
291
+ for hash_group_index, object_id_size_tuple in enumerate(
292
+ hb_result.hash_bucket_group_to_obj_id_tuple
293
+ ):
294
+ if object_id_size_tuple:
295
+ all_hash_group_idx_to_obj_id[hash_group_index].append(
296
+ object_id_size_tuple[0],
297
+ )
298
+ all_hash_group_idx_to_size_bytes[
299
+ hash_group_index
300
+ ] += object_id_size_tuple[1].item()
301
+ all_hash_group_idx_to_num_rows[
302
+ hash_group_index
303
+ ] += object_id_size_tuple[2].item()
304
+ logger.info(
305
+ f"Got {total_input_records_count} hash bucket records from hash bucketing step..."
306
+ )
307
+
308
+ total_hb_record_count = total_input_records_count
309
+ mutable_compaction_audit.set_hash_bucket_processed_size_bytes(
310
+ hb_data_processed_size_bytes.item()
311
+ )
312
+
313
+ # BSP Step 2: Merge
314
+ # NOTE: DELETE-type deltas are stored in Plasma object store
315
+ # in prepare_deletes and therefore don't need to included
316
+ # in merge task resource estimation
317
+ merge_start = time.monotonic()
318
+ merge_results, merge_invoke_end = _merge(
319
+ params,
320
+ task_resource_options_provider,
321
+ merge_resource_options_provider,
322
+ all_hash_group_idx_to_size_bytes,
323
+ all_hash_group_idx_to_num_rows,
324
+ round_completion_info,
325
+ previous_compacted_delta_manifest,
326
+ all_hash_group_idx_to_obj_id,
327
+ compacted_partition,
328
+ delete_strategy,
329
+ delete_file_envelopes,
330
+ )
331
+ logger.info(f"Got {len(merge_results)} merge results.")
332
+
333
+ merge_results_retrieved_at: float = time.time()
334
+ merge_end: float = time.monotonic()
335
+
336
+ total_dd_record_count = sum([ddr.deduped_record_count for ddr in merge_results])
337
+ total_deleted_record_count = sum(
338
+ [ddr.deleted_record_count for ddr in merge_results]
339
+ )
340
+ logger.info(
341
+ f"Deduped {total_dd_record_count} records and deleted {total_deleted_record_count} records..."
342
+ )
343
+
344
+ mutable_compaction_audit.set_input_records(total_input_records_count.item())
345
+
346
+ telemetry_time_merge = mutable_compaction_audit.save_step_stats(
347
+ CompactionSessionAuditInfo.MERGE_STEP_NAME,
348
+ merge_results,
349
+ merge_results_retrieved_at,
350
+ merge_invoke_end - merge_start,
351
+ merge_end - merge_start,
352
+ )
353
+
354
+ mutable_compaction_audit.set_records_deduped(total_dd_record_count.item())
355
+ mutable_compaction_audit.set_records_deleted(total_deleted_record_count.item())
356
+ record_info_msg: str = (
357
+ f"Hash bucket records: {total_hb_record_count},"
358
+ f" Deduped records: {total_dd_record_count}, "
359
+ f" Deleted records: {total_deleted_record_count}, "
360
+ )
361
+ logger.info(record_info_msg)
362
+ telemetry_this_round = telemetry_time_hb + telemetry_time_merge
363
+ previous_telemetry = (
364
+ mutable_compaction_audit.telemetry_time_in_seconds
365
+ if mutable_compaction_audit.telemetry_time_in_seconds
366
+ else 0.0
367
+ )
368
+ mutable_compaction_audit.set_telemetry_time_in_seconds(
369
+ telemetry_this_round + previous_telemetry
370
+ )
371
+ return merge_results
372
+
373
+
374
+ def _merge(
375
+ params: CompactPartitionParams,
376
+ task_resource_options_provider: callable,
377
+ merge_resource_options_provider: callable,
378
+ all_hash_group_idx_to_size_bytes: dict,
379
+ all_hash_group_idx_to_num_rows: dict,
380
+ round_completion_info: RoundCompletionInfo,
381
+ previous_compacted_delta_manifest: Manifest,
382
+ all_hash_group_idx_to_obj_id: dict,
383
+ compacted_partition: Partition,
384
+ delete_strategy: DeleteStrategy,
385
+ delete_file_envelopes: DeleteFileEnvelope,
386
+ ) -> tuple[List[MergeResult], float]:
387
+ merge_options_provider = functools.partial(
388
+ task_resource_options_provider,
389
+ pg_config=params.pg_config,
390
+ resource_amount_provider=merge_resource_options_provider,
391
+ num_hash_groups=params.hash_group_count,
392
+ hash_group_size_bytes=all_hash_group_idx_to_size_bytes,
393
+ hash_group_num_rows=all_hash_group_idx_to_num_rows,
394
+ total_memory_buffer_percentage=params.total_memory_buffer_percentage,
395
+ round_completion_info=round_completion_info,
396
+ compacted_delta_manifest=previous_compacted_delta_manifest,
397
+ primary_keys=params.primary_keys,
398
+ deltacat_storage=params.deltacat_storage,
399
+ deltacat_storage_kwargs=params.deltacat_storage_kwargs,
400
+ ray_custom_resources=params.ray_custom_resources,
401
+ memory_logs_enabled=params.memory_logs_enabled,
402
+ )
403
+
404
+ def merge_input_provider(index, item) -> dict[str, MergeInput]:
405
+ return {
406
+ "input": MergeInput.of(
407
+ merge_file_groups_provider=RemoteMergeFileGroupsProvider(
408
+ hash_group_index=item[0],
409
+ dfe_groups_refs=item[1],
410
+ hash_bucket_count=params.hash_bucket_count,
411
+ num_hash_groups=params.hash_group_count,
412
+ object_store=params.object_store,
413
+ ),
414
+ write_to_partition=compacted_partition,
415
+ compacted_file_content_type=params.compacted_file_content_type,
416
+ primary_keys=params.primary_keys,
417
+ sort_keys=params.sort_keys,
418
+ merge_task_index=index,
419
+ drop_duplicates=params.drop_duplicates,
420
+ max_records_per_output_file=params.records_per_compacted_file,
421
+ enable_profiler=params.enable_profiler,
422
+ metrics_config=params.metrics_config,
423
+ s3_table_writer_kwargs=params.s3_table_writer_kwargs,
424
+ read_kwargs_provider=params.read_kwargs_provider,
425
+ round_completion_info=round_completion_info,
426
+ object_store=params.object_store,
427
+ deltacat_storage=params.deltacat_storage,
428
+ deltacat_storage_kwargs=params.deltacat_storage_kwargs,
429
+ delete_strategy=delete_strategy,
430
+ delete_file_envelopes=delete_file_envelopes,
431
+ memory_logs_enabled=params.memory_logs_enabled,
432
+ disable_copy_by_reference=params.disable_copy_by_reference,
433
+ )
434
+ }
435
+
436
+ merge_tasks_pending = invoke_parallel(
437
+ items=all_hash_group_idx_to_obj_id.items(),
438
+ ray_task=mg.merge,
439
+ max_parallelism=params.task_max_parallelism,
440
+ options_provider=merge_options_provider,
441
+ kwargs_provider=merge_input_provider,
442
+ )
443
+ merge_invoke_end = time.monotonic()
444
+ logger.info(f"Getting {len(merge_tasks_pending)} merge results...")
445
+ merge_results: List[MergeResult] = ray.get(merge_tasks_pending)
446
+
447
+ return merge_results, merge_invoke_end
448
+
449
+
450
+ def _hash_bucket(
451
+ params: CompactPartitionParams,
452
+ uniform_deltas: List[DeltaAnnotated],
453
+ ) -> tuple[List[HashBucketResult], float]:
454
+
455
+ hb_options_provider = functools.partial(
456
+ task_resource_options_provider,
457
+ pg_config=params.pg_config,
458
+ resource_amount_provider=hash_bucket_resource_options_provider,
459
+ previous_inflation=params.previous_inflation,
460
+ average_record_size_bytes=params.average_record_size_bytes,
461
+ total_memory_buffer_percentage=params.total_memory_buffer_percentage,
462
+ primary_keys=params.primary_keys,
463
+ ray_custom_resources=params.ray_custom_resources,
464
+ memory_logs_enabled=params.memory_logs_enabled,
465
+ )
466
+
467
+ def hash_bucket_input_provider(index, item) -> dict[str, HashBucketInput]:
468
+ return {
469
+ "input": HashBucketInput.of(
470
+ item,
471
+ primary_keys=params.primary_keys,
472
+ hb_task_index=index,
473
+ num_hash_buckets=params.hash_bucket_count,
474
+ num_hash_groups=params.hash_group_count,
475
+ enable_profiler=params.enable_profiler,
476
+ metrics_config=params.metrics_config,
477
+ read_kwargs_provider=params.read_kwargs_provider,
478
+ object_store=params.object_store,
479
+ deltacat_storage=params.deltacat_storage,
480
+ deltacat_storage_kwargs=params.deltacat_storage_kwargs,
481
+ memory_logs_enabled=params.memory_logs_enabled,
482
+ )
483
+ }
484
+
485
+ hb_tasks_pending = invoke_parallel(
486
+ items=uniform_deltas,
487
+ ray_task=hb.hash_bucket,
488
+ max_parallelism=params.task_max_parallelism,
489
+ options_provider=hb_options_provider,
490
+ kwargs_provider=hash_bucket_input_provider,
491
+ )
492
+ hb_invoke_end = time.monotonic()
493
+
494
+ logger.info(f"Getting {len(hb_tasks_pending)} hash bucket results...")
495
+ hb_results: List[HashBucketResult] = ray.get(hb_tasks_pending)
496
+ logger.info(f"Got {len(hb_results)} hash bucket results.")
497
+
498
+ return (hb_results, hb_invoke_end)
499
+
500
+
501
+ def _run_local_merge(
502
+ params: CompactPartitionParams,
503
+ uniform_deltas: List[DeltaAnnotated],
504
+ compacted_partition: Partition,
505
+ round_completion_info: RoundCompletionInfo,
506
+ delete_strategy: Optional[DeleteStrategy],
507
+ delete_file_envelopes: Optional[DeleteFileEnvelope],
508
+ mutable_compaction_audit: CompactionSessionAuditInfo,
509
+ previous_compacted_delta_manifest: Optional[Manifest],
510
+ total_input_records_count: np.int64,
511
+ ) -> tuple[List[MergeResult], np.int64]:
512
+ local_merge_input: MergeInput = generate_local_merge_input(
513
+ params,
514
+ uniform_deltas,
515
+ compacted_partition,
516
+ round_completion_info,
517
+ delete_strategy,
518
+ delete_file_envelopes,
519
+ )
520
+ estimated_da_bytes = (
521
+ mutable_compaction_audit.estimated_in_memory_size_bytes_during_discovery
522
+ )
523
+ estimated_num_records: int = sum(
524
+ [
525
+ entry.meta.record_count
526
+ for delta in uniform_deltas
527
+ for entry in delta.manifest.entries
528
+ ]
529
+ )
530
+ local_merge_options = local_merge_resource_options_provider(
531
+ estimated_da_size=estimated_da_bytes,
532
+ estimated_num_rows=estimated_num_records,
533
+ total_memory_buffer_percentage=params.total_memory_buffer_percentage,
534
+ round_completion_info=round_completion_info,
535
+ compacted_delta_manifest=previous_compacted_delta_manifest,
536
+ ray_custom_resources=params.ray_custom_resources,
537
+ primary_keys=params.primary_keys,
538
+ memory_logs_enabled=params.memory_logs_enabled,
539
+ )
540
+ local_merge_result = ray.get(
541
+ mg.merge.options(**local_merge_options).remote(local_merge_input)
542
+ )
543
+ total_input_records_count += local_merge_result.input_record_count
544
+ merge_results = [local_merge_result]
545
+ return merge_results, total_input_records_count
546
+
547
+
548
+ def _process_merge_results(
549
+ params: CompactPartitionParams,
550
+ merge_results: List[MergeResult],
551
+ mutable_compaction_audit: CompactionSessionAuditInfo,
552
+ ) -> tuple[Delta, List[MaterializeResult], dict]:
553
+ mat_results = []
554
+ for merge_result in merge_results:
555
+ mat_results.extend(merge_result.materialize_results)
556
+
557
+ mat_results: List[MaterializeResult] = sorted(
558
+ mat_results, key=lambda m: m.task_index
559
+ )
560
+ hb_id_to_entry_indices_range = {}
561
+ file_index = 0
562
+ previous_task_index = -1
563
+
564
+ duplicate_hash_bucket_mat_results = 0
565
+ for mat_result in mat_results:
566
+ assert (
567
+ mat_result.pyarrow_write_result.files >= 1
568
+ ), "At least one file must be materialized"
569
+ if mat_result.task_index == previous_task_index:
570
+ duplicate_hash_bucket_mat_results += 1
571
+ else:
572
+ duplicate_hash_bucket_mat_results = 0
573
+ assert duplicate_hash_bucket_mat_results < params.num_rounds, (
574
+ f"Duplicate record count ({duplicate_hash_bucket_mat_results}) is as large "
575
+ f"as or greater than params.num_rounds, which is {params.num_rounds}"
576
+ )
577
+ hb_id_to_entry_indices_range[str(mat_result.task_index)] = (
578
+ file_index,
579
+ file_index + mat_result.pyarrow_write_result.files,
580
+ )
581
+
582
+ file_index += mat_result.pyarrow_write_result.files
583
+ previous_task_index = mat_result.task_index
584
+
585
+ s3_utils.upload(
586
+ mutable_compaction_audit.audit_url,
587
+ str(json.dumps(mutable_compaction_audit)),
588
+ **params.s3_client_kwargs,
589
+ )
590
+ deltas: List[Delta] = [m.delta for m in mat_results]
591
+ # Note: An appropriate last stream position must be set
592
+ # to avoid correctness issue.
593
+ merged_delta: Delta = Delta.merge_deltas(
594
+ deltas,
595
+ stream_position=params.last_stream_position_to_compact,
596
+ )
597
+
598
+ return merged_delta, mat_results, hb_id_to_entry_indices_range
599
+
600
+
601
+ def _upload_compaction_audit(
602
+ params: CompactPartitionParams,
603
+ mutable_compaction_audit: CompactionSessionAuditInfo,
604
+ round_completion_info: RoundCompletionInfo,
605
+ ) -> None:
606
+
607
+ # After all incremental delta related calculations, we update
608
+ # the input sizes to accommodate the compacted table
609
+ if round_completion_info:
610
+ mutable_compaction_audit.set_input_file_count(
611
+ (mutable_compaction_audit.input_file_count or 0)
612
+ + round_completion_info.compacted_pyarrow_write_result.files
613
+ )
614
+ mutable_compaction_audit.set_input_size_bytes(
615
+ (mutable_compaction_audit.input_size_bytes or 0.0)
616
+ + round_completion_info.compacted_pyarrow_write_result.file_bytes
617
+ )
618
+ mutable_compaction_audit.set_input_records(
619
+ (mutable_compaction_audit.input_records or 0)
620
+ + round_completion_info.compacted_pyarrow_write_result.records
621
+ )
622
+
623
+ s3_utils.upload(
624
+ mutable_compaction_audit.audit_url,
625
+ str(json.dumps(mutable_compaction_audit)),
626
+ **params.s3_client_kwargs,
627
+ )
628
+ return
629
+
630
+
631
+ def _write_new_round_completion_file(
632
+ params: CompactPartitionParams,
633
+ mutable_compaction_audit: CompactionSessionAuditInfo,
634
+ compacted_partition: Partition,
635
+ audit_url: str,
636
+ hb_id_to_entry_indices_range: dict,
637
+ rcf_source_partition_locator: rcf.PartitionLocator,
638
+ new_compacted_delta_locator: DeltaLocator,
639
+ pyarrow_write_result: PyArrowWriteResult,
640
+ ) -> ExecutionCompactionResult:
641
+ input_inflation = None
642
+ input_average_record_size_bytes = None
643
+ # Note: we only consider inflation for incremental delta
644
+ if (
645
+ mutable_compaction_audit.input_size_bytes
646
+ and mutable_compaction_audit.hash_bucket_processed_size_bytes
647
+ ):
648
+ input_inflation = (
649
+ mutable_compaction_audit.hash_bucket_processed_size_bytes
650
+ / mutable_compaction_audit.input_size_bytes
651
+ )
652
+
653
+ if (
654
+ mutable_compaction_audit.hash_bucket_processed_size_bytes
655
+ and mutable_compaction_audit.input_records
656
+ ):
657
+ input_average_record_size_bytes = (
658
+ mutable_compaction_audit.hash_bucket_processed_size_bytes
659
+ / mutable_compaction_audit.input_records
660
+ )
661
+
662
+ logger.info(
663
+ f"The inflation of input deltas={input_inflation}"
664
+ f" and average record size={input_average_record_size_bytes}"
665
+ )
666
+
667
+ new_round_completion_info = RoundCompletionInfo.of(
668
+ high_watermark=params.last_stream_position_to_compact,
669
+ compacted_delta_locator=new_compacted_delta_locator,
670
+ compacted_pyarrow_write_result=pyarrow_write_result,
671
+ sort_keys_bit_width=params.bit_width_of_sort_keys,
672
+ manifest_entry_copied_by_reference_ratio=mutable_compaction_audit.untouched_file_ratio,
673
+ compaction_audit_url=audit_url,
674
+ hash_bucket_count=params.hash_bucket_count,
675
+ hb_index_to_entry_range=hb_id_to_entry_indices_range,
676
+ compactor_version=CompactorVersion.V2.value,
677
+ input_inflation=input_inflation,
678
+ input_average_record_size_bytes=input_average_record_size_bytes,
679
+ )
680
+
681
+ logger.info(
682
+ f"Partition-{params.source_partition_locator.partition_values},"
683
+ f"compacted at: {params.last_stream_position_to_compact},"
684
+ )
685
+ logger.info(
686
+ f"Checking if partition {rcf_source_partition_locator} is inplace compacted against {params.destination_partition_locator}..."
687
+ )
688
+ is_inplace_compacted: bool = (
689
+ rcf_source_partition_locator.partition_values
690
+ == params.destination_partition_locator.partition_values
691
+ and rcf_source_partition_locator.stream_id
692
+ == params.destination_partition_locator.stream_id
693
+ )
694
+ if is_inplace_compacted:
695
+ logger.info(
696
+ "Overriding round completion file source partition locator as in-place compacted. "
697
+ + f"Got compacted partition partition_id of {compacted_partition.locator.partition_id} "
698
+ f"and rcf source partition_id of {rcf_source_partition_locator.partition_id}."
699
+ )
700
+ rcf_source_partition_locator = compacted_partition.locator
701
+
702
+ round_completion_file_s3_url = rcf.write_round_completion_file(
703
+ params.compaction_artifact_s3_bucket,
704
+ rcf_source_partition_locator,
705
+ compacted_partition.locator,
706
+ new_round_completion_info,
707
+ **params.s3_client_kwargs,
708
+ )
709
+
710
+ return ExecutionCompactionResult(
711
+ compacted_partition,
712
+ new_round_completion_info,
713
+ round_completion_file_s3_url,
714
+ is_inplace_compacted,
715
+ )
716
+
717
+
718
+ def _commit_compaction_result(
719
+ params: CompactPartitionParams,
720
+ execute_compaction_result: ExecutionCompactionResult,
721
+ ) -> None:
722
+ compaction_session_type: str = (
723
+ "INPLACE" if execute_compaction_result.is_inplace_compacted else "NON-INPLACE"
724
+ )
725
+ logger.info(
726
+ f"Partition-{params.source_partition_locator} -> "
727
+ f"{compaction_session_type} Compaction session data processing completed"
728
+ )
729
+ if execute_compaction_result.new_compacted_partition:
730
+ previous_partition: Optional[Partition] = None
731
+ if execute_compaction_result.is_inplace_compacted:
732
+ previous_partition: Optional[
733
+ Partition
734
+ ] = params.deltacat_storage.get_partition(
735
+ params.source_partition_locator.stream_locator,
736
+ params.source_partition_locator.partition_values,
737
+ **params.deltacat_storage_kwargs,
738
+ )
739
+ # NOTE: Retrieving the previous partition again as the partition_id may have changed by the time commit_partition is called.
740
+ logger.info(
741
+ f"Committing compacted partition to: {execute_compaction_result.new_compacted_partition.locator} "
742
+ f"using previous partition: {previous_partition.locator if previous_partition else None}"
743
+ )
744
+ committed_partition: Partition = params.deltacat_storage.commit_partition(
745
+ execute_compaction_result.new_compacted_partition,
746
+ previous_partition,
747
+ **params.deltacat_storage_kwargs,
748
+ )
749
+ logger.info(f"Committed compacted partition: {committed_partition}")
750
+ else:
751
+ logger.warning("No new partition was committed during compaction.")
752
+
753
+ logger.info(f"Completed compaction session for: {params.source_partition_locator}")