-
Notifications
You must be signed in to change notification settings - Fork 1.3k
Expand file tree
/
Copy pathnodes.py
More file actions
1047 lines (906 loc) · 38.9 KB
/
Copy pathnodes.py
File metadata and controls
1047 lines (906 loc) · 38.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 logging
from datetime import datetime, timedelta, timezone
from typing import Any, Dict, List, Optional, Set, Union
import dill
import pandas as pd
import pyarrow as pa
import ray
from ray.data import Dataset
from feast import BatchFeatureView, FeatureView, StreamFeatureView
from feast.aggregation import Aggregation
from feast.data_source import DataSource
from feast.feature_view_utils import get_transformation_function, has_transformation
from feast.infra.common.serde import SerializedArtifacts
from feast.infra.compute_engines.dag.context import ExecutionContext
from feast.infra.compute_engines.dag.model import DAGFormat
from feast.infra.compute_engines.dag.node import DAGNode
from feast.infra.compute_engines.dag.value import DAGValue
from feast.infra.compute_engines.ray.config import RayComputeEngineConfig
from feast.infra.compute_engines.ray.utils import (
safe_batch_processor,
write_to_online_store,
)
from feast.infra.compute_engines.utils import (
ENTITY_TS_ALIAS,
create_offline_store_retrieval_job,
)
from feast.infra.ray_initializer import get_ray_wrapper
from feast.infra.ray_shared_utils import (
apply_field_mapping,
broadcast_join,
distributed_windowed_join,
)
logger = logging.getLogger(__name__)
class RayReadNode(DAGNode):
"""
Ray node for reading data from offline stores.
"""
def __init__(
self,
name: str,
source: DataSource,
column_info,
config: RayComputeEngineConfig,
start_time: Optional[datetime] = None,
end_time: Optional[datetime] = None,
):
super().__init__(name)
self.source = source
self.column_info = column_info
self.config = config
self.start_time = start_time
self.end_time = end_time
def execute(self, context: ExecutionContext) -> DAGValue:
"""Execute the read operation to load data from the offline store."""
try:
retrieval_job = create_offline_store_retrieval_job(
data_source=self.source,
column_info=self.column_info,
context=context,
start_time=self.start_time,
end_time=self.end_time,
)
if hasattr(retrieval_job, "to_ray_dataset"):
ray_dataset = retrieval_job.to_ray_dataset()
else:
try:
arrow_table = retrieval_job.to_arrow()
ray_wrapper = get_ray_wrapper()
ray_dataset = ray_wrapper.from_arrow(arrow_table)
except Exception:
df = retrieval_job.to_df()
ray_wrapper = get_ray_wrapper()
ray_dataset = ray_wrapper.from_pandas(df)
field_mapping = getattr(self.source, "field_mapping", None)
if field_mapping:
ray_dataset = apply_field_mapping(ray_dataset, field_mapping)
return DAGValue(
data=ray_dataset,
format=DAGFormat.RAY,
metadata={
"source": "offline_store",
"source_type": type(self.source).__name__,
"start_time": self.start_time,
"end_time": self.end_time,
},
)
except Exception as e:
logger.error(f"Ray read node failed: {e}")
raise
class RayJoinNode(DAGNode):
"""
Ray node for joining entity dataframes with feature data.
"""
def __init__(
self,
name: str,
column_info,
config: RayComputeEngineConfig,
is_historical_retrieval: bool = False,
):
super().__init__(name)
self.column_info = column_info
self.config = config
self.is_historical_retrieval = is_historical_retrieval
def execute(self, context: ExecutionContext) -> DAGValue:
"""Execute the join operation."""
input_value = self.get_single_input_value(context)
input_value.assert_format(DAGFormat.RAY)
feature_dataset: Dataset = input_value.data
# If this is not a historical retrieval, just return the feature data
if not self.is_historical_retrieval or context.entity_df is None:
return DAGValue(
data=feature_dataset,
format=DAGFormat.RAY,
metadata={"joined": False},
)
entity_df = context.entity_df
if isinstance(entity_df, pd.DataFrame):
ray_wrapper = get_ray_wrapper()
entity_dataset = ray_wrapper.from_pandas(entity_df)
else:
entity_dataset = entity_df
join_keys = self.column_info.join_keys
timestamp_col = self.column_info.timestamp_column
requested_feats = getattr(self.column_info, "feature_cols", [])
# Check if the feature dataset contains aggregated features (from aggregation node)
# If so, we don't need point-in-time join logic - just simple join on entity keys
is_aggregated = (
input_value.metadata.get("aggregated", False)
if input_value.metadata
else False
)
feature_size = feature_dataset.size_bytes()
if is_aggregated:
# For aggregated features, do simple join on entity keys
feature_df = feature_dataset.to_pandas()
feature_ref = ray.put(feature_df)
@safe_batch_processor
def join_with_aggregated_features(batch: pd.DataFrame) -> pd.DataFrame:
features = ray.get(feature_ref)
if join_keys:
result = pd.merge(
batch,
features,
on=join_keys,
how="left",
suffixes=("", "_feature"),
)
else:
result = batch.copy()
return result
joined_dataset = entity_dataset.map_batches(
join_with_aggregated_features,
batch_format="pandas",
concurrency=self.config.max_workers or 12,
)
else:
if feature_size <= self.config.broadcast_join_threshold_mb * 1024 * 1024:
# Use broadcast join for small feature datasets
joined_dataset = broadcast_join(
entity_dataset,
feature_dataset.to_pandas(),
join_keys,
timestamp_col,
requested_feats,
)
else:
# Use distributed join for large datasets
joined_dataset = distributed_windowed_join(
entity_dataset,
feature_dataset,
join_keys,
timestamp_col,
requested_feats,
)
return DAGValue(
data=joined_dataset,
format=DAGFormat.RAY,
metadata={
"joined": True,
"join_keys": join_keys,
"join_strategy": "broadcast"
if feature_size <= self.config.broadcast_join_threshold_mb * 1024 * 1024
else "distributed",
},
)
class RayFilterNode(DAGNode):
"""
Ray node for filtering data based on TTL and custom conditions.
"""
def __init__(
self,
name: str,
column_info,
config: RayComputeEngineConfig,
ttl: Optional[timedelta] = None,
filter_condition: Optional[str] = None,
):
super().__init__(name)
self.column_info = column_info
self.config = config
self.ttl = ttl
self.filter_condition = filter_condition
def execute(self, context: ExecutionContext) -> DAGValue:
"""Execute the filter operation."""
input_value = self.get_single_input_value(context)
input_value.assert_format(DAGFormat.RAY)
dataset: Dataset = input_value.data
@safe_batch_processor
def apply_filters(batch: pd.DataFrame) -> pd.DataFrame:
"""Apply TTL and custom filters to the batch."""
filtered_batch = batch.copy()
# Apply TTL filter if specified
if self.ttl:
timestamp_col = self.column_info.timestamp_column
if timestamp_col in filtered_batch.columns:
# Convert to datetime if not already
if not pd.api.types.is_datetime64_any_dtype(
filtered_batch[timestamp_col]
):
filtered_batch[timestamp_col] = pd.to_datetime(
filtered_batch[timestamp_col]
)
# For historical retrieval, use entity timestamp for TTL calculation
if ENTITY_TS_ALIAS in filtered_batch.columns:
# Use entity timestamp for TTL calculation (historical retrieval)
if not pd.api.types.is_datetime64_any_dtype(
filtered_batch[ENTITY_TS_ALIAS]
):
filtered_batch[ENTITY_TS_ALIAS] = pd.to_datetime(
filtered_batch[ENTITY_TS_ALIAS]
)
# Apply TTL filter with both upper and lower bounds:
# 1. feature.ts <= entity.event_timestamp (upper bound)
# 2. feature.ts >= entity.event_timestamp - ttl (lower bound)
upper_bound = filtered_batch[ENTITY_TS_ALIAS]
lower_bound = filtered_batch[ENTITY_TS_ALIAS] - self.ttl
filtered_batch = filtered_batch[
(filtered_batch[timestamp_col] <= upper_bound)
& (filtered_batch[timestamp_col] >= lower_bound)
]
else:
# Use current time for TTL calculation (real-time retrieval)
# Check if timestamp column is timezone-aware
if isinstance(
filtered_batch[timestamp_col].dtype, pd.DatetimeTZDtype
):
# Use timezone-aware current time
current_time = datetime.now(timezone.utc)
else:
# Use naive datetime
current_time = datetime.now()
ttl_threshold = current_time - self.ttl
# Apply TTL filter
filtered_batch = filtered_batch[
filtered_batch[timestamp_col] >= ttl_threshold
]
# Apply custom filter condition if specified
if self.filter_condition:
try:
filtered_batch = filtered_batch.query(self.filter_condition)
except Exception as e:
logger.warning(f"Custom filter failed: {e}")
return filtered_batch
filtered_dataset = dataset.map_batches(apply_filters, batch_format="pandas")
return DAGValue(
data=filtered_dataset,
format=DAGFormat.RAY,
metadata={
"filtered": True,
"ttl": self.ttl,
"filter_condition": self.filter_condition,
},
)
class RayAggregationNode(DAGNode):
"""
Ray node for performing aggregations on feature data.
"""
def __init__(
self,
name: str,
aggregations: List[Aggregation],
group_by_keys: List[str],
timestamp_col: str,
config: RayComputeEngineConfig,
enable_tiling: bool = False,
hop_size: Optional[timedelta] = None,
):
super().__init__(name)
self.aggregations = aggregations
self.group_by_keys = group_by_keys
self.timestamp_col = timestamp_col
self.config = config
self.enable_tiling = enable_tiling
self.hop_size = hop_size
def execute(self, context: ExecutionContext) -> DAGValue:
"""Execute the aggregation operation."""
input_value = self.get_single_input_value(context)
input_value.assert_format(DAGFormat.RAY)
dataset: Dataset = input_value.data
# Check if tiling should be used
has_time_windows = any(agg.time_window for agg in self.aggregations)
if self.enable_tiling and has_time_windows:
return self._execute_tiled_aggregation(dataset)
else:
return self._execute_standard_aggregation(dataset)
def _execute_tiled_aggregation(self, dataset: Dataset) -> DAGValue:
"""
Execute tiled aggregation.
Flow:
1. Convert Ray Dataset → pandas
2. Generate cumulative tiles
3. Convert to windowed aggregations
4. Convert pandas → Ray Dataset
"""
from feast.aggregation.tiling.orchestrator import apply_sawtooth_window_tiling
from feast.aggregation.tiling.tile_subtraction import (
convert_cumulative_to_windowed,
deduplicate_keep_latest,
)
ray_wrapper = get_ray_wrapper()
input_pdf = dataset.to_pandas()
for agg in self.aggregations:
if agg.time_window is None:
raise ValueError(
f"Tiling is enabled but aggregation on column '{agg.column}' has no time_window set. "
f"Either set time_window for all aggregations or disable tiling by setting enable_tiling=False."
)
# Group aggregations by time window
window_to_aggs: Dict[timedelta, List[Aggregation]] = {}
for agg in self.aggregations:
if agg.time_window:
if agg.time_window not in window_to_aggs:
window_to_aggs[agg.time_window] = []
window_to_aggs[agg.time_window].append(agg)
# Process each time window in pandas
windowed_pdfs = []
for window_size, window_aggs in window_to_aggs.items():
# Step 1: Generate cumulative tiles
tiles_pdf = apply_sawtooth_window_tiling(
df=input_pdf,
aggregations=window_aggs,
group_by_keys=self.group_by_keys,
timestamp_col=self.timestamp_col,
window_size=window_size,
hop_size=self.hop_size or timedelta(minutes=5),
)
if tiles_pdf.empty:
continue
# Step 2: Convert to windowed aggregations
windowed_pdf = convert_cumulative_to_windowed(
tiles_df=tiles_pdf,
entity_keys=self.group_by_keys,
timestamp_col=self.timestamp_col,
window_size=window_size,
aggregations=window_aggs,
)
if not windowed_pdf.empty:
windowed_pdfs.append(windowed_pdf)
if not windowed_pdfs:
# No results, return empty Ray Dataset
aggregated_dataset = ray_wrapper.from_pandas(pd.DataFrame())
else:
# Step 3: Join all windows in pandas (outer merge on entity keys + timestamp)
if len(windowed_pdfs) == 1:
final_pdf = windowed_pdfs[0]
else:
final_pdf = windowed_pdfs[0]
join_keys = self.group_by_keys + [self.timestamp_col]
for pdf in windowed_pdfs[1:]:
final_pdf = pd.merge(
final_pdf,
pdf,
on=join_keys,
how="outer",
suffixes=("", "_dup"),
)
# Drop duplicate columns from merge
final_pdf = final_pdf.loc[
:, ~final_pdf.columns.str.endswith("_dup")
]
# Step 4: Deduplicate in pandas (keep latest timestamp per entity)
if self.timestamp_col in final_pdf.columns and not final_pdf.empty:
final_pdf = deduplicate_keep_latest(
final_pdf, self.group_by_keys, self.timestamp_col
)
aggregated_dataset = ray_wrapper.from_pandas(final_pdf)
return DAGValue(
data=aggregated_dataset,
format=DAGFormat.RAY,
metadata={
"aggregated": True,
"aggregations": len(self.aggregations),
"group_by_keys": self.group_by_keys,
"tiled": True,
},
)
def _execute_standard_aggregation(self, dataset: Dataset) -> DAGValue:
"""Execute standard aggregation without tiling."""
# Convert aggregations to Ray's groupby format
agg_dict = {}
for agg in self.aggregations:
feature_name = agg.resolved_name(agg.time_window)
if agg.function == "count":
agg_dict[feature_name] = (agg.column, "count")
elif agg.function == "sum":
agg_dict[feature_name] = (agg.column, "sum")
elif agg.function == "mean" or agg.function == "avg":
agg_dict[feature_name] = (agg.column, "mean")
elif agg.function == "min":
agg_dict[feature_name] = (agg.column, "min")
elif agg.function == "max":
agg_dict[feature_name] = (agg.column, "max")
elif agg.function == "std":
agg_dict[feature_name] = (agg.column, "std")
elif agg.function == "var":
agg_dict[feature_name] = (agg.column, "var")
elif agg.function == "count_distinct":
agg_dict[feature_name] = (agg.column, "nunique")
else:
raise ValueError(f"Unknown aggregation function: {agg.function}.")
# Apply aggregations using pandas fallback (Ray's native groupby has compatibility issues)
if self.group_by_keys and agg_dict:
# Use pandas-based aggregation for entire dataset
aggregated_dataset = self._fallback_pandas_aggregation(dataset, agg_dict)
else:
# No group keys or aggregations, return original dataset
aggregated_dataset = dataset
return DAGValue(
data=aggregated_dataset,
format=DAGFormat.RAY,
metadata={
"aggregated": True,
"aggregations": len(self.aggregations),
"group_by_keys": self.group_by_keys,
},
)
def _fallback_pandas_aggregation(self, dataset: Dataset, agg_dict: dict) -> Dataset:
"""Fallback to pandas-based aggregation for the entire dataset."""
# Convert entire dataset to pandas for aggregation
df = dataset.to_pandas()
if df.empty:
return dataset
# Group by the specified keys
if self.group_by_keys:
grouped = df.groupby(self.group_by_keys)
else:
# If no group keys, apply aggregations to entire dataset
grouped = df.groupby(lambda x: 0) # Dummy grouping
# Apply each aggregation
agg_results = []
for feature_name, (column, function) in agg_dict.items():
if column in df.columns:
if function == "count":
result = grouped[column].count()
elif function == "sum":
result = grouped[column].sum()
elif function == "mean":
result = grouped[column].mean()
elif function == "min":
result = grouped[column].min()
elif function == "max":
result = grouped[column].max()
elif function == "std":
result = grouped[column].std()
elif function == "var":
result = grouped[column].var()
elif function == "nunique":
result = grouped[column].nunique()
else:
raise ValueError(f"Unknown aggregation function: {function}.")
result.name = feature_name
agg_results.append(result)
# Combine aggregation results
if agg_results:
result_df = pd.concat(agg_results, axis=1)
# Reset index to make group keys regular columns
if self.group_by_keys:
result_df = result_df.reset_index()
# Convert back to Ray Dataset
ray_wrapper = get_ray_wrapper()
return ray_wrapper.from_pandas(result_df)
else:
return dataset
class RayDedupNode(DAGNode):
"""
Ray node for deduplicating records.
Two dedup strategies are provided:
* **Materialization** (``is_materialization=True``): per-block
``drop_duplicates``. This is streaming-friendly because it never needs
to see all blocks at once. Any cross-block duplicates are resolved by
the online store, which does an UPSERT and therefore naturally keeps the
last-written value. This avoids the ``groupby().map_groups()`` full
shuffle that would otherwise block until every single block was produced.
* **Historical retrieval** (``is_materialization=False``): global
``groupby().map_groups()``. Correctness is required here because the
entity-timestamp join must return exactly one feature row per
(entity, query-timestamp) pair.
"""
def __init__(
self,
name: str,
column_info,
config: RayComputeEngineConfig,
is_materialization: bool = False,
):
super().__init__(name)
self.column_info = column_info
self.config = config
self.is_materialization = is_materialization
def execute(self, context: ExecutionContext) -> DAGValue:
"""Execute the deduplication operation."""
input_value = self.get_single_input_value(context)
input_value.assert_format(DAGFormat.RAY)
dataset: Dataset = input_value.data
join_keys = self.column_info.join_keys
timestamp_col = self.column_info.timestamp_column
if join_keys:
if self.is_materialization:
# Per-block dedup: streaming-safe, no full shuffle required.
# Cross-block duplicates are handled by the online-store UPSERT.
#
# IMPORTANT: do NOT call dataset.schema() here. For streaming
# datasets backed by slow map_batches actors, .schema() triggers
# eager block execution to
# infer the output type. Those blocks are consumed and LOST —
# they never reach the write stage. We therefore defer the
# column-existence check to inside _dedup_block, which runs in
# a worker per block without interfering with streaming.
_join_keys = list(join_keys)
_ts_col = timestamp_col
def _dedup_block(block: pd.DataFrame) -> pd.DataFrame:
available = [k for k in _join_keys if k in block.columns]
if not available:
return block
if _ts_col and _ts_col in block.columns:
block = block.sort_values(_ts_col, ascending=False)
return block.drop_duplicates(subset=available)
dataset = dataset.map_batches(_dedup_block, batch_format="pandas")
else:
# Global dedup via groupby: required for historical retrieval
# where the entity–timestamp join must return exactly one row
# per (entity, query-timestamp) pair.
# NOTE: groupby().map_groups() is a full shuffle and blocks
# until ALL upstream blocks are produced. Use only when
# correctness across partition boundaries is mandatory.
available_join_keys = [
k for k in join_keys if k in dataset.schema().names
]
available_ts_col = (
timestamp_col if timestamp_col in dataset.schema().names else None
)
if available_join_keys:
def _keep_latest_in_group(group: pd.DataFrame) -> pd.DataFrame:
if available_ts_col and available_ts_col in group.columns:
group = group.sort_values(available_ts_col, ascending=False)
return group.head(1)
dataset = dataset.groupby(available_join_keys).map_groups(
_keep_latest_in_group, batch_format="pandas"
)
deduped_dataset = dataset
return DAGValue(
data=deduped_dataset,
format=DAGFormat.RAY,
metadata={"deduped": True},
)
class RayTransformationNode(DAGNode):
"""
Ray node for applying feature transformations.
"""
def __init__(
self,
name: str,
transformation,
config: RayComputeEngineConfig,
):
super().__init__(name)
self.transformation = transformation
self.transformation_name = getattr(transformation, "name", "unknown")
self.config = config
def execute(self, context: ExecutionContext) -> DAGValue:
"""Execute the transformation operation."""
input_value = self.get_single_input_value(context)
input_value.assert_format(DAGFormat.RAY)
dataset: Dataset = input_value.data
# Check transformation mode
from feast.transformation.mode import TransformationMode
transformation_mode = getattr(
self.transformation, "mode", TransformationMode.PYTHON
)
is_ray_native = transformation_mode in (TransformationMode.RAY, "ray")
if is_ray_native:
transformation_func = None
if hasattr(self.transformation, "udf") and callable(
self.transformation.udf
):
transformation_func = self.transformation.udf
elif callable(self.transformation):
transformation_func = self.transformation
if transformation_func:
transformed_dataset = transformation_func(dataset)
else:
logger.warning(
"No transformation function available in RAY mode, returning original dataset"
)
transformed_dataset = dataset
else:
transformation_serialized = None
if hasattr(self.transformation, "udf") and callable(
self.transformation.udf
):
transformation_serialized = dill.dumps(self.transformation.udf)
elif callable(self.transformation):
transformation_serialized = dill.dumps(self.transformation)
@safe_batch_processor
def apply_transformation_with_serialized_udf(
batch: pd.DataFrame,
) -> pd.DataFrame:
"""Apply the transformation using pre-serialized UDF."""
if transformation_serialized:
transformation_func = dill.loads(transformation_serialized)
transformed_batch = transformation_func(batch)
else:
logger.warning(
"No serialized transformation available, returning original batch"
)
transformed_batch = batch
return transformed_batch
num_gpus = getattr(self.config, "num_gpus", None) or None
task_options: Dict[str, Any] = dict(
getattr(self.config, "worker_task_options", None) or {}
)
if num_gpus:
task_options["num_gpus"] = num_gpus
batch_format = (
getattr(self.config, "gpu_batch_format", "pandas")
if num_gpus
else "pandas"
)
# Only the scheduling-relevant subset flows into map_batches
_MAP_BATCHES_RESOURCE_KEYS = {
"num_gpus",
"num_cpus",
"accelerator_type",
"resources",
}
map_kwargs: Dict[str, Any] = {
"batch_format": batch_format,
"concurrency": self.config.max_workers or 12,
**{
k: v
for k, v in task_options.items()
if k in _MAP_BATCHES_RESOURCE_KEYS
},
}
transformed_dataset = dataset.map_batches(
apply_transformation_with_serialized_udf,
**map_kwargs,
)
return DAGValue(
data=transformed_dataset,
format=DAGFormat.RAY,
metadata={
"transformed": True,
"transformation": self.transformation_name,
},
)
class RayDerivedReadNode(DAGNode):
"""
Ray node for reading derived feature views after parent dependencies are materialized.
This node ensures that parent feature views are fully materialized before reading from their sink_source.
"""
def __init__(
self,
name: str,
feature_view: FeatureView,
parent_dependencies: List[DAGNode],
config: RayComputeEngineConfig,
column_info,
is_materialization: bool = True,
):
super().__init__(name)
self.feature_view = feature_view
self.config = config
self.column_info = column_info
self.is_materialization = is_materialization
# Add parent dependencies to ensure they execute first
for parent in parent_dependencies:
self.add_input(parent)
def execute(self, context: ExecutionContext) -> DAGValue:
"""Execute the derived read operation after parents are materialized."""
parent_values = self.get_input_values(context)
if not parent_values:
raise ValueError(
f"No parent data available for derived view {self.feature_view.name}"
)
parent_value = parent_values[0]
parent_value.assert_format(DAGFormat.RAY)
if has_transformation(self.feature_view):
transformation_func = get_transformation_function(self.feature_view)
if callable(transformation_func):
def apply_transformation(batch: pd.DataFrame) -> pd.DataFrame:
return transformation_func(batch)
transformed_dataset = parent_value.data.map_batches(
apply_transformation,
batch_format="pandas",
concurrency=self.config.max_workers or 12,
)
return DAGValue(
data=transformed_dataset,
format=DAGFormat.RAY,
metadata={
"source": "derived_from_parent",
"source_description": f"Transformed data from parent for {self.feature_view.name}",
},
)
return DAGValue(
data=parent_value.data,
format=DAGFormat.RAY,
metadata={
"source": "derived_from_parent",
"source_description": f"Data from parent for {self.feature_view.name}",
},
)
class RayWriteNode(DAGNode):
"""
Ray node for writing results to online/offline stores and sink_source paths.
This node handles writing intermediate results for derived feature views.
"""
def __init__(
self,
name: str,
feature_view: Union[BatchFeatureView, StreamFeatureView, FeatureView],
inputs=None,
config: Optional[RayComputeEngineConfig] = None,
):
super().__init__(name, inputs=inputs)
self.feature_view = feature_view
self.config = config
def execute(self, context: ExecutionContext) -> DAGValue:
"""Execute the write operation."""
input_value = self.get_single_input_value(context)
input_value.assert_format(DAGFormat.RAY)
dataset: Dataset = input_value.data
serialized_artifacts = SerializedArtifacts.serialize(
feature_view=self.feature_view, repo_config=context.repo_config
)
@safe_batch_processor
def write_batch_with_serialized_artifacts(batch: pd.DataFrame) -> pd.DataFrame:
"""Write each batch using pre-serialized artifacts."""
(
feature_view,
online_store,
offline_store,
repo_config,
) = serialized_artifacts.unserialize()
arrow_table = pa.Table.from_pandas(batch)
# Write to online store if enabled
write_to_online_store(
arrow_table=arrow_table,
feature_view=feature_view,
online_store=online_store,
repo_config=repo_config,
)
# Write to offline store if enabled
if getattr(feature_view, "offline", False):
offline_store.offline_write_batch(
config=repo_config,
feature_view=feature_view,
table=arrow_table,
progress=lambda x: None,
)
return batch
# Resolve write concurrency from config.
# write_concurrency takes precedence; falls back to max_workers, then 1.
if self.config is not None and self.config.write_concurrency is not None:
_write_concurrency = self.config.write_concurrency
elif self.config is not None and self.config.max_workers is not None:
_write_concurrency = self.config.max_workers
else:
_write_concurrency = 1
written_dataset = dataset.map_batches(
write_batch_with_serialized_artifacts,
batch_format="pandas",
concurrency=_write_concurrency,
)
written_dataset = written_dataset.materialize()
return DAGValue(
data=written_dataset,
format=DAGFormat.RAY,
metadata={
"written": True,
"feature_view": self.feature_view.name,
"online": getattr(self.feature_view, "online", False),
"offline": getattr(self.feature_view, "offline", False),
"batch_source_path": getattr(
getattr(self.feature_view, "batch_source", None), "path", "unknown"
),
},
)
class RayValidationNode(DAGNode):
"""
Ray node for validating feature data against the declared schema.
Checks that all expected columns are present and logs warnings for
type mismatches. Validation runs once on the first batch to avoid
per-batch overhead; the full dataset is passed through unchanged.
"""
def __init__(
self,
name: str,
expected_columns: Dict[str, Optional[pa.DataType]],
json_columns: Optional[Set[str]] = None,
inputs: Optional[List[DAGNode]] = None,
):
super().__init__(name, inputs=inputs)
self.expected_columns = expected_columns
self.json_columns = json_columns or set()
def execute(self, context: ExecutionContext) -> DAGValue:
input_value = self.get_single_input_value(context)
dataset = input_value.data
if not self.expected_columns:
context.node_outputs[self.name] = input_value
return input_value
expected_names = set(self.expected_columns.keys())
schema = dataset.schema()
actual_columns = set(schema.names)
missing = expected_names - actual_columns
if missing:
raise ValueError(
f"[Validation: {self.name}] Missing expected columns: {missing}. "
f"Actual columns: {sorted(actual_columns)}"
)
for col_name, expected_type in self.expected_columns.items():
if expected_type is None:
continue
actual_field = schema.field(col_name)
actual_type = actual_field.type
if actual_type != expected_type:
# Map type compatibility
if pa.types.is_map(expected_type) and (
pa.types.is_map(actual_type)
or pa.types.is_struct(actual_type)
or pa.types.is_list(actual_type)
):
continue
# JSON type compatibility (large_string / string)
if pa.types.is_large_string(expected_type) and (
pa.types.is_string(actual_type)
or pa.types.is_large_string(actual_type)
):
continue
# Struct type compatibility
if pa.types.is_struct(expected_type) and (
pa.types.is_struct(actual_type)
or pa.types.is_map(actual_type)
or pa.types.is_list(actual_type)
):
continue
logger.warning(
"[Validation: %s] Column '%s' type mismatch: expected %s, got %s",
self.name,
col_name,
expected_type,
actual_type,