deltacat 2.0.0b7__py3-none-any.whl → 2.0.0b10__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 +27 -6
- deltacat/api.py +478 -123
- deltacat/aws/s3u.py +2 -2
- deltacat/benchmarking/conftest.py +1 -1
- deltacat/catalog/main/impl.py +12 -6
- deltacat/catalog/model/catalog.py +65 -47
- deltacat/catalog/model/properties.py +1 -3
- deltacat/compute/__init__.py +14 -0
- deltacat/compute/converter/constants.py +5 -0
- deltacat/compute/converter/converter_session.py +78 -36
- deltacat/compute/converter/model/convert_input.py +24 -4
- deltacat/compute/converter/model/convert_result.py +61 -0
- deltacat/compute/converter/model/converter_session_params.py +52 -10
- deltacat/compute/converter/pyiceberg/overrides.py +181 -62
- deltacat/compute/converter/steps/convert.py +84 -36
- deltacat/compute/converter/steps/dedupe.py +25 -4
- deltacat/compute/converter/utils/convert_task_options.py +42 -13
- deltacat/compute/converter/utils/iceberg_columns.py +5 -0
- deltacat/compute/converter/utils/io.py +82 -11
- deltacat/compute/converter/utils/s3u.py +13 -4
- deltacat/compute/jobs/__init__.py +0 -0
- deltacat/compute/jobs/client.py +404 -0
- deltacat/constants.py +4 -4
- deltacat/daft/daft_scan.py +7 -3
- deltacat/daft/translator.py +126 -0
- deltacat/examples/basic_logging.py +5 -3
- deltacat/examples/hello_world.py +4 -2
- deltacat/examples/indexer/__init__.py +0 -0
- deltacat/examples/indexer/aws/__init__.py +0 -0
- deltacat/examples/indexer/gcp/__init__.py +0 -0
- deltacat/examples/indexer/indexer.py +163 -0
- deltacat/examples/indexer/job_runner.py +199 -0
- deltacat/io/__init__.py +13 -0
- deltacat/io/dataset/__init__.py +0 -0
- deltacat/io/dataset/deltacat_dataset.py +91 -0
- deltacat/io/datasink/__init__.py +0 -0
- deltacat/io/datasink/deltacat_datasink.py +207 -0
- deltacat/io/datasource/__init__.py +0 -0
- deltacat/io/datasource/deltacat_datasource.py +580 -0
- deltacat/io/reader/__init__.py +0 -0
- deltacat/io/reader/deltacat_read_api.py +172 -0
- deltacat/storage/__init__.py +2 -0
- deltacat/storage/model/expression/__init__.py +47 -0
- deltacat/storage/model/expression/expression.py +656 -0
- deltacat/storage/model/expression/visitor.py +248 -0
- deltacat/storage/model/metafile.py +74 -42
- deltacat/storage/model/scan/push_down.py +32 -5
- deltacat/storage/model/types.py +5 -3
- deltacat/storage/rivulet/__init__.py +4 -4
- deltacat/tests/_io/reader/__init__.py +0 -0
- deltacat/tests/_io/reader/test_deltacat_read_api.py +0 -0
- deltacat/tests/compute/converter/test_convert_session.py +209 -46
- deltacat/tests/local_deltacat_storage/__init__.py +1 -0
- deltacat/tests/storage/model/test_expression.py +327 -0
- deltacat/tests/storage/rivulet/fs/test_file_location_provider.py +2 -1
- deltacat/tests/storage/rivulet/test_dataset.py +1 -1
- deltacat/tests/storage/rivulet/test_manifest.py +1 -1
- deltacat/tests/storage/rivulet/writer/test_memtable_dataset_writer.py +1 -1
- deltacat/tests/test_deltacat_api.py +50 -9
- deltacat/types/media.py +141 -43
- deltacat/types/tables.py +35 -7
- deltacat/utils/daft.py +2 -2
- deltacat/utils/filesystem.py +39 -9
- deltacat/utils/polars.py +128 -0
- deltacat/utils/pyarrow.py +151 -15
- deltacat/utils/ray_utils/concurrency.py +1 -1
- deltacat/utils/ray_utils/runtime.py +56 -4
- deltacat/utils/url.py +1284 -0
- {deltacat-2.0.0b7.dist-info → deltacat-2.0.0b10.dist-info}/METADATA +9 -6
- {deltacat-2.0.0b7.dist-info → deltacat-2.0.0b10.dist-info}/RECORD +73 -48
- {deltacat-2.0.0b7.dist-info → deltacat-2.0.0b10.dist-info}/LICENSE +0 -0
- {deltacat-2.0.0b7.dist-info → deltacat-2.0.0b10.dist-info}/WHEEL +0 -0
- {deltacat-2.0.0b7.dist-info → deltacat-2.0.0b10.dist-info}/top_level.txt +0 -0
@@ -1,6 +1,10 @@
|
|
1
1
|
from __future__ import annotations
|
2
2
|
from typing import Optional, Dict
|
3
|
-
from deltacat.compute.converter.constants import
|
3
|
+
from deltacat.compute.converter.constants import (
|
4
|
+
DEFAULT_CONVERTER_TASK_MAX_PARALLELISM,
|
5
|
+
)
|
6
|
+
from deltacat.constants import DEFAULT_NAMESPACE
|
7
|
+
from fsspec import AbstractFileSystem
|
4
8
|
|
5
9
|
|
6
10
|
class ConverterSessionParams(dict):
|
@@ -18,15 +22,15 @@ class ConverterSessionParams(dict):
|
|
18
22
|
assert (
|
19
23
|
params.get("iceberg_warehouse_bucket_name") is not None
|
20
24
|
), "iceberg_warehouse_bucket_name is a required arg"
|
21
|
-
assert (
|
22
|
-
params.get("iceberg_namespace") is not None
|
23
|
-
), "iceberg_namespace is a required arg"
|
24
25
|
result = ConverterSessionParams(params)
|
25
26
|
|
27
|
+
result.iceberg_namespace = params.get("iceberg_namespace", DEFAULT_NAMESPACE)
|
26
28
|
result.enforce_primary_key_uniqueness = params.get(
|
27
29
|
"enforce_primary_key_uniqueness", False
|
28
30
|
)
|
29
|
-
result.
|
31
|
+
result.compact_previous_position_delete_files = params.get(
|
32
|
+
"compact_previous_position_delete_files", False
|
33
|
+
)
|
30
34
|
|
31
35
|
# For Iceberg v3 spec, option to produce delete vector that can establish 1:1 mapping with data files.
|
32
36
|
result.position_delete_for_multiple_data_files = params.get(
|
@@ -36,6 +40,10 @@ class ConverterSessionParams(dict):
|
|
36
40
|
"task_max_parallelism", DEFAULT_CONVERTER_TASK_MAX_PARALLELISM
|
37
41
|
)
|
38
42
|
result.merge_keys = params.get("merge_keys", None)
|
43
|
+
result.s3_client_kwargs = params.get("s3_client_kwargs", {})
|
44
|
+
result.s3_file_system = params.get("s3_file_system", None)
|
45
|
+
result.s3_prefix_override = params.get("s3_prefix_override", None)
|
46
|
+
|
39
47
|
return result
|
40
48
|
|
41
49
|
@property
|
@@ -54,6 +62,10 @@ class ConverterSessionParams(dict):
|
|
54
62
|
def iceberg_namespace(self) -> str:
|
55
63
|
return self["iceberg_namespace"]
|
56
64
|
|
65
|
+
@iceberg_namespace.setter
|
66
|
+
def iceberg_namespace(self, iceberg_namespace) -> None:
|
67
|
+
self["iceberg_namespace"] = iceberg_namespace
|
68
|
+
|
57
69
|
@property
|
58
70
|
def enforce_primary_key_uniqueness(self) -> bool:
|
59
71
|
return self["enforce_primary_key_uniqueness"]
|
@@ -63,12 +75,16 @@ class ConverterSessionParams(dict):
|
|
63
75
|
self["enforce_primary_key_uniqueness"] = enforce_primary_key_uniqueness
|
64
76
|
|
65
77
|
@property
|
66
|
-
def
|
67
|
-
return self["
|
78
|
+
def compact_previous_position_delete_files(self) -> bool:
|
79
|
+
return self["compact_previous_position_delete_files"]
|
68
80
|
|
69
|
-
@
|
70
|
-
def
|
71
|
-
self
|
81
|
+
@compact_previous_position_delete_files.setter
|
82
|
+
def compact_previous_position_delete_files(
|
83
|
+
self, compact_previous_position_delete_files
|
84
|
+
) -> None:
|
85
|
+
self[
|
86
|
+
"compact_previous_position_delete_files"
|
87
|
+
] = compact_previous_position_delete_files
|
72
88
|
|
73
89
|
@property
|
74
90
|
def position_delete_for_multiple_data_files(self) -> bool:
|
@@ -97,3 +113,29 @@ class ConverterSessionParams(dict):
|
|
97
113
|
@merge_keys.setter
|
98
114
|
def merge_keys(self, merge_keys) -> None:
|
99
115
|
self["merge_keys"] = merge_keys
|
116
|
+
|
117
|
+
@property
|
118
|
+
def s3_client_kwargs(self) -> Dict:
|
119
|
+
return self["s3_client_kwargs"]
|
120
|
+
|
121
|
+
@s3_client_kwargs.setter
|
122
|
+
def s3_client_kwargs(self, s3_client_kwargs) -> None:
|
123
|
+
self["s3_client_kwargs"] = s3_client_kwargs
|
124
|
+
|
125
|
+
@property
|
126
|
+
def s3_file_system(self) -> AbstractFileSystem:
|
127
|
+
return self["s3_file_system"]
|
128
|
+
|
129
|
+
@s3_file_system.setter
|
130
|
+
def s3_file_system(self, s3_file_system) -> None:
|
131
|
+
self["s3_file_system"] = s3_file_system
|
132
|
+
|
133
|
+
@property
|
134
|
+
def location_provider_prefix_override(self) -> str:
|
135
|
+
return self["location_provider_prefix_override"]
|
136
|
+
|
137
|
+
@location_provider_prefix_override.setter
|
138
|
+
def location_provider_prefix_override(
|
139
|
+
self, location_provider_prefix_override
|
140
|
+
) -> None:
|
141
|
+
self["location_provider_prefix_override"] = location_provider_prefix_override
|
@@ -1,68 +1,200 @@
|
|
1
1
|
from collections import defaultdict
|
2
2
|
import logging
|
3
3
|
from deltacat import logs
|
4
|
+
import pyarrow
|
4
5
|
import pyarrow.parquet as pq
|
6
|
+
from pyiceberg.io.pyarrow import (
|
7
|
+
parquet_path_to_id_mapping,
|
8
|
+
StatisticsCollector,
|
9
|
+
MetricModeTypes,
|
10
|
+
DataFileStatistics,
|
11
|
+
MetricsMode,
|
12
|
+
StatsAggregator,
|
13
|
+
)
|
14
|
+
from typing import Dict, List, Set
|
15
|
+
from deltacat.compute.converter.utils.iceberg_columns import (
|
16
|
+
ICEBERG_RESERVED_FIELD_ID_FOR_FILE_PATH_COLUMN,
|
17
|
+
ICEBERG_RESERVED_FIELD_ID_FOR_POS_COLUMN,
|
18
|
+
)
|
19
|
+
from pyiceberg.io.pyarrow import (
|
20
|
+
compute_statistics_plan,
|
21
|
+
)
|
22
|
+
from pyiceberg.manifest import (
|
23
|
+
DataFile,
|
24
|
+
DataFileContent,
|
25
|
+
FileFormat,
|
26
|
+
)
|
27
|
+
from pyiceberg.table import _min_sequence_number, _open_manifest
|
28
|
+
from pyiceberg.utils.concurrent import ExecutorFactory
|
29
|
+
from itertools import chain
|
30
|
+
from pyiceberg.typedef import (
|
31
|
+
KeyDefaultDict,
|
32
|
+
)
|
33
|
+
|
5
34
|
|
6
35
|
logger = logs.configure_deltacat_logger(logging.getLogger(__name__))
|
7
36
|
|
8
37
|
|
9
|
-
def
|
10
|
-
|
11
|
-
|
12
|
-
|
13
|
-
|
14
|
-
|
15
|
-
|
16
|
-
|
17
|
-
|
18
|
-
|
19
|
-
|
38
|
+
def parquet_path_to_id_mapping_override(schema):
|
39
|
+
res = parquet_path_to_id_mapping(schema)
|
40
|
+
# Override here to insert position delete reserved column field IDs
|
41
|
+
res["file_path"] = ICEBERG_RESERVED_FIELD_ID_FOR_FILE_PATH_COLUMN
|
42
|
+
res["pos"] = ICEBERG_RESERVED_FIELD_ID_FOR_POS_COLUMN
|
43
|
+
return res
|
44
|
+
|
45
|
+
|
46
|
+
def data_file_statistics_from_parquet_metadata(
|
47
|
+
parquet_metadata: pq.FileMetaData,
|
48
|
+
stats_columns: Dict[int, StatisticsCollector],
|
49
|
+
parquet_column_mapping: Dict[str, int],
|
50
|
+
) -> DataFileStatistics:
|
51
|
+
"""
|
52
|
+
Overrides original Pyiceberg function: Compute and return DataFileStatistics that includes the following.
|
53
|
+
|
54
|
+
- record_count
|
55
|
+
- column_sizes
|
56
|
+
- value_counts
|
57
|
+
- null_value_counts
|
58
|
+
- nan_value_counts
|
59
|
+
- column_aggregates
|
60
|
+
- split_offsets
|
61
|
+
|
62
|
+
Args:
|
63
|
+
parquet_metadata (pyarrow.parquet.FileMetaData): A pyarrow metadata object.
|
64
|
+
stats_columns (Dict[int, StatisticsCollector]): The statistics gathering plan. It is required to
|
65
|
+
set the mode for column metrics collection
|
66
|
+
parquet_column_mapping (Dict[str, int]): The mapping of the parquet file name to the field ID
|
67
|
+
"""
|
68
|
+
column_sizes: Dict[int, int] = {}
|
69
|
+
value_counts: Dict[int, int] = {}
|
70
|
+
split_offsets: List[int] = []
|
71
|
+
|
72
|
+
null_value_counts: Dict[int, int] = {}
|
73
|
+
nan_value_counts: Dict[int, int] = {}
|
74
|
+
|
75
|
+
col_aggs = {}
|
76
|
+
|
77
|
+
invalidate_col: Set[int] = set()
|
78
|
+
for r in range(parquet_metadata.num_row_groups):
|
79
|
+
# References:
|
80
|
+
# https://github.com/apache/iceberg/blob/fc381a81a1fdb8f51a0637ca27cd30673bd7aad3/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java#L232
|
81
|
+
# https://github.com/apache/parquet-mr/blob/ac29db4611f86a07cc6877b416aa4b183e09b353/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184
|
82
|
+
|
83
|
+
row_group = parquet_metadata.row_group(r)
|
84
|
+
|
85
|
+
data_offset = row_group.column(0).data_page_offset
|
86
|
+
dictionary_offset = row_group.column(0).dictionary_page_offset
|
87
|
+
|
88
|
+
if row_group.column(0).has_dictionary_page and dictionary_offset < data_offset:
|
89
|
+
split_offsets.append(dictionary_offset)
|
90
|
+
else:
|
91
|
+
split_offsets.append(data_offset)
|
92
|
+
|
93
|
+
for pos in range(parquet_metadata.num_columns):
|
94
|
+
column = row_group.column(pos)
|
95
|
+
field_id = parquet_column_mapping[column.path_in_schema]
|
96
|
+
if field_id in stats_columns:
|
97
|
+
stats_col = stats_columns[field_id]
|
98
|
+
|
99
|
+
column_sizes.setdefault(field_id, 0)
|
100
|
+
column_sizes[field_id] += column.total_compressed_size
|
101
|
+
|
102
|
+
if stats_col.mode == MetricsMode(MetricModeTypes.NONE):
|
103
|
+
continue
|
104
|
+
|
105
|
+
value_counts[field_id] = (
|
106
|
+
value_counts.get(field_id, 0) + column.num_values
|
107
|
+
)
|
108
|
+
|
109
|
+
if column.is_stats_set:
|
110
|
+
try:
|
111
|
+
statistics = column.statistics
|
112
|
+
|
113
|
+
if statistics.has_null_count:
|
114
|
+
null_value_counts[field_id] = (
|
115
|
+
null_value_counts.get(field_id, 0)
|
116
|
+
+ statistics.null_count
|
117
|
+
)
|
118
|
+
|
119
|
+
if stats_col.mode == MetricsMode(MetricModeTypes.COUNTS):
|
120
|
+
continue
|
121
|
+
|
122
|
+
if field_id not in col_aggs:
|
123
|
+
col_aggs[field_id] = StatsAggregator(
|
124
|
+
stats_col.iceberg_type,
|
125
|
+
statistics.physical_type,
|
126
|
+
stats_col.mode.length,
|
127
|
+
)
|
128
|
+
|
129
|
+
col_aggs[field_id].update_min(statistics.min)
|
130
|
+
col_aggs[field_id].update_max(statistics.max)
|
131
|
+
|
132
|
+
except pyarrow.lib.ArrowNotImplementedError as e:
|
133
|
+
invalidate_col.add(field_id)
|
134
|
+
logger.warning(e)
|
135
|
+
else:
|
136
|
+
# Note: Removed original adding columns without stats to invalid column logic here
|
137
|
+
logger.warning(
|
138
|
+
"PyArrow statistics missing for column %d when writing file", pos
|
139
|
+
)
|
140
|
+
|
141
|
+
split_offsets.sort()
|
142
|
+
|
143
|
+
for field_id in invalidate_col:
|
144
|
+
del col_aggs[field_id]
|
145
|
+
del null_value_counts[field_id]
|
146
|
+
|
147
|
+
return DataFileStatistics(
|
148
|
+
record_count=parquet_metadata.num_rows,
|
149
|
+
column_sizes=column_sizes,
|
150
|
+
value_counts=value_counts,
|
151
|
+
null_value_counts=null_value_counts,
|
152
|
+
nan_value_counts=nan_value_counts,
|
153
|
+
column_aggregates=col_aggs,
|
154
|
+
split_offsets=split_offsets,
|
20
155
|
)
|
21
156
|
|
157
|
+
|
158
|
+
def parquet_files_dict_to_iceberg_data_files(io, table_metadata, files_dict):
|
22
159
|
data_file_content_type = DataFileContent.POSITION_DELETES
|
23
160
|
iceberg_files = []
|
24
161
|
schema = table_metadata.schema()
|
25
|
-
for
|
26
|
-
for
|
27
|
-
|
28
|
-
|
29
|
-
|
30
|
-
|
31
|
-
_check_pyarrow_schema_compatible(
|
32
|
-
schema, parquet_metadata.schema.to_arrow_schema()
|
33
|
-
)
|
162
|
+
for partition_value, file_paths in files_dict.items():
|
163
|
+
for file_path in file_paths:
|
164
|
+
logger.info(f"DEBUG_file_path:{file_path}")
|
165
|
+
input_file = io.new_input(file_path)
|
166
|
+
with input_file.open() as input_stream:
|
167
|
+
parquet_metadata = pq.read_metadata(input_stream)
|
34
168
|
|
35
|
-
|
36
|
-
parquet_metadata=parquet_metadata,
|
37
|
-
stats_columns=compute_statistics_plan(
|
38
|
-
schema, table_metadata.properties
|
39
|
-
),
|
40
|
-
parquet_column_mapping=parquet_path_to_id_mapping(schema),
|
41
|
-
)
|
169
|
+
# Removed _check_pyarrow_schema_compatible() here since reserved columns does not comply to all rules.
|
42
170
|
|
43
|
-
|
44
|
-
|
45
|
-
|
46
|
-
|
47
|
-
|
48
|
-
|
49
|
-
|
50
|
-
|
51
|
-
|
52
|
-
|
53
|
-
|
54
|
-
|
55
|
-
|
56
|
-
|
171
|
+
statistics = data_file_statistics_from_parquet_metadata(
|
172
|
+
parquet_metadata=parquet_metadata,
|
173
|
+
stats_columns=compute_statistics_plan(
|
174
|
+
schema, table_metadata.properties
|
175
|
+
),
|
176
|
+
parquet_column_mapping=parquet_path_to_id_mapping_override(schema),
|
177
|
+
)
|
178
|
+
|
179
|
+
data_file = DataFile(
|
180
|
+
content=data_file_content_type,
|
181
|
+
file_path=file_path,
|
182
|
+
file_format=FileFormat.PARQUET,
|
183
|
+
partition=partition_value,
|
184
|
+
file_size_in_bytes=len(input_file),
|
185
|
+
sort_order_id=None,
|
186
|
+
spec_id=table_metadata.default_spec_id,
|
187
|
+
equality_ids=None,
|
188
|
+
key_metadata=None,
|
189
|
+
**statistics.to_serialized_dict(),
|
190
|
+
)
|
191
|
+
iceberg_files.append(data_file)
|
57
192
|
return iceberg_files
|
58
193
|
|
59
194
|
|
60
195
|
def fetch_all_bucket_files(table):
|
61
196
|
# step 1: filter manifests using partition summaries
|
62
197
|
# the filter depends on the partition spec used to write the manifest file, so create a cache of filters for each spec id
|
63
|
-
from pyiceberg.typedef import (
|
64
|
-
KeyDefaultDict,
|
65
|
-
)
|
66
198
|
|
67
199
|
data_scan = table.scan()
|
68
200
|
snapshot = data_scan.snapshot()
|
@@ -78,23 +210,8 @@ def fetch_all_bucket_files(table):
|
|
78
210
|
|
79
211
|
# step 2: filter the data files in each manifest
|
80
212
|
# this filter depends on the partition spec used to write the manifest file
|
81
|
-
from pyiceberg.expressions.visitors import _InclusiveMetricsEvaluator
|
82
|
-
from pyiceberg.types import (
|
83
|
-
strtobool,
|
84
|
-
)
|
85
|
-
from pyiceberg.table import _min_sequence_number, _open_manifest
|
86
|
-
from pyiceberg.utils.concurrent import ExecutorFactory
|
87
|
-
from itertools import chain
|
88
|
-
from pyiceberg.manifest import DataFileContent
|
89
|
-
|
90
213
|
partition_evaluators = KeyDefaultDict(data_scan._build_partition_evaluator)
|
91
|
-
|
92
|
-
data_scan.table_metadata.schema(),
|
93
|
-
data_scan.row_filter,
|
94
|
-
data_scan.case_sensitive,
|
95
|
-
strtobool(data_scan.options.get("include_empty_files", "false")),
|
96
|
-
).eval
|
97
|
-
|
214
|
+
residual_evaluators = KeyDefaultDict(data_scan._build_residual_evaluator)
|
98
215
|
min_sequence_number = _min_sequence_number(manifests)
|
99
216
|
|
100
217
|
# {"bucket_index": List[DataFile]}
|
@@ -111,7 +228,8 @@ def fetch_all_bucket_files(table):
|
|
111
228
|
data_scan.io,
|
112
229
|
manifest,
|
113
230
|
partition_evaluators[manifest.partition_spec_id],
|
114
|
-
|
231
|
+
residual_evaluators[manifest.partition_spec_id],
|
232
|
+
data_scan._build_metrics_evaluator(),
|
115
233
|
)
|
116
234
|
for manifest in manifests
|
117
235
|
if data_scan._check_sequence_number(min_sequence_number, manifest)
|
@@ -122,9 +240,10 @@ def fetch_all_bucket_files(table):
|
|
122
240
|
file_sequence_number = manifest_entry.sequence_number
|
123
241
|
data_file_tuple = (file_sequence_number, data_file)
|
124
242
|
partition_value = data_file.partition
|
243
|
+
|
125
244
|
if data_file.content == DataFileContent.DATA:
|
126
245
|
data_entries[partition_value].append(data_file_tuple)
|
127
|
-
|
246
|
+
elif data_file.content == DataFileContent.POSITION_DELETES:
|
128
247
|
positional_delete_entries[partition_value].append(data_file_tuple)
|
129
248
|
elif data_file.content == DataFileContent.EQUALITY_DELETES:
|
130
249
|
equality_data_entries[partition_value].append(data_file_tuple)
|
@@ -15,7 +15,10 @@ from deltacat.compute.converter.utils.io import (
|
|
15
15
|
from deltacat.compute.converter.utils.converter_session_utils import (
|
16
16
|
partition_value_record_to_partition_value_string,
|
17
17
|
)
|
18
|
-
|
18
|
+
from deltacat.compute.converter.pyiceberg.overrides import (
|
19
|
+
parquet_files_dict_to_iceberg_data_files,
|
20
|
+
)
|
21
|
+
from deltacat.compute.converter.model.convert_result import ConvertResult
|
19
22
|
from deltacat import logs
|
20
23
|
|
21
24
|
logger = logs.configure_deltacat_logger(logging.getLogger(__name__))
|
@@ -27,17 +30,22 @@ def convert(convert_input: ConvertInput):
|
|
27
30
|
convert_task_index = convert_input.convert_task_index
|
28
31
|
iceberg_table_warehouse_prefix = convert_input.iceberg_table_warehouse_prefix
|
29
32
|
identifier_fields = convert_input.identifier_fields
|
30
|
-
|
33
|
+
table_io = convert_input.table_io
|
34
|
+
table_metadata = convert_input.table_metadata
|
35
|
+
compact_previous_position_delete_files = (
|
36
|
+
convert_input.compact_previous_position_delete_files
|
37
|
+
)
|
31
38
|
position_delete_for_multiple_data_files = (
|
32
39
|
convert_input.position_delete_for_multiple_data_files
|
33
40
|
)
|
34
41
|
max_parallel_data_file_download = convert_input.max_parallel_data_file_download
|
35
42
|
s3_file_system = convert_input.s3_file_system
|
43
|
+
s3_client_kwargs = convert_input.s3_client_kwargs
|
36
44
|
if not position_delete_for_multiple_data_files:
|
37
45
|
raise NotImplementedError(
|
38
46
|
f"Distributed file level position delete compute is not supported yet"
|
39
47
|
)
|
40
|
-
if
|
48
|
+
if compact_previous_position_delete_files:
|
41
49
|
raise NotImplementedError(f"Compact previous position delete not supported yet")
|
42
50
|
|
43
51
|
logger.info(f"Starting convert task index: {convert_task_index}")
|
@@ -52,9 +60,15 @@ def convert(convert_input: ConvertInput):
|
|
52
60
|
convert_input_files.partition_value
|
53
61
|
)
|
54
62
|
partition_value = convert_input_files.partition_value
|
55
|
-
|
56
|
-
|
57
|
-
|
63
|
+
|
64
|
+
if partition_value_str:
|
65
|
+
iceberg_table_warehouse_prefix_with_partition = (
|
66
|
+
f"{iceberg_table_warehouse_prefix}/{partition_value_str}"
|
67
|
+
)
|
68
|
+
else:
|
69
|
+
iceberg_table_warehouse_prefix_with_partition = (
|
70
|
+
f"{iceberg_table_warehouse_prefix}"
|
71
|
+
)
|
58
72
|
enforce_primary_key_uniqueness = convert_input.enforce_primary_key_uniqueness
|
59
73
|
total_pos_delete_table = []
|
60
74
|
if applicable_equality_delete_files:
|
@@ -65,8 +79,10 @@ def convert(convert_input: ConvertInput):
|
|
65
79
|
identifier_columns=identifier_fields,
|
66
80
|
equality_delete_files_list=applicable_equality_delete_files,
|
67
81
|
iceberg_table_warehouse_prefix_with_partition=iceberg_table_warehouse_prefix_with_partition,
|
82
|
+
convert_task_index=convert_task_index,
|
68
83
|
max_parallel_data_file_download=max_parallel_data_file_download,
|
69
84
|
s3_file_system=s3_file_system,
|
85
|
+
s3_client_kwargs=s3_client_kwargs,
|
70
86
|
)
|
71
87
|
if pos_delete_after_converting_equality_delete:
|
72
88
|
total_pos_delete_table.append(pos_delete_after_converting_equality_delete)
|
@@ -76,31 +92,71 @@ def convert(convert_input: ConvertInput):
|
|
76
92
|
all_data_files=all_data_files_for_this_bucket,
|
77
93
|
data_files_downloaded=applicable_data_files,
|
78
94
|
)
|
79
|
-
|
95
|
+
logger.info(
|
96
|
+
f"[Convert task {convert_task_index}]: Got {len(data_files_to_dedupe)} files to dedupe."
|
97
|
+
)
|
98
|
+
(
|
99
|
+
pos_delete_after_dedupe,
|
100
|
+
data_file_to_dedupe_record_count,
|
101
|
+
data_file_to_dedupe_size,
|
102
|
+
) = dedupe_data_files(
|
80
103
|
data_file_to_dedupe=data_files_to_dedupe,
|
81
|
-
identify_column_name_concatenated=identifier_fields[0],
|
82
104
|
identifier_columns=identifier_fields,
|
83
105
|
merge_sort_column=sc._ORDERED_RECORD_IDX_COLUMN_NAME,
|
106
|
+
s3_client_kwargs=s3_client_kwargs,
|
107
|
+
)
|
108
|
+
logger.info(
|
109
|
+
f"[Convert task {convert_task_index}]: Dedupe produced {len(pos_delete_after_dedupe)} position delete records."
|
84
110
|
)
|
85
111
|
total_pos_delete_table.append(pos_delete_after_dedupe)
|
86
112
|
|
87
113
|
total_pos_delete = pa.concat_tables(total_pos_delete_table)
|
88
|
-
|
89
|
-
|
90
|
-
|
91
|
-
s3_table_writer_kwargs={},
|
92
|
-
s3_file_system=s3_file_system,
|
114
|
+
|
115
|
+
logger.info(
|
116
|
+
f"[Convert task {convert_task_index}]: Total position delete produced:{len(total_pos_delete)}"
|
93
117
|
)
|
94
118
|
|
119
|
+
to_be_added_files_list = []
|
120
|
+
if total_pos_delete:
|
121
|
+
to_be_added_files_list_parquet = upload_table_with_retry(
|
122
|
+
table=total_pos_delete,
|
123
|
+
s3_url_prefix=iceberg_table_warehouse_prefix_with_partition,
|
124
|
+
s3_table_writer_kwargs={},
|
125
|
+
s3_file_system=s3_file_system,
|
126
|
+
)
|
127
|
+
|
128
|
+
to_be_added_files_dict = defaultdict()
|
129
|
+
to_be_added_files_dict[partition_value] = to_be_added_files_list_parquet
|
130
|
+
|
131
|
+
logger.info(
|
132
|
+
f"[Convert task {convert_task_index}]: Produced {len(to_be_added_files_list_parquet)} position delete files."
|
133
|
+
)
|
134
|
+
to_be_added_files_list = parquet_files_dict_to_iceberg_data_files(
|
135
|
+
io=table_io,
|
136
|
+
table_metadata=table_metadata,
|
137
|
+
files_dict=to_be_added_files_dict,
|
138
|
+
)
|
139
|
+
|
95
140
|
to_be_delete_files_dict = defaultdict()
|
96
141
|
if applicable_equality_delete_files:
|
97
142
|
to_be_delete_files_dict[partition_value] = [
|
98
143
|
equality_delete_file[1]
|
99
144
|
for equality_delete_file in applicable_equality_delete_files
|
100
145
|
]
|
101
|
-
|
102
|
-
|
103
|
-
|
146
|
+
|
147
|
+
convert_res = ConvertResult.of(
|
148
|
+
convert_task_index=convert_task_index,
|
149
|
+
to_be_added_files=to_be_added_files_list,
|
150
|
+
to_be_deleted_files=to_be_delete_files_dict,
|
151
|
+
position_delete_record_count=len(total_pos_delete),
|
152
|
+
input_data_files_record_count=data_file_to_dedupe_record_count,
|
153
|
+
input_data_files_hash_columns_in_memory_sizes=data_file_to_dedupe_size,
|
154
|
+
position_delete_in_memory_sizes=int(total_pos_delete.nbytes),
|
155
|
+
position_delete_on_disk_sizes=sum(
|
156
|
+
file.file_size_in_bytes for file in to_be_added_files_list
|
157
|
+
),
|
158
|
+
)
|
159
|
+
return convert_res
|
104
160
|
|
105
161
|
|
106
162
|
def get_additional_applicable_data_files(all_data_files, data_files_downloaded):
|
@@ -120,11 +176,6 @@ def filter_rows_to_be_deleted(
|
|
120
176
|
equality_delete_table[identifier_column],
|
121
177
|
)
|
122
178
|
position_delete_table = data_file_table.filter(equality_deletes)
|
123
|
-
logger.info(f"positional_delete_table:{position_delete_table.to_pydict()}")
|
124
|
-
logger.info(f"data_file_table:{data_file_table.to_pydict()}")
|
125
|
-
logger.info(
|
126
|
-
f"length_pos_delete_table, {len(position_delete_table)}, length_data_table:{len(data_file_table)}"
|
127
|
-
)
|
128
179
|
return position_delete_table
|
129
180
|
|
130
181
|
|
@@ -149,25 +200,15 @@ def compute_pos_delete_converting_equality_deletes(
|
|
149
200
|
return new_position_delete_table
|
150
201
|
|
151
202
|
|
152
|
-
def download_bucketed_table(data_files, equality_delete_files):
|
153
|
-
from deltacat.utils.pyarrow import s3_file_to_table
|
154
|
-
|
155
|
-
compacted_table = s3_file_to_table(
|
156
|
-
[data_file.file_path for data_file in data_files]
|
157
|
-
)
|
158
|
-
equality_delete_table = s3_file_to_table(
|
159
|
-
[eq_file.file_path for eq_file in equality_delete_files]
|
160
|
-
)
|
161
|
-
return compacted_table, equality_delete_table
|
162
|
-
|
163
|
-
|
164
203
|
def compute_pos_delete_with_limited_parallelism(
|
165
204
|
data_files_list,
|
166
205
|
identifier_columns,
|
167
206
|
equality_delete_files_list,
|
168
207
|
iceberg_table_warehouse_prefix_with_partition,
|
208
|
+
convert_task_index,
|
169
209
|
max_parallel_data_file_download,
|
170
210
|
s3_file_system,
|
211
|
+
s3_client_kwargs,
|
171
212
|
):
|
172
213
|
for data_files, equality_delete_files in zip(
|
173
214
|
data_files_list, equality_delete_files_list
|
@@ -182,6 +223,7 @@ def compute_pos_delete_with_limited_parallelism(
|
|
182
223
|
sc._ORDERED_RECORD_IDX_COLUMN_NAME,
|
183
224
|
],
|
184
225
|
sequence_number=data_file[0],
|
226
|
+
s3_client_kwargs=s3_client_kwargs,
|
185
227
|
)
|
186
228
|
data_table_total.append(data_table)
|
187
229
|
data_table_total = pa.concat_tables(data_table_total)
|
@@ -191,6 +233,7 @@ def compute_pos_delete_with_limited_parallelism(
|
|
191
233
|
equality_delete_table = download_data_table_and_append_iceberg_columns(
|
192
234
|
data_files=equality_delete[1],
|
193
235
|
columns_to_download=identifier_columns,
|
236
|
+
s3_client_kwargs=s3_client_kwargs,
|
194
237
|
)
|
195
238
|
equality_delete_table_total.append(equality_delete_table)
|
196
239
|
equality_delete_table_total = pa.concat_tables(equality_delete_table_total)
|
@@ -201,11 +244,16 @@ def compute_pos_delete_with_limited_parallelism(
|
|
201
244
|
iceberg_table_warehouse_prefix_with_partition=iceberg_table_warehouse_prefix_with_partition,
|
202
245
|
identifier_columns=identifier_columns,
|
203
246
|
s3_file_system=s3_file_system,
|
247
|
+
s3_client_kwargs=s3_client_kwargs,
|
204
248
|
)
|
205
|
-
if not new_pos_delete_table:
|
206
|
-
logger.info("No records deleted based on equality delete converstion")
|
207
249
|
|
208
250
|
logger.info(
|
209
|
-
f"
|
251
|
+
f"[Convert task {convert_task_index}]: Find deletes got {len(data_table_total)} data table records, "
|
252
|
+
f"{len(equality_delete_table_total)} equality deletes as input, "
|
253
|
+
f"Produced {len(new_pos_delete_table)} position deletes based off find deletes input."
|
210
254
|
)
|
255
|
+
|
256
|
+
if not new_pos_delete_table:
|
257
|
+
logger.info("No records deleted based on equality delete convertion")
|
258
|
+
|
211
259
|
return new_pos_delete_table
|
@@ -4,16 +4,21 @@ import deltacat.compute.converter.utils.iceberg_columns as sc
|
|
4
4
|
from deltacat.compute.converter.utils.io import (
|
5
5
|
download_data_table_and_append_iceberg_columns,
|
6
6
|
)
|
7
|
+
import logging
|
8
|
+
from deltacat import logs
|
9
|
+
|
10
|
+
logger = logs.configure_deltacat_logger(logging.getLogger(__name__))
|
7
11
|
|
8
12
|
|
9
13
|
def dedupe_data_files(
|
10
14
|
data_file_to_dedupe,
|
11
|
-
identify_column_name_concatenated,
|
12
15
|
identifier_columns,
|
13
16
|
merge_sort_column,
|
17
|
+
s3_client_kwargs,
|
14
18
|
):
|
15
19
|
data_file_table = []
|
16
20
|
|
21
|
+
downloaded_data_file_record_count = 0
|
17
22
|
# Sort data files by file sequence number first
|
18
23
|
data_file_to_dedupe = sorted(data_file_to_dedupe, key=lambda f: f[0])
|
19
24
|
for file_tuple in data_file_to_dedupe:
|
@@ -27,11 +32,20 @@ def dedupe_data_files(
|
|
27
32
|
sc._ORDERED_RECORD_IDX_COLUMN_NAME,
|
28
33
|
],
|
29
34
|
sequence_number=sequence_number,
|
35
|
+
s3_client_kwargs=s3_client_kwargs,
|
30
36
|
)
|
37
|
+
downloaded_data_file_record_count += len(data_file_to_dedupe_table)
|
31
38
|
data_file_table.append(data_file_to_dedupe_table)
|
32
39
|
|
33
40
|
final_data_to_dedupe = pa.concat_tables(data_file_table)
|
34
41
|
|
42
|
+
assert len(final_data_to_dedupe) == downloaded_data_file_record_count, (
|
43
|
+
f"Mismatch record count while performing table concat, Got {len(final_data_to_dedupe)} in final table, "
|
44
|
+
f"while input table length is: {downloaded_data_file_record_count}"
|
45
|
+
)
|
46
|
+
|
47
|
+
logger.info(f"Length of pyarrow table to dedupe:{len(final_data_to_dedupe)}")
|
48
|
+
|
35
49
|
record_idx_iterator = iter(range(len(final_data_to_dedupe)))
|
36
50
|
|
37
51
|
# Append global record index to used as aggregate column
|
@@ -40,7 +54,7 @@ def dedupe_data_files(
|
|
40
54
|
)
|
41
55
|
|
42
56
|
final_data_table_indices = final_data_to_dedupe.group_by(
|
43
|
-
|
57
|
+
sc._IDENTIFIER_COLUMNS_HASH_COLUMN_NAME, use_threads=False
|
44
58
|
).aggregate([(sc._GLOBAL_RECORD_IDX_COLUMN_NAME, "max")])
|
45
59
|
|
46
60
|
pos_delete_indices = pc.invert(
|
@@ -55,6 +69,13 @@ def dedupe_data_files(
|
|
55
69
|
final_data_table_to_delete = final_data_to_dedupe.filter(pos_delete_indices)
|
56
70
|
|
57
71
|
final_data_table_to_delete = final_data_table_to_delete.drop(
|
58
|
-
[
|
72
|
+
[sc._IDENTIFIER_COLUMNS_HASH_COLUMN_NAME, sc._GLOBAL_RECORD_IDX_COLUMN_NAME]
|
73
|
+
)
|
74
|
+
logger.info(
|
75
|
+
f"Deduped {len(final_data_table_to_delete)} Records based off identifier columns."
|
76
|
+
)
|
77
|
+
return (
|
78
|
+
final_data_table_to_delete,
|
79
|
+
len(final_data_to_dedupe),
|
80
|
+
int(final_data_to_dedupe.nbytes),
|
59
81
|
)
|
60
|
-
return final_data_table_to_delete
|