-
Notifications
You must be signed in to change notification settings - Fork 1.2k
feat: Compute Engine Initial Implementation #5223
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
1183a9a
2825ee4
4210f68
a398075
68c48dc
1c9ae31
25af94e
ed0cdf4
6b57e94
227f8f4
e9362de
68cf242
3a5cf92
c1ba3d6
95f757d
e5445a2
263024b
2433064
87e51c7
c698b64
0877c03
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,119 @@ | ||
| # 🧠 ComputeEngine (WIP) | ||
|
|
||
| The `ComputeEngine` is Feast’s pluggable abstraction for executing feature pipelines — including transformations, aggregations, joins, and materializations/get_historical_features — on a backend of your choice (e.g., Spark, PyArrow, Pandas, Ray). | ||
|
|
||
| It powers both: | ||
|
|
||
| - `materialize()` – for batch and stream generation of features to offline/online stores | ||
| - `get_historical_features()` – for point-in-time correct training dataset retrieval | ||
|
|
||
| This system builds and executes DAGs (Directed Acyclic Graphs) of typed operations, enabling modular and scalable workflows. | ||
|
|
||
| --- | ||
|
|
||
| ## 🧠 Core Concepts | ||
|
|
||
| | Component | Description | | ||
| |--------------------|----------------------------------------------------------------------| | ||
| | `ComputeEngine` | Interface for executing materialization and retrieval tasks | | ||
| | `FeatureBuilder` | Constructs a DAG from Feature View definition for a specific backend | | ||
| | `DAGNode` | Represents a logical operation (read, aggregate, join, etc.) | | ||
| | `ExecutionPlan` | Executes nodes in dependency order and stores intermediate outputs | | ||
| | `ExecutionContext` | Holds config, registry, stores, entity data, and node outputs | | ||
|
|
||
| --- | ||
|
|
||
| ## ✨ Available Engines | ||
|
|
||
| ### 🔥 SparkComputeEngine | ||
|
|
||
| - Distributed DAG execution via Apache Spark | ||
| - Supports point-in-time joins and large-scale materialization | ||
| - Integrates with `SparkOfflineStore` and `SparkMaterializationJob` | ||
|
|
||
| ### 🧪 LocalComputeEngine (WIP) | ||
|
|
||
| - Runs on Arrow + Pandas (or optionally DuckDB) | ||
| - Designed for local dev, testing, or lightweight feature generation | ||
|
|
||
| --- | ||
|
|
||
| ## 🛠️ Feature Builder Flow | ||
| ```markdown | ||
| SourceReadNode | ||
| | | ||
| v | ||
| JoinNode (Only for get_historical_features with entity df) | ||
| | | ||
| v | ||
| FilterNode (Always included; applies TTL or user-defined filters) | ||
| | | ||
| v | ||
| AggregationNode (If aggregations are defined in FeatureView) | ||
| | | ||
| v | ||
| DeduplicationNode (If no aggregation is defined for get_historical_features) | ||
| | | ||
| v | ||
| TransformationNode (If feature_transformation is defined) | ||
| | | ||
| v | ||
| ValidationNode (If enable_validation = True) | ||
| | | ||
| v | ||
| Output | ||
| ├──> RetrievalOutput (For get_historical_features) | ||
| └──> OnlineStoreWrite / OfflineStoreWrite (For materialize) | ||
| ``` | ||
|
|
||
| Each step is implemented as a `DAGNode`. An `ExecutionPlan` executes these nodes in topological order, caching `DAGValue` outputs. | ||
|
|
||
| --- | ||
|
|
||
| ## 🧩 Implementing a Custom Compute Engine | ||
|
|
||
| To create your own compute engine: | ||
|
|
||
| 1. **Implement the interface** | ||
|
|
||
| ```python | ||
| from feast.infra.compute_engines.base import ComputeEngine | ||
| from feast.infra.materialization.batch_materialization_engine import MaterializationTask, MaterializationJob | ||
| from feast.infra.compute_engines.tasks import HistoricalRetrievalTask | ||
| class MyComputeEngine(ComputeEngine): | ||
| def materialize(self, task: MaterializationTask) -> MaterializationJob: | ||
| ... | ||
|
|
||
| def get_historical_features(self, task: HistoricalRetrievalTask) -> RetrievalJob: | ||
| ... | ||
| ``` | ||
|
|
||
| 2. Create a FeatureBuilder | ||
| ```python | ||
| from feast.infra.compute_engines.feature_builder import FeatureBuilder | ||
|
|
||
| class CustomFeatureBuilder(FeatureBuilder): | ||
| def build_source_node(self): ... | ||
| def build_aggregation_node(self, input_node): ... | ||
| def build_join_node(self, input_node): ... | ||
| def build_filter_node(self, input_node): | ||
| def build_dedup_node(self, input_node): | ||
| def build_transformation_node(self, input_node): ... | ||
| def build_output_nodes(self, input_node): ... | ||
| ``` | ||
|
|
||
| 3. Define DAGNode subclasses | ||
| * ReadNode, AggregationNode, JoinNode, WriteNode, etc. | ||
| * Each DAGNode.execute(context) -> DAGValue | ||
|
|
||
| 4. Return an ExecutionPlan | ||
| * ExecutionPlan stores DAG nodes in topological order | ||
| * Automatically handles intermediate value caching | ||
|
|
||
| ## 🚧 Roadmap | ||
| - [x] Modular, backend-agnostic DAG execution framework | ||
| - [x] Spark engine with native support for materialization + PIT joins | ||
| - [ ] PyArrow + Pandas engine for local compute | ||
| - [ ] Native multi-feature-view DAG optimization | ||
| - [ ] DAG validation, metrics, and debug output | ||
| - [ ] Scalable distributed backend via Ray or Polars | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -6,6 +6,7 @@ | |
| import dill | ||
|
|
||
| from feast import flags_helper | ||
| from feast.aggregation import Aggregation | ||
| from feast.data_source import DataSource | ||
| from feast.entity import Entity | ||
| from feast.feature_view import FeatureView | ||
|
|
@@ -40,7 +41,8 @@ class BatchFeatureView(FeatureView): | |
| schema: The schema of the feature view, including feature, timestamp, and entity | ||
| columns. If not specified, can be inferred from the underlying data source. | ||
| source: The batch source of data where this group of features is stored. | ||
| online: A boolean indicating whether online retrieval is enabled for this feature view. | ||
| online: A boolean indicating whether online retrieval and write to online store is enabled for this feature view. | ||
| offline: A boolean indicating whether offline retrieval and write to offline store is enabled for this feature view. | ||
| description: A human-readable description. | ||
| tags: A dictionary of key-value pairs to store arbitrary metadata. | ||
| owner: The owner of the batch feature view, typically the email of the primary maintainer. | ||
|
|
@@ -55,6 +57,7 @@ class BatchFeatureView(FeatureView): | |
| entity_columns: List[Field] | ||
| features: List[Field] | ||
| online: bool | ||
| offline: bool | ||
| description: str | ||
| tags: Dict[str, str] | ||
| owner: str | ||
|
|
@@ -63,6 +66,8 @@ class BatchFeatureView(FeatureView): | |
| udf: Optional[Callable[[Any], Any]] | ||
| udf_string: Optional[str] | ||
| feature_transformation: Transformation | ||
| batch_engine: Optional[Field] | ||
| aggregations: Optional[List[Aggregation]] | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If I'm to be honest, I don't love putting the transformation in the FeatureView. I think it'd be more intuitive to put Aggregation under Transformation and make the FeatureViews purely represent schemas available for online or offline.
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree. This is only to be consistent with the current In some way that also make sense but for Feast it will needs much more work to refactor the For Chrono they put it into a GroupBy API, which is similar to our FeatureView: I can look into how to merge |
||
|
|
||
| def __init__( | ||
| self, | ||
|
|
@@ -73,13 +78,16 @@ def __init__( | |
| entities: Optional[List[Entity]] = None, | ||
| ttl: Optional[timedelta] = None, | ||
| tags: Optional[Dict[str, str]] = None, | ||
| online: bool = True, | ||
| online: bool = False, | ||
| offline: bool = True, | ||
| description: str = "", | ||
| owner: str = "", | ||
| schema: Optional[List[Field]] = None, | ||
| udf: Optional[Callable[[Any], Any]], | ||
| udf_string: Optional[str] = "", | ||
| feature_transformation: Optional[Transformation] = None, | ||
| batch_engine: Optional[Field] = None, | ||
| aggregations: Optional[List[Aggregation]] = None, | ||
| ): | ||
| if not flags_helper.is_test(): | ||
| warnings.warn( | ||
|
|
@@ -103,13 +111,16 @@ def __init__( | |
| self.feature_transformation = ( | ||
| feature_transformation or self.get_feature_transformation() | ||
| ) | ||
| self.batch_engine = batch_engine | ||
| self.aggregations = aggregations or [] | ||
|
|
||
| super().__init__( | ||
| name=name, | ||
| entities=entities, | ||
| ttl=ttl, | ||
| tags=tags, | ||
| online=online, | ||
| offline=offline, | ||
| description=description, | ||
| owner=owner, | ||
| schema=schema, | ||
|
|
@@ -144,18 +155,21 @@ def batch_feature_view( | |
| source: Optional[DataSource] = None, | ||
| tags: Optional[Dict[str, str]] = None, | ||
| online: bool = True, | ||
| offline: bool = True, | ||
| description: str = "", | ||
| owner: str = "", | ||
| schema: Optional[List[Field]] = None, | ||
| ): | ||
| """ | ||
| Args: | ||
| name: | ||
| mode: | ||
| entities: | ||
| ttl: | ||
| source: | ||
| tags: | ||
| online: | ||
| offline: | ||
| description: | ||
| owner: | ||
| schema: | ||
|
|
@@ -181,6 +195,7 @@ def decorator(user_function): | |
| source=source, | ||
| tags=tags, | ||
| online=online, | ||
| offline=offline, | ||
| description=description, | ||
| owner=owner, | ||
| schema=schema, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,83 @@ | ||
| from abc import ABC | ||
| from typing import Union | ||
|
|
||
| import pyarrow as pa | ||
|
|
||
| from feast import RepoConfig | ||
| from feast.infra.compute_engines.dag.context import ColumnInfo, ExecutionContext | ||
| from feast.infra.compute_engines.tasks import HistoricalRetrievalTask | ||
| from feast.infra.materialization.batch_materialization_engine import ( | ||
| MaterializationJob, | ||
| MaterializationTask, | ||
| ) | ||
| from feast.infra.offline_stores.offline_store import OfflineStore | ||
| from feast.infra.online_stores.online_store import OnlineStore | ||
| from feast.infra.registry.registry import Registry | ||
| from feast.utils import _get_column_names | ||
|
|
||
|
|
||
| class ComputeEngine(ABC): | ||
| """ | ||
| The interface that Feast uses to control the compute system that handles materialization and get_historical_features. | ||
| Each engine must implement: | ||
| - materialize(): to generate and persist features | ||
| - get_historical_features(): to perform point-in-time correct joins | ||
| Engines should use FeatureBuilder and DAGNode abstractions to build modular, pluggable workflows. | ||
| """ | ||
|
|
||
| def __init__( | ||
| self, | ||
| *, | ||
| registry: Registry, | ||
| repo_config: RepoConfig, | ||
| offline_store: OfflineStore, | ||
| online_store: OnlineStore, | ||
| **kwargs, | ||
| ): | ||
| self.registry = registry | ||
| self.repo_config = repo_config | ||
| self.offline_store = offline_store | ||
| self.online_store = online_store | ||
|
|
||
| def materialize(self, task: MaterializationTask) -> MaterializationJob: | ||
| raise NotImplementedError | ||
|
|
||
| def get_historical_features(self, task: HistoricalRetrievalTask) -> pa.Table: | ||
| raise NotImplementedError | ||
|
|
||
| def get_execution_context( | ||
| self, | ||
| task: Union[MaterializationTask, HistoricalRetrievalTask], | ||
| ) -> ExecutionContext: | ||
| entity_defs = [ | ||
| self.registry.get_entity(name, task.project) | ||
| for name in task.feature_view.entities | ||
| ] | ||
| entity_df = None | ||
| if hasattr(task, "entity_df") and task.entity_df is not None: | ||
| entity_df = task.entity_df | ||
|
|
||
| column_info = self.get_column_info(task) | ||
| return ExecutionContext( | ||
| project=task.project, | ||
| repo_config=self.repo_config, | ||
| offline_store=self.offline_store, | ||
| online_store=self.online_store, | ||
| entity_defs=entity_defs, | ||
| column_info=column_info, | ||
| entity_df=entity_df, | ||
| ) | ||
|
|
||
| def get_column_info( | ||
| self, | ||
| task: Union[MaterializationTask, HistoricalRetrievalTask], | ||
| ) -> ColumnInfo: | ||
| join_keys, feature_cols, ts_col, created_ts_col = _get_column_names( | ||
| task.feature_view, self.registry.list_entities(task.project) | ||
| ) | ||
| return ColumnInfo( | ||
| join_keys=join_keys, | ||
| feature_cols=feature_cols, | ||
| ts_col=ts_col, | ||
| created_ts_col=created_ts_col, | ||
| ) |
Uh oh!
There was an error while loading. Please reload this page.