-
Notifications
You must be signed in to change notification settings - Fork 1.3k
Expand file tree
/
Copy pathdask.py
More file actions
1264 lines (1090 loc) · 44.9 KB
/
Copy pathdask.py
File metadata and controls
1264 lines (1090 loc) · 44.9 KB
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
import json
import os
import uuid
from datetime import date, datetime, timezone
from pathlib import Path
from typing import Any, Callable, Dict, List, Literal, Optional, Tuple, Union
import dask
import dask.dataframe as dd
import numpy as np
import pandas as pd
import pyarrow
import pyarrow.compute as pc
import pyarrow.dataset
import pyarrow.parquet as pq
import pytz
from feast.data_source import DataSource
from feast.errors import (
FeastJoinKeysDuringMaterialization,
SavedDatasetLocationAlreadyExists,
)
from feast.feature_logging import LoggingConfig, LoggingSource
from feast.feature_view import DUMMY_ENTITY_ID, DUMMY_ENTITY_VAL, FeatureView
from feast.infra.offline_stores.file_source import (
FileLoggingDestination,
FileSource,
SavedDatasetFileStorage,
)
from feast.infra.offline_stores.offline_store import (
OfflineStore,
RetrievalJob,
RetrievalMetadata,
)
from feast.infra.offline_stores.offline_utils import (
DEFAULT_ENTITY_DF_EVENT_TIMESTAMP_COL,
get_pyarrow_schema_from_batch_source,
)
from feast.infra.registry.base_registry import BaseRegistry
from feast.monitoring.monitoring_utils import (
MONITORING_DIR,
MONITORING_PARQUET_FILES,
monitoring_parquet_meta,
normalize_monitoring_row,
opt_float,
)
from feast.on_demand_feature_view import OnDemandFeatureView
from feast.repo_config import FeastConfigBaseModel, RepoConfig
from feast.saved_dataset import SavedDatasetStorage
from feast.utils import (
_get_requested_feature_views_to_features_dict,
compute_non_entity_date_range,
)
# DaskRetrievalJob will cast string objects to string[pyarrow] from dask version 2023.7.1
# This is not the desired behavior for our use case, so we set the convert-string option to False
# See (https://github.com/dask/dask/issues/10881#issuecomment-1923327936)
dask.config.set({"dataframe.convert-string": False})
class DaskOfflineStoreConfig(FeastConfigBaseModel):
"""Offline store config for dask store"""
type: Union[Literal["dask"], Literal["file"]] = "dask"
""" Offline store type selector"""
class DaskRetrievalJob(RetrievalJob):
def __init__(
self,
evaluation_function: Callable,
full_feature_names: bool,
repo_path: str,
on_demand_feature_views: Optional[List[OnDemandFeatureView]] = None,
metadata: Optional[RetrievalMetadata] = None,
):
"""Initialize a lazy historical retrieval job"""
# The evaluation function executes a stored procedure to compute a historical retrieval.
self.evaluation_function = evaluation_function
self._full_feature_names = full_feature_names
self._on_demand_feature_views = on_demand_feature_views or []
self._metadata = metadata
self.repo_path = repo_path
@property
def full_feature_names(self) -> bool:
return self._full_feature_names
@property
def on_demand_feature_views(self) -> List[OnDemandFeatureView]:
return self._on_demand_feature_views
def _to_df_internal(self, timeout: Optional[int] = None) -> pd.DataFrame:
# Only execute the evaluation function to build the final historical retrieval dataframe at the last moment.
df = self.evaluation_function().compute()
df = df.reset_index(drop=True)
return df
def _to_arrow_internal(self, timeout: Optional[int] = None):
# Only execute the evaluation function to build the final historical retrieval dataframe at the last moment.
df = self.evaluation_function().compute()
return pyarrow.Table.from_pandas(df)
def persist(
self,
storage: SavedDatasetStorage,
allow_overwrite: Optional[bool] = False,
timeout: Optional[int] = None,
):
assert isinstance(storage, SavedDatasetFileStorage)
# Check if the specified location already exists.
if not allow_overwrite and os.path.exists(storage.file_options.uri):
raise SavedDatasetLocationAlreadyExists(location=storage.file_options.uri)
absolute_path = FileSource.get_uri_for_file_path(
repo_path=self.repo_path, uri=storage.file_options.uri
)
filesystem, path = FileSource.create_filesystem_and_path(
str(absolute_path),
storage.file_options.s3_endpoint_override,
)
if path.endswith(".parquet"):
pyarrow.parquet.write_table(
self.to_arrow(), where=path, filesystem=filesystem
)
else:
# otherwise assume destination is directory
pyarrow.parquet.write_to_dataset(
self.to_arrow(), root_path=path, filesystem=filesystem
)
@property
def metadata(self) -> Optional[RetrievalMetadata]:
return self._metadata
def supports_remote_storage_export(self) -> bool:
return False
class DaskOfflineStore(OfflineStore):
@staticmethod
def get_historical_features(
config: RepoConfig,
feature_views: List[FeatureView],
feature_refs: List[str],
entity_df: Optional[Union[pd.DataFrame, dd.DataFrame, str]],
registry: BaseRegistry,
project: str,
full_feature_names: bool = False,
**kwargs,
) -> RetrievalJob:
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
for fv in feature_views:
assert isinstance(fv.batch_source, FileSource)
non_entity_mode = entity_df is None
if non_entity_mode:
start_date, end_date = compute_non_entity_date_range(
feature_views,
start_date=kwargs.get("start_date"),
end_date=kwargs.get("end_date"),
)
entity_df = pd.DataFrame(
{DEFAULT_ENTITY_DF_EVENT_TIMESTAMP_COL: [end_date]}
)
else:
if not isinstance(entity_df, pd.DataFrame) and not isinstance(
entity_df, dd.DataFrame
):
raise ValueError(
f"Please provide an entity_df of type pd.DataFrame or dask.dataframe.DataFrame instead of type {type(entity_df)}"
)
entity_df_event_timestamp_col = DEFAULT_ENTITY_DF_EVENT_TIMESTAMP_COL # local modifiable copy of global variable
if entity_df_event_timestamp_col not in entity_df.columns:
datetime_columns = entity_df.select_dtypes(
include=["datetime", "datetimetz"]
).columns
if len(datetime_columns) == 1:
print(
f"Using {datetime_columns[0]} as the event timestamp. To specify a column explicitly, please name it {DEFAULT_ENTITY_DF_EVENT_TIMESTAMP_COL}."
)
entity_df_event_timestamp_col = datetime_columns[0]
else:
raise ValueError(
f"Please provide an entity_df with a column named {DEFAULT_ENTITY_DF_EVENT_TIMESTAMP_COL} representing the time of events."
)
(
feature_views_to_features,
on_demand_feature_views_to_features,
) = _get_requested_feature_views_to_features_dict(
feature_refs,
feature_views,
registry.list_on_demand_feature_views(config.project),
)
entity_df_event_timestamp_range = (
(start_date, end_date)
if non_entity_mode
else _get_entity_df_event_timestamp_range(
entity_df, entity_df_event_timestamp_col
)
)
# Create lazy function that is only called from the RetrievalJob object
def evaluate_historical_retrieval():
# Create a copy of entity_df to prevent modifying the original
entity_df_with_features = entity_df.copy()
entity_df_event_timestamp_col_type = entity_df_with_features.dtypes[
entity_df_event_timestamp_col
]
# TODO: need to figure out why the value of entity_df_event_timestamp_col_type.tz is pytz.UTC
if (
not hasattr(entity_df_event_timestamp_col_type, "tz")
or entity_df_event_timestamp_col_type.tz != pytz.UTC
):
# Make sure all event timestamp fields are tz-aware. We default tz-naive fields to UTC
entity_df_with_features[entity_df_event_timestamp_col] = (
entity_df_with_features[entity_df_event_timestamp_col].apply(
lambda x: (
x
if x.tzinfo is not None
else x.replace(tzinfo=timezone.utc)
)
)
)
# Convert event timestamp column to datetime and normalize time zone to UTC
# This is necessary to avoid issues with pd.merge_asof
if isinstance(entity_df_with_features, dd.DataFrame):
entity_df_with_features[entity_df_event_timestamp_col] = (
dd.to_datetime(
entity_df_with_features[entity_df_event_timestamp_col],
utc=True,
)
)
else:
entity_df_with_features[entity_df_event_timestamp_col] = (
pd.to_datetime(
entity_df_with_features[entity_df_event_timestamp_col],
utc=True,
)
)
# Sort event timestamp values
entity_df_with_features = entity_df_with_features.sort_values(
entity_df_event_timestamp_col
)
all_join_keys = []
# Load feature view data from sources and join them incrementally
for feature_view, features in feature_views_to_features.items():
timestamp_field = feature_view.batch_source.timestamp_field
created_timestamp_column = (
feature_view.batch_source.created_timestamp_column
)
# Build a list of entity columns to join on (from the right table)
join_keys = []
for entity_column in feature_view.entity_columns:
join_key = feature_view.projection.join_key_map.get(
entity_column.name, entity_column.name
)
join_keys.append(join_key)
right_entity_key_columns = [
timestamp_field,
created_timestamp_column,
] + join_keys
right_entity_key_columns = [c for c in right_entity_key_columns if c]
all_join_keys = list(set(all_join_keys + join_keys))
df_to_join = _read_datasource(
feature_view.batch_source, config.repo_path
)
df_to_join, timestamp_field = _field_mapping(
df_to_join,
feature_view,
features,
right_entity_key_columns,
entity_df_event_timestamp_col,
timestamp_field,
full_feature_names,
)
# In non-entity mode, if the synthetic entity_df lacks join keys, cross join to build a snapshot
# of all entities as-of the requested timestamp, then rely on TTL and deduplication to select
# the appropriate latest rows per entity.
current_join_keys = join_keys
if non_entity_mode:
current_join_keys = []
df_to_join = _merge(
entity_df_with_features, df_to_join, current_join_keys
)
df_to_join = _normalize_timestamp(
df_to_join, timestamp_field, created_timestamp_column
)
df_to_join = _filter_ttl(
df_to_join,
feature_view,
entity_df_event_timestamp_col,
timestamp_field,
)
df_to_join = _drop_duplicates(
df_to_join,
all_join_keys,
timestamp_field,
created_timestamp_column,
entity_df_event_timestamp_col,
)
entity_df_with_features = _drop_columns(
df_to_join, features, timestamp_field, created_timestamp_column
)
# Ensure that we delete dataframes to free up memory
del df_to_join
return entity_df_with_features.persist()
job = DaskRetrievalJob(
evaluation_function=evaluate_historical_retrieval,
full_feature_names=full_feature_names,
on_demand_feature_views=OnDemandFeatureView.get_requested_odfvs(
feature_refs, project, registry
),
metadata=RetrievalMetadata(
features=feature_refs,
keys=list(set(entity_df.columns) - {entity_df_event_timestamp_col}),
min_event_timestamp=entity_df_event_timestamp_range[0],
max_event_timestamp=entity_df_event_timestamp_range[1],
),
repo_path=str(config.repo_path),
)
return job
@staticmethod
def pull_latest_from_table_or_query(
config: RepoConfig,
data_source: DataSource,
join_key_columns: List[str],
feature_name_columns: List[str],
timestamp_field: str,
created_timestamp_column: Optional[str],
start_date: Optional[datetime] = None,
end_date: Optional[datetime] = None,
) -> RetrievalJob:
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
assert isinstance(data_source, FileSource)
def evaluate_func():
df = DaskOfflineStore.evaluate_offline_job(
config=config,
data_source=data_source,
join_key_columns=join_key_columns,
timestamp_field=timestamp_field,
created_timestamp_column=created_timestamp_column,
start_date=start_date,
end_date=end_date,
)
ts_columns = (
[timestamp_field, created_timestamp_column]
if created_timestamp_column
else [timestamp_field]
)
columns_to_extract = set(
join_key_columns + feature_name_columns + ts_columns
)
if join_key_columns:
df = df.drop_duplicates(
join_key_columns, keep="last", ignore_index=True
)
else:
df[DUMMY_ENTITY_ID] = DUMMY_ENTITY_VAL
columns_to_extract.add(DUMMY_ENTITY_ID)
if feature_name_columns:
df = df[list(columns_to_extract)]
return df.persist()
# When materializing a single feature view, we don't need full feature names. On demand transforms aren't materialized
return DaskRetrievalJob(
evaluation_function=evaluate_func,
full_feature_names=False,
repo_path=str(config.repo_path),
)
@staticmethod
def evaluate_offline_job(
config: RepoConfig,
data_source: FileSource,
join_key_columns: List[str],
timestamp_field: str,
created_timestamp_column: Optional[str] = None,
start_date: Optional[datetime] = None,
end_date: Optional[datetime] = None,
) -> dd.DataFrame:
# Create lazy function that is only called from the RetrievalJob object
source_df = _read_datasource(data_source, config.repo_path)
# Validate join keys against source columns BEFORE calling _normalize_timestamp.
# _normalize_timestamp ends with df.persist() which initialises Dask's global
# ThreadPoolExecutor. When the process subsequently exits (e.g. after raising
# FeastJoinKeysDuringMaterialization), Python's atexit handler calls
# ThreadPoolExecutor.shutdown(wait=True) which can block indefinitely, hanging
# the subprocess and preventing the parent from reading its output.
# df.columns is derived from the Parquet schema metadata — no computation needed.
source_columns = set(source_df.columns)
if not set(join_key_columns).issubset(source_columns):
raise FeastJoinKeysDuringMaterialization(
data_source.path, set(join_key_columns), source_columns
)
source_df = _normalize_timestamp(
source_df, timestamp_field, created_timestamp_column
)
# try-catch block is added to deal with this issue https://github.com/dask/dask/issues/8939.
# TODO(kevjumba): remove try catch when fix is merged upstream in Dask.
try:
if created_timestamp_column:
source_df = source_df.sort_values(
by=created_timestamp_column,
)
source_df = source_df.sort_values(by=timestamp_field)
except ZeroDivisionError:
# Use 1 partition to get around case where everything in timestamp column is the same so the partition algorithm doesn't
# try to divide by zero.
if created_timestamp_column:
source_df = source_df.sort_values(
by=created_timestamp_column, npartitions=1
)
source_df = source_df.sort_values(by=timestamp_field, npartitions=1)
# TODO: The old implementation is inclusive of start_date and exclusive of end_date.
# Which is inconsistent with other offline stores.
if start_date or end_date:
if start_date and end_date:
source_df = source_df[
source_df[timestamp_field].between(
start_date, end_date, inclusive="both"
)
]
elif start_date:
source_df = source_df[source_df[timestamp_field] >= start_date]
elif end_date:
source_df = source_df[source_df[timestamp_field] <= end_date]
source_df = source_df.persist()
return source_df
@staticmethod
def pull_all_from_table_or_query(
config: RepoConfig,
data_source: DataSource,
join_key_columns: List[str],
feature_name_columns: List[str],
timestamp_field: str,
created_timestamp_column: Optional[str] = None,
start_date: Optional[datetime] = None,
end_date: Optional[datetime] = None,
) -> RetrievalJob:
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
assert isinstance(data_source, FileSource)
def evaluate_func():
df = DaskOfflineStore.evaluate_offline_job(
config=config,
data_source=data_source,
join_key_columns=join_key_columns,
timestamp_field=timestamp_field,
created_timestamp_column=created_timestamp_column,
start_date=start_date,
end_date=end_date,
)
ts_columns = (
[timestamp_field, created_timestamp_column]
if created_timestamp_column
else [timestamp_field]
)
columns_to_extract = set(
join_key_columns + feature_name_columns + ts_columns
)
if not join_key_columns:
df[DUMMY_ENTITY_ID] = DUMMY_ENTITY_VAL
columns_to_extract.add(DUMMY_ENTITY_ID)
# TODO: Decides if we want to field mapping for pull_latest_from_table_or_query
# This is default for other offline store.
if feature_name_columns:
df = df[list(columns_to_extract)]
return df.persist()
# When materializing a single feature view, we don't need full feature names. On demand transforms aren't materialized
return DaskRetrievalJob(
evaluation_function=evaluate_func,
full_feature_names=False,
repo_path=str(config.repo_path),
)
@staticmethod
def write_logged_features(
config: RepoConfig,
data: Union[pyarrow.Table, Path],
source: LoggingSource,
logging_config: LoggingConfig,
registry: BaseRegistry,
):
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
destination = logging_config.destination
assert isinstance(destination, FileLoggingDestination)
if isinstance(data, Path):
# Since this code will be mostly used from Go-created thread, it's better to avoid producing new threads
data = pyarrow.parquet.read_table(data, use_threads=False, pre_buffer=False)
if config.repo_path is not None and not Path(destination.path).is_absolute():
absolute_path = config.repo_path / destination.path
else:
absolute_path = Path(destination.path)
filesystem, path = FileSource.create_filesystem_and_path(
str(absolute_path),
destination.s3_endpoint_override,
)
pyarrow.dataset.write_dataset(
data,
base_dir=path,
basename_template=f"{uuid.uuid4().hex}-{{i}}.parquet",
partitioning=destination.partition_by,
filesystem=filesystem,
use_threads=False,
format=pyarrow.dataset.ParquetFileFormat(),
existing_data_behavior="overwrite_or_ignore",
)
@staticmethod
def offline_write_batch(
config: RepoConfig,
feature_view: FeatureView,
table: pyarrow.Table,
progress: Optional[Callable[[int], Any]],
):
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
assert isinstance(feature_view.batch_source, FileSource)
pa_schema, column_names = get_pyarrow_schema_from_batch_source(
config, feature_view.batch_source
)
if column_names != table.column_names:
raise ValueError(
f"The input pyarrow table has schema {table.schema} with the incorrect columns {table.column_names}. "
f"The schema is expected to be {pa_schema} with the columns (in this exact order) to be {column_names}."
)
file_options = feature_view.batch_source.file_options
absolute_path = FileSource.get_uri_for_file_path(
repo_path=config.repo_path, uri=file_options.uri
)
filesystem, path = FileSource.create_filesystem_and_path(
str(absolute_path), file_options.s3_endpoint_override
)
prev_table = pyarrow.parquet.read_table(
path, filesystem=filesystem, memory_map=True
)
if table.schema != prev_table.schema:
table = table.cast(prev_table.schema)
new_table = pyarrow.concat_tables([table, prev_table])
writer = pyarrow.parquet.ParquetWriter(
path, table.schema, filesystem=filesystem
)
writer.write_table(new_table)
writer.close()
@staticmethod
def compute_monitoring_metrics(
config: RepoConfig,
data_source: DataSource,
feature_columns: List[Tuple[str, str]],
timestamp_field: str,
start_date: Optional[datetime] = None,
end_date: Optional[datetime] = None,
histogram_bins: int = 20,
top_n: int = 10,
) -> List[Dict[str, Any]]:
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
assert isinstance(data_source, FileSource)
table = _dask_read_batch_arrow(data_source, config.repo_path)
table = _dask_filter_arrow_by_timestamp(
table, timestamp_field, start_date, end_date
)
results: List[Dict[str, Any]] = []
for name, ftype in feature_columns:
if name not in table.column_names:
continue
col = table[name]
if ftype == "numeric":
m = _dask_compute_numeric_metrics(col, histogram_bins)
elif ftype == "categorical":
m = _dask_compute_categorical_metrics(col, top_n)
else:
continue
m["feature_name"] = name
results.append(m)
return results
@staticmethod
def get_monitoring_max_timestamp(
config: RepoConfig,
data_source: DataSource,
timestamp_field: str,
) -> Optional[datetime]:
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
assert isinstance(data_source, FileSource)
absolute_path = FileSource.get_uri_for_file_path(
repo_path=config.repo_path,
uri=data_source.file_options.uri,
)
filesystem, path = FileSource.create_filesystem_and_path(
str(absolute_path), data_source.file_options.s3_endpoint_override
)
try:
t = pq.read_table(path, filesystem=filesystem, columns=[timestamp_field])
except Exception:
return None
if t.num_rows == 0:
return None
arr = t[timestamp_field]
mx = pc.max(arr) # type: ignore[attr-defined]
val = mx.as_py()
if val is None:
return None
if isinstance(val, datetime):
return val if val.tzinfo else val.replace(tzinfo=timezone.utc)
if isinstance(val, date):
return datetime.combine(val, datetime.min.time(), tzinfo=timezone.utc)
return None
@staticmethod
def ensure_monitoring_tables(config: RepoConfig) -> None:
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
base = os.path.join(_dask_monitoring_base(config), MONITORING_DIR)
os.makedirs(base, exist_ok=True)
tables = [
monitoring_parquet_meta(t)
for t in ("feature", "feature_view", "feature_service", "job")
]
for fname, columns, _ in tables:
fpath = _dask_monitoring_path(config, fname)
if not os.path.isfile(fpath):
os.makedirs(os.path.dirname(fpath), exist_ok=True)
pd.DataFrame(columns=columns).to_parquet(fpath, index=False)
@staticmethod
def save_monitoring_metrics(
config: RepoConfig,
metric_type: str,
metrics: List[Dict[str, Any]],
) -> None:
if not metrics:
return
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
fname, columns, pk = _dask_mon_table_meta(metric_type)
path = _dask_monitoring_path(config, fname)
_dask_parquet_upsert(path, columns, pk, metrics)
@staticmethod
def query_monitoring_metrics(
config: RepoConfig,
project: str,
metric_type: str,
filters: Optional[Dict[str, Any]] = None,
start_date: Optional[date] = None,
end_date: Optional[date] = None,
) -> List[Dict[str, Any]]:
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
fname, columns, _ = _dask_mon_table_meta(metric_type)
path = _dask_monitoring_path(config, fname)
return _dask_parquet_query(
path, columns, project, filters, start_date, end_date
)
@staticmethod
def clear_monitoring_baseline(
config: RepoConfig,
project: str,
feature_view_name: Optional[str] = None,
feature_name: Optional[str] = None,
data_source_type: Optional[str] = None,
) -> None:
assert isinstance(config.offline_store, DaskOfflineStoreConfig)
path = _dask_monitoring_path(config, MONITORING_PARQUET_FILES["feature"])
tab = _dask_read_parquet_if_exists(path)
if tab is None or tab.num_rows == 0:
return
df = tab.to_pandas()
mask = df["project_id"] == project
if feature_view_name is not None:
mask = mask & (df["feature_view_name"] == feature_view_name)
if feature_name is not None:
mask = mask & (df["feature_name"] == feature_name)
if data_source_type is not None:
mask = mask & (df["data_source_type"] == data_source_type)
mask = mask & (df["is_baseline"].isin([True, 1]))
df.loc[mask, "is_baseline"] = False
pq.write_table(pyarrow.Table.from_pandas(df, preserve_index=False), path)
def _dask_monitoring_base(config: RepoConfig) -> str:
base = config.repo_path
return str(base) if base else "."
def _dask_monitoring_path(config: RepoConfig, filename: str) -> str:
return os.path.join(_dask_monitoring_base(config), MONITORING_DIR, filename)
def _dask_mon_table_meta(metric_type: str):
return monitoring_parquet_meta(metric_type)
def _dask_read_parquet_if_exists(path: str) -> Optional[pyarrow.Table]:
if not os.path.isfile(path):
return None
return pq.read_table(path)
def _dask_read_batch_arrow(
data_source: FileSource, repo_path: Optional[Path]
) -> pyarrow.Table:
absolute_path = FileSource.get_uri_for_file_path(
repo_path=repo_path,
uri=data_source.file_options.uri,
)
filesystem, path = FileSource.create_filesystem_and_path(
str(absolute_path), data_source.file_options.s3_endpoint_override
)
return pq.read_table(path, filesystem=filesystem)
def _dask_filter_arrow_by_timestamp(
table: pyarrow.Table,
timestamp_field: str,
start_date: Optional[datetime],
end_date: Optional[datetime],
) -> pyarrow.Table:
if start_date is None and end_date is None:
return table
arr = table[timestamp_field]
mask = None
if start_date is not None:
mask = pc.greater_equal(arr, pyarrow.scalar(start_date)) # type: ignore[attr-defined]
if end_date is not None:
m2 = pc.less_equal(arr, pyarrow.scalar(end_date)) # type: ignore[attr-defined]
mask = m2 if mask is None else pc.and_(mask, m2) # type: ignore[attr-defined]
return table.filter(mask)
def _dask_compute_numeric_metrics(
column: pyarrow.ChunkedArray, histogram_bins: int
) -> Dict[str, Any]:
total = len(column)
null_count = column.null_count
result: Dict[str, Any] = {
"feature_type": "numeric",
"row_count": total,
"null_count": null_count,
"null_rate": null_count / total if total > 0 else 0.0,
"mean": None,
"stddev": None,
"min_val": None,
"max_val": None,
"p50": None,
"p75": None,
"p90": None,
"p95": None,
"p99": None,
"histogram": None,
}
valid = pc.drop_null(column) # type: ignore[attr-defined]
if len(valid) == 0:
return result
float_array = pc.cast(valid, pyarrow.float64())
result["mean"] = opt_float(pc.mean(float_array).as_py()) # type: ignore[attr-defined]
result["stddev"] = opt_float(pc.stddev(float_array, ddof=1).as_py()) # type: ignore[attr-defined]
min_max = pc.min_max(float_array) # type: ignore[attr-defined]
result["min_val"] = min_max["min"].as_py()
result["max_val"] = min_max["max"].as_py()
quantiles = pc.quantile(float_array, q=[0.50, 0.75, 0.90, 0.95, 0.99]) # type: ignore[attr-defined]
q_values = quantiles.to_pylist()
result["p50"] = q_values[0]
result["p75"] = q_values[1]
result["p90"] = q_values[2]
result["p95"] = q_values[3]
result["p99"] = q_values[4]
np_array = float_array.to_numpy()
counts, bin_edges = np.histogram(np_array, bins=histogram_bins)
result["histogram"] = {
"bins": bin_edges.tolist(),
"counts": counts.tolist(),
"bin_width": float(bin_edges[1] - bin_edges[0]) if len(bin_edges) > 1 else 0,
}
return result
def _dask_compute_categorical_metrics(
column: pyarrow.ChunkedArray, top_n: int
) -> Dict[str, Any]:
total = len(column)
null_count = column.null_count
result: Dict[str, Any] = {
"feature_type": "categorical",
"row_count": total,
"null_count": null_count,
"null_rate": null_count / total if total > 0 else 0.0,
"mean": None,
"stddev": None,
"min_val": None,
"max_val": None,
"p50": None,
"p75": None,
"p90": None,
"p95": None,
"p99": None,
"histogram": None,
}
valid = pc.drop_null(column) # type: ignore[attr-defined]
if len(valid) == 0:
return result
value_counts = pc.value_counts(valid) # type: ignore[attr-defined]
entries = [
{"value": vc["values"].as_py(), "count": vc["counts"].as_py()}
for vc in value_counts
]
entries.sort(key=lambda x: x["count"], reverse=True)
unique_count = len(entries)
top_entries = entries[:top_n]
other_count = sum(e["count"] for e in entries[top_n:])
result["histogram"] = {
"values": top_entries,
"other_count": other_count,
"unique_count": unique_count,
}
return result
def _dask_parquet_upsert(
path: str,
columns: List[str],
pk_cols: List[str],
new_rows: List[Dict[str, Any]],
) -> None:
os.makedirs(os.path.dirname(path), exist_ok=True)
prepared: List[Dict[str, Any]] = []
for row in new_rows:
r = dict(row)
if (
"histogram" in r
and r["histogram"] is not None
and not isinstance(r["histogram"], str)
):
r["histogram"] = json.dumps(r["histogram"])
prepared.append(r)
new_df = pd.DataFrame(prepared, columns=columns)
existing = _dask_read_parquet_if_exists(path)
if existing is not None:
old_df = existing.to_pandas()
combined = pd.concat([old_df, new_df], ignore_index=True)
else:
combined = new_df
combined = combined.drop_duplicates(subset=pk_cols, keep="last")
table = pyarrow.Table.from_pandas(combined, preserve_index=False)
pq.write_table(table, path)
def _dask_parquet_query(
path: str,
columns: List[str],
project: str,
filters: Optional[Dict[str, Any]],
start_date: Optional[date],
end_date: Optional[date],
) -> List[Dict[str, Any]]:
tab = _dask_read_parquet_if_exists(path)
if tab is None or tab.num_rows == 0:
return []
df = tab.to_pandas()
if project:
df = df[df["project_id"] == project]
if filters:
for key, value in filters.items():
if value is not None:
df = df[df[key] == value]
if "metric_date" in df.columns:
if start_date is not None:
df = df[df["metric_date"] >= start_date]
if end_date is not None:
df = df[df["metric_date"] <= end_date]
df = df.sort_values("metric_date", ascending=True)
else:
df = df.sort_values("job_id", ascending=True)
results = []
for _, row in df.iterrows():
record = {c: row.get(c) for c in columns}
normalize_monitoring_row(record)
for key in ("metric_date", "computed_at"):
val = record.get(key)
if (
val is not None
and not isinstance(val, str)
and hasattr(val, "isoformat")
):
record[key] = val.isoformat()
results.append(record)
return results
def _get_entity_df_event_timestamp_range(
entity_df: Union[pd.DataFrame, str],
entity_df_event_timestamp_col: str,
) -> Tuple[datetime, datetime]:
if not isinstance(entity_df, pd.DataFrame):
raise ValueError(
f"Please provide an entity_df of type {type(pd.DataFrame)} instead of type {type(entity_df)}"
)
entity_df_event_timestamp = entity_df.loc[
:, entity_df_event_timestamp_col
].infer_objects()
if pd.api.types.is_string_dtype(entity_df_event_timestamp):
entity_df_event_timestamp = pd.to_datetime(entity_df_event_timestamp, utc=True)
return (
entity_df_event_timestamp.min().to_pydatetime(),
entity_df_event_timestamp.max().to_pydatetime(),
)
def _read_datasource(data_source, repo_path) -> dd.DataFrame:
storage_options = (
{
"client_kwargs": {
"endpoint_url": data_source.file_options.s3_endpoint_override
}
}
if data_source.file_options.s3_endpoint_override
else None
)
path = FileSource.get_uri_for_file_path(
repo_path=repo_path,
uri=data_source.file_options.uri,
)
return dd.read_parquet(