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
@@ -0,0 +1,152 @@
|
|
1
|
+
import logging
|
2
|
+
import functools
|
3
|
+
from deltacat.constants import PYARROW_INFLATION_MULTIPLIER
|
4
|
+
from deltacat.storage import (
|
5
|
+
PartitionLocator,
|
6
|
+
Delta,
|
7
|
+
interface as unimplemented_deltacat_storage,
|
8
|
+
)
|
9
|
+
from deltacat import logs
|
10
|
+
from deltacat.compute.compactor.utils import io as io_v1
|
11
|
+
from deltacat.compute.compactor import DeltaAnnotated
|
12
|
+
from typing import Dict, List, Optional, Any
|
13
|
+
from deltacat.compute.compactor_v2.constants import (
|
14
|
+
MIN_FILES_IN_BATCH,
|
15
|
+
MIN_DELTA_BYTES_IN_BATCH,
|
16
|
+
)
|
17
|
+
from deltacat.compute.compactor.model.compaction_session_audit_info import (
|
18
|
+
CompactionSessionAuditInfo,
|
19
|
+
)
|
20
|
+
from deltacat.compute.compactor_v2.utils.task_options import (
|
21
|
+
estimate_manifest_entry_size_bytes,
|
22
|
+
)
|
23
|
+
from deltacat.compute.compactor_v2.utils.content_type_params import (
|
24
|
+
append_content_type_params,
|
25
|
+
)
|
26
|
+
|
27
|
+
logger = logs.configure_deltacat_logger(logging.getLogger(__name__))
|
28
|
+
|
29
|
+
|
30
|
+
def discover_deltas(
|
31
|
+
source_partition_locator: PartitionLocator,
|
32
|
+
last_stream_position_to_compact: int,
|
33
|
+
rebase_source_partition_locator: Optional[PartitionLocator] = None,
|
34
|
+
rebase_source_partition_high_watermark: Optional[int] = None,
|
35
|
+
rcf_high_watermark: Optional[int] = None,
|
36
|
+
deltacat_storage=unimplemented_deltacat_storage,
|
37
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = {},
|
38
|
+
list_deltas_kwargs: Optional[Dict[str, Any]] = {},
|
39
|
+
) -> List[Delta]:
|
40
|
+
|
41
|
+
previous_compacted_high_watermark = (
|
42
|
+
rebase_source_partition_high_watermark or rcf_high_watermark
|
43
|
+
)
|
44
|
+
|
45
|
+
delta_source_partition_locator = (
|
46
|
+
rebase_source_partition_locator or source_partition_locator
|
47
|
+
)
|
48
|
+
|
49
|
+
result = []
|
50
|
+
|
51
|
+
delta_source_incremental_deltas = io_v1._discover_deltas(
|
52
|
+
delta_source_partition_locator,
|
53
|
+
previous_compacted_high_watermark,
|
54
|
+
last_stream_position_to_compact,
|
55
|
+
deltacat_storage,
|
56
|
+
deltacat_storage_kwargs,
|
57
|
+
list_deltas_kwargs,
|
58
|
+
)
|
59
|
+
|
60
|
+
result.extend(delta_source_incremental_deltas)
|
61
|
+
|
62
|
+
logger.info(
|
63
|
+
f"Length of input deltas from delta source table is {len(delta_source_incremental_deltas)}"
|
64
|
+
f" from ({previous_compacted_high_watermark}, {last_stream_position_to_compact}]"
|
65
|
+
)
|
66
|
+
|
67
|
+
if rebase_source_partition_locator:
|
68
|
+
previous_compacted_deltas = io_v1._discover_deltas(
|
69
|
+
source_partition_locator,
|
70
|
+
None,
|
71
|
+
None,
|
72
|
+
deltacat_storage,
|
73
|
+
deltacat_storage_kwargs,
|
74
|
+
list_deltas_kwargs,
|
75
|
+
)
|
76
|
+
|
77
|
+
result.extend(previous_compacted_deltas)
|
78
|
+
|
79
|
+
logger.info(
|
80
|
+
f"Length of input deltas from previous compacted table is {len(previous_compacted_deltas)}"
|
81
|
+
f" from ({None}, {None}]"
|
82
|
+
)
|
83
|
+
|
84
|
+
return result
|
85
|
+
|
86
|
+
|
87
|
+
def create_uniform_input_deltas(
|
88
|
+
input_deltas: List[Delta],
|
89
|
+
hash_bucket_count: int,
|
90
|
+
compaction_audit: CompactionSessionAuditInfo,
|
91
|
+
min_delta_bytes: Optional[float] = MIN_DELTA_BYTES_IN_BATCH,
|
92
|
+
min_file_counts: Optional[float] = MIN_FILES_IN_BATCH,
|
93
|
+
previous_inflation: Optional[float] = PYARROW_INFLATION_MULTIPLIER,
|
94
|
+
enable_input_split: Optional[bool] = False,
|
95
|
+
deltacat_storage=unimplemented_deltacat_storage,
|
96
|
+
deltacat_storage_kwargs: Optional[Dict[str, Any]] = {},
|
97
|
+
) -> List[DeltaAnnotated]:
|
98
|
+
|
99
|
+
delta_bytes = 0
|
100
|
+
delta_manifest_entries_count = 0
|
101
|
+
estimated_da_bytes = 0
|
102
|
+
input_da_list = []
|
103
|
+
|
104
|
+
for delta in input_deltas:
|
105
|
+
if enable_input_split:
|
106
|
+
append_content_type_params(
|
107
|
+
delta=delta,
|
108
|
+
deltacat_storage=deltacat_storage,
|
109
|
+
deltacat_storage_kwargs=deltacat_storage_kwargs,
|
110
|
+
)
|
111
|
+
|
112
|
+
manifest_entries = delta.manifest.entries
|
113
|
+
delta_manifest_entries_count += len(manifest_entries)
|
114
|
+
|
115
|
+
for entry_index in range(len(manifest_entries)):
|
116
|
+
entry = manifest_entries[entry_index]
|
117
|
+
delta_bytes += entry.meta.content_length
|
118
|
+
estimated_da_bytes += estimate_manifest_entry_size_bytes(
|
119
|
+
entry=entry, previous_inflation=previous_inflation
|
120
|
+
)
|
121
|
+
|
122
|
+
delta_annotated = DeltaAnnotated.of(delta)
|
123
|
+
input_da_list.append(delta_annotated)
|
124
|
+
|
125
|
+
logger.info(f"Input deltas to compact this round: " f"{len(input_da_list)}")
|
126
|
+
logger.info(f"Input delta bytes to compact: {delta_bytes}")
|
127
|
+
logger.info(f"Input delta files to compact: {delta_manifest_entries_count}")
|
128
|
+
|
129
|
+
if not input_da_list:
|
130
|
+
raise RuntimeError("No input deltas to compact!")
|
131
|
+
|
132
|
+
size_estimation_function = functools.partial(
|
133
|
+
estimate_manifest_entry_size_bytes, previous_inflation=previous_inflation
|
134
|
+
)
|
135
|
+
|
136
|
+
rebatched_da_list = DeltaAnnotated.rebatch(
|
137
|
+
input_da_list,
|
138
|
+
min_delta_bytes=min_delta_bytes,
|
139
|
+
min_file_counts=min_file_counts,
|
140
|
+
estimation_function=size_estimation_function,
|
141
|
+
)
|
142
|
+
|
143
|
+
compaction_audit.set_input_size_bytes(delta_bytes)
|
144
|
+
compaction_audit.set_input_file_count(delta_manifest_entries_count)
|
145
|
+
compaction_audit.set_estimated_in_memory_size_bytes_during_discovery(
|
146
|
+
estimated_da_bytes
|
147
|
+
)
|
148
|
+
|
149
|
+
logger.info(f"Hash bucket count: {hash_bucket_count}")
|
150
|
+
logger.info(f"Input uniform delta count: {len(rebatched_da_list)}")
|
151
|
+
|
152
|
+
return rebatched_da_list
|
@@ -0,0 +1,341 @@
|
|
1
|
+
import logging
|
2
|
+
from typing import List, Optional, Iterable
|
3
|
+
|
4
|
+
import numpy as np
|
5
|
+
import pyarrow as pa
|
6
|
+
import pyarrow.compute as pc
|
7
|
+
import uuid
|
8
|
+
import hashlib
|
9
|
+
from deltacat.compute.compactor_v2.constants import (
|
10
|
+
TOTAL_BYTES_IN_SHA1_HASH,
|
11
|
+
PK_DELIMITER,
|
12
|
+
MAX_SIZE_OF_RECORD_BATCH_IN_GIB,
|
13
|
+
)
|
14
|
+
import time
|
15
|
+
from deltacat.compute.compactor.model.delta_file_envelope import DeltaFileEnvelope
|
16
|
+
from deltacat import logs
|
17
|
+
from deltacat.compute.compactor.utils import system_columns as sc
|
18
|
+
from deltacat.io.object_store import IObjectStore
|
19
|
+
from deltacat.utils.performance import timed_invocation
|
20
|
+
|
21
|
+
logger = logs.configure_deltacat_logger(logging.getLogger(__name__))
|
22
|
+
|
23
|
+
|
24
|
+
def _append_sha1_hash_to_table(table: pa.Table, hash_column: pa.Array) -> pa.Table:
|
25
|
+
hash_column_np = hash_column.to_numpy()
|
26
|
+
|
27
|
+
result = []
|
28
|
+
for hash_value in hash_column_np:
|
29
|
+
result.append(hashlib.sha1(hash_value.encode("utf-8")).hexdigest())
|
30
|
+
|
31
|
+
return sc.append_pk_hash_string_column(table, result)
|
32
|
+
|
33
|
+
|
34
|
+
def _is_sha1_desired(hash_columns: List[pa.Array]) -> bool:
|
35
|
+
total_size = 0
|
36
|
+
total_len = 0
|
37
|
+
|
38
|
+
for hash_column in hash_columns:
|
39
|
+
total_size += hash_column.nbytes
|
40
|
+
total_len += len(hash_column)
|
41
|
+
|
42
|
+
logger.info(
|
43
|
+
f"Found total length of hash column={total_len} and total_size={total_size}"
|
44
|
+
)
|
45
|
+
|
46
|
+
return total_size > TOTAL_BYTES_IN_SHA1_HASH * total_len
|
47
|
+
|
48
|
+
|
49
|
+
def _append_table_by_hash_bucket(
|
50
|
+
pki_table: pa.Table, hash_bucket_to_table: np.ndarray
|
51
|
+
) -> int:
|
52
|
+
|
53
|
+
hb_pk_table, sort_latency = timed_invocation(
|
54
|
+
lambda: pki_table.sort_by(sc._HASH_BUCKET_IDX_COLUMN_NAME)
|
55
|
+
)
|
56
|
+
logger.info(f"Sorting a pk table of length {len(pki_table)} took {sort_latency}s")
|
57
|
+
|
58
|
+
hb_pk_grouped_by, groupby_latency = timed_invocation(
|
59
|
+
lambda: hb_pk_table.group_by(sc._HASH_BUCKET_IDX_COLUMN_NAME).aggregate(
|
60
|
+
[(sc._HASH_BUCKET_IDX_COLUMN_NAME, "count")]
|
61
|
+
)
|
62
|
+
)
|
63
|
+
|
64
|
+
logger.info(
|
65
|
+
f"Grouping a pki table of length {len(pki_table)} took {groupby_latency}s"
|
66
|
+
)
|
67
|
+
|
68
|
+
group_count_array = hb_pk_grouped_by[f"{sc._HASH_BUCKET_IDX_COLUMN_NAME}_count"]
|
69
|
+
hb_group_array = hb_pk_grouped_by[sc._HASH_BUCKET_IDX_COLUMN_NAME]
|
70
|
+
|
71
|
+
result_len = 0
|
72
|
+
for i, group_count in enumerate(group_count_array):
|
73
|
+
hb_idx = hb_group_array[i].as_py()
|
74
|
+
pyarrow_table = hb_pk_table.slice(offset=result_len, length=group_count.as_py())
|
75
|
+
pyarrow_table = pyarrow_table.drop(
|
76
|
+
[sc._HASH_BUCKET_IDX_COLUMN_NAME, sc._PK_HASH_STRING_COLUMN_NAME]
|
77
|
+
)
|
78
|
+
if hash_bucket_to_table[hb_idx] is None:
|
79
|
+
hash_bucket_to_table[hb_idx] = []
|
80
|
+
hash_bucket_to_table[hb_idx].append(pyarrow_table)
|
81
|
+
result_len += len(pyarrow_table)
|
82
|
+
|
83
|
+
return result_len
|
84
|
+
|
85
|
+
|
86
|
+
def _optimized_group_record_batches_by_hash_bucket(
|
87
|
+
pki_table: pa.Table, num_buckets: int
|
88
|
+
):
|
89
|
+
|
90
|
+
input_table_len = len(pki_table)
|
91
|
+
|
92
|
+
hash_bucket_to_tables = np.empty([num_buckets], dtype="object")
|
93
|
+
hb_to_table = np.empty([num_buckets], dtype="object")
|
94
|
+
|
95
|
+
# This split will ensure that the sort is not performed on a very huge table
|
96
|
+
# resulting in ArrowInvalid: offset overflow while concatenating arrays
|
97
|
+
# Known issue with Arrow: https://github.com/apache/arrow/issues/25822
|
98
|
+
table_batches, to_batches_latency = timed_invocation(lambda: pki_table.to_batches())
|
99
|
+
|
100
|
+
logger.info(f"to_batches took {to_batches_latency} for {len(pki_table)} rows")
|
101
|
+
|
102
|
+
current_bytes = 0
|
103
|
+
record_batches = []
|
104
|
+
result_len = 0
|
105
|
+
for record_batch in table_batches:
|
106
|
+
current_bytes += record_batch.nbytes
|
107
|
+
record_batches.append(record_batch)
|
108
|
+
if current_bytes >= MAX_SIZE_OF_RECORD_BATCH_IN_GIB:
|
109
|
+
logger.info(
|
110
|
+
f"Total number of record batches without exceeding {MAX_SIZE_OF_RECORD_BATCH_IN_GIB} "
|
111
|
+
f"is {len(record_batches)} and size {current_bytes}"
|
112
|
+
)
|
113
|
+
appended_len, append_latency = timed_invocation(
|
114
|
+
_append_table_by_hash_bucket,
|
115
|
+
pa.Table.from_batches(record_batches),
|
116
|
+
hash_bucket_to_tables,
|
117
|
+
)
|
118
|
+
logger.info(
|
119
|
+
f"Appended the hash bucketed batch of {appended_len} in {append_latency}s"
|
120
|
+
)
|
121
|
+
|
122
|
+
result_len += appended_len
|
123
|
+
current_bytes = 0
|
124
|
+
record_batches.clear()
|
125
|
+
|
126
|
+
if record_batches:
|
127
|
+
appended_len, append_latency = timed_invocation(
|
128
|
+
_append_table_by_hash_bucket,
|
129
|
+
pa.Table.from_batches(record_batches),
|
130
|
+
hash_bucket_to_tables,
|
131
|
+
)
|
132
|
+
result_len += appended_len
|
133
|
+
current_bytes = 0
|
134
|
+
record_batches.clear()
|
135
|
+
|
136
|
+
concat_start = time.monotonic()
|
137
|
+
for hb, tables in enumerate(hash_bucket_to_tables):
|
138
|
+
if tables:
|
139
|
+
assert hb_to_table[hb] is None, f"The HB index is repeated {hb}"
|
140
|
+
hb_to_table[hb] = pa.concat_tables(tables)
|
141
|
+
|
142
|
+
concat_end = time.monotonic()
|
143
|
+
logger.info(
|
144
|
+
f"Total time taken to concat all record batches with length "
|
145
|
+
f"{input_table_len}: {concat_end - concat_start}s"
|
146
|
+
)
|
147
|
+
|
148
|
+
assert (
|
149
|
+
input_table_len == result_len
|
150
|
+
), f"Grouping has resulted in record loss as {result_len} != {input_table_len}"
|
151
|
+
|
152
|
+
return hb_to_table
|
153
|
+
|
154
|
+
|
155
|
+
def group_by_pk_hash_bucket(
|
156
|
+
table: pa.Table, num_buckets: int, primary_keys: List[str]
|
157
|
+
) -> np.ndarray:
|
158
|
+
table = generate_pk_hash_column([table], primary_keys, requires_sha1=True)[0]
|
159
|
+
|
160
|
+
# group hash bucket record indices
|
161
|
+
result = group_record_indices_by_hash_bucket(
|
162
|
+
table,
|
163
|
+
num_buckets,
|
164
|
+
)
|
165
|
+
|
166
|
+
return result
|
167
|
+
|
168
|
+
|
169
|
+
def generate_pk_hash_column(
|
170
|
+
tables: List[pa.Table],
|
171
|
+
primary_keys: Optional[List[str]] = None,
|
172
|
+
requires_sha1: bool = False,
|
173
|
+
) -> List[pa.Table]:
|
174
|
+
"""
|
175
|
+
Returns a new table list after generating the primary key hash if desired.
|
176
|
+
|
177
|
+
1. If there are no primary keys, each hash will be unique uuid/sha1 hex
|
178
|
+
2. If there are more than 0 primary keys, returns a table with pk hash column appended.
|
179
|
+
"""
|
180
|
+
|
181
|
+
def _generate_pk_hash(table: pa.Table) -> pa.Array:
|
182
|
+
pk_columns = []
|
183
|
+
for pk_name in primary_keys:
|
184
|
+
pk_columns.append(pc.cast(table[pk_name], pa.string()))
|
185
|
+
|
186
|
+
pk_columns.append(PK_DELIMITER)
|
187
|
+
hash_column = pc.binary_join_element_wise(*pk_columns)
|
188
|
+
return hash_column
|
189
|
+
|
190
|
+
def _generate_uuid(table: pa.Table) -> pa.Array:
|
191
|
+
hash_column = pa.array(
|
192
|
+
[uuid.uuid4().hex for _ in range(len(table))], pa.string()
|
193
|
+
)
|
194
|
+
return hash_column
|
195
|
+
|
196
|
+
start = time.monotonic()
|
197
|
+
|
198
|
+
hash_column_list = []
|
199
|
+
|
200
|
+
can_sha1 = False
|
201
|
+
if primary_keys:
|
202
|
+
hash_column_list = [_generate_pk_hash(table) for table in tables]
|
203
|
+
|
204
|
+
can_sha1 = requires_sha1 or _is_sha1_desired(hash_column_list)
|
205
|
+
else:
|
206
|
+
hash_column_list = [_generate_uuid(table) for table in tables]
|
207
|
+
|
208
|
+
logger.info(
|
209
|
+
f"can_generate_sha1={can_sha1} for the table and requires_sha1={requires_sha1}"
|
210
|
+
)
|
211
|
+
|
212
|
+
result = []
|
213
|
+
|
214
|
+
total_len = 0
|
215
|
+
total_size = 0
|
216
|
+
for index, table in enumerate(tables):
|
217
|
+
if can_sha1:
|
218
|
+
table = _append_sha1_hash_to_table(table, hash_column_list[index])
|
219
|
+
else:
|
220
|
+
table = table.append_column(
|
221
|
+
sc._PK_HASH_STRING_COLUMN_FIELD, hash_column_list[index]
|
222
|
+
)
|
223
|
+
|
224
|
+
total_len += len(table)
|
225
|
+
total_size += hash_column_list[index].nbytes
|
226
|
+
|
227
|
+
result.append(table)
|
228
|
+
|
229
|
+
end = time.monotonic()
|
230
|
+
|
231
|
+
logger.info(
|
232
|
+
f"Took {end - start}s to generate pk hash of len: {total_len}"
|
233
|
+
f" for size: {total_size} bytes"
|
234
|
+
)
|
235
|
+
|
236
|
+
return result
|
237
|
+
|
238
|
+
|
239
|
+
def group_record_indices_by_hash_bucket(
|
240
|
+
pki_table: pa.Table, num_buckets: int
|
241
|
+
) -> np.ndarray:
|
242
|
+
"""
|
243
|
+
Groups the record indices by it's corresponding hash bucket. Hence, this method may
|
244
|
+
create num_buckets tables as a result.
|
245
|
+
"""
|
246
|
+
|
247
|
+
input_table_len = len(pki_table)
|
248
|
+
|
249
|
+
hash_bucket_id_col_list = np.empty([input_table_len], dtype="int32")
|
250
|
+
bucketing_start_time = time.monotonic()
|
251
|
+
|
252
|
+
for index, hash_value in enumerate(sc.pk_hash_string_column_np(pki_table)):
|
253
|
+
hash_bucket = pk_digest_to_hash_bucket_index(hash_value, num_buckets)
|
254
|
+
hash_bucket_id_col_list[index] = hash_bucket
|
255
|
+
|
256
|
+
pki_table = sc.append_hash_bucket_idx_col(pki_table, hash_bucket_id_col_list)
|
257
|
+
bucketing_end_time = time.monotonic()
|
258
|
+
|
259
|
+
logger.info(
|
260
|
+
f"Took {bucketing_end_time - bucketing_start_time}s to generate the "
|
261
|
+
f"hb index for {len(pki_table)} rows"
|
262
|
+
)
|
263
|
+
|
264
|
+
result, group_latency = timed_invocation(
|
265
|
+
_optimized_group_record_batches_by_hash_bucket,
|
266
|
+
pki_table=pki_table,
|
267
|
+
num_buckets=num_buckets,
|
268
|
+
)
|
269
|
+
|
270
|
+
logger.info(
|
271
|
+
f"Final grouping of table with {input_table_len} records took: {group_latency}s"
|
272
|
+
)
|
273
|
+
|
274
|
+
return result
|
275
|
+
|
276
|
+
|
277
|
+
def group_hash_bucket_indices(
|
278
|
+
hash_bucket_object_groups: np.ndarray,
|
279
|
+
num_buckets: int,
|
280
|
+
num_groups: int,
|
281
|
+
object_store: Optional[IObjectStore] = None,
|
282
|
+
) -> np.ndarray:
|
283
|
+
"""
|
284
|
+
This method persists all tables for a given hash bucket into the object store
|
285
|
+
and returns the object references for each hash group.
|
286
|
+
"""
|
287
|
+
|
288
|
+
hash_bucket_group_to_obj_id_size_tuple = np.empty([num_groups], dtype="object")
|
289
|
+
|
290
|
+
if hash_bucket_object_groups is None:
|
291
|
+
return hash_bucket_group_to_obj_id_size_tuple
|
292
|
+
|
293
|
+
hb_group_to_object = np.empty([num_groups], dtype="object")
|
294
|
+
hash_group_to_size = np.empty([num_groups], dtype="int64")
|
295
|
+
hash_group_to_num_rows = np.empty([num_groups], dtype="int64")
|
296
|
+
|
297
|
+
for hb_index, obj in enumerate(hash_bucket_object_groups):
|
298
|
+
if obj:
|
299
|
+
hb_group = hash_bucket_index_to_hash_group_index(hb_index, num_groups)
|
300
|
+
if hb_group_to_object[hb_group] is None:
|
301
|
+
hb_group_to_object[hb_group] = np.empty([num_buckets], dtype="object")
|
302
|
+
hash_group_to_size[hb_group] = np.int64(0)
|
303
|
+
hash_group_to_num_rows[hb_group] = np.int64(0)
|
304
|
+
hb_group_to_object[hb_group][hb_index] = obj
|
305
|
+
for dfe in obj:
|
306
|
+
casted_dfe: DeltaFileEnvelope = dfe
|
307
|
+
hash_group_to_size[hb_group] += casted_dfe.table_size_bytes
|
308
|
+
hash_group_to_num_rows[hb_group] += casted_dfe.table_num_rows
|
309
|
+
|
310
|
+
for hb_group, obj in enumerate(hb_group_to_object):
|
311
|
+
if obj is None:
|
312
|
+
continue
|
313
|
+
object_ref = object_store.put(obj)
|
314
|
+
hash_bucket_group_to_obj_id_size_tuple[hb_group] = (
|
315
|
+
object_ref,
|
316
|
+
hash_group_to_size[hb_group],
|
317
|
+
hash_group_to_num_rows[hb_group],
|
318
|
+
)
|
319
|
+
del object_ref
|
320
|
+
return hash_bucket_group_to_obj_id_size_tuple
|
321
|
+
|
322
|
+
|
323
|
+
def hash_bucket_index_to_hash_group_index(hb_index: int, num_groups: int) -> int:
|
324
|
+
return hb_index % num_groups
|
325
|
+
|
326
|
+
|
327
|
+
def hash_group_index_to_hash_bucket_indices(
|
328
|
+
hb_group: int, num_buckets: int, num_groups: int
|
329
|
+
) -> Iterable[int]:
|
330
|
+
|
331
|
+
if hb_group > num_buckets:
|
332
|
+
return []
|
333
|
+
|
334
|
+
return range(hb_group, num_buckets, num_groups)
|
335
|
+
|
336
|
+
|
337
|
+
def pk_digest_to_hash_bucket_index(digest: str, num_buckets: int) -> int:
|
338
|
+
"""
|
339
|
+
Generates the hash bucket index from the given digest.
|
340
|
+
"""
|
341
|
+
return int(digest, 16) % num_buckets
|
@@ -0,0 +1,221 @@
|
|
1
|
+
from typing import Dict, Optional, List, Tuple
|
2
|
+
from deltacat.types.media import ContentEncoding, ContentType
|
3
|
+
from deltacat.types.partial_download import PartialParquetParameters
|
4
|
+
from deltacat.storage import (
|
5
|
+
Manifest,
|
6
|
+
ManifestEntry,
|
7
|
+
interface as unimplemented_deltacat_storage,
|
8
|
+
)
|
9
|
+
from deltacat.compute.compactor.model.delta_annotated import DeltaAnnotated
|
10
|
+
from deltacat.compute.compactor.model.round_completion_info import RoundCompletionInfo
|
11
|
+
from deltacat.compute.compactor_v2.utils.primary_key_index import (
|
12
|
+
hash_group_index_to_hash_bucket_indices,
|
13
|
+
)
|
14
|
+
from deltacat.compute.compactor_v2.constants import TOTAL_MEMORY_BUFFER_PERCENTAGE
|
15
|
+
|
16
|
+
|
17
|
+
def _get_parquet_type_params_if_exist(
|
18
|
+
entry: ManifestEntry,
|
19
|
+
) -> Optional[PartialParquetParameters]:
|
20
|
+
if (
|
21
|
+
entry.meta
|
22
|
+
and entry.meta.content_type == ContentType.PARQUET
|
23
|
+
and entry.meta.content_encoding == ContentEncoding.IDENTITY
|
24
|
+
and entry.meta.content_type_parameters
|
25
|
+
):
|
26
|
+
for type_params in entry.meta.content_type_parameters:
|
27
|
+
if isinstance(type_params, PartialParquetParameters):
|
28
|
+
return type_params
|
29
|
+
return None
|
30
|
+
|
31
|
+
|
32
|
+
def _calculate_parquet_column_size(
|
33
|
+
type_params: PartialParquetParameters, columns: List[str]
|
34
|
+
):
|
35
|
+
column_size = 0.0
|
36
|
+
for rg in type_params.row_groups_to_download:
|
37
|
+
columns_found = 0
|
38
|
+
row_group_meta = type_params.pq_metadata.row_group(rg)
|
39
|
+
for col in range(row_group_meta.num_columns):
|
40
|
+
column_meta = row_group_meta.column(col)
|
41
|
+
if column_meta.path_in_schema in columns:
|
42
|
+
columns_found += 1
|
43
|
+
column_size += column_meta.total_uncompressed_size
|
44
|
+
assert columns_found == len(columns), (
|
45
|
+
"Columns not found in the parquet data as "
|
46
|
+
f"{columns_found} != {len(columns)}"
|
47
|
+
)
|
48
|
+
return column_size
|
49
|
+
|
50
|
+
|
51
|
+
def estimate_manifest_entry_size_bytes(
|
52
|
+
entry: ManifestEntry, previous_inflation: float, **kwargs
|
53
|
+
) -> float:
|
54
|
+
if entry.meta.source_content_length:
|
55
|
+
return entry.meta.source_content_length
|
56
|
+
|
57
|
+
type_params = _get_parquet_type_params_if_exist(entry=entry)
|
58
|
+
|
59
|
+
if type_params:
|
60
|
+
return type_params.in_memory_size_bytes
|
61
|
+
|
62
|
+
return entry.meta.content_length * previous_inflation
|
63
|
+
|
64
|
+
|
65
|
+
def estimate_manifest_entry_num_rows(
|
66
|
+
entry: ManifestEntry,
|
67
|
+
average_record_size_bytes: float,
|
68
|
+
previous_inflation: float,
|
69
|
+
**kwargs,
|
70
|
+
) -> int:
|
71
|
+
if entry.meta.record_count:
|
72
|
+
return entry.meta.record_count
|
73
|
+
|
74
|
+
type_params = _get_parquet_type_params_if_exist(entry=entry)
|
75
|
+
|
76
|
+
if type_params:
|
77
|
+
return type_params.num_rows
|
78
|
+
|
79
|
+
total_size_bytes = estimate_manifest_entry_size_bytes(
|
80
|
+
entry=entry, previous_inflation=previous_inflation, **kwargs
|
81
|
+
)
|
82
|
+
|
83
|
+
return int(total_size_bytes / average_record_size_bytes)
|
84
|
+
|
85
|
+
|
86
|
+
def estimate_manifest_entry_column_size_bytes(
|
87
|
+
entry: ManifestEntry, columns: Optional[List[str]] = None
|
88
|
+
) -> Optional[float]:
|
89
|
+
if not columns:
|
90
|
+
return 0
|
91
|
+
|
92
|
+
type_params = _get_parquet_type_params_if_exist(entry=entry)
|
93
|
+
|
94
|
+
if type_params and type_params.pq_metadata:
|
95
|
+
return _calculate_parquet_column_size(type_params=type_params, columns=columns)
|
96
|
+
|
97
|
+
return None
|
98
|
+
|
99
|
+
|
100
|
+
def hash_bucket_resource_options_provider(
|
101
|
+
index: int,
|
102
|
+
item: DeltaAnnotated,
|
103
|
+
previous_inflation: float,
|
104
|
+
average_record_size_bytes: float,
|
105
|
+
primary_keys: List[str] = None,
|
106
|
+
**kwargs,
|
107
|
+
) -> Dict:
|
108
|
+
size_bytes = 0.0
|
109
|
+
num_rows = 0
|
110
|
+
total_pk_size = 0
|
111
|
+
|
112
|
+
if not item.manifest or not item.manifest.entries:
|
113
|
+
return {"CPU": 0.01}
|
114
|
+
|
115
|
+
for entry in item.manifest.entries:
|
116
|
+
entry_size = estimate_manifest_entry_size_bytes(
|
117
|
+
entry=entry, previous_inflation=previous_inflation
|
118
|
+
)
|
119
|
+
num_rows += estimate_manifest_entry_num_rows(
|
120
|
+
entry=entry,
|
121
|
+
previous_inflation=previous_inflation,
|
122
|
+
average_record_size_bytes=average_record_size_bytes,
|
123
|
+
)
|
124
|
+
size_bytes += entry_size
|
125
|
+
|
126
|
+
if primary_keys:
|
127
|
+
pk_size = estimate_manifest_entry_column_size_bytes(
|
128
|
+
entry=entry,
|
129
|
+
columns=primary_keys,
|
130
|
+
)
|
131
|
+
|
132
|
+
if pk_size is None:
|
133
|
+
total_pk_size += entry_size
|
134
|
+
else:
|
135
|
+
total_pk_size += pk_size
|
136
|
+
|
137
|
+
# total size + pk size + pk hash column + hash bucket index column
|
138
|
+
# Refer to hash_bucket step for more details.
|
139
|
+
total_memory = size_bytes + total_pk_size + num_rows * 20 + num_rows * 4
|
140
|
+
|
141
|
+
# Consider buffer
|
142
|
+
total_memory = total_memory * (1 + TOTAL_MEMORY_BUFFER_PERCENTAGE / 100.0)
|
143
|
+
|
144
|
+
return {"num_cpus": 0.01, "memory": total_memory}
|
145
|
+
|
146
|
+
|
147
|
+
def merge_resource_options_provider(
|
148
|
+
index: int,
|
149
|
+
item: Tuple[int, List],
|
150
|
+
num_hash_groups: int,
|
151
|
+
hash_group_size_bytes: Dict[int, int],
|
152
|
+
hash_group_num_rows: Dict[int, int],
|
153
|
+
round_completion_info: Optional[RoundCompletionInfo] = None,
|
154
|
+
compacted_delta_manifest: Optional[Manifest] = None,
|
155
|
+
primary_keys: Optional[List[str]] = None,
|
156
|
+
deltacat_storage=unimplemented_deltacat_storage,
|
157
|
+
deltacat_storage_kwargs: Optional[Dict] = {},
|
158
|
+
**kwargs,
|
159
|
+
) -> Dict:
|
160
|
+
hb_group_idx = item[0]
|
161
|
+
|
162
|
+
data_size = hash_group_size_bytes.get(hb_group_idx, 0)
|
163
|
+
num_rows = hash_group_num_rows.get(hb_group_idx, 0)
|
164
|
+
|
165
|
+
pk_size_bytes = 0
|
166
|
+
|
167
|
+
if (
|
168
|
+
round_completion_info
|
169
|
+
and compacted_delta_manifest
|
170
|
+
and round_completion_info.hb_index_to_entry_range
|
171
|
+
):
|
172
|
+
|
173
|
+
previous_inflation = (
|
174
|
+
round_completion_info.compacted_pyarrow_write_result.pyarrow_bytes
|
175
|
+
/ round_completion_info.compacted_pyarrow_write_result.file_bytes
|
176
|
+
)
|
177
|
+
average_record_size = (
|
178
|
+
round_completion_info.compacted_pyarrow_write_result.pyarrow_bytes
|
179
|
+
/ round_completion_info.compacted_pyarrow_write_result.records
|
180
|
+
)
|
181
|
+
|
182
|
+
iterable = hash_group_index_to_hash_bucket_indices(
|
183
|
+
hb_group_idx, round_completion_info.hash_bucket_count, num_hash_groups
|
184
|
+
)
|
185
|
+
|
186
|
+
for hb_idx in iterable:
|
187
|
+
entry_start, entry_end = round_completion_info.hb_index_to_entry_range[
|
188
|
+
str(hb_idx)
|
189
|
+
]
|
190
|
+
for entry_index in range(entry_start, entry_end):
|
191
|
+
entry = compacted_delta_manifest.entries[entry_index]
|
192
|
+
|
193
|
+
current_entry_size = estimate_manifest_entry_size_bytes(
|
194
|
+
entry=entry, previous_inflation=previous_inflation
|
195
|
+
)
|
196
|
+
current_entry_rows = estimate_manifest_entry_num_rows(
|
197
|
+
entry=entry,
|
198
|
+
average_record_size_bytes=average_record_size,
|
199
|
+
previous_inflation=previous_inflation,
|
200
|
+
)
|
201
|
+
|
202
|
+
data_size += current_entry_size
|
203
|
+
num_rows += current_entry_rows
|
204
|
+
|
205
|
+
if primary_keys:
|
206
|
+
pk_size = estimate_manifest_entry_column_size_bytes(
|
207
|
+
entry=entry,
|
208
|
+
columns=primary_keys,
|
209
|
+
)
|
210
|
+
|
211
|
+
if pk_size is None:
|
212
|
+
pk_size_bytes += current_entry_size
|
213
|
+
else:
|
214
|
+
pk_size_bytes += pk_size
|
215
|
+
|
216
|
+
# total data downloaded + primary key hash column + primary key column + dict size for merge
|
217
|
+
total_memory = data_size + pk_size_bytes + num_rows * 20 + num_rows * 20
|
218
|
+
|
219
|
+
total_memory = total_memory * (1 + TOTAL_MEMORY_BUFFER_PERCENTAGE / 100.0)
|
220
|
+
|
221
|
+
return {"num_cpus": 0.01, "memory": total_memory}
|