Building a Feature Store from Scratch: Why We Skipped Feast and Built Our Own

Key Takeaways

  • A feature store is not a database — it's a system that solves feature consistency, point-in-time correctness, and the training/serving skew problem that silently kills ML models in production.
  • Off-the-shelf options like Feast, Tecton, and Hopsworks make strong assumptions about your infrastructure. If your data platform doesn't match those assumptions, you'll spend more time fighting the tool than building features.
  • Our architecture uses Apache Iceberg for the offline store, Redis for the online store, and a thin Python feature server that handles both training and serving with the same feature definitions.
  • Point-in-time correctness is the hardest problem to get right, and it's the one most tutorials skip. We'll show exactly how to implement it.
  • Building your own feature store is not always the right call. We'll be honest about when you should just use Feast and move on.

The Problem That Pushed Us to Build

I've been running the ML platform at a mid-size fintech for about three years. We have around 40 models in production, mostly fraud detection, credit scoring, and recommendation systems. When I joined, feature engineering looked like this: every ML engineer had their own collection of SQL queries and Python scripts that computed features, saved them as Parquet files in S3, and loaded them at training time. Serving was a separate pipeline entirely — a different set of scripts that computed the same features in near real-time and pushed them into a Redis cluster.

The problems were exactly what you'd expect. Features drifted between training and serving. Nobody could reproduce a training dataset from six months ago. A single feature change required updating code in three different repositories. And every time a new engineer joined, it took them two weeks just to understand where features came from.

We needed a feature store. That much was obvious. What wasn't obvious was whether we should build one or adopt something off the shelf.

What a Feature Store Actually Does (and Doesn't)

Before diving into our build-vs-buy analysis, let me clarify what a feature store actually needs to do. There's a lot of marketing noise in this space, and it's worth being precise.

A feature store must solve four core problems:

  1. Feature consistency: The same feature definition is used in training and serving. No more "the model was trained with a 30-day rolling average but serving computes a 7-day rolling average because someone copied the wrong query."
  2. Point-in-time correctness: When you build a training dataset, each row gets the feature values that would have been available at the time of the event — not future data that would cause leakage.
  3. Feature discovery and reuse: Engineers can find existing features instead of recomputing them. A feature registry with metadata, lineage, and documentation.
  4. Dual storage: An offline store optimized for batch reads (training) and an online store optimized for low-latency lookups (serving).

What a feature store does not need to do: it doesn't need to orchestrate your pipelines (that's Airflow/Dagster), it doesn't need to train models (that's your ML framework), and it doesn't need to serve predictions (that's your model serving layer). I've seen feature store vendors scope-creep into all of these areas, and it creates more problems than it solves.

Why Feast Didn't Work for Us

We spent three months evaluating Feast seriously. We ran a proof of concept, attended the community calls, read the source code. Feast is a solid project, and for many teams it's the right choice. But it wasn't the right choice for us, for specific technical reasons.

Problem 1: Offline store assumptions. Feast's offline store interface assumes you're using BigQuery, Snowflake, Redshift, or file-based sources. Our data platform is built on Apache Iceberg tables in S3, queried through Spark and Trino. Feast has a contrib Spark offline store, but it was poorly maintained at the time and didn't support Iceberg's time-travel features that we needed for reproducibility.

Problem 2: Materialization latency. Feast's materialization process (moving features from offline to online store) is batch-oriented. For our fraud detection models, we needed feature freshness under 60 seconds. Feast's streaming support was experimental and required adopting their specific Kafka integration, which didn't match our existing Flink-based streaming infrastructure.

Problem 3: Point-in-time join performance. Feast's get_historical_features() pulls entity data into the driver process and performs point-in-time joins there. With our training datasets containing 50M+ rows and 200+ features, this approach was unacceptably slow. We needed pushdown into the compute engine.

We also evaluated Tecton (too expensive for our scale, and we didn't want vendor lock-in on something this critical) and Hopsworks (strong platform but required adopting their entire ecosystem, which conflicted with our existing Kubernetes infrastructure).

Comparison: Feature Store Options in 2026

Criteria Feast Tecton Hopsworks Our Custom Build
Cost Free (OSS) $$$ (managed) Free (OSS) / Managed Engineering time only
Offline Store BQ, Snowflake, Redshift, Files Spark, Snowflake, Rift Hudi on S3 Iceberg on S3 (custom)
Online Store Redis, DynamoDB, Datastore DynamoDB (managed) RonDB / MySQL Redis Cluster
Streaming Features Experimental (Kafka push) Native (Spark/Rift) Native (Flink/Spark) Native (Flink sinks)
Point-in-Time Joins Driver-side (slow at scale) Server-side (fast) Spark-based Spark/Trino pushdown
Feature Freshness Minutes (batch materialization) Seconds Minutes to seconds Seconds (Flink path)
Setup Complexity Low Low (managed) Medium-High High (initial investment)
Maintenance Burden Low None (managed) Medium Medium-High

Our Architecture: The Three-Layer Feature Store

After the evaluation, we decided to build. The architecture has three layers: the feature registry (definitions and metadata), the offline store (Iceberg tables for training), and the online store (Redis for serving). A thin feature server API sits in front of both stores and handles all reads.

The single most important design decision we made: feature definitions are code, stored in a Git repository, versioned and reviewed like any other software artifact. No UI-based feature creation. No YAML-driven magic. Python classes with explicit schemas.

Feature Definitions as Python Classes

Every feature group is a Python class that declares its entity, schema, data source, and computation logic. Here's a real example from our fraud detection pipeline:

from dataclasses import dataclass, field
from datetime import timedelta
from typing import List

from feature_store.core import FeatureGroup, Entity, Feature, DataSource
from feature_store.types import Float64, Int64, String, Timestamp


@dataclass
class UserTransactionFeatures(FeatureGroup):
    """Rolling transaction statistics per user for fraud scoring."""

    name: str = "user_transaction_features"
    entity: Entity = Entity(name="user_id", dtype=String)
    ttl: timedelta = timedelta(hours=24)
    online: bool = True
    tags: List[str] = field(default_factory=lambda: ["fraud", "transactions"])

    features: List[Feature] = field(default_factory=lambda: [
        Feature("tx_count_1h", Int64, "Transaction count in last 1 hour"),
        Feature("tx_count_24h", Int64, "Transaction count in last 24 hours"),
        Feature("tx_amount_sum_1h", Float64, "Sum of transaction amounts, 1h window"),
        Feature("tx_amount_avg_30d", Float64, "Average transaction amount, 30d window"),
        Feature("tx_amount_std_30d", Float64, "Std deviation of amounts, 30d window"),
        Feature("unique_merchants_7d", Int64, "Unique merchants in last 7 days"),
        Feature("max_single_tx_24h", Float64, "Largest single transaction, 24h"),
        Feature("is_new_device", Int64, "1 if device not seen in last 90 days"),
    ])

    source: DataSource = DataSource(
        table="analytics.transactions",
        timestamp_field="event_timestamp",
        created_timestamp_field="created_at",
    )

    def compute_batch(self, spark, entity_df, start_ts, end_ts):
        """Compute features for offline store (training)."""
        transactions = spark.read.format("iceberg").load(
            "s3://data-lake/analytics.db/transactions"
        ).filter(
            (col("event_timestamp") >= start_ts) &
            (col("event_timestamp") <= end_ts)
        )

        one_hour = transactions.filter(
            col("event_timestamp") >= col("_point_in_time") - expr("INTERVAL 1 HOUR")
        )

        twenty_four_hours = transactions.filter(
            col("event_timestamp") >= col("_point_in_time") - expr("INTERVAL 24 HOURS")
        )

        thirty_days = transactions.filter(
            col("event_timestamp") >= col("_point_in_time") - expr("INTERVAL 30 DAYS")
        )

        return entity_df.join(
            one_hour.groupBy("user_id").agg(
                count("*").alias("tx_count_1h"),
                sum("amount").alias("tx_amount_sum_1h"),
            ),
            on="user_id",
            how="left",
        ).join(
            twenty_four_hours.groupBy("user_id").agg(
                count("*").alias("tx_count_24h"),
                max("amount").alias("max_single_tx_24h"),
            ),
            on="user_id",
            how="left",
        ).join(
            thirty_days.groupBy("user_id").agg(
                avg("amount").alias("tx_amount_avg_30d"),
                stddev("amount").alias("tx_amount_std_30d"),
            ),
            on="user_id",
            how="left",
        ).fillna(0)

This is verbose, but that's intentional. Every feature has an explicit type and description. The computation logic is visible in one place. Code review catches mistakes before they hit production. No hidden transformations buried in YAML config files.

The Offline Store: Iceberg Tables in S3

The offline store is a collection of Iceberg tables, one per feature group, partitioned by date. We chose Iceberg over Delta Lake for three reasons: snapshot isolation for concurrent reads/writes, time-travel for reproducible training datasets, and native support in both Spark and Trino (we use Spark for heavy batch computation and Trino for ad-hoc queries during feature exploration).

Each feature table has this schema pattern:

CREATE TABLE feature_store.user_transaction_features (
    user_id         STRING      NOT NULL,
    event_timestamp TIMESTAMP   NOT NULL,
    created_at      TIMESTAMP   NOT NULL,
    tx_count_1h         BIGINT,
    tx_count_24h        BIGINT,
    tx_amount_sum_1h    DOUBLE,
    tx_amount_avg_30d   DOUBLE,
    tx_amount_std_30d   DOUBLE,
    unique_merchants_7d BIGINT,
    max_single_tx_24h   DOUBLE,
    is_new_device       BIGINT
)
USING iceberg
PARTITIONED BY (days(event_timestamp))
LOCATION 's3://feature-store/user_transaction_features'
TBLPROPERTIES (
    'write.format.default' = 'parquet',
    'write.parquet.compression-codec' = 'zstd'
);

The event_timestamp column is the business time when the feature value became valid. The created_at column tracks when the row was actually written to the store. This distinction matters for late-arriving data: if a batch job backfills features for yesterday, event_timestamp is yesterday but created_at is today.

The Online Store: Redis Cluster

For serving, we use a six-node Redis Cluster. The key schema is straightforward:

# Key format: {feature_group}:{entity_id}
# Value: MessagePack-serialized dict of feature values + metadata

import msgpack
import redis
from datetime import datetime, timezone

class OnlineStore:
    def __init__(self, redis_url: str):
        self.client = redis.RedisCluster.from_url(redis_url)

    def write_features(
        self, feature_group: str, entity_id: str,
        features: dict, event_timestamp: datetime
    ):
        key = f"{feature_group}:{entity_id}"
        value = msgpack.packb({
            "features": features,
            "event_timestamp": event_timestamp.isoformat(),
            "written_at": datetime.now(timezone.utc).isoformat(),
        })
        # TTL from feature group definition
        ttl_seconds = self.get_ttl(feature_group)
        self.client.setex(key, ttl_seconds, value)

    def read_features(
        self, feature_group: str, entity_id: str
    ) -> dict | None:
        key = f"{feature_group}:{entity_id}"
        raw = self.client.get(key)
        if raw is None:
            return None
        data = msgpack.unpackb(raw, raw=False)
        return data["features"]

    def read_batch(
        self, feature_group: str, entity_ids: list[str]
    ) -> dict[str, dict]:
        """Pipeline read for multiple entities."""
        pipe = self.client.pipeline(transaction=False)
        keys = [f"{feature_group}:{eid}" for eid in entity_ids]
        for key in keys:
            pipe.get(key)
        results = pipe.execute()

        output = {}
        for eid, raw in zip(entity_ids, results):
            if raw is not None:
                data = msgpack.unpackb(raw, raw=False)
                output[eid] = data["features"]
        return output

We use MessagePack instead of JSON for serialization because it's roughly 30% smaller and 3x faster to serialize/deserialize than JSON for our typical feature payloads. With an average of 50-80 features per entity, that adds up at 10K+ requests per second.

Point-in-Time Correctness: The Hard Part

Point-in-time correctness is the problem that justifies a feature store's existence. Without it, you have data leakage, your offline metrics don't match production performance, and your models quietly degrade in ways that are extremely difficult to debug.

The concept is simple: when building a training example for an event that occurred at time T, you should only use feature values that were available at time T. You must not use features computed from data that arrived after T.

The implementation is where it gets tricky. Here's our point-in-time join logic:

from pyspark.sql import SparkSession, DataFrame
from pyspark.sql.functions import col, max as spark_max
from pyspark.sql.window import Window


def point_in_time_join(
    entity_df: DataFrame,
    feature_df: DataFrame,
    entity_column: str,
    entity_timestamp_column: str = "event_timestamp",
    feature_timestamp_column: str = "event_timestamp",
    ttl_seconds: int | None = None,
) -> DataFrame:
    """
    Join features to entities using point-in-time semantics.

    For each entity row, finds the most recent feature row where
    feature.event_timestamp <= entity.event_timestamp.

    Optionally enforces a TTL: features older than ttl_seconds
    before the entity timestamp are treated as missing.
    """
    # Rename feature timestamp to avoid collision
    feature_df = feature_df.withColumnRenamed(
        feature_timestamp_column, "_feature_ts"
    )
    entity_df = entity_df.withColumnRenamed(
        entity_timestamp_column, "_entity_ts"
    )

    # Join on entity key where feature timestamp <= entity timestamp
    joined = entity_df.join(
        feature_df,
        on=(
            (entity_df[entity_column] == feature_df[entity_column]) &
            (feature_df["_feature_ts"] <= entity_df["_entity_ts"])
        ),
        how="left",
    ).drop(feature_df[entity_column])

    # Apply TTL filter if specified
    if ttl_seconds is not None:
        joined = joined.filter(
            (col("_feature_ts").isNull()) |
            (
                col("_entity_ts").cast("long") -
                col("_feature_ts").cast("long") <= ttl_seconds
            )
        )

    # For each entity row, keep only the most recent feature row
    window = Window.partitionBy(
        entity_column, "_entity_ts"
    ).orderBy(col("_feature_ts").desc())

    result = joined.withColumn(
        "_rank", row_number().over(window)
    ).filter(
        col("_rank") == 1
    ).drop("_rank", "_feature_ts")

    # Restore original column name
    result = result.withColumnRenamed("_entity_ts", entity_timestamp_column)

    return result

The critical detail is the inequality join: feature_ts <= entity_ts. This ensures you never look into the future. The window function with row_number() then picks the most recent valid feature row for each entity. The optional TTL prevents using stale features — if the most recent feature update for a user was 90 days ago, it's probably better to treat that as missing rather than using ancient data.

This runs entirely inside Spark, pushed down to the Iceberg scan layer. For our 50M-row training sets with 200+ features across 15 feature groups, the full join completes in about 12 minutes on our Spark cluster. The equivalent operation in Feast's driver-side approach was taking over three hours and frequently OOM-killing the driver process.

Feature Freshness: Batch vs. Streaming Paths

Not all features need the same freshness. Our architecture supports two materialization paths:

Batch path (hourly/daily): A Dagster job runs the compute_batch() method for each feature group, writes results to the Iceberg offline store, and then pushes the latest values to Redis. This handles features like 30-day rolling averages, lifetime statistics, and slowly-changing attributes.

Streaming path (sub-minute): Flink jobs consume from Kafka topics, compute windowed aggregations, and write directly to both Redis (for immediate serving) and Iceberg (for eventual training consistency). This handles features like "transaction count in last 1 hour" that the fraud model needs in real-time.

# Simplified materialization job (Dagster)
from dagster import op, job, schedule
from feature_store.registry import FeatureRegistry
from feature_store.stores import OfflineStore, OnlineStore


@op
def materialize_feature_group(context, feature_group_name: str):
    registry = FeatureRegistry()
    fg = registry.get_feature_group(feature_group_name)
    offline = OfflineStore()
    online = OnlineStore()

    spark = SparkSession.builder.getOrCreate()

    # Compute features for the current window
    end_ts = context.partition_key  # e.g., "2026-01-15T00:00:00"
    start_ts = end_ts - fg.ttl

    features_df = fg.compute_batch(spark, entity_df=None, start_ts=start_ts, end_ts=end_ts)

    # Write to offline store (Iceberg)
    offline.write(fg.name, features_df)
    context.log.info(f"Wrote {features_df.count()} rows to offline store")

    # Push latest values to online store (Redis)
    latest = features_df.groupBy(fg.entity.name).agg(
        spark_max("event_timestamp").alias("latest_ts")
    )
    latest_features = features_df.join(
        latest,
        on=[fg.entity.name, features_df.event_timestamp == latest.latest_ts],
    )

    # Collect and push to Redis (for small-medium entity counts)
    rows = latest_features.collect()
    for row in rows:
        online.write_features(
            fg.name,
            entity_id=row[fg.entity.name],
            features={f.name: row[f.name] for f in fg.features},
            event_timestamp=row["event_timestamp"],
        )

    context.log.info(f"Pushed {len(rows)} entities to online store")


@job
def materialize_all():
    registry = FeatureRegistry()
    for fg in registry.list_feature_groups():
        if fg.batch_schedule:
            materialize_feature_group(fg.name)

The Feature Server API

The feature server is a FastAPI application that provides a unified interface for both training and serving. ML engineers interact with features through this API — they never read from Redis or Iceberg directly.

from fastapi import FastAPI, HTTPException
from pydantic import BaseModel
from typing import Optional

app = FastAPI(title="Feature Server")

class OnlineFeatureRequest(BaseModel):
    feature_group: str
    entity_ids: list[str]
    features: Optional[list[str]] = None  # None = all features

class OnlineFeatureResponse(BaseModel):
    features: dict[str, dict[str, float | int | str | None]]
    metadata: dict[str, str]

@app.post("/features/online", response_model=OnlineFeatureResponse)
async def get_online_features(request: OnlineFeatureRequest):
    """Low-latency feature retrieval for model serving."""
    store = get_online_store()
    registry = get_registry()

    fg = registry.get_feature_group(request.feature_group)
    if fg is None:
        raise HTTPException(404, f"Feature group '{request.feature_group}' not found")

    results = store.read_batch(fg.name, request.entity_ids)

    # Filter to requested features if specified
    if request.features:
        results = {
            eid: {k: v for k, v in feats.items() if k in request.features}
            for eid, feats in results.items()
        }

    # Fill missing entities with None values
    all_feature_names = request.features or [f.name for f in fg.features]
    for eid in request.entity_ids:
        if eid not in results:
            results[eid] = {f: None for f in all_feature_names}

    return OnlineFeatureResponse(
        features=results,
        metadata={"feature_group": fg.name, "entity_count": str(len(results))},
    )


class TrainingDataRequest(BaseModel):
    entity_source: str  # SQL query or table name
    feature_groups: list[str]
    start_time: str
    end_time: str

@app.post("/features/training")
async def get_training_features(request: TrainingDataRequest):
    """Generate a point-in-time correct training dataset."""
    # This kicks off a Spark job and returns a job ID
    job_id = submit_training_data_job(
        entity_source=request.entity_source,
        feature_groups=request.feature_groups,
        start_time=request.start_time,
        end_time=request.end_time,
    )
    return {"job_id": job_id, "status": "submitted"}

The online endpoint averages 4ms p50 and 12ms p99 latency at our current throughput. The training endpoint is async because point-in-time joins over large datasets take minutes — it returns a job ID that the caller polls until the result is ready in S3.

Backfilling: Rewriting History Without Breaking It

Backfilling is the process of computing feature values for historical time periods, usually because you've added a new feature or fixed a bug in an existing computation. It sounds straightforward until you realize the implications.

When you backfill a feature, you need to:

  1. Compute the feature using only data available at each historical point. You can't use the current version of a lookup table that was different six months ago.
  2. Write the results without overwriting existing feature values that downstream models may depend on. Iceberg's snapshot isolation helps here — you can write to a new snapshot while readers continue using the old one.
  3. Validate the backfilled values against the existing data to catch computation errors before they propagate.
  4. Atomically swap the old and new feature data once validation passes.

We handle this with a shadow-write pattern. Backfilled features land in a separate Iceberg branch (Iceberg supports Git-like branching natively). A validation job compares statistical distributions of the backfilled features against the existing ones and flags anomalies. Only after a human reviews and approves the validation report does the branch get merged into the main feature table.

This saved us from a serious incident once. An engineer backfilled a feature and accidentally used a filter that excluded null values in a join, silently dropping 15% of entities. The validation caught the entity count discrepancy and blocked the merge.

Monitoring: Catching Drift Before the Model Does

Feature monitoring runs on two levels:

Operational monitoring: Is the materialization pipeline running on schedule? Are features landing in Redis within the expected freshness window? Are there any entities with missing features? We track these with Prometheus metrics and alert through PagerDuty.

Statistical monitoring: Are the feature distributions shifting? We compute daily statistics (mean, stddev, quantiles, null rate, cardinality) for every feature and compare against a 30-day rolling baseline. A shift beyond 3 standard deviations triggers an alert to the feature owner.

from dataclasses import dataclass
from datetime import datetime

@dataclass
class FeatureStats:
    feature_name: str
    computed_at: datetime
    count: int
    null_count: int
    mean: float | None
    stddev: float | None
    p25: float | None
    p50: float | None
    p75: float | None
    p99: float | None
    min_val: float | None
    max_val: float | None
    cardinality: int | None

def detect_drift(
    current: FeatureStats,
    baseline: list[FeatureStats],
    threshold_sigma: float = 3.0,
) -> list[str]:
    """Compare current stats against baseline window. Return list of alerts."""
    alerts = []
    if not baseline:
        return alerts

    # Null rate drift
    baseline_null_rates = [b.null_count / b.count for b in baseline if b.count > 0]
    if baseline_null_rates:
        mean_null_rate = sum(baseline_null_rates) / len(baseline_null_rates)
        std_null_rate = (
            sum((r - mean_null_rate) ** 2 for r in baseline_null_rates)
            / len(baseline_null_rates)
        ) ** 0.5
        current_null_rate = current.null_count / current.count if current.count > 0 else 0

        if std_null_rate > 0 and abs(current_null_rate - mean_null_rate) > threshold_sigma * std_null_rate:
            alerts.append(
                f"Null rate drift: {current_null_rate:.3f} vs baseline {mean_null_rate:.3f} "
                f"(+/- {std_null_rate:.3f})"
            )

    # Mean drift for numeric features
    baseline_means = [b.mean for b in baseline if b.mean is not None]
    if baseline_means and current.mean is not None:
        mean_of_means = sum(baseline_means) / len(baseline_means)
        std_of_means = (
            sum((m - mean_of_means) ** 2 for m in baseline_means)
            / len(baseline_means)
        ) ** 0.5

        if std_of_means > 0 and abs(current.mean - mean_of_means) > threshold_sigma * std_of_means:
            alerts.append(
                f"Mean drift: {current.mean:.4f} vs baseline {mean_of_means:.4f} "
                f"(+/- {std_of_means:.4f})"
            )

    return alerts

This monitoring has caught real issues: a data source schema change that caused a feature to silently become all nulls, a timezone bug that shifted a date-based feature by one day, and a gradual distribution shift in transaction amounts during a holiday period that would have degraded the fraud model if we hadn't retrained proactively.

When to Build vs. When to Use Feast

I want to be honest about this, because "we built our own" stories often gloss over the costs. Building a feature store took our team of three platform engineers roughly six months of focused work. It took another three months to stabilize it, write documentation, and onboard the ML team. We're now nine months in and still spending about 20% of one engineer's time on maintenance and improvements.

Build your own if:

  • You have a non-standard data platform that doesn't match the assumptions of existing tools (our case: Iceberg + Flink + Trino).
  • You need sub-minute feature freshness and your streaming infrastructure doesn't align with what off-the-shelf stores expect.
  • You have a dedicated platform engineering team that can absorb the ongoing maintenance burden.
  • Your scale justifies it — we have 40+ models and 200+ features. If you have 3 models, this is massive over-engineering.

Use Feast (or another existing solution) if:

  • Your data lives in BigQuery, Snowflake, or Redshift. Feast integrates beautifully with these.
  • Batch feature freshness (minutes to hours) is acceptable for your use cases.
  • Your team is small and needs to ship models, not build infrastructure.
  • You're just getting started with ML in production. Feature consistency and point-in-time joins matter more than where the bits are stored.
  • Your training datasets are under 10M rows. Feast's driver-side joins work fine at this scale.

The best feature store is the one your team actually uses. A perfect custom system that only the platform team understands is worse than an imperfect off-the-shelf tool that every ML engineer can operate independently.

Lessons After Nine Months in Production

Here's what we've learned operating this system:

  1. Feature definitions are the easy part. The hard parts are backfilling, monitoring, and debugging data quality issues that span multiple systems. Plan for these from day one.
  2. TTLs prevent more bugs than tests do. Aggressively setting TTLs on feature groups forces you to think about feature freshness and prevents stale data from silently degrading models.
  3. The registry is the product. ML engineers don't interact with Redis or Iceberg. They interact with the feature registry and the API. Invest in good documentation, search, and lineage tracking for your features.
  4. Streaming features are 10x harder than batch features. We started with batch-only and added streaming later for the features that genuinely needed it. Don't start with streaming.
  5. Version everything. Feature definitions, computation code, schemas, even the monitoring thresholds. When something breaks at 3 AM, you need to know exactly what changed.

If you're considering building a feature store, start with the simplest possible version: a feature registry, an offline store with point-in-time joins, and a basic online store. You can always add streaming, monitoring, and backfilling capabilities later. But get point-in-time correctness right from the beginning — it's the foundation everything else depends on, and retrofitting it is painful.

The code examples in this article are simplified versions of our production code, but the architecture and patterns are exactly what we run. If you have questions about specific implementation details, feel free to reach out or leave a comment below.

Leave a Comment