deltacat 0.1.18b14__py3-none-any.whl → 0.1.18b16__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.
- deltacat/__init__.py +1 -1
- deltacat/aws/clients.py +17 -6
- deltacat/aws/redshift/model/manifest.py +4 -0
- deltacat/aws/s3u.py +24 -1
- deltacat/compute/compactor/compaction_session.py +42 -18
- deltacat/compute/compactor/model/compact_partition_params.py +297 -58
- deltacat/compute/compactor/model/compaction_session_audit_info.py +163 -9
- deltacat/compute/compactor/model/delta_annotated.py +95 -9
- deltacat/compute/compactor/model/delta_file_envelope.py +14 -2
- deltacat/compute/compactor/model/round_completion_info.py +17 -1
- deltacat/compute/compactor/repartition_session.py +4 -1
- deltacat/compute/compactor/steps/dedupe.py +9 -6
- deltacat/compute/compactor/steps/hash_bucket.py +24 -3
- deltacat/compute/compactor/steps/materialize.py +11 -6
- deltacat/compute/compactor/steps/repartition.py +22 -1
- deltacat/compute/compactor/utils/io.py +40 -23
- deltacat/compute/compactor/utils/sort_key.py +5 -0
- deltacat/compute/compactor/utils/system_columns.py +43 -0
- deltacat/compute/compactor_v2/compaction_session.py +509 -0
- deltacat/compute/compactor_v2/constants.py +37 -0
- deltacat/compute/compactor_v2/model/hash_bucket_input.py +78 -0
- deltacat/compute/compactor_v2/model/hash_bucket_result.py +12 -0
- deltacat/compute/compactor_v2/model/merge_input.py +143 -0
- deltacat/compute/compactor_v2/model/merge_result.py +12 -0
- deltacat/compute/compactor_v2/steps/__init__.py +0 -0
- deltacat/compute/compactor_v2/steps/hash_bucket.py +203 -0
- deltacat/compute/compactor_v2/steps/merge.py +469 -0
- deltacat/compute/compactor_v2/utils/__init__.py +0 -0
- deltacat/compute/compactor_v2/utils/content_type_params.py +66 -0
- deltacat/compute/compactor_v2/utils/dedupe.py +58 -0
- deltacat/compute/compactor_v2/utils/io.py +152 -0
- deltacat/compute/compactor_v2/utils/primary_key_index.py +341 -0
- deltacat/compute/compactor_v2/utils/task_options.py +221 -0
- deltacat/compute/metastats/meta_stats.py +4 -2
- deltacat/compute/metastats/stats.py +1 -0
- deltacat/compute/metastats/utils/io.py +4 -0
- deltacat/compute/stats/utils/io.py +20 -5
- deltacat/exceptions.py +4 -0
- deltacat/io/memcached_object_store.py +37 -14
- deltacat/logs.py +4 -3
- deltacat/storage/interface.py +8 -1
- deltacat/storage/model/types.py +2 -1
- deltacat/tests/aws/test_clients.py +16 -3
- deltacat/tests/compute/__init__.py +0 -0
- deltacat/tests/compute/common.py +96 -0
- deltacat/tests/compute/compactor/__init__.py +0 -0
- deltacat/tests/compute/compactor/steps/__init__.py +0 -0
- deltacat/tests/{test_repartition.py → compute/compactor/steps/test_repartition.py} +34 -8
- deltacat/tests/compute/compactor/utils/__init__.py +0 -0
- deltacat/tests/{compactor → compute/compactor}/utils/test_io.py +47 -5
- deltacat/tests/compute/compactor_v2/__init__.py +0 -0
- deltacat/tests/{compactor → compute}/test_compact_partition_params.py +14 -30
- deltacat/tests/compute/test_compaction_session_incremental.py +363 -0
- deltacat/tests/compute/testcases.py +395 -0
- deltacat/tests/io/test_memcached_object_store.py +5 -4
- deltacat/tests/local_deltacat_storage/__init__.py +62 -19
- deltacat/tests/test_utils/pyarrow.py +49 -0
- deltacat/tests/test_utils/utils.py +13 -0
- deltacat/tests/utils/data/__init__.py +0 -0
- deltacat/tests/utils/test_daft.py +76 -0
- deltacat/tests/utils/test_pyarrow.py +133 -0
- deltacat/tests/utils/test_resources.py +23 -20
- deltacat/types/media.py +1 -0
- deltacat/types/partial_download.py +83 -0
- deltacat/types/tables.py +6 -0
- deltacat/utils/arguments.py +25 -0
- deltacat/utils/daft.py +87 -0
- deltacat/utils/placement.py +20 -3
- deltacat/utils/pyarrow.py +218 -1
- deltacat/utils/ray_utils/concurrency.py +26 -1
- deltacat/utils/resources.py +72 -1
- deltacat/utils/s3fs.py +21 -0
- {deltacat-0.1.18b14.dist-info → deltacat-0.1.18b16.dist-info}/METADATA +17 -3
- {deltacat-0.1.18b14.dist-info → deltacat-0.1.18b16.dist-info}/RECORD +79 -47
- {deltacat-0.1.18b14.dist-info → deltacat-0.1.18b16.dist-info}/WHEEL +1 -1
- /deltacat/{tests/compactor → compute/compactor_v2}/__init__.py +0 -0
- /deltacat/{tests/compactor/utils → compute/compactor_v2/model}/__init__.py +0 -0
- {deltacat-0.1.18b14.dist-info → deltacat-0.1.18b16.dist-info}/LICENSE +0 -0
- {deltacat-0.1.18b14.dist-info → deltacat-0.1.18b16.dist-info}/top_level.txt +0 -0
@@ -3,7 +3,7 @@ import logging
|
|
3
3
|
import time
|
4
4
|
from contextlib import nullcontext
|
5
5
|
from itertools import chain
|
6
|
-
from typing import Generator, List, Optional, Tuple
|
6
|
+
from typing import Any, Dict, Generator, List, Optional, Tuple
|
7
7
|
import numpy as np
|
8
8
|
import pyarrow as pa
|
9
9
|
import ray
|
@@ -91,7 +91,11 @@ def _group_file_records_by_pk_hash_bucket(
|
|
91
91
|
is_src_delta: np.bool_ = True,
|
92
92
|
read_kwargs_provider: Optional[ReadKwargsProvider] = None,
|
93
93
|
deltacat_storage=unimplemented_deltacat_storage,
|
94
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
95
|
+
**kwargs,
|
94
96
|
) -> Tuple[Optional[DeltaFileEnvelopeGroups], int]:
|
97
|
+
if deltacat_storage_kwargs is None:
|
98
|
+
deltacat_storage_kwargs = {}
|
95
99
|
# read input parquet s3 objects into a list of delta file envelopes
|
96
100
|
delta_file_envelopes, total_record_count = _read_delta_file_envelopes(
|
97
101
|
annotated_delta,
|
@@ -99,6 +103,8 @@ def _group_file_records_by_pk_hash_bucket(
|
|
99
103
|
sort_key_names,
|
100
104
|
read_kwargs_provider,
|
101
105
|
deltacat_storage,
|
106
|
+
deltacat_storage_kwargs,
|
107
|
+
**kwargs,
|
102
108
|
)
|
103
109
|
if delta_file_envelopes is None:
|
104
110
|
return None, 0
|
@@ -134,8 +140,11 @@ def _read_delta_file_envelopes(
|
|
134
140
|
sort_key_names: List[str],
|
135
141
|
read_kwargs_provider: Optional[ReadKwargsProvider],
|
136
142
|
deltacat_storage=unimplemented_deltacat_storage,
|
143
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
144
|
+
**kwargs,
|
137
145
|
) -> Tuple[Optional[List[DeltaFileEnvelope]], int]:
|
138
|
-
|
146
|
+
if deltacat_storage_kwargs is None:
|
147
|
+
deltacat_storage_kwargs = {}
|
139
148
|
columns_to_read = list(chain(primary_keys, sort_key_names))
|
140
149
|
# TODO (rootliu) compare performance of column read from unpartitioned vs partitioned file
|
141
150
|
# https://arrow.apache.org/docs/python/parquet.html#writing-to-partitioned-datasets
|
@@ -145,6 +154,7 @@ def _read_delta_file_envelopes(
|
|
145
154
|
columns=columns_to_read,
|
146
155
|
file_reader_kwargs_provider=read_kwargs_provider,
|
147
156
|
storage_type=StorageType.LOCAL,
|
157
|
+
**deltacat_storage_kwargs,
|
148
158
|
)
|
149
159
|
annotations = annotated_delta.annotations
|
150
160
|
assert (
|
@@ -182,7 +192,11 @@ def _timed_hash_bucket(
|
|
182
192
|
read_kwargs_provider: Optional[ReadKwargsProvider] = None,
|
183
193
|
object_store: Optional[IObjectStore] = None,
|
184
194
|
deltacat_storage=unimplemented_deltacat_storage,
|
195
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
196
|
+
**kwargs,
|
185
197
|
):
|
198
|
+
if deltacat_storage_kwargs is None:
|
199
|
+
deltacat_storage_kwargs = {}
|
186
200
|
task_id = get_current_ray_task_id()
|
187
201
|
worker_id = get_current_ray_worker_id()
|
188
202
|
with memray.Tracker(
|
@@ -207,6 +221,8 @@ def _timed_hash_bucket(
|
|
207
221
|
is_src_delta,
|
208
222
|
read_kwargs_provider,
|
209
223
|
deltacat_storage,
|
224
|
+
deltacat_storage_kwargs,
|
225
|
+
**kwargs,
|
210
226
|
)
|
211
227
|
hash_bucket_group_to_obj_id, _ = group_hash_bucket_indices(
|
212
228
|
delta_file_envelope_groups, num_buckets, num_groups, object_store
|
@@ -235,8 +251,11 @@ def hash_bucket(
|
|
235
251
|
read_kwargs_provider: Optional[ReadKwargsProvider],
|
236
252
|
object_store: Optional[IObjectStore],
|
237
253
|
deltacat_storage=unimplemented_deltacat_storage,
|
254
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
255
|
+
**kwargs,
|
238
256
|
) -> HashBucketResult:
|
239
|
-
|
257
|
+
if deltacat_storage_kwargs is None:
|
258
|
+
deltacat_storage_kwargs = {}
|
240
259
|
logger.info(f"Starting hash bucket task...")
|
241
260
|
hash_bucket_result, duration = timed_invocation(
|
242
261
|
func=_timed_hash_bucket,
|
@@ -250,6 +269,8 @@ def hash_bucket(
|
|
250
269
|
read_kwargs_provider=read_kwargs_provider,
|
251
270
|
object_store=object_store,
|
252
271
|
deltacat_storage=deltacat_storage,
|
272
|
+
deltacat_storage_kwargs=deltacat_storage_kwargs,
|
273
|
+
**kwargs,
|
253
274
|
)
|
254
275
|
|
255
276
|
emit_metrics_time = 0.0
|
@@ -69,7 +69,11 @@ def materialize(
|
|
69
69
|
s3_table_writer_kwargs: Optional[Dict[str, Any]] = None,
|
70
70
|
object_store: Optional[IObjectStore] = None,
|
71
71
|
deltacat_storage=unimplemented_deltacat_storage,
|
72
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
72
73
|
):
|
74
|
+
if deltacat_storage_kwargs is None:
|
75
|
+
deltacat_storage_kwargs = {}
|
76
|
+
|
73
77
|
def _stage_delta_from_manifest_entry_reference_list(
|
74
78
|
manifest_entry_list_reference: List[ManifestEntry],
|
75
79
|
partition: Partition,
|
@@ -105,6 +109,7 @@ def materialize(
|
|
105
109
|
max_records_per_entry=max_records_per_output_file,
|
106
110
|
content_type=compacted_file_content_type,
|
107
111
|
s3_table_writer_kwargs=s3_table_writer_kwargs,
|
112
|
+
**deltacat_storage_kwargs,
|
108
113
|
)
|
109
114
|
compacted_table_size = TABLE_CLASS_TO_SIZE_FUNC[type(compacted_table)](
|
110
115
|
compacted_table
|
@@ -116,11 +121,10 @@ def materialize(
|
|
116
121
|
)
|
117
122
|
manifest = delta.manifest
|
118
123
|
manifest_records = manifest.meta.record_count
|
119
|
-
assert (
|
120
|
-
manifest_records == len(compacted_table),
|
124
|
+
assert manifest_records == len(compacted_table), (
|
121
125
|
f"Unexpected Error: Materialized delta manifest record count "
|
122
126
|
f"({manifest_records}) does not equal compacted table record count "
|
123
|
-
f"({len(compacted_table)})"
|
127
|
+
f"({len(compacted_table)})"
|
124
128
|
)
|
125
129
|
materialize_result = MaterializeResult.of(
|
126
130
|
delta=delta,
|
@@ -187,10 +191,11 @@ def materialize(
|
|
187
191
|
src_stream_position_np.item(),
|
188
192
|
)
|
189
193
|
dl_digest = delta_locator.digest()
|
190
|
-
|
191
194
|
manifest = manifest_cache.setdefault(
|
192
195
|
dl_digest,
|
193
|
-
deltacat_storage.get_delta_manifest(
|
196
|
+
deltacat_storage.get_delta_manifest(
|
197
|
+
delta_locator, **deltacat_storage_kwargs
|
198
|
+
),
|
194
199
|
)
|
195
200
|
|
196
201
|
if read_kwargs_provider is None:
|
@@ -236,6 +241,7 @@ def materialize(
|
|
236
241
|
Delta.of(delta_locator, None, None, None, manifest),
|
237
242
|
src_file_idx_np.item(),
|
238
243
|
file_reader_kwargs_provider=read_kwargs_provider,
|
244
|
+
**deltacat_storage_kwargs,
|
239
245
|
)
|
240
246
|
logger.debug(
|
241
247
|
f"Time taken for materialize task"
|
@@ -253,7 +259,6 @@ def materialize(
|
|
253
259
|
materialized_results.append(_materialize(record_batch_tables.remaining))
|
254
260
|
|
255
261
|
logger.info(f"Got {count_of_src_dfl} source delta files during materialize")
|
256
|
-
|
257
262
|
referenced_manifest_delta = (
|
258
263
|
_stage_delta_from_manifest_entry_reference_list(
|
259
264
|
manifest_entry_list_reference, partition
|
@@ -4,7 +4,7 @@ from contextlib import nullcontext
|
|
4
4
|
import pyarrow.compute as pc
|
5
5
|
from deltacat.constants import SIGNED_INT64_MIN_VALUE, SIGNED_INT64_MAX_VALUE
|
6
6
|
import pyarrow as pa
|
7
|
-
from typing import List, Optional
|
7
|
+
from typing import Any, Dict, List, Optional
|
8
8
|
from deltacat.types.media import StorageType, ContentType
|
9
9
|
import ray
|
10
10
|
from deltacat import logs
|
@@ -56,8 +56,11 @@ def repartition_range(
|
|
56
56
|
destination_partition: Partition,
|
57
57
|
repartition_args: dict,
|
58
58
|
max_records_per_output_file: int,
|
59
|
+
s3_table_writer_kwargs: Optional[Dict[str, Any]] = None,
|
59
60
|
repartitioned_file_content_type: ContentType = ContentType.PARQUET,
|
60
61
|
deltacat_storage=unimplemented_deltacat_storage,
|
62
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
63
|
+
**kwargs,
|
61
64
|
):
|
62
65
|
"""
|
63
66
|
Repartitions a list of Arrow tables based on specified ranges and stores the repartitioned tables.
|
@@ -85,6 +88,8 @@ def repartition_range(
|
|
85
88
|
in the tables, an error will be raised. For each partition range, a new file is created. This could result in
|
86
89
|
more output files than input files.
|
87
90
|
"""
|
91
|
+
if deltacat_storage_kwargs is None:
|
92
|
+
deltacat_storage_kwargs = {}
|
88
93
|
column: str = repartition_args["column"]
|
89
94
|
partition_ranges: List = repartition_args["ranges"]
|
90
95
|
if len(partition_ranges) == 0:
|
@@ -141,6 +146,8 @@ def repartition_range(
|
|
141
146
|
destination_partition,
|
142
147
|
max_records_per_entry=max_records_per_output_file,
|
143
148
|
content_type=repartitioned_file_content_type,
|
149
|
+
s3_table_writer_kwargs=s3_table_writer_kwargs,
|
150
|
+
**deltacat_storage_kwargs,
|
144
151
|
)
|
145
152
|
partition_deltas.append(partition_delta)
|
146
153
|
|
@@ -161,9 +168,14 @@ def _timed_repartition(
|
|
161
168
|
max_records_per_output_file: int,
|
162
169
|
enable_profiler: bool,
|
163
170
|
read_kwargs_provider: Optional[ReadKwargsProvider],
|
171
|
+
s3_table_writer_kwargs: Optional[Dict[str, Any]] = None,
|
164
172
|
repartitioned_file_content_type: ContentType = ContentType.PARQUET,
|
165
173
|
deltacat_storage=unimplemented_deltacat_storage,
|
174
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
175
|
+
**kwargs,
|
166
176
|
) -> RepartitionResult:
|
177
|
+
if deltacat_storage_kwargs is None:
|
178
|
+
deltacat_storage_kwargs = {}
|
167
179
|
task_id = get_current_ray_task_id()
|
168
180
|
worker_id = get_current_ray_worker_id()
|
169
181
|
with memray.Tracker(
|
@@ -180,8 +192,10 @@ def _timed_repartition(
|
|
180
192
|
destination_partition=destination_partition,
|
181
193
|
repartition_args=repartition_args,
|
182
194
|
max_records_per_output_file=max_records_per_output_file,
|
195
|
+
s3_table_writer_kwargs=s3_table_writer_kwargs,
|
183
196
|
repartitioned_file_content_type=repartitioned_file_content_type,
|
184
197
|
deltacat_storage=deltacat_storage,
|
198
|
+
deltacat_storage_kwargs=deltacat_storage_kwargs,
|
185
199
|
)
|
186
200
|
else:
|
187
201
|
raise NotImplementedError(
|
@@ -199,9 +213,14 @@ def repartition(
|
|
199
213
|
enable_profiler: bool,
|
200
214
|
metrics_config: Optional[MetricsConfig],
|
201
215
|
read_kwargs_provider: Optional[ReadKwargsProvider],
|
216
|
+
s3_table_writer_kwargs: Optional[Dict[str, Any]] = None,
|
202
217
|
repartitioned_file_content_type: ContentType = ContentType.PARQUET,
|
203
218
|
deltacat_storage=unimplemented_deltacat_storage,
|
219
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
220
|
+
**kwargs,
|
204
221
|
) -> RepartitionResult:
|
222
|
+
if deltacat_storage_kwargs is None:
|
223
|
+
deltacat_storage_kwargs = {}
|
205
224
|
logger.info(f"Starting repartition task...")
|
206
225
|
repartition_result, duration = timed_invocation(
|
207
226
|
func=_timed_repartition,
|
@@ -212,8 +231,10 @@ def repartition(
|
|
212
231
|
max_records_per_output_file=max_records_per_output_file,
|
213
232
|
enable_profiler=enable_profiler,
|
214
233
|
read_kwargs_provider=read_kwargs_provider,
|
234
|
+
s3_table_writer_kwargs=s3_table_writer_kwargs,
|
215
235
|
repartitioned_file_content_type=repartitioned_file_content_type,
|
216
236
|
deltacat_storage=deltacat_storage,
|
237
|
+
deltacat_storage_kwargs=deltacat_storage_kwargs,
|
217
238
|
)
|
218
239
|
if metrics_config:
|
219
240
|
emit_timer_metrics(
|
@@ -10,11 +10,12 @@ from deltacat.constants import (
|
|
10
10
|
from deltacat.storage import (
|
11
11
|
PartitionLocator,
|
12
12
|
Delta,
|
13
|
+
ManifestEntry,
|
13
14
|
interface as unimplemented_deltacat_storage,
|
14
15
|
)
|
15
16
|
from deltacat import logs
|
16
17
|
from deltacat.compute.compactor import DeltaAnnotated
|
17
|
-
from typing import Dict, List, Optional, Tuple, Union
|
18
|
+
from typing import Dict, List, Optional, Tuple, Union, Any
|
18
19
|
from deltacat.compute.compactor import HighWatermark
|
19
20
|
from deltacat.compute.compactor.model.compaction_session_audit_info import (
|
20
21
|
CompactionSessionAuditInfo,
|
@@ -31,23 +32,30 @@ def discover_deltas(
|
|
31
32
|
rebase_source_partition_locator: Optional[PartitionLocator],
|
32
33
|
rebase_source_partition_high_watermark: Optional[int],
|
33
34
|
deltacat_storage=unimplemented_deltacat_storage,
|
34
|
-
|
35
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = {},
|
36
|
+
list_deltas_kwargs: Optional[Dict[str, Any]] = {},
|
35
37
|
) -> Tuple[List[Delta], int]:
|
36
|
-
|
38
|
+
if deltacat_storage_kwargs is None:
|
39
|
+
deltacat_storage_kwargs = {}
|
37
40
|
# Source One: new deltas from uncompacted table for incremental compaction or deltas from compacted table for rebase
|
38
|
-
|
39
|
-
source_partition_locator,
|
41
|
+
start_position_exclusive = (
|
40
42
|
high_watermark.get(source_partition_locator)
|
41
43
|
if isinstance(high_watermark, dict)
|
42
|
-
else high_watermark
|
44
|
+
else high_watermark
|
45
|
+
)
|
46
|
+
input_deltas = _discover_deltas(
|
47
|
+
source_partition_locator,
|
48
|
+
start_position_exclusive,
|
43
49
|
last_stream_position_to_compact
|
44
50
|
if not rebase_source_partition_locator
|
45
51
|
else deltacat_storage.get_partition(
|
46
52
|
source_partition_locator.stream_locator,
|
47
53
|
source_partition_locator.partition_values,
|
54
|
+
**deltacat_storage_kwargs,
|
48
55
|
).stream_position,
|
49
56
|
deltacat_storage,
|
50
|
-
|
57
|
+
deltacat_storage_kwargs,
|
58
|
+
list_deltas_kwargs,
|
51
59
|
)
|
52
60
|
|
53
61
|
# Source Two: delta from compacted table for incremental compaction or new deltas from uncompacted table for rebase
|
@@ -56,6 +64,7 @@ def discover_deltas(
|
|
56
64
|
compacted_partition = deltacat_storage.get_partition(
|
57
65
|
compacted_partition_locator.stream_locator,
|
58
66
|
compacted_partition_locator.partition_values,
|
67
|
+
**deltacat_storage_kwargs,
|
59
68
|
)
|
60
69
|
previous_last_stream_position_compacted = (
|
61
70
|
compacted_partition.stream_position if compacted_partition else -1
|
@@ -67,7 +76,8 @@ def discover_deltas(
|
|
67
76
|
None,
|
68
77
|
previous_last_stream_position_compacted,
|
69
78
|
deltacat_storage,
|
70
|
-
|
79
|
+
deltacat_storage_kwargs,
|
80
|
+
list_deltas_kwargs,
|
71
81
|
)
|
72
82
|
logger.info(
|
73
83
|
f"Length of input deltas from uncompacted table {len(input_deltas)} up to {last_stream_position_to_compact},"
|
@@ -80,7 +90,8 @@ def discover_deltas(
|
|
80
90
|
rebase_source_partition_high_watermark,
|
81
91
|
last_stream_position_to_compact,
|
82
92
|
deltacat_storage,
|
83
|
-
|
93
|
+
deltacat_storage_kwargs,
|
94
|
+
list_deltas_kwargs,
|
84
95
|
)
|
85
96
|
logger.info(
|
86
97
|
f"Length of input deltas from uncompacted table {len(input_deltas_new)} up to {last_stream_position_to_compact},"
|
@@ -99,6 +110,8 @@ def limit_input_deltas(
|
|
99
110
|
input_deltas_stats: Dict[int, DeltaStats],
|
100
111
|
compaction_audit: CompactionSessionAuditInfo,
|
101
112
|
deltacat_storage=unimplemented_deltacat_storage,
|
113
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
114
|
+
**kwargs,
|
102
115
|
) -> Tuple[List[DeltaAnnotated], int, HighWatermark, bool]:
|
103
116
|
# TODO (pdames): when row counts are available in metadata, use them
|
104
117
|
# instead of bytes - memory consumption depends more on number of
|
@@ -108,6 +121,8 @@ def limit_input_deltas(
|
|
108
121
|
# this assumption could be removed, but we'd still need to know the max
|
109
122
|
# resources we COULD get for this cluster, and the amount of memory
|
110
123
|
# available per CPU should remain fixed across the cluster.
|
124
|
+
if deltacat_storage_kwargs is None:
|
125
|
+
deltacat_storage_kwargs = {}
|
111
126
|
worker_cpus = int(cluster_resources["CPU"])
|
112
127
|
worker_obj_store_mem = float(cluster_resources["object_store_memory"])
|
113
128
|
logger.info(f"Total worker object store memory: {worker_obj_store_mem}")
|
@@ -135,7 +150,7 @@ def limit_input_deltas(
|
|
135
150
|
for stream_pos, delta_stats in input_deltas_stats.items()
|
136
151
|
}
|
137
152
|
for delta in input_deltas:
|
138
|
-
manifest = deltacat_storage.get_delta_manifest(delta)
|
153
|
+
manifest = deltacat_storage.get_delta_manifest(delta, **deltacat_storage_kwargs)
|
139
154
|
delta.manifest = manifest
|
140
155
|
position = delta.stream_position
|
141
156
|
delta_stats = input_deltas_stats.get(delta.stream_position, DeltaStats())
|
@@ -258,6 +273,8 @@ def fit_input_deltas(
|
|
258
273
|
compaction_audit: CompactionSessionAuditInfo,
|
259
274
|
hash_bucket_count: Optional[int],
|
260
275
|
deltacat_storage=unimplemented_deltacat_storage,
|
276
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = None,
|
277
|
+
**kwargs,
|
261
278
|
) -> Tuple[List[DeltaAnnotated], int, HighWatermark, bool]:
|
262
279
|
"""
|
263
280
|
This method tries to fit all the input deltas to run into the existing cluster. Contrary to
|
@@ -277,6 +294,8 @@ def fit_input_deltas(
|
|
277
294
|
Tuple of list of annotated deltas, recommended hash bucket count, high watermark,
|
278
295
|
and whether multiple rounds are required (which is always False)
|
279
296
|
"""
|
297
|
+
if deltacat_storage_kwargs is None:
|
298
|
+
deltacat_storage_kwargs = {}
|
280
299
|
worker_cpus = int(cluster_resources["CPU"])
|
281
300
|
total_memory = float(cluster_resources["memory"])
|
282
301
|
high_watermark = HighWatermark()
|
@@ -306,8 +325,8 @@ def fit_input_deltas(
|
|
306
325
|
# We assume that the cluster is capable of distributing all tasks
|
307
326
|
# correctly. Hence, the correct in-memory size will be in the ratio of
|
308
327
|
# in-disk size.
|
309
|
-
def estimate_size(
|
310
|
-
return (content_length * 1.0 / delta_bytes) * total_memory
|
328
|
+
def estimate_size(manifest_entry: ManifestEntry):
|
329
|
+
return (manifest_entry.meta.content_length * 1.0 / delta_bytes) * total_memory
|
311
330
|
|
312
331
|
# Assuming each CPU consumes equal amount of memory
|
313
332
|
min_delta_bytes = total_memory / worker_cpus
|
@@ -341,18 +360,16 @@ def _discover_deltas(
|
|
341
360
|
start_position_exclusive: Optional[int],
|
342
361
|
end_position_inclusive: int,
|
343
362
|
deltacat_storage=unimplemented_deltacat_storage,
|
344
|
-
|
363
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = {},
|
364
|
+
list_deltas_kwargs: Optional[Dict[str, Any]] = {},
|
345
365
|
) -> List[Delta]:
|
346
|
-
|
347
|
-
|
348
|
-
|
349
|
-
|
350
|
-
|
351
|
-
deltas_list_result = deltacat_storage.
|
352
|
-
|
353
|
-
table_name=table_name,
|
354
|
-
partition_values=partition_values,
|
355
|
-
table_version=table_version,
|
366
|
+
if deltacat_storage_kwargs is None:
|
367
|
+
deltacat_storage_kwargs = {}
|
368
|
+
|
369
|
+
kwargs = {**deltacat_storage_kwargs, **list_deltas_kwargs}
|
370
|
+
|
371
|
+
deltas_list_result = deltacat_storage.list_partition_deltas(
|
372
|
+
partition_like=source_partition_locator,
|
356
373
|
first_stream_position=start_position_exclusive,
|
357
374
|
last_stream_position=end_position_inclusive,
|
358
375
|
ascending_order=True,
|
@@ -9,6 +9,8 @@ def validate_sort_keys(
|
|
9
9
|
source_partition_locator: PartitionLocator,
|
10
10
|
sort_keys: List[SortKey],
|
11
11
|
deltacat_storage,
|
12
|
+
deltacat_storage_kwargs,
|
13
|
+
**kwargs,
|
12
14
|
) -> int:
|
13
15
|
"""
|
14
16
|
Validates the input sort keys to ensure that they are unique, are using
|
@@ -16,6 +18,8 @@ def validate_sort_keys(
|
|
16
18
|
sum of bit widths across sort key data types is less-than-or-equal-to
|
17
19
|
256. Returns the sum of bit widths across all sort keys.
|
18
20
|
"""
|
21
|
+
if deltacat_storage_kwargs is None:
|
22
|
+
deltacat_storage_kwargs = {}
|
19
23
|
total_sort_keys_bit_width = 0
|
20
24
|
if sort_keys:
|
21
25
|
sort_key_names = [key.key_name for key in sort_keys]
|
@@ -27,6 +31,7 @@ def validate_sort_keys(
|
|
27
31
|
stream_locator.namespace,
|
28
32
|
stream_locator.table_name,
|
29
33
|
stream_locator.table_version,
|
34
|
+
**deltacat_storage_kwargs,
|
30
35
|
)
|
31
36
|
if isinstance(table_version_schema, pa.Schema):
|
32
37
|
for sort_key_name in sort_key_names:
|
@@ -22,6 +22,13 @@ _PK_HASH_COLUMN_FIELD = pa.field(
|
|
22
22
|
_PK_HASH_COLUMN_TYPE,
|
23
23
|
)
|
24
24
|
|
25
|
+
_PK_HASH_STRING_COLUMN_NAME = _get_sys_col_name("hash_str")
|
26
|
+
_PK_HASH_STRING_COLUMN_TYPE = pa.string()
|
27
|
+
_PK_HASH_STRING_COLUMN_FIELD = pa.field(
|
28
|
+
_PK_HASH_STRING_COLUMN_NAME,
|
29
|
+
_PK_HASH_STRING_COLUMN_TYPE,
|
30
|
+
)
|
31
|
+
|
25
32
|
_DEDUPE_TASK_IDX_COLUMN_NAME = _get_sys_col_name("dedupe_task_idx")
|
26
33
|
_DEDUPE_TASK_IDX_COLUMN_TYPE = pa.int32()
|
27
34
|
_DEDUPE_TASK_IDX_COLUMN_FIELD = pa.field(
|
@@ -36,6 +43,12 @@ _PARTITION_STREAM_POSITION_COLUMN_FIELD = pa.field(
|
|
36
43
|
_PARTITION_STREAM_POSITION_COLUMN_TYPE,
|
37
44
|
)
|
38
45
|
|
46
|
+
_HASH_BUCKET_IDX_COLUMN_NAME = _get_sys_col_name("hash_bucket_idx")
|
47
|
+
_HASH_BUCKET_IDX_COLUMN_TYPE = pa.int32()
|
48
|
+
_HASH_BUCKET_IDX_COLUMN_FIELD = pa.field(
|
49
|
+
_HASH_BUCKET_IDX_COLUMN_NAME, _HASH_BUCKET_IDX_COLUMN_TYPE
|
50
|
+
)
|
51
|
+
|
39
52
|
_ORDERED_FILE_IDX_COLUMN_NAME = _get_sys_col_name("file_index")
|
40
53
|
_ORDERED_FILE_IDX_COLUMN_TYPE = pa.int32()
|
41
54
|
_ORDERED_FILE_IDX_COLUMN_FIELD = pa.field(
|
@@ -76,10 +89,18 @@ def get_pk_hash_column_array(obj) -> Union[pa.Array, pa.ChunkedArray]:
|
|
76
89
|
return pa.array(obj, _PK_HASH_COLUMN_TYPE)
|
77
90
|
|
78
91
|
|
92
|
+
def get_pk_hash_string_column_array(obj) -> Union[pa.Array, pa.ChunkedArray]:
|
93
|
+
return pa.array(obj, _PK_HASH_STRING_COLUMN_TYPE)
|
94
|
+
|
95
|
+
|
79
96
|
def pk_hash_column_np(table: pa.Table) -> np.ndarray:
|
80
97
|
return table[_PK_HASH_COLUMN_NAME].to_numpy()
|
81
98
|
|
82
99
|
|
100
|
+
def pk_hash_string_column_np(table: pa.Table) -> np.ndarray:
|
101
|
+
return table[_PK_HASH_STRING_COLUMN_NAME].to_numpy()
|
102
|
+
|
103
|
+
|
83
104
|
def pk_hash_column(table: pa.Table) -> pa.ChunkedArray:
|
84
105
|
return table[_PK_HASH_COLUMN_NAME]
|
85
106
|
|
@@ -143,6 +164,10 @@ def get_delta_type_column_array(obj) -> Union[pa.Array, pa.ChunkedArray]:
|
|
143
164
|
)
|
144
165
|
|
145
166
|
|
167
|
+
def get_hash_bucket_idx_column_array(obj) -> Union[pa.Array, pa.ChunkedArray]:
|
168
|
+
return pa.array(obj, _HASH_BUCKET_IDX_COLUMN_TYPE)
|
169
|
+
|
170
|
+
|
146
171
|
def get_is_source_column_array(obj) -> Union[pa.Array, pa.ChunkedArray]:
|
147
172
|
return pa.array(
|
148
173
|
obj,
|
@@ -232,6 +257,24 @@ def append_pk_hash_column(table: pa.Table, pk_hashes) -> pa.Table:
|
|
232
257
|
return table
|
233
258
|
|
234
259
|
|
260
|
+
def append_pk_hash_string_column(table: pa.Table, pk_hashes) -> pa.Table:
|
261
|
+
|
262
|
+
table = table.append_column(
|
263
|
+
_PK_HASH_STRING_COLUMN_FIELD, get_pk_hash_string_column_array(pk_hashes)
|
264
|
+
)
|
265
|
+
return table
|
266
|
+
|
267
|
+
|
268
|
+
def append_hash_bucket_idx_col(table: pa.Table, hash_bucket_indexes) -> pa.Table:
|
269
|
+
|
270
|
+
table = table.append_column(
|
271
|
+
_HASH_BUCKET_IDX_COLUMN_FIELD,
|
272
|
+
get_hash_bucket_idx_column_array(hash_bucket_indexes),
|
273
|
+
)
|
274
|
+
|
275
|
+
return table
|
276
|
+
|
277
|
+
|
235
278
|
def append_record_idx_col(table: pa.Table, ordered_record_indices) -> pa.Table:
|
236
279
|
|
237
280
|
table = table.append_column(
|