-
Notifications
You must be signed in to change notification settings - Fork 172
/
delta_sharing.py
276 lines (243 loc) · 10.9 KB
/
delta_sharing.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
#
# Copyright (C) 2021 The Delta Lake Project Authors.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
from itertools import chain
from typing import BinaryIO, List, Optional, Sequence, TextIO, Tuple, Union
from pathlib import Path
import pandas as pd
from delta_sharing.protocol import CdfOptions
try:
from pyspark.sql import DataFrame as PySparkDataFrame
except ImportError:
pass
from delta_sharing.protocol import DeltaSharingProfile, Schema, Share, Table
from delta_sharing.reader import DeltaSharingReader
from delta_sharing.rest_client import DataSharingRestClient
from requests.exceptions import HTTPError
def _parse_url(url: str) -> Tuple[str, str, str, str]:
"""
:param url: a url under the format "<profile>#<share>.<schema>.<table>"
:return: a tuple with parsed (profile, share, schema, table)
"""
shape_index = url.rfind("#")
if shape_index < 0:
raise ValueError(f"Invalid 'url': {url}")
profile = url[0:shape_index]
fragments = url[shape_index + 1 :].split(".")
if len(fragments) != 3:
raise ValueError(f"Invalid 'url': {url}")
share, schema, table = fragments
if len(profile) == 0 or len(share) == 0 or len(schema) == 0 or len(table) == 0:
raise ValueError(f"Invalid 'url': {url}")
return (profile, share, schema, table)
def load_as_pandas(
url: str,
limit: Optional[int] = None,
version: Optional[int] = None,
timestamp: Optional[str] = None
) -> pd.DataFrame:
"""
Load the shared table using the given url as a pandas DataFrame.
:param url: a url under the format "<profile>#<share>.<schema>.<table>"
:param limit: a non-negative int. Load only the ``limit`` rows if the parameter is specified.
Use this optional parameter to explore the shared table without loading the entire table to
the memory.
:param version: an optional non-negative int. Load the snapshot of table at version
:return: A pandas DataFrame representing the shared table.
"""
profile_json, share, schema, table = _parse_url(url)
profile = DeltaSharingProfile.read_from_file(profile_json)
return DeltaSharingReader(
table=Table(name=table, share=share, schema=schema),
rest_client=DataSharingRestClient(profile),
limit=limit,
version=version,
timestamp=timestamp
).to_pandas()
def load_as_spark(
url: str,
version: Optional[int] = None,
timestamp: Optional[str] = None
) -> "PySparkDataFrame": # noqa: F821
"""
Load the shared table using the given url as a Spark DataFrame. `PySpark` must be installed,
and the application must be a PySpark application with the Apache Spark Connector for Delta
Sharing installed. Only one of version/timestamp is supported at one time.
:param url: a url under the format "<profile>#<share>.<schema>.<table>".
:param version: an optional non-negative int. Load the snapshot of table at version.
:param timestamp: an optional string. Load the snapshot of table at version corresponding
to the timestamp.
:return: A Spark DataFrame representing the shared table.
"""
try:
from pyspark.sql import SparkSession
except ImportError:
raise ImportError("Unable to import pyspark. `load_as_spark` requires PySpark.")
spark = SparkSession.getActiveSession()
assert spark is not None, (
"No active SparkSession was found. "
"`load_as_spark` requires running in a PySpark application."
)
df = spark.read.format("deltaSharing")
if version is not None:
df.option("versionAsOf", version)
if timestamp is not None:
df.option("timestampAsOf", timestamp)
return df.load(url)
def load_table_changes_as_spark(
url: str,
starting_version: Optional[int] = None,
ending_version: Optional[int] = None,
starting_timestamp: Optional[str] = None,
ending_timestamp: Optional[str] = None
) -> "PySparkDataFrame": # noqa: F821
"""
Load the table changes of a shared table as a Spark DataFrame using the given url.
`PySpark` must be installed, and the application must be a PySpark application with
the Apache Spark Connector for Delta Sharing installed.
Either starting_version or starting_timestamp need to be provided. And only one starting/ending
parameter is accepted by the server. If the end parameter is not provided, the API will use the
latest table version for it. The parameter range is inclusive in the query.
:param url: a url under the format "<profile>#<share>.<schema>.<table>".
:param starting_version: The starting version of table changes.
:param ending_version: The ending version of table changes.
:param starting_timestamp: The starting timestamp of table changes.
:param ending_timestamp: The ending timestamp of table changes.
:return: A Spark DataFrame representing the table changes.
"""
try:
from pyspark.sql import SparkSession
except ImportError:
raise ImportError(
"Unable to import pyspark. `load_table_changes_as_spark` requires PySpark.")
spark = SparkSession.getActiveSession()
assert spark is not None, (
"No active SparkSession was found. "
"`load_table_changes_as_spark` requires running in a PySpark application."
)
df = spark.read.format("deltaSharing").option("readChangeFeed", "true")
if starting_version is not None:
df.option("startingVersion", starting_version)
if ending_version is not None:
df.option("endingVersion", ending_version)
if starting_timestamp is not None:
df.option("startingTimestamp", starting_timestamp)
if ending_timestamp is not None:
df.option("endingTimestamp", ending_timestamp)
return df.load(url)
def load_table_changes_as_pandas(
url: str,
starting_version: Optional[int] = None,
ending_version: Optional[int] = None,
starting_timestamp: Optional[str] = None,
ending_timestamp: Optional[str] = None
) -> pd.DataFrame:
"""
Load the table changes of shared table as a pandas DataFrame using the given url.
Either starting_version or starting_timestamp need to be provided. And only one starting/ending
parameter is accepted by the server. If the end parameter is not provided, the API will use the
latest table version for it. The parameter range is inclusive in the query.
:param url: a url under the format "<profile>#<share>.<schema>.<table>".
:param starting_version: The starting version of table changes.
:param ending_version: The ending version of table changes.
:param starting_timestamp: The starting timestamp of table changes.
:param ending_timestamp: The ending timestamp of table changes.
:return: A pandas DataFrame representing the shared table.
"""
profile_json, share, schema, table = _parse_url(url)
profile = DeltaSharingProfile.read_from_file(profile_json)
return DeltaSharingReader(
table=Table(name=table, share=share, schema=schema),
rest_client=DataSharingRestClient(profile),
).table_changes_to_pandas(CdfOptions(
starting_version=starting_version,
ending_version=ending_version,
starting_timestamp=starting_timestamp,
ending_timestamp=ending_timestamp,
))
class SharingClient:
"""
A Delta Sharing client to query shares/schemas/tables from a Delta Sharing Server.
"""
def __init__(self, profile: Union[str, BinaryIO, TextIO, Path, DeltaSharingProfile]):
if not isinstance(profile, DeltaSharingProfile):
profile = DeltaSharingProfile.read_from_file(profile)
self._profile = profile
self._rest_client = DataSharingRestClient(profile)
def __list_all_tables_in_share(self, share: Share) -> Sequence[Table]:
tables: List[Table] = []
page_token: Optional[str] = None
while True:
response = self._rest_client.list_all_tables(share=share, page_token=page_token)
tables.extend(response.tables)
page_token = response.next_page_token
if page_token is None:
return tables
def list_shares(self) -> Sequence[Share]:
"""
List shares that can be accessed by you in a Delta Sharing Server.
:return: the shares that can be accessed.
"""
shares: List[Share] = []
page_token: Optional[str] = None
while True:
response = self._rest_client.list_shares(page_token=page_token)
shares.extend(response.shares)
page_token = response.next_page_token
if page_token is None:
return shares
def list_schemas(self, share: Share) -> Sequence[Schema]:
"""
List schemas in a share that can be accessed by you in a Delta Sharing Server.
:param share: the share to list.
:return: the schemas in a share.
"""
schemas: List[Schema] = []
page_token: Optional[str] = None
while True:
response = self._rest_client.list_schemas(share=share, page_token=page_token)
schemas.extend(response.schemas)
page_token = response.next_page_token
if page_token is None:
return schemas
def list_tables(self, schema: Schema) -> Sequence[Table]:
"""
List tables in a schema that can be accessed by you in a Delta Sharing Server.
:param schema: the schema to list.
:return: the tables in a schema.
"""
tables: List[Table] = []
page_token: Optional[str] = None
while True:
response = self._rest_client.list_tables(schema=schema, page_token=page_token)
tables.extend(response.tables)
page_token = response.next_page_token
if page_token is None:
return tables
def list_all_tables(self) -> Sequence[Table]:
"""
List all tables that can be accessed by you in a Delta Sharing Server.
:return: all tables that can be accessed.
"""
shares = self.list_shares()
try:
return list(chain(*(self.__list_all_tables_in_share(share) for share in shares)))
except HTTPError as e:
if e.response.status_code == 404:
# The server doesn't support all-tables API. Fallback to the old APIs instead.
schemas = chain(*(self.list_schemas(share) for share in shares))
return list(chain(*(self.list_tables(schema) for schema in schemas)))
else:
raise e