forked from logicalclocks/hopsworks-api
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathspark.py
1535 lines (1383 loc) · 58.5 KB
/
spark.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
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# Copyright 2020 Logical Clocks 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
import copy
import json
import os
import re
import shutil
import uuid
import warnings
from datetime import date, datetime, timezone
from typing import TYPE_CHECKING, Any, Dict, List, Optional, TypeVar, Union
if TYPE_CHECKING:
import great_expectations
from pyspark.rdd import RDD
from pyspark.sql import DataFrame
import numpy as np
import pandas as pd
import tzlocal
from hsfs.constructor import query
# in case importing in %%local
from hsfs.core.vector_db_client import VectorDbClient
try:
import pyspark
from pyspark import SparkFiles
from pyspark.rdd import RDD
from pyspark.sql import DataFrame, SparkSession, SQLContext
from pyspark.sql.avro.functions import from_avro, to_avro
from pyspark.sql.functions import (
array,
col,
concat,
from_json,
lit,
struct,
udf,
)
from pyspark.sql.types import (
ArrayType,
BinaryType,
BooleanType,
ByteType,
DateType,
DecimalType,
DoubleType,
FloatType,
IntegerType,
LongType,
ShortType,
StringType,
StructField,
StructType,
TimestampType,
)
if pd.__version__ >= "2.0.0" and pyspark.__version__ < "3.2.3":
def iteritems(self):
return self.items()
pd.DataFrame.iteritems = iteritems
except ImportError:
pass
from hopsworks_common import client
from hopsworks_common.client.exceptions import FeatureStoreException
from hsfs import (
feature,
feature_view,
training_dataset,
training_dataset_feature,
transformation_function,
util,
)
from hsfs import feature_group as fg_mod
from hsfs.core import (
dataset_api,
delta_engine,
hudi_engine,
kafka_engine,
transformation_function_engine,
)
from hsfs.core.constants import HAS_AVRO, HAS_GREAT_EXPECTATIONS
from hsfs.decorators import uses_great_expectations
from hsfs.storage_connector import StorageConnector
from hsfs.training_dataset_split import TrainingDatasetSplit
if HAS_GREAT_EXPECTATIONS:
import great_expectations
if HAS_AVRO:
import avro
class Engine:
HIVE_FORMAT = "hive"
KAFKA_FORMAT = "kafka"
APPEND = "append"
OVERWRITE = "overwrite"
def __init__(self):
self._spark_session = SparkSession.builder.enableHiveSupport().getOrCreate()
self._spark_context = self._spark_session.sparkContext
# self._spark_context.setLogLevel("DEBUG")
self._jvm = self._spark_context._jvm
self._spark_session.conf.set("hive.exec.dynamic.partition", "true")
self._spark_session.conf.set("hive.exec.dynamic.partition.mode", "nonstrict")
self._spark_session.conf.set("spark.sql.hive.convertMetastoreParquet", "false")
self._spark_session.conf.set("spark.sql.session.timeZone", "UTC")
self._dataset_api = dataset_api.DatasetApi()
def sql(
self,
sql_query,
feature_store,
connector,
dataframe_type,
read_options,
schema=None,
):
if not connector:
result_df = self._sql_offline(sql_query, feature_store)
else:
result_df = connector.read(sql_query, None, {}, None)
self.set_job_group("", "")
return self._return_dataframe_type(result_df, dataframe_type)
def is_flyingduck_query_supported(self, query, read_options=None):
return False # we do not support flyingduck on pyspark clients
def _sql_offline(self, sql_query, feature_store):
# set feature store
self._spark_session.sql("USE {}".format(feature_store))
return self._spark_session.sql(sql_query)
def show(self, sql_query, feature_store, n, online_conn, read_options=None):
return self.sql(
sql_query, feature_store, online_conn, "default", read_options
).show(n)
def read_vector_db(
self,
feature_group: fg_mod.FeatureGroup,
n: int = None,
dataframe_type: str = "default",
) -> Union[
pd.DataFrame, np.ndarray, List[List[Any]], TypeVar("pyspark.sql.DataFrame")
]:
results = VectorDbClient.read_feature_group(feature_group, n)
feature_names = [f.name for f in feature_group.features]
dataframe_type = dataframe_type.lower()
if dataframe_type in ["default", "spark"]:
if len(results) == 0:
return self._spark_session.createDataFrame(
self._spark_session.sparkContext.emptyRDD(), StructType()
)
else:
return self._spark_session.createDataFrame(results, feature_names)
else:
df = pd.DataFrame(results, columns=feature_names, index=None)
return self._return_dataframe_type(df, dataframe_type)
def set_job_group(self, group_id, description):
self._spark_session.sparkContext.setJobGroup(group_id, description)
def register_external_temporary_table(self, external_fg, alias):
if isinstance(external_fg, fg_mod.ExternalFeatureGroup):
external_dataset = external_fg.storage_connector.read(
external_fg.query,
external_fg.data_format,
external_fg.options,
external_fg.storage_connector._get_path(external_fg.path),
)
elif isinstance(external_fg, fg_mod.FeatureGroup):
external_dataset = external_fg.storage_connector.read(
None,
external_fg.timeTravelFormat,
None,
external_fg.storage_connector._get_path(external_fg.path),
)
else:
external_dataset = external_fg.dataframe
if external_fg.location:
self._spark_session.sparkContext.textFile(external_fg.location).collect()
external_dataset.createOrReplaceTempView(alias)
return external_dataset
def register_hudi_temporary_table(
self, hudi_fg_alias, feature_store_id, feature_store_name, read_options
):
hudi_engine_instance = hudi_engine.HudiEngine(
feature_store_id,
feature_store_name,
hudi_fg_alias.feature_group,
self._spark_context,
self._spark_session,
)
hudi_engine_instance.register_temporary_table(
hudi_fg_alias,
read_options,
)
hudi_engine_instance.reconcile_hudi_schema(
self.save_empty_dataframe, hudi_fg_alias, read_options
)
def register_delta_temporary_table(
self, delta_fg_alias, feature_store_id, feature_store_name, read_options
):
delta_engine_instance = delta_engine.DeltaEngine(
feature_store_id,
feature_store_name,
delta_fg_alias.feature_group,
self._spark_session,
self._spark_context,
)
delta_engine_instance.register_temporary_table(
delta_fg_alias,
read_options,
)
def _return_dataframe_type(self, dataframe, dataframe_type):
if dataframe_type.lower() in ["default", "spark"]:
return dataframe
# Converting to pandas dataframe if return type is not spark
if isinstance(dataframe, DataFrame):
dataframe = dataframe.toPandas()
if dataframe_type.lower() == "pandas":
return dataframe
if dataframe_type.lower() == "numpy":
return dataframe.values
if dataframe_type.lower() == "python":
return dataframe.values.tolist()
raise TypeError(
"Dataframe type `{}` not supported on this platform.".format(dataframe_type)
)
def convert_to_default_dataframe(self, dataframe):
if isinstance(dataframe, list):
dataframe = np.array(dataframe)
if isinstance(dataframe, np.ndarray):
if dataframe.ndim != 2:
raise TypeError(
"Cannot convert numpy array that do not have two dimensions to a dataframe. "
"The number of dimensions are: {}".format(dataframe.ndim)
)
num_cols = dataframe.shape[1]
dataframe_dict = {}
for n_col in list(range(num_cols)):
col_name = "col_" + str(n_col)
dataframe_dict[col_name] = dataframe[:, n_col]
dataframe = pd.DataFrame(dataframe_dict)
if isinstance(dataframe, pd.DataFrame):
# convert timestamps to current timezone
local_tz = tzlocal.get_localzone()
# make shallow copy so the original df does not get changed
dataframe_copy = dataframe.copy(deep=False)
for c in dataframe_copy.columns:
if isinstance(
dataframe_copy[c].dtype, pd.core.dtypes.dtypes.DatetimeTZDtype
):
# convert to utc timestamp
dataframe_copy[c] = dataframe_copy[c].dt.tz_convert(None)
if dataframe_copy[c].dtype == np.dtype("datetime64[ns]"):
# set the timezone to the client's timezone because that is
# what spark expects.
dataframe_copy[c] = dataframe_copy[c].dt.tz_localize(
str(local_tz), ambiguous="infer", nonexistent="shift_forward"
)
dataframe = self._spark_session.createDataFrame(dataframe_copy)
elif isinstance(dataframe, RDD):
dataframe = dataframe.toDF()
if isinstance(dataframe, DataFrame):
upper_case_features = [
c for c in dataframe.columns if any(re.finditer("[A-Z]", c))
]
space_features = [c for c in dataframe.columns if " " in c]
if len(upper_case_features) > 0:
warnings.warn(
"The ingested dataframe contains upper case letters in feature names: `{}`. "
"Feature names are sanitized to lower case in the feature store.".format(
upper_case_features
),
util.FeatureGroupWarning,
stacklevel=1,
)
if len(space_features) > 0:
warnings.warn(
"The ingested dataframe contains feature names with spaces: `{}`. "
"Feature names are sanitized to use underscore '_' in the feature store.".format(
space_features
),
util.FeatureGroupWarning,
stacklevel=1,
)
lowercase_dataframe = dataframe.select(
[col(x).alias(util.autofix_feature_name(x)) for x in dataframe.columns]
)
# for streaming dataframes this will be handled in DeltaStreamerTransformer.java class
if not lowercase_dataframe.isStreaming:
nullable_schema = copy.deepcopy(lowercase_dataframe.schema)
for struct_field in nullable_schema:
struct_field.nullable = True
lowercase_dataframe = self._spark_session.createDataFrame(
lowercase_dataframe.rdd, nullable_schema
)
return lowercase_dataframe
if dataframe == "spine":
return None
raise TypeError(
"The provided dataframe type is not recognized. Supported types are: spark rdds, spark dataframes, "
"pandas dataframes, python 2D lists, and numpy 2D arrays. The provided dataframe has type: {}".format(
type(dataframe)
)
)
def save_dataframe(
self,
feature_group,
dataframe,
operation,
online_enabled,
storage,
offline_write_options,
online_write_options,
validation_id=None,
):
try:
# Currently on-demand transformation functions not supported in external feature groups.
if (
not isinstance(feature_group, fg_mod.ExternalFeatureGroup)
and feature_group.transformation_functions
):
dataframe = self._apply_transformation_function(
feature_group.transformation_functions, dataframe
)
if (
isinstance(feature_group, fg_mod.ExternalFeatureGroup)
and feature_group.online_enabled
) or feature_group.stream:
self._save_online_dataframe(
feature_group, dataframe, online_write_options
)
else:
if storage == "offline" or not online_enabled:
self._save_offline_dataframe(
feature_group,
dataframe,
operation,
offline_write_options,
validation_id,
)
elif storage == "online":
self._save_online_dataframe(
feature_group, dataframe, online_write_options
)
elif online_enabled and storage is None:
self._save_offline_dataframe(
feature_group,
dataframe,
operation,
offline_write_options,
)
self._save_online_dataframe(
feature_group, dataframe, online_write_options
)
except Exception as e:
raise FeatureStoreException(e).with_traceback(e.__traceback__) from e
def save_stream_dataframe(
self,
feature_group: Union[fg_mod.FeatureGroup, fg_mod.ExternalFeatureGroup],
dataframe,
query_name,
output_mode,
await_termination: bool,
timeout,
checkpoint_dir: Optional[str],
write_options: Optional[Dict[str, Any]],
):
if feature_group.transformation_functions:
dataframe = self._apply_transformation_function(
feature_group.transformation_functions, dataframe
)
write_options = kafka_engine.get_kafka_config(
feature_group.feature_store_id, write_options, engine="spark"
)
serialized_df = self._online_fg_to_avro(
feature_group, self._encode_complex_features(feature_group, dataframe)
)
project_id = str(feature_group.feature_store.project_id)
feature_group_id = str(feature_group._id)
subject_id = str(feature_group.subject["id"]).encode("utf8")
if query_name is None:
query_name = (
f"insert_stream_{project_id}_{feature_group_id}"
f"_{feature_group.name}_{feature_group.version}_onlinefs"
)
query = (
serialized_df.withColumn(
"headers",
array(
struct(
lit("projectId").alias("key"),
lit(project_id.encode("utf8")).alias("value"),
),
struct(
lit("featureGroupId").alias("key"),
lit(feature_group_id.encode("utf8")).alias("value"),
),
struct(
lit("subjectId").alias("key"), lit(subject_id).alias("value")
),
),
)
.writeStream.outputMode(output_mode)
.format(self.KAFKA_FORMAT)
.option(
"checkpointLocation",
"/Projects/"
+ client.get_instance()._project_name
+ "/Resources/"
+ query_name
+ "-checkpoint"
if checkpoint_dir is None
else checkpoint_dir,
)
.options(**write_options)
.option("topic", feature_group._online_topic_name)
.queryName(query_name)
.start()
)
if await_termination:
query.awaitTermination(timeout)
return query
def _save_offline_dataframe(
self,
feature_group,
dataframe,
operation,
write_options,
validation_id=None,
):
if feature_group.time_travel_format == "HUDI":
hudi_engine_instance = hudi_engine.HudiEngine(
feature_group.feature_store_id,
feature_group.feature_store_name,
feature_group,
self._spark_session,
self._spark_context,
)
hudi_engine_instance.save_hudi_fg(
dataframe, self.APPEND, operation, write_options, validation_id
)
elif feature_group.time_travel_format == "DELTA":
delta_engine_instance = delta_engine.DeltaEngine(
feature_group.feature_store_id,
feature_group.feature_store_name,
feature_group,
self._spark_session,
self._spark_context,
)
delta_engine_instance.save_delta_fg(dataframe, write_options, validation_id)
else:
dataframe.write.format(self.HIVE_FORMAT).mode(self.APPEND).options(
**write_options
).partitionBy(
feature_group.partition_key if feature_group.partition_key else []
).saveAsTable(feature_group._get_table_name())
def _save_online_dataframe(self, feature_group, dataframe, write_options):
write_options = kafka_engine.get_kafka_config(
feature_group.feature_store_id, write_options, engine="spark"
)
serialized_df = self._online_fg_to_avro(
feature_group, self._encode_complex_features(feature_group, dataframe)
)
project_id = str(feature_group.feature_store.project_id).encode("utf8")
feature_group_id = str(feature_group._id).encode("utf8")
subject_id = str(feature_group.subject["id"]).encode("utf8")
serialized_df.withColumn(
"headers",
array(
struct(lit("projectId").alias("key"), lit(project_id).alias("value")),
struct(
lit("featureGroupId").alias("key"),
lit(feature_group_id).alias("value"),
),
struct(lit("subjectId").alias("key"), lit(subject_id).alias("value")),
),
).write.format(self.KAFKA_FORMAT).options(**write_options).option(
"topic", feature_group._online_topic_name
).save()
def _encode_complex_features(
self,
feature_group: Union[fg_mod.FeatureGroup, fg_mod.ExternalFeatureGroup],
dataframe: Union[RDD, DataFrame],
):
"""Encodes all complex type features to binary using their avro type as schema."""
return dataframe.select(
[
field["name"]
if field["name"] not in feature_group.get_complex_features()
else to_avro(
field["name"], feature_group._get_feature_avro_schema(field["name"])
).alias(field["name"])
for field in json.loads(feature_group.avro_schema)["fields"]
]
)
def _online_fg_to_avro(
self,
feature_group: Union[fg_mod.FeatureGroup, fg_mod.ExternalFeatureGroup],
dataframe: Union[DataFrame, RDD],
):
"""Packs all features into named struct to be serialized to single avro/binary
column. And packs primary key into arry to be serialized for partitioning.
"""
return dataframe.select(
[
# be aware: primary_key array should always be sorted
to_avro(
concat(
*[
col(f).cast("string")
for f in sorted(feature_group.primary_key)
]
)
).alias("key"),
to_avro(
struct(
[
field["name"]
for field in json.loads(feature_group.avro_schema)["fields"]
]
),
feature_group._get_encoded_avro_schema(),
).alias("value"),
]
)
def get_training_data(
self,
training_dataset: training_dataset.TrainingDataset,
feature_view_obj: feature_view.FeatureView,
query_obj: query.Query,
read_options: Dict[str, Any],
dataframe_type: str,
training_dataset_version: int = None,
):
"""
Function that creates or retrieves already created the training dataset.
# Arguments
training_dataset_obj `TrainingDataset`: The training dataset metadata object.
feature_view_obj `FeatureView`: The feature view object for the which the training data is being created.
query_obj `Query`: The query object that contains the query used to create the feature view.
read_options `Dict[str, Any]`: Dictionary that can be used to specify extra parameters for reading data.
dataframe_type `str`: The type of dataframe returned.
training_dataset_version `int`: Version of training data to be retrieved.
# Raises
`ValueError`: If the training dataset statistics could not be retrieved.
"""
return self.write_training_dataset(
training_dataset,
query_obj,
read_options,
None,
read_options=read_options,
to_df=True,
feature_view_obj=feature_view_obj,
training_dataset_version=training_dataset_version,
)
def split_labels(self, df, labels, dataframe_type):
if labels:
if isinstance(df, pd.DataFrame):
labels_df = df[labels]
df_new = df.drop(columns=labels)
else:
labels_df = df.select(*labels)
df_new = df.drop(*labels)
return (
self._return_dataframe_type(df_new, dataframe_type),
self._return_dataframe_type(labels_df, dataframe_type),
)
else:
return self._return_dataframe_type(df, dataframe_type), None
def drop_columns(self, df, drop_cols):
return df.drop(*drop_cols)
def write_training_dataset(
self,
training_dataset: training_dataset.TrainingDataset,
query_obj: query.Query,
user_write_options: Dict[str, Any],
save_mode: str,
read_options: Dict[str, Any] = None,
feature_view_obj: feature_view.FeatureView = None,
to_df: bool = False,
training_dataset_version: Optional[int] = None,
):
"""
Function that creates or retrieves already created the training dataset.
# Arguments
training_dataset `TrainingDataset`: The training dataset metadata object.
query_obj `Query`: The query object that contains the query used to create the feature view.
user_write_options `Dict[str, Any]`: Dictionary that can be used to specify extra parameters for writing data using spark.
save_mode `str`: Spark save mode to be used while writing data.
read_options `Dict[str, Any]`: Dictionary that can be used to specify extra parameters for reading data.
feature_view_obj `FeatureView`: The feature view object for the which the training data is being created.
to_df `bool`: Return dataframe instead of writing the data.
training_dataset_version `Optional[int]`: Version of training data to be retrieved.
# Raises
`ValueError`: If the training dataset statistics could not be retrieved.
"""
write_options = self.write_options(
training_dataset.data_format, user_write_options
)
if read_options is None:
read_options = {}
if len(training_dataset.splits) == 0:
if isinstance(query_obj, query.Query):
dataset = self.convert_to_default_dataframe(
query_obj.read(read_options=read_options)
)
else:
raise ValueError("Dataset should be a query.")
# if training_dataset_version is None:
transformation_function_engine.TransformationFunctionEngine.compute_and_set_feature_statistics(
training_dataset, feature_view_obj, dataset
)
# else:
# transformation_function_engine.TransformationFunctionEngine.get_and_set_feature_statistics(
# training_dataset, feature_view_obj, training_dataset_version
# )
if training_dataset.coalesce:
dataset = dataset.coalesce(1)
path = training_dataset.location + "/" + training_dataset.name
return self._write_training_dataset_single(
feature_view_obj.transformation_functions,
dataset,
training_dataset.storage_connector,
training_dataset.data_format,
write_options,
save_mode,
path,
to_df=to_df,
)
else:
split_dataset = self._split_df(
query_obj, training_dataset, read_options=read_options
)
for key in split_dataset:
if training_dataset.coalesce:
split_dataset[key] = split_dataset[key].coalesce(1)
split_dataset[key] = split_dataset[key].cache()
if training_dataset_version is None:
transformation_function_engine.TransformationFunctionEngine.compute_and_set_feature_statistics(
training_dataset, feature_view_obj, split_dataset
)
else:
transformation_function_engine.TransformationFunctionEngine.get_and_set_feature_statistics(
training_dataset, feature_view_obj, training_dataset_version
)
return self._write_training_dataset_splits(
training_dataset,
split_dataset,
write_options,
save_mode,
to_df=to_df,
transformation_functions=feature_view_obj.transformation_functions,
)
def _split_df(self, query_obj, training_dataset, read_options=None):
if read_options is None:
read_options = {}
if (
training_dataset.splits[0].split_type
== TrainingDatasetSplit.TIME_SERIES_SPLIT
):
event_time = query_obj._left_feature_group.event_time
if event_time not in [_feature.name for _feature in query_obj.features]:
query_obj.append_feature(
query_obj._left_feature_group.__getattr__(event_time)
)
return self._time_series_split(
training_dataset,
query_obj.read(read_options=read_options),
event_time,
drop_event_time=True,
)
else:
return self._time_series_split(
training_dataset,
query_obj.read(read_options=read_options),
event_time,
)
else:
return self._random_split(
query_obj.read(read_options=read_options), training_dataset
)
def _random_split(self, dataset, training_dataset):
splits = [(split.name, split.percentage) for split in training_dataset.splits]
split_weights = [split[1] for split in splits]
split_dataset = dataset.randomSplit(split_weights, training_dataset.seed)
return dict([(split[0], split_dataset[i]) for i, split in enumerate(splits)])
def _time_series_split(
self, training_dataset, dataset, event_time, drop_event_time=False
):
# duplicate the code from util module to avoid udf errors on windows
def check_timestamp_format_from_date_string(input_date):
date_format_patterns = {
r"^([0-9]{4})([0-9]{2})([0-9]{2})$": "%Y%m%d",
r"^([0-9]{4})([0-9]{2})([0-9]{2})([0-9]{2})$": "%Y%m%d%H",
r"^([0-9]{4})([0-9]{2})([0-9]{2})([0-9]{2})([0-9]{2})$": "%Y%m%d%H%M",
r"^([0-9]{4})([0-9]{2})([0-9]{2})([0-9]{2})([0-9]{2})([0-9]{2})$": "%Y%m%d%H%M%S",
r"^([0-9]{4})([0-9]{2})([0-9]{2})([0-9]{2})([0-9]{2})([0-9]{2})([0-9]{3})$": "%Y%m%d%H%M%S%f",
r"^([0-9]{4})([0-9]{2})([0-9]{2})T([0-9]{2})([0-9]{2})([0-9]{2})([0-9]{6})Z$": "ISO",
}
normalized_date = (
input_date.replace("/", "")
.replace("-", "")
.replace(" ", "")
.replace(":", "")
.replace(".", "")
)
date_format = None
for pattern in date_format_patterns:
date_format_pattern = re.match(pattern, normalized_date)
if date_format_pattern:
date_format = date_format_patterns[pattern]
break
if date_format is None:
raise ValueError(
"Unable to identify format of the provided date value : "
+ input_date
)
return normalized_date, date_format
def get_timestamp_from_date_string(input_date):
norm_input_date, date_format = check_timestamp_format_from_date_string(
input_date
)
try:
if date_format != "ISO":
date_time = datetime.strptime(norm_input_date, date_format)
else:
date_time = datetime.fromisoformat(input_date[:-1])
except ValueError as err:
raise ValueError(
"Unable to parse the normalized input date value : "
+ norm_input_date
+ " with format "
+ date_format
) from err
if date_time.tzinfo is None:
date_time = date_time.replace(tzinfo=timezone.utc)
return int(float(date_time.timestamp()) * 1000)
def convert_event_time_to_timestamp(event_time):
if not event_time:
return None
if isinstance(event_time, str):
return get_timestamp_from_date_string(event_time)
elif isinstance(event_time, pd._libs.tslibs.timestamps.Timestamp):
# convert to unix epoch time in milliseconds.
event_time = event_time.to_pydatetime()
# convert to unix epoch time in milliseconds.
if event_time.tzinfo is None:
event_time = event_time.replace(tzinfo=timezone.utc)
return int(event_time.timestamp() * 1000)
elif isinstance(event_time, datetime):
# convert to unix epoch time in milliseconds.
if event_time.tzinfo is None:
event_time = event_time.replace(tzinfo=timezone.utc)
return int(event_time.timestamp() * 1000)
elif isinstance(event_time, date):
# convert to unix epoch time in milliseconds.
event_time = datetime(*event_time.timetuple()[:7])
if event_time.tzinfo is None:
event_time = event_time.replace(tzinfo=timezone.utc)
return int(event_time.timestamp() * 1000)
elif isinstance(event_time, int):
if event_time == 0:
raise ValueError("Event time should be greater than 0.")
# jdbc supports timestamp precision up to second only.
if len(str(event_time)) <= 10:
event_time = event_time * 1000
return event_time
else:
raise ValueError(
"Given event time should be in `datetime`, `date`, `str` or `int` type"
)
# registering the UDF
_convert_event_time_to_timestamp = udf(
convert_event_time_to_timestamp, LongType()
)
result_dfs = {}
ts_col = _convert_event_time_to_timestamp(col(event_time))
for split in training_dataset.splits:
result_df = dataset.filter(ts_col >= split.start_time).filter(
ts_col < split.end_time
)
if drop_event_time:
result_df = result_df.drop(event_time)
result_dfs[split.name] = result_df
return result_dfs
def _write_training_dataset_splits(
self,
training_dataset,
feature_dataframes,
write_options,
save_mode,
to_df=False,
transformation_functions: List[
transformation_function.TransformationFunction
] = None,
):
for split_name, feature_dataframe in feature_dataframes.items():
split_path = training_dataset.location + "/" + str(split_name)
feature_dataframes[split_name] = self._write_training_dataset_single(
transformation_functions,
feature_dataframe,
training_dataset.storage_connector,
training_dataset.data_format,
write_options,
save_mode,
split_path,
to_df=to_df,
)
if to_df:
return feature_dataframes
def _write_training_dataset_single(
self,
transformation_functions,
feature_dataframe,
storage_connector,
data_format,
write_options,
save_mode,
path,
to_df=False,
):
# apply transformation functions (they are applied separately to each split)
feature_dataframe = self._apply_transformation_function(
transformation_functions, dataset=feature_dataframe
)
if to_df:
return feature_dataframe
# TODO: currently not supported petastorm, hdf5 and npy file formats
if data_format.lower() == "tsv":
data_format = "csv"
path = self.setup_storage_connector(storage_connector, path)
feature_dataframe.write.format(data_format).options(**write_options).mode(
save_mode
).save(path)
feature_dataframe.unpersist()
def read(
self, storage_connector, data_format, read_options, location, dataframe_type
):
if not data_format:
raise FeatureStoreException("data_format is not specified")
if isinstance(location, str):
if data_format.lower() in ["delta", "parquet", "hudi", "orc", "bigquery"]:
# All the above data format readers can handle partitioning
# by their own, they don't need /**
# for bigquery, argument location can be a SQL query
path = location
else:
path = location + "/**"
if data_format.lower() == "tsv":
data_format = "csv"
else:
path = None
path = self.setup_storage_connector(storage_connector, path)
return self._return_dataframe_type(
self._spark_session.read.format(data_format)
.options(**(read_options if read_options else {}))
.load(path),
dataframe_type=dataframe_type,
)
def read_stream(
self,
storage_connector,
message_format,
schema,
options,
include_metadata,
):
# ideally all this logic should be in the storage connector in case we add more
# streaming storage connectors...
stream = self._spark_session.readStream.format(
storage_connector.SPARK_FORMAT
) # todo SPARK_FORMAT available only for KAFKA connectors
# set user options last so that they overwrite any default options
stream = stream.options(**storage_connector.spark_options(), **options)
if storage_connector.type == StorageConnector.KAFKA:
return self._read_stream_kafka(
stream, message_format, schema, include_metadata
)
def _read_stream_kafka(self, stream, message_format, schema, include_metadata):
kafka_cols = [
col("key"),
col("topic"),
col("partition"),
col("offset"),
col("timestamp"),
col("timestampType"),
]
if message_format == "avro" and schema is not None:
# check if vallid avro schema
avro.schema.parse(schema)
df = stream.load()
if include_metadata is True:
return df.select(
*kafka_cols, from_avro(df.value, schema).alias("value")
).select(*kafka_cols, col("value.*"))
return df.select(from_avro(df.value, schema).alias("value")).select(
col("value.*")
)
elif message_format == "json" and schema is not None: