-
Notifications
You must be signed in to change notification settings - Fork 400
/
writer.py
327 lines (285 loc) · 11.5 KB
/
writer.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
import json
import uuid
from dataclasses import dataclass
from datetime import date, datetime
from decimal import Decimal
from typing import (
TYPE_CHECKING,
Any,
Dict,
Iterable,
Iterator,
List,
Mapping,
Optional,
Union,
)
if TYPE_CHECKING:
import pandas as pd
import pyarrow as pa
import pyarrow.dataset as ds
import pyarrow.fs as pa_fs
from pyarrow.lib import RecordBatchReader
from typing_extensions import Literal
from .deltalake import PyDeltaTableError
from .deltalake import write_new_deltalake as _write_new_deltalake
from .table import DeltaTable
try:
import pandas as pd
except ModuleNotFoundError:
_has_pandas = False
else:
_has_pandas = True
class DeltaTableProtocolError(PyDeltaTableError):
pass
@dataclass
class AddAction:
path: str
size: int
partition_values: Mapping[str, Optional[str]]
modification_time: int
data_change: bool
stats: str
def write_deltalake(
table_or_uri: Union[str, DeltaTable],
data: Union[
"pd.DataFrame",
pa.Table,
pa.RecordBatch,
Iterable[pa.RecordBatch],
RecordBatchReader,
],
schema: Optional[pa.Schema] = None,
partition_by: Optional[List[str]] = None,
filesystem: Optional[pa_fs.FileSystem] = None,
mode: Literal["error", "append", "overwrite", "ignore"] = "error",
file_options: Optional[ds.ParquetFileWriteOptions] = None,
max_open_files: int = 1024,
max_rows_per_file: int = 0,
min_rows_per_group: int = 0,
max_rows_per_group: int = 1048576,
name: Optional[str] = None,
description: Optional[str] = None,
configuration: Optional[Mapping[str, Optional[str]]] = None,
) -> None:
"""Write to a Delta Lake table (Experimental)
If the table does not already exist, it will be created.
This function only supports protocol version 1 currently. If an attempting
to write to an existing table with a higher min_writer_version, this
function will throw DeltaTableProtocolError.
Note that this function does NOT register this table in a data catalog.
:param table_or_uri: URI of a table or a DeltaTable object.
:param data: Data to write. If passing iterable, the schema must also be given.
:param schema: Optional schema to write.
:param partition_by: List of columns to partition the table by. Only required
when creating a new table.
:param filesystem: Optional filesystem to pass to PyArrow. If not provided will
be inferred from uri.
:param mode: How to handle existing data. Default is to error if table already exists.
If 'append', will add new data.
If 'overwrite', will replace table with new data.
If 'ignore', will not write anything if table already exists.
:param file_options: Optional write options for Parquet (ParquetFileWriteOptions).
Can be provided with defaults using ParquetFileWriteOptions().make_write_options().
Please refer to https://github.com/apache/arrow/blob/master/python/pyarrow/_dataset_parquet.pyx#L492-L533
for the list of available options
:param max_open_files: Limits the maximum number of
files that can be left open while writing. If an attempt is made to open
too many files then the least recently used file will be closed.
If this setting is set too low you may end up fragmenting your
data into many small files.
:param max_rows_per_file: Maximum number of rows per file.
If greater than 0 then this will limit how many rows are placed in any single file.
Otherwise there will be no limit and one file will be created in each output directory
unless files need to be closed to respect max_open_files
:param min_rows_per_group: Minimum number of rows per group. When the value is set,
the dataset writer will batch incoming data and only write the row groups to the disk
when sufficient rows have accumulated.
:param max_rows_per_group: Maximum number of rows per group.
If the value is set, then the dataset writer may split up large incoming batches into multiple row groups.
If this value is set, then min_rows_per_group should also be set.
:param name: User-provided identifier for this table.
:param description: User-provided description for this table.
:param configuration: A map containing configuration options for the metadata action.
"""
if _has_pandas and isinstance(data, pd.DataFrame):
data = pa.Table.from_pandas(data)
if schema is None:
if isinstance(data, RecordBatchReader):
schema = data.schema
elif isinstance(data, Iterable):
raise ValueError("You must provide schema if data is Iterable")
else:
schema = data.schema
if isinstance(table_or_uri, str):
table = try_get_deltatable(table_or_uri)
table_uri = table_or_uri
else:
table = table_or_uri
table_uri = table_uri = table._table.table_uri()
__enforce_append_only(table=table, configuration=configuration, mode=mode)
# TODO: Pass through filesystem once it is complete
# if filesystem is None:
# filesystem = pa_fs.PyFileSystem(DeltaStorageHandler(table_uri))
if table: # already exists
if mode == "error":
raise AssertionError("DeltaTable already exists.")
elif mode == "ignore":
return
current_version = table.version()
if partition_by:
assert partition_by == table.metadata().partition_columns
if table.protocol().min_writer_version > 1:
raise DeltaTableProtocolError(
"This table's min_writer_version is "
f"{table.protocol().min_writer_version}, "
"but this method only supports version 1."
)
else: # creating a new table
current_version = -1
# TODO: Don't allow writing to non-empty directory
# Blocked on: Finish filesystem implementation in fs.py
# assert len(filesystem.get_file_info(pa_fs.FileSelector(table_uri, allow_not_found=True))) == 0
if partition_by:
partition_schema = pa.schema([schema.field(name) for name in partition_by])
partitioning = ds.partitioning(partition_schema, flavor="hive")
else:
partitioning = None
add_actions: List[AddAction] = []
def visitor(written_file: Any) -> None:
partition_values = get_partitions_from_path(table_uri, written_file.path)
stats = get_file_stats_from_metadata(written_file.metadata)
add_actions.append(
AddAction(
written_file.path,
written_file.metadata.serialized_size,
partition_values,
int(datetime.now().timestamp()),
True,
json.dumps(stats, cls=DeltaJSONEncoder),
)
)
ds.write_dataset(
data,
base_dir=table_uri,
basename_template=f"{current_version + 1}-{uuid.uuid4()}-{{i}}.parquet",
format="parquet",
partitioning=partitioning,
# It will not accept a schema if using a RBR
schema=schema if not isinstance(data, RecordBatchReader) else None,
file_visitor=visitor,
existing_data_behavior="overwrite_or_ignore",
file_options=file_options,
max_open_files=max_open_files,
max_rows_per_file=max_rows_per_file,
min_rows_per_group=min_rows_per_group,
max_rows_per_group=max_rows_per_group,
)
if table is None:
_write_new_deltalake( # type: ignore[call-arg]
table_uri,
schema,
add_actions,
mode,
partition_by or [],
name,
description,
configuration,
)
else:
table._table.create_write_transaction(
add_actions,
mode,
partition_by or [],
)
def __enforce_append_only(
table: Optional[DeltaTable],
configuration: Optional[Mapping[str, Optional[str]]],
mode: str,
) -> None:
"""Throw ValueError if table configuration contains delta.appendOnly and mode is not append"""
if table:
configuration = table.metadata().configuration
config_delta_append_only = (
configuration and configuration.get("delta.appendOnly", "false") == "true"
)
if config_delta_append_only and mode != "append":
raise ValueError(
f"If configuration has delta.appendOnly = 'true', mode must be 'append'. Mode is currently {mode}"
)
class DeltaJSONEncoder(json.JSONEncoder):
def default(self, obj: Any) -> Any:
if isinstance(obj, bytes):
return obj.decode("unicode_escape")
elif isinstance(obj, date):
return obj.isoformat()
elif isinstance(obj, datetime):
return obj.isoformat()
elif isinstance(obj, Decimal):
return str(obj)
# Let the base class default method raise the TypeError
return json.JSONEncoder.default(self, obj)
def try_get_deltatable(table_uri: str) -> Optional[DeltaTable]:
try:
return DeltaTable(table_uri)
except PyDeltaTableError as err:
if "Not a Delta table" not in str(err):
raise
return None
def get_partitions_from_path(base_path: str, path: str) -> Dict[str, str]:
path = path.split(base_path, maxsplit=1)[1]
parts = path.split("/")
parts.pop() # remove filename
out = {}
for part in parts:
if part == "":
continue
key, value = part.split("=", maxsplit=1)
out[key] = value
return out
def get_file_stats_from_metadata(
metadata: Any,
) -> Dict[str, Union[int, Dict[str, Any]]]:
stats = {
"numRecords": metadata.num_rows,
"minValues": {},
"maxValues": {},
"nullCount": {},
}
def iter_groups(metadata: Any) -> Iterator[Any]:
for i in range(metadata.num_row_groups):
yield metadata.row_group(i)
for column_idx in range(metadata.num_columns):
name = metadata.row_group(0).column(column_idx).path_in_schema
# If stats missing, then we can't know aggregate stats
if all(
group.column(column_idx).is_stats_set for group in iter_groups(metadata)
):
stats["nullCount"][name] = sum(
group.column(column_idx).statistics.null_count
for group in iter_groups(metadata)
)
# I assume for now this is based on data type, and thus is
# consistent between groups
if metadata.row_group(0).column(column_idx).statistics.has_min_max:
# Min and Max are recorded in physical type, not logical type
# https://stackoverflow.com/questions/66753485/decoding-parquet-min-max-statistics-for-decimal-type
# TODO: Add logic to decode physical type for DATE, DECIMAL
logical_type = (
metadata.row_group(0)
.column(column_idx)
.statistics.logical_type.type
)
#
if logical_type not in ["STRING", "INT", "TIMESTAMP", "NONE"]:
continue
# import pdb; pdb.set_trace()
stats["minValues"][name] = min(
group.column(column_idx).statistics.min
for group in iter_groups(metadata)
)
stats["maxValues"][name] = max(
group.column(column_idx).statistics.max
for group in iter_groups(metadata)
)
return stats