forked from logicalclocks/hopsworks-api
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathdelta_engine.py
227 lines (191 loc) · 8.35 KB
/
delta_engine.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
#
# Copyright 2024 Hopsworks AB
#
# 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 __future__ import annotations
from hopsworks_common.client.exceptions import FeatureStoreException
from hsfs import feature_group_commit, util
from hsfs.core import feature_group_api
try:
from delta.tables import DeltaTable
except ImportError:
pass
class DeltaEngine:
DELTA_SPARK_FORMAT = "delta"
DELTA_QUERY_TIME_TRAVEL_AS_OF_INSTANT = "timestampAsOf"
def __init__(
self,
feature_store_id,
feature_store_name,
feature_group,
spark_session,
spark_context,
):
self._feature_group = feature_group
self._spark_context = spark_context
self._spark_session = spark_session
self._feature_store_id = feature_store_id
self._feature_store_name = feature_store_name
self._feature_group_api = feature_group_api.FeatureGroupApi()
def save_delta_fg(self, dataset, write_options, validation_id=None):
fg_commit = self._write_delta_dataset(dataset, write_options)
fg_commit.validation_id = validation_id
return self._feature_group_api.commit(self._feature_group, fg_commit)
def register_temporary_table(self, delta_fg_alias, read_options):
location = self._feature_group.prepare_spark_location()
delta_options = self._setup_delta_read_opts(delta_fg_alias, read_options)
self._spark_session.read.format(self.DELTA_SPARK_FORMAT).options(
**delta_options
).load(location).createOrReplaceTempView(
delta_fg_alias.alias
)
def _setup_delta_read_opts(self, delta_fg_alias, read_options):
delta_options = {}
if delta_fg_alias.left_feature_group_end_timestamp is None and (
delta_fg_alias.left_feature_group_start_timestamp is None
or delta_fg_alias.left_feature_group_start_timestamp == 0
):
# snapshot query latest state
delta_options = {}
elif (
delta_fg_alias.left_feature_group_end_timestamp is not None
and delta_fg_alias.left_feature_group_start_timestamp is None
):
# snapshot query with end time
_delta_commit_end_time = util.get_delta_datestr_from_timestamp(
delta_fg_alias.left_feature_group_end_timestamp
)
delta_options = {
self.DELTA_QUERY_TIME_TRAVEL_AS_OF_INSTANT: _delta_commit_end_time,
}
if read_options:
delta_options.update(read_options)
return delta_options
def delete_record(self, delete_df):
location = self._feature_group.prepare_spark_location()
if not DeltaTable.isDeltaTable(
self._spark_session, location
):
raise FeatureStoreException(
f"This is no data available in Feature group {self._feature_group.name}, or it not DELTA enabled "
)
else:
fg_source_table = DeltaTable.forPath(
self._spark_session, location
)
source_alias = (
f"{self._feature_group.name}_{self._feature_group.version}_source"
)
updates_alias = (
f"{self._feature_group.name}_{self._feature_group.version}_updates"
)
merge_query_str = self._generate_merge_query(source_alias, updates_alias)
fg_source_table.alias(source_alias).merge(
delete_df.alias(updates_alias), merge_query_str
).whenMatchedDelete().execute()
fg_commit = self._get_last_commit_metadata(
self._spark_session, location
)
return self._feature_group_api.commit(self._feature_group, fg_commit)
def _write_delta_dataset(self, dataset, write_options):
location = self._feature_group.prepare_spark_location()
if write_options is None:
write_options = {}
if not DeltaTable.isDeltaTable(
self._spark_session, location
):
(
dataset.write.format(DeltaEngine.DELTA_SPARK_FORMAT)
.options(**write_options)
.partitionBy(
self._feature_group.partition_key
if self._feature_group.partition_key
else []
)
.mode("append")
.save(location)
)
else:
fg_source_table = DeltaTable.forPath(
self._spark_session, location
)
source_alias = (
f"{self._feature_group.name}_{self._feature_group.version}_source"
)
updates_alias = (
f"{self._feature_group.name}_{self._feature_group.version}_updates"
)
merge_query_str = self._generate_merge_query(source_alias, updates_alias)
fg_source_table.alias(source_alias).merge(
dataset.alias(updates_alias), merge_query_str
).whenMatchedUpdateAll().whenNotMatchedInsertAll().execute()
return self._get_last_commit_metadata(
self._spark_session, location
)
def vacuum(self, retention_hours):
location = self._feature_group.prepare_spark_location()
delta_table = DeltaTable.forPath(self._spark_session, location)
# Vacuum the table
# https://docs.delta.io/1.0.1/api/python/index.html#delta.tables.DeltaTable.vacuum
delta_table.vacuum(retention_hours)
def _generate_merge_query(self, source_alias, updates_alias):
merge_query_list = []
primary_key = self._feature_group.primary_key
# add event time to primary key for upserts
if self._feature_group.event_time is not None:
primary_key.append(self._feature_group.event_time)
# add partition key for upserts
if self._feature_group.partition_key:
primary_key = primary_key + self._feature_group.partition_key
for pk in primary_key:
merge_query_list.append(f"{source_alias}.{pk} == {updates_alias}.{pk}")
megrge_query_str = " AND ".join(merge_query_list)
return megrge_query_str
@staticmethod
def _get_last_commit_metadata(spark_context, base_path):
fg_source_table = DeltaTable.forPath(spark_context, base_path)
# Get info about the latest commit
last_commit = fg_source_table.history(1).first().asDict()
version = last_commit["version"]
commit_timestamp = util.convert_event_time_to_timestamp(
last_commit["timestamp"]
)
commit_date_string = util.get_hudi_datestr_from_timestamp(commit_timestamp)
operation_metrics = last_commit["operationMetrics"]
# Get info about the oldest remaining commit
oldest_commit = fg_source_table.history().orderBy("version").first().asDict()
oldest_commit_timestamp = util.convert_event_time_to_timestamp(
oldest_commit["timestamp"]
)
if version == 0:
fg_commit = feature_group_commit.FeatureGroupCommit(
commitid=None,
commit_date_string=commit_date_string,
commit_time=commit_timestamp,
rows_inserted=operation_metrics["numOutputRows"],
rows_updated=0,
rows_deleted=0,
last_active_commit_time=oldest_commit_timestamp,
)
else:
fg_commit = feature_group_commit.FeatureGroupCommit(
commitid=None,
commit_date_string=commit_date_string,
commit_time=commit_timestamp,
rows_inserted=operation_metrics["numTargetRowsInserted"],
rows_updated=operation_metrics["numTargetRowsUpdated"],
rows_deleted=operation_metrics["numTargetRowsDeleted"],
last_active_commit_time=oldest_commit_timestamp,
)
return fg_commit