<?xml version="1.0" encoding="utf-8"?>
<feed xmlns="http://www.w3.org/2005/Atom"><title>Apache DataFusion Blog - Tim Saucer (rerun.io)</title><link href="https://datafusion.apache.org/blog/" rel="alternate"/><link href="https://datafusion.apache.org/blog/feeds/tim-saucer-rerunio.atom.xml" rel="self"/><id>https://datafusion.apache.org/blog/</id><updated>2026-03-31T00:00:00+00:00</updated><entry><title>Writing Custom Table Providers in Apache DataFusion</title><link href="https://datafusion.apache.org/blog/2026/03/31/writing-table-providers" rel="alternate"/><published>2026-03-31T00:00:00+00:00</published><updated>2026-03-31T00:00:00+00:00</updated><author><name>Tim Saucer (rerun.io)</name></author><id>tag:datafusion.apache.org,2026-03-31:/blog/2026/03/31/writing-table-providers</id><summary type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;One of DataFusion's greatest strengths is its extensibility. If your data lives
in a custom format, behind an API, or in a system that DataFusion does not
natively support, you can teach DataFusion to read it by implementing a
&lt;strong&gt;custom table provider&lt;/strong&gt;. This post walks through the three layers you …&lt;/p&gt;</summary><content type="html">&lt;!--
{% comment %}
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements.  See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to you under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License.  You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
{% endcomment %}
--&gt;

&lt;p&gt;One of DataFusion's greatest strengths is its extensibility. If your data lives
in a custom format, behind an API, or in a system that DataFusion does not
natively support, you can teach DataFusion to read it by implementing a
&lt;strong&gt;custom table provider&lt;/strong&gt;. This post walks through the three layers you need to
understand to design a table provider and where planning and execution work should happen.&lt;/p&gt;
&lt;h2 id="the-three-layers"&gt;The Three Layers&lt;a class="headerlink" href="#the-three-layers" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;When DataFusion executes a query against a table, three abstractions collaborate
to produce results:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html"&gt;TableProvider&lt;/a&gt;&lt;/strong&gt; -- Describes the table (schema, capabilities) and
   produces an execution plan when queried. This is part of the &lt;strong&gt;Logical Plan&lt;/strong&gt;.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;ExecutionPlan&lt;/a&gt;&lt;/strong&gt; -- Describes &lt;em&gt;how&lt;/em&gt; to compute the result: partitioning,
   ordering, and child plan relationships. This is part of the &lt;strong&gt;Physical Plan&lt;/strong&gt;.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/type.SendableRecordBatchStream.html"&gt;SendableRecordBatchStream&lt;/a&gt;&lt;/strong&gt; -- The async stream that &lt;em&gt;actually does the
   work&lt;/em&gt;, yielding &lt;code&gt;RecordBatch&lt;/code&gt;es one at a time.&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Think of these as a funnel: &lt;code&gt;TableProvider::scan()&lt;/code&gt; is called once during
planning to create an &lt;code&gt;ExecutionPlan&lt;/code&gt;, then &lt;code&gt;ExecutionPlan::execute()&lt;/code&gt; is called
once per partition to create a stream, and those streams are where rows are
actually produced during execution.&lt;/p&gt;
&lt;h2 id="background-logical-and-physical-planning"&gt;Background: Logical and Physical Planning&lt;a class="headerlink" href="#background-logical-and-physical-planning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Before diving into the three layers, it helps to understand how DataFusion
processes a query. There are several phases between a SQL string (or DataFrame
call) and streaming results:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;SQL / DataFrame API
  → Logical Plan          (abstract: what to compute)
  → Logical Optimization  (rewrite rules that preserve semantics)
  → Physical Plan         (concrete: how to compute it)
  → Physical Optimization (hardware- and data-aware rewrites)
  → Execution             (streaming RecordBatches)
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="logical-planning"&gt;Logical Planning&lt;a class="headerlink" href="#logical-planning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;A &lt;strong&gt;logical plan&lt;/strong&gt; describes &lt;em&gt;what&lt;/em&gt; the query computes without specifying &lt;em&gt;how&lt;/em&gt;.
It is a tree of relational operators -- &lt;code&gt;TableScan&lt;/code&gt;, &lt;code&gt;Filter&lt;/code&gt;, &lt;code&gt;Projection&lt;/code&gt;,
&lt;code&gt;Aggregate&lt;/code&gt;, &lt;code&gt;Join&lt;/code&gt;, &lt;code&gt;Sort&lt;/code&gt;, &lt;code&gt;Limit&lt;/code&gt;, and so on. The logical optimizer rewrites
this tree to reduce work while preserving the query's meaning. Some logical
optimizations include:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Predicate pushdown&lt;/strong&gt; -- moves filters as close to the data source as
  possible, so fewer rows flow through the rest of the plan.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Projection pruning&lt;/strong&gt; -- eliminates columns that are never referenced
  downstream, reducing memory and I/O.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Expression simplification&lt;/strong&gt; -- rewrites expressions like &lt;code&gt;1 = 1&lt;/code&gt; or
  &lt;code&gt;x AND true&lt;/code&gt; into simpler forms.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Subquery decorrelation&lt;/strong&gt; -- converts correlated &lt;code&gt;IN&lt;/code&gt; / &lt;code&gt;EXISTS&lt;/code&gt; subqueries
  into more efficient semi-joins.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Limit pushdown&lt;/strong&gt; -- pushes &lt;code&gt;LIMIT&lt;/code&gt; earlier in the plan so operators
  produce less data.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="physical-planning"&gt;Physical Planning&lt;a class="headerlink" href="#physical-planning" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;strong&gt;physical planner&lt;/strong&gt; converts the optimized logical plan into an
&lt;code&gt;ExecutionPlan&lt;/code&gt; tree -- the concrete plan that will actually run. This is where
decisions like "use a hash join vs. a sort-merge join" or "how many partitions
to scan" are made. The physical optimizer then refines this tree further with rewrites such as:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;Distribution enforcement&lt;/strong&gt; -- inserts &lt;code&gt;RepartitionExec&lt;/code&gt; nodes so that data
  is partitioned correctly for joins and aggregations.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Sort enforcement&lt;/strong&gt; -- inserts &lt;code&gt;SortExec&lt;/code&gt; nodes where ordering is required,
  and removes them where the data is already sorted.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Join selection&lt;/strong&gt; -- picks the most efficient join strategy based on
  statistics and table sizes.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Aggregate optimization&lt;/strong&gt; -- combines partial and final aggregation stages,
  and can use exact statistics to skip scanning entirely.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="why-this-matters-for-table-providers"&gt;Why This Matters for Table Providers&lt;a class="headerlink" href="#why-this-matters-for-table-providers" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Your &lt;code&gt;TableProvider&lt;/code&gt; sits at the boundary between logical and physical planning.
During logical optimization, DataFusion determines which filters and projections
&lt;em&gt;could&lt;/em&gt; be pushed down to the source. When &lt;code&gt;scan()&lt;/code&gt; is called during physical
planning, those hints are passed to you. By implementing capabilities like
&lt;code&gt;supports_filters_pushdown&lt;/code&gt;, you influence what the optimizer can do -- and the
metadata you declare in your &lt;code&gt;ExecutionPlan&lt;/code&gt; (partitioning, ordering) directly
affects which physical optimizations apply.&lt;/p&gt;
&lt;h2 id="choosing-the-right-starting-point"&gt;Choosing the Right Starting Point&lt;a class="headerlink" href="#choosing-the-right-starting-point" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Not every custom data source requires implementing all three layers from
scratch. DataFusion provides building blocks that let you plug in at whatever
level makes sense:&lt;/p&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;If your data is...&lt;/th&gt;
&lt;th&gt;Start with&lt;/th&gt;
&lt;th&gt;You implement&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;Already in &lt;code&gt;RecordBatch&lt;/code&gt;es in memory&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/memory/struct.MemTable.html"&gt;MemTable&lt;/a&gt;&lt;/td&gt;
&lt;td&gt;Nothing -- just construct it&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;An async stream of batches&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/stream/struct.StreamTable.html"&gt;StreamTable&lt;/a&gt;&lt;/td&gt;
&lt;td&gt;A stream factory&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;A logical transformation of other tables&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/view/struct.ViewTable.html"&gt;ViewTable&lt;/a&gt; wrapping a logical plan&lt;/td&gt;
&lt;td&gt;The logical plan&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;A variant of an existing file format&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt; with a custom &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/trait.FileFormat.html"&gt;FileFormat&lt;/a&gt; wrapping an existing one&lt;/td&gt;
&lt;td&gt;A thin &lt;code&gt;FileFormat&lt;/code&gt; wrapper&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;Files in a custom format on disk or object storage&lt;/td&gt;
&lt;td&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt; with a custom &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/trait.FileFormat.html"&gt;FileFormat&lt;/a&gt;, &lt;a href="https://docs.rs/datafusion-datasource/latest/datafusion_datasource/file/trait.FileSource.html"&gt;FileSource&lt;/a&gt;, and &lt;a href="https://docs.rs/datafusion-datasource/latest/datafusion_datasource/file_stream/trait.FileOpener.html"&gt;FileOpener&lt;/a&gt;&lt;/td&gt;
&lt;td&gt;The format, source, and opener&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;A custom source needing full control&lt;/td&gt;
&lt;td&gt;&lt;code&gt;TableProvider&lt;/code&gt; + &lt;code&gt;ExecutionPlan&lt;/code&gt; + stream&lt;/td&gt;
&lt;td&gt;All three layers&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;If your data is file-based, &lt;code&gt;ListingTable&lt;/code&gt; handles file discovery, partition
column inference, and plan construction -- you only need to implement
&lt;code&gt;FileFormat&lt;/code&gt;, &lt;code&gt;FileSource&lt;/code&gt;, and &lt;code&gt;FileOpener&lt;/code&gt; to describe how to read your
files. See the &lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/custom_data_source/custom_file_format.rs"&gt;custom_file_format example&lt;/a&gt; for a minimal wrapping approach,
or &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/parquet/struct.ParquetSource.html"&gt;ParquetSource&lt;/a&gt; and &lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/file_format/parquet/struct.ParquetOpener.html"&gt;ParquetOpener&lt;/a&gt; for a full custom implementation to
use as a reference.&lt;/p&gt;
&lt;p&gt;The rest of this post focuses on the full &lt;code&gt;TableProvider&lt;/code&gt; + &lt;code&gt;ExecutionPlan&lt;/code&gt; +
stream path, which gives you complete control and applies to any data source.&lt;/p&gt;
&lt;h2 id="layer-1-tableprovider"&gt;Layer 1: TableProvider&lt;a class="headerlink" href="#layer-1-tableprovider" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;A &lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html"&gt;TableProvider&lt;/a&gt; represents a queryable data source. For a minimal read-only
table, you need four methods:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl TableProvider for MyTable {
    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }

    fn schema(&amp;amp;self) -&amp;gt; SchemaRef {
        Arc::clone(&amp;amp;self.schema)
    }

    fn table_type(&amp;amp;self) -&amp;gt; TableType {
        TableType::Base
    }

    async fn scan(
        &amp;amp;self,
        state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        // Build and return an ExecutionPlan -- don't do any execution work here -- keep lightweight!
        Ok(Arc::new(MyExecPlan::new(
            Arc::clone(&amp;amp;self.schema),
            projection,
            limit,
        )))
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The &lt;code&gt;scan&lt;/code&gt; method is the heart of &lt;code&gt;TableProvider&lt;/code&gt;. It receives three pushdown
hints from the optimizer, each reducing the amount of data your source needs
to produce:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;projection&lt;/code&gt;&lt;/strong&gt; -- Which columns are needed. This reduces the &lt;strong&gt;width&lt;/strong&gt; of
  the output. If your source supports it, read only these columns rather than
  the full schema.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;filters&lt;/code&gt;&lt;/strong&gt; -- Predicates the engine would like you to apply during the
  scan. This reduces the &lt;strong&gt;number of rows&lt;/strong&gt; by skipping data that does not
  match. Implement &lt;code&gt;supports_filters_pushdown&lt;/code&gt; to advertise which filters you
  can handle.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;limit&lt;/code&gt;&lt;/strong&gt; -- A row count cap. This also reduces the &lt;strong&gt;number of rows&lt;/strong&gt; --
  if you can stop reading early once you have produced enough rows, this avoids
  unnecessary work.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;You can also use the &lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html#method.scan_with_args"&gt;scan_with_args()&lt;/a&gt;
variant that provides additional pushdown information for other advanced use cases.&lt;/p&gt;
&lt;h3 id="keep-scan-lightweight"&gt;Keep &lt;code&gt;scan()&lt;/code&gt; Lightweight&lt;a class="headerlink" href="#keep-scan-lightweight" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;This is a critical point: &lt;strong&gt;&lt;code&gt;scan()&lt;/code&gt; runs during planning, not execution.&lt;/strong&gt; It
should return quickly. Best practice is to avoid performing I/O, network
calls, or heavy computation here. The &lt;code&gt;scan&lt;/code&gt; method's job is to &lt;em&gt;describe&lt;/em&gt; how
the data will be produced, not to produce it. All the real work belongs in the
stream (Layer 3).&lt;/p&gt;
&lt;p&gt;A common pitfall is to fetch data or open connections in &lt;code&gt;scan()&lt;/code&gt;. This blocks
the planning thread and can cause timeouts or deadlocks, especially if the query
involves multiple tables or subqueries that all need to be planned before
execution begins.&lt;/p&gt;
&lt;h3 id="existing-implementations-to-learn-from"&gt;Existing Implementations to Learn From&lt;a class="headerlink" href="#existing-implementations-to-learn-from" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion ships several &lt;code&gt;TableProvider&lt;/code&gt; implementations that are excellent
references:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/memory/struct.MemTable.html"&gt;MemTable&lt;/a&gt;&lt;/strong&gt; -- Holds data in memory as &lt;code&gt;Vec&amp;lt;RecordBatch&amp;gt;&lt;/code&gt;. The simplest
  possible provider; great for tests and small datasets.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/stream/struct.StreamTable.html"&gt;StreamTable&lt;/a&gt;&lt;/strong&gt; -- Wraps a user-provided stream factory. Useful when your
  data arrives as a continuous stream (e.g., from Kafka or a socket).&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html"&gt;ListingTable&lt;/a&gt;&lt;/strong&gt; -- The file-based data source behind DataFusion's
  built-in Parquet, CSV, and JSON support. Demonstrates sophisticated filter
  and projection pushdown, file pruning, and schema inference.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/view/struct.ViewTable.html"&gt;ViewTable&lt;/a&gt;&lt;/strong&gt; -- Wraps a logical plan, representing a SQL view. Useful
  if your provider is best expressed as a transformation of other tables.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="layer-2-executionplan"&gt;Layer 2: ExecutionPlan&lt;a class="headerlink" href="#layer-2-executionplan" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;An &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;ExecutionPlan&lt;/a&gt; is a node in the physical query plan tree. Your table
provider's &lt;code&gt;scan()&lt;/code&gt; method returns one. The required methods are:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;impl ExecutionPlan for MyExecPlan {
    fn name(&amp;amp;self) -&amp;gt; &amp;amp;str { "MyExecPlan" }

    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }

    fn properties(&amp;amp;self) -&amp;gt; &amp;amp;PlanProperties {
        &amp;amp;self.properties
    }

    fn children(&amp;amp;self) -&amp;gt; Vec&amp;lt;&amp;amp;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        vec![]  // Leaf node -- no children
    }

    fn with_new_children(
        self: Arc&amp;lt;Self&amp;gt;,
        children: Vec&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        assert!(children.is_empty());
        Ok(self)
    }

    fn execute(
        &amp;amp;self,
        partition: usize,
        context: Arc&amp;lt;TaskContext&amp;gt;,
    ) -&amp;gt; Result&amp;lt;SendableRecordBatchStream&amp;gt; {
        // This is where you build and return your stream
        // ...
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The key properties to set correctly in &lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.PlanProperties.html"&gt;PlanProperties&lt;/a&gt; are &lt;strong&gt;output
partitioning&lt;/strong&gt; and &lt;strong&gt;output ordering&lt;/strong&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Output partitioning&lt;/strong&gt; tells the engine how many partitions your data has,
which determines parallelism. If your source naturally partitions data (e.g.,
by file or by shard), expose that here.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Output ordering&lt;/strong&gt; declares whether your data is naturally sorted. This
enables the optimizer to avoid inserting a &lt;code&gt;SortExec&lt;/code&gt; when a query requires
ordered data. Getting this right can be a significant performance win.&lt;/p&gt;
&lt;h3 id="partitioning-strategies"&gt;Partitioning Strategies&lt;a class="headerlink" href="#partitioning-strategies" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Since &lt;code&gt;execute()&lt;/code&gt; is called once per partition, partitioning directly controls
the parallelism of your table scan. Each partition produces an independent
stream that DataFusion schedules as a &lt;strong&gt;task&lt;/strong&gt; on the tokio runtime. It is
important to distinguish tasks from threads: tasks are lightweight units of
async work that are multiplexed onto a thread pool. You can have many more
tasks (partitions) than physical threads -- the runtime will interleave them
efficiently as they await I/O or yield.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Start simple: match your data's natural layout.&lt;/strong&gt; If you have 4 files, expose
4 partitions. If your source has 8 shards, expose 8 partitions. DataFusion will
insert a &lt;code&gt;RepartitionExec&lt;/code&gt; above your scan when downstream operators need a
different distribution. You can also implement the
&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.repartitioned"&gt;&lt;code&gt;repartitioned&lt;/code&gt;&lt;/a&gt;
method on your &lt;code&gt;ExecutionPlan&lt;/code&gt; to let DataFusion request a different partition
count directly from your source, avoiding the extra operator entirely.&lt;/p&gt;
&lt;p&gt;Consider how your data source naturally divides its data:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;By file or object:&lt;/strong&gt; If you are reading from S3, each file can be a
  partition. DataFusion will read them in parallel.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;By shard or region:&lt;/strong&gt; If your source is a sharded database, each shard
  maps naturally to a partition.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;By key range:&lt;/strong&gt; If your data is keyed (e.g., by timestamp or customer ID),
  you can split it into ranges.&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Advanced: aligning with &lt;code&gt;target_partitions&lt;/code&gt;.&lt;/strong&gt; Once you have something
working, you can tune further. Having &lt;em&gt;too many&lt;/em&gt; partitions is not free: each
partition adds scheduling overhead, and downstream operators may need to
repartition the data anyway. The session configuration exposes a
&lt;strong&gt;target partition count&lt;/strong&gt; that reflects how many partitions the optimizer
expects to work with:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;async fn scan(
    &amp;amp;self,
    state: &amp;amp;dyn Session,
    projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
    filters: &amp;amp;[Expr],
    limit: Option&amp;lt;usize&amp;gt;,
) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
    let target_partitions = state.config().target_partitions();
    // Optionally coalesce or split partitions to match target_partitions.
    // ...
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If your source produces data in exactly &lt;code&gt;target_partitions&lt;/code&gt; partitions, the
optimizer is less likely to insert a &lt;code&gt;RepartitionExec&lt;/code&gt; above your scan.
For small datasets, &lt;code&gt;target_partitions&lt;/code&gt; may be set to 1, which avoids any
repartitioning overhead entirely.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Advanced: declaring hash partitioning.&lt;/strong&gt; If your source stores data
pre-partitioned by a specific key (e.g., &lt;code&gt;customer_id&lt;/code&gt;), you can declare this
in your output partitioning. For a query like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT customer_id, SUM(amount)
FROM my_table
GROUP BY customer_id;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If you declare your output partitioning as &lt;code&gt;Hash([customer_id], N)&lt;/code&gt;, the
optimizer recognizes that the data is already distributed correctly for the
aggregation and eliminates the &lt;code&gt;RepartitionExec&lt;/code&gt; that would otherwise appear
in the plan. You can verify this with &lt;code&gt;EXPLAIN&lt;/code&gt; (more on this below).&lt;/p&gt;
&lt;p&gt;Conversely, if you report &lt;code&gt;UnknownPartitioning&lt;/code&gt;, DataFusion must assume the
worst case and will always insert repartitioning operators as needed.&lt;/p&gt;
&lt;h3 id="keep-execute-lightweight-too"&gt;Keep &lt;code&gt;execute()&lt;/code&gt; Lightweight Too&lt;a class="headerlink" href="#keep-execute-lightweight-too" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Like &lt;code&gt;scan()&lt;/code&gt;, the &lt;code&gt;execute()&lt;/code&gt; method should construct and return a stream
without doing heavy work. The actual data production happens when the stream
is polled. Do not block on async operations here -- build the stream and let
the runtime drive it.&lt;/p&gt;
&lt;h3 id="existing-implementations-to-learn-from_1"&gt;Existing Implementations to Learn From&lt;a class="headerlink" href="#existing-implementations-to-learn-from_1" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/stream/struct.StreamingTableExec.html"&gt;StreamingTableExec&lt;/a&gt;&lt;/strong&gt; -- Executes a streaming table scan. It takes a
  stream factory (a closure that produces streams) and handles partitioning.
  Good reference for wrapping external streams.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/datasource/struct.DataSourceExec.html"&gt;DataSourceExec&lt;/a&gt;&lt;/strong&gt; -- The execution plan behind DataFusion's built-in file
  scanning (Parquet, CSV, JSON). It demonstrates sophisticated partitioning,
  filter pushdown, and projection pushdown.&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id="layer-3-sendablerecordbatchstream"&gt;Layer 3: SendableRecordBatchStream&lt;a class="headerlink" href="#layer-3-sendablerecordbatchstream" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/type.SendableRecordBatchStream.html"&gt;SendableRecordBatchStream&lt;/a&gt; is where the real work happens. It is defined as:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;type SendableRecordBatchStream =
    Pin&amp;lt;Box&amp;lt;dyn RecordBatchStream&amp;lt;Item = Result&amp;lt;RecordBatch&amp;gt;&amp;gt; + Send&amp;gt;&amp;gt;;
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This is an async stream of &lt;code&gt;RecordBatch&lt;/code&gt;es that can be sent across threads. When
the DataFusion runtime polls this stream, your code runs: reading files, calling
APIs, transforming data, etc.&lt;/p&gt;
&lt;h3 id="using-recordbatchstreamadapter"&gt;Using RecordBatchStreamAdapter&lt;a class="headerlink" href="#using-recordbatchstreamadapter" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The easiest way to create a &lt;code&gt;SendableRecordBatchStream&lt;/code&gt; is with
&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/stream/struct.RecordBatchStreamAdapter.html"&gt;RecordBatchStreamAdapter&lt;/a&gt;. It bridges any &lt;code&gt;futures::Stream&amp;lt;Item =
Result&amp;lt;RecordBatch&amp;gt;&amp;gt;&lt;/code&gt; into the &lt;code&gt;SendableRecordBatchStream&lt;/code&gt; type:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use datafusion::physical_plan::stream::RecordBatchStreamAdapter;

fn execute(
    &amp;amp;self,
    partition: usize,
    context: Arc&amp;lt;TaskContext&amp;gt;,
) -&amp;gt; Result&amp;lt;SendableRecordBatchStream&amp;gt; {
    let schema = self.schema();
    let config = self.config.clone();

    let stream = futures::stream::once(async move {
        // ALL the heavy work happens here, inside the stream:
        // - Open connections
        // - Read data from external sources
        // - Transform and batch the results
        let batches = fetch_data_from_source(&amp;amp;config).await?;
        Ok(batches)
    })
    .flat_map(|result| match result {
        Ok(batch) =&amp;gt; futures::stream::iter(vec![Ok(batch)]),
        Err(e) =&amp;gt; futures::stream::iter(vec![Err(e)]),
    });

    Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream)))
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h3 id="blocking-work-use-a-separate-thread-pool"&gt;Blocking Work: Use a Separate Thread Pool&lt;a class="headerlink" href="#blocking-work-use-a-separate-thread-pool" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;If your stream performs &lt;strong&gt;blocking&lt;/strong&gt; work -- such as blocking I/O, or CPU work
that runs for hundreds of milliseconds without yielding -- you must avoid
blocking the tokio async runtime. Short CPU work (e.g., parsing a batch in a
few milliseconds) is fine to do inline as long as your code yields back to the
runtime frequently. But for long-running synchronous work that cannot yield,
offload to a dedicated thread pool and send results back through a channel:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn execute(
    &amp;amp;self,
    partition: usize,
    context: Arc&amp;lt;TaskContext&amp;gt;,
) -&amp;gt; Result&amp;lt;SendableRecordBatchStream&amp;gt; {
    let schema = self.schema();
    let config = self.config.clone();

    let (tx, rx) = tokio::sync::mpsc::channel(2);

    // Spawn blocking work on a dedicated thread pool
    tokio::task::spawn_blocking(move || {
        let batches = generate_data(&amp;amp;config);
        for batch in batches {
            if tx.blocking_send(Ok(batch)).is_err() {
                break; // Receiver dropped, query was cancelled
            }
        }
    });

    let stream = tokio_stream::wrappers::ReceiverStream::new(rx);
    Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream)))
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;This pattern keeps the async runtime responsive while long-running synchronous
work runs on its own threads. For a working example that shows how to configure
separate thread pools for I/O and CPU work, see the
&lt;a href="https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/query_planning/thread_pools.rs"&gt;thread_pools example&lt;/a&gt;
in the DataFusion repository.&lt;/p&gt;
&lt;h2 id="where-should-the-work-happen"&gt;Where Should the Work Happen?&lt;a class="headerlink" href="#where-should-the-work-happen" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;This table summarizes what belongs at each layer:&lt;/p&gt;
&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;Layer&lt;/th&gt;
&lt;th&gt;Runs During&lt;/th&gt;
&lt;th&gt;Should Do&lt;/th&gt;
&lt;th&gt;Should NOT Do&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;TableProvider::scan()&lt;/code&gt;&lt;/td&gt;
&lt;td&gt;Planning&lt;/td&gt;
&lt;td&gt;Build an &lt;code&gt;ExecutionPlan&lt;/code&gt; with metadata&lt;/td&gt;
&lt;td&gt;I/O, network calls, heavy computation&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;ExecutionPlan::execute()&lt;/code&gt;&lt;/td&gt;
&lt;td&gt;Execution (once per partition)&lt;/td&gt;
&lt;td&gt;Construct a stream, set up channels&lt;/td&gt;
&lt;td&gt;Block on async work, read data&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;&lt;code&gt;RecordBatchStream&lt;/code&gt; (polling)&lt;/td&gt;
&lt;td&gt;Execution&lt;/td&gt;
&lt;td&gt;All I/O, computation, data production&lt;/td&gt;
&lt;td&gt;--&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;The guiding principle: &lt;strong&gt;push work as late as possible.&lt;/strong&gt; Planning should be
fast so the optimizer can do its job. Execution setup should be fast so all
partitions can start promptly. The stream is where you spend time producing
data.&lt;/p&gt;
&lt;h3 id="why-this-matters"&gt;Why This Matters&lt;a class="headerlink" href="#why-this-matters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;When &lt;code&gt;scan()&lt;/code&gt; does heavy work, several problems arise:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;&lt;strong&gt;Planning becomes slow.&lt;/strong&gt; If a query touches 10 tables and each &lt;code&gt;scan()&lt;/code&gt;
   takes 500ms, planning alone takes 5 seconds before any data flows.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Execution is single-threaded.&lt;/strong&gt; &lt;code&gt;scan()&lt;/code&gt; runs on a single thread during
   planning, so any work done there cannot benefit from the parallel execution
   that DataFusion provides across partitions.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;The optimizer cannot help.&lt;/strong&gt; The optimizer runs between planning and
   execution. If you have already fetched data during planning, optimizations
   like predicate pushdown or partition pruning cannot reduce the work.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;Resource management breaks down.&lt;/strong&gt; DataFusion manages concurrency and
   memory during execution. Work done during planning bypasses these controls.&lt;/li&gt;
&lt;/ol&gt;
&lt;h2 id="filter-pushdown-doing-less-work"&gt;Filter Pushdown: Doing Less Work&lt;a class="headerlink" href="#filter-pushdown-doing-less-work" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;One of the most impactful optimizations you can add to a custom table provider
is &lt;strong&gt;filter pushdown&lt;/strong&gt; -- letting the source skip data that the query does not
need, rather than reading everything and filtering it afterward.&lt;/p&gt;
&lt;h3 id="how-filter-pushdown-works"&gt;How Filter Pushdown Works&lt;a class="headerlink" href="#how-filter-pushdown-works" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;When DataFusion plans a query with a &lt;code&gt;WHERE&lt;/code&gt; clause, it passes the filter
predicates to your &lt;code&gt;scan()&lt;/code&gt; method as the &lt;code&gt;filters&lt;/code&gt; parameter. By default,
DataFusion assumes your provider cannot handle any filters and inserts a
&lt;code&gt;FilterExec&lt;/code&gt; node above your scan to apply them. But if your source &lt;em&gt;can&lt;/em&gt;
evaluate some predicates during scanning -- for example, by skipping files,
partitions, or row groups that cannot match -- you can eliminate a huge amount
of unnecessary I/O.&lt;/p&gt;
&lt;p&gt;To opt in, implement &lt;code&gt;supports_filters_pushdown&lt;/code&gt;:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;fn supports_filters_pushdown(
    &amp;amp;self,
    filters: &amp;amp;[&amp;amp;Expr],
) -&amp;gt; Result&amp;lt;Vec&amp;lt;TableProviderFilterPushDown&amp;gt;&amp;gt; {
    Ok(filters.iter().map(|f| {
        match f {
            // We can fully evaluate equality filters on
            // the partition column at the source
            Expr::BinaryExpr(BinaryExpr {
                left, op: Operator::Eq, right
            }) if is_partition_column(left) || is_partition_column(right) =&amp;gt; {
                TableProviderFilterPushDown::Exact
            }
            // All other filters: let DataFusion handle them
            _ =&amp;gt; TableProviderFilterPushDown::Unsupported,
        }
    }).collect())
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The three possible responses for each filter are:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;Exact&lt;/code&gt;&lt;/strong&gt; -- Your source guarantees that no output rows will have a false
  value for this predicate. Because the filter is fully evaluated at the source,
  DataFusion will &lt;strong&gt;not&lt;/strong&gt; add a &lt;code&gt;FilterExec&lt;/code&gt; for it.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;Inexact&lt;/code&gt;&lt;/strong&gt; -- Your source has the ability to reduce the data produced, but
  the output may still include rows that do not satisfy the predicate. For
  example, you might skip entire files based on metadata statistics but not
  filter individual rows within a file. DataFusion will still add a &lt;code&gt;FilterExec&lt;/code&gt;
  above your scan to remove any remaining rows that slipped through.&lt;/li&gt;
&lt;li&gt;&lt;strong&gt;&lt;code&gt;Unsupported&lt;/code&gt;&lt;/strong&gt; -- Your source ignores this filter entirely. DataFusion
  handles it.&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id="why-filter-pushdown-matters"&gt;Why Filter Pushdown Matters&lt;a class="headerlink" href="#why-filter-pushdown-matters" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;Consider a table with 1 billion rows partitioned by &lt;code&gt;region&lt;/code&gt;, and a query:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;SELECT * FROM events WHERE region = 'us-east-1' AND event_type = 'click';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;&lt;strong&gt;Without filter pushdown:&lt;/strong&gt; Your table provider reads all 1 billion rows
across all regions. DataFusion then applies both filters, discarding the vast
majority of the data.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;With filter pushdown on &lt;code&gt;region&lt;/code&gt;:&lt;/strong&gt; Your &lt;code&gt;scan()&lt;/code&gt; method sees the
&lt;code&gt;region = 'us-east-1'&lt;/code&gt; filter and constructs an execution plan that only reads
the &lt;code&gt;us-east-1&lt;/code&gt; partition. If that partition holds 100 million rows, you have
just eliminated 90% of the I/O. DataFusion still applies the &lt;code&gt;event_type&lt;/code&gt;
filter via &lt;code&gt;FilterExec&lt;/code&gt; if you reported it as &lt;code&gt;Unsupported&lt;/code&gt;.&lt;/p&gt;
&lt;h3 id="only-push-down-filters-when-the-data-source-can-do-better"&gt;Only Push Down Filters When the Data Source Can Do Better&lt;a class="headerlink" href="#only-push-down-filters-when-the-data-source-can-do-better" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;DataFusion already pushes filters as close to the data source as possible, typically placing them directly above the scan. &lt;code&gt;FilterExec&lt;/code&gt; is also highly optimized, with vectorized evaluation and type-specialized kernels for fast predicate evaluation.&lt;/p&gt;
&lt;p&gt;Because of this, you should only implement filter pushdown when your data source
can do strictly better -- for example, by avoiding I/O entirely through
skipping files or partitions based on metadata. If your data source cannot
eliminate I/O in this way, it is usually better to let DataFusion handle the
filter, as its in-memory execution is already highly efficient.&lt;/p&gt;
&lt;h3 id="using-explain-to-debug-your-table-provider"&gt;Using EXPLAIN to Debug Your Table Provider&lt;a class="headerlink" href="#using-explain-to-debug-your-table-provider" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;The &lt;code&gt;EXPLAIN&lt;/code&gt; statement is your best tool for understanding what DataFusion is
actually doing with your table provider. It shows the physical plan that
DataFusion will execute, including any operators it inserted:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-sql"&gt;EXPLAIN SELECT * FROM events WHERE region = 'us-east-1' AND event_type = 'click';
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;If you are using DataFrames, call &lt;code&gt;.explain(false, false)&lt;/code&gt; for the logical plan
or &lt;code&gt;.explain(false, true)&lt;/code&gt; for the physical plan. You can also print the plans
in verbose mode with &lt;code&gt;.explain(true, true)&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Before filter pushdown&lt;/strong&gt;, the plan might look like:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;FilterExec: region@0 = us-east-1 AND event_type@1 = click
  MyExecPlan: partitions=50
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Here DataFusion is reading all 50 partitions and filtering everything
afterward. The &lt;code&gt;FilterExec&lt;/code&gt; above your scan is doing all the predicate work.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;After implementing pushdown for &lt;code&gt;region&lt;/code&gt;&lt;/strong&gt; (reported as &lt;code&gt;Exact&lt;/code&gt;):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;FilterExec: event_type@1 = click
  MyExecPlan: partitions=5, filter=[region = us-east-1]
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;Now your exec reads only the 5 partitions for &lt;code&gt;us-east-1&lt;/code&gt;, and the remaining
&lt;code&gt;FilterExec&lt;/code&gt; only handles the &lt;code&gt;event_type&lt;/code&gt; predicate. The &lt;code&gt;region&lt;/code&gt; filter has
been fully absorbed by your scan.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;After implementing pushdown for both filters&lt;/strong&gt; (both &lt;code&gt;Exact&lt;/code&gt;):&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-text"&gt;MyExecPlan: partitions=5, filter=[region = us-east-1 AND event_type = click]
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;No &lt;code&gt;FilterExec&lt;/code&gt; at all -- your source handles everything.&lt;/p&gt;
&lt;p&gt;Similarly, &lt;code&gt;EXPLAIN&lt;/code&gt; will reveal whether DataFusion is inserting unnecessary
&lt;code&gt;SortExec&lt;/code&gt; or &lt;code&gt;RepartitionExec&lt;/code&gt; nodes that you could eliminate by declaring
better output properties. Whenever your queries seem slower than expected,
&lt;code&gt;EXPLAIN&lt;/code&gt; is the first place to look.&lt;/p&gt;
&lt;h3 id="a-complete-filter-pushdown-example"&gt;A Complete Filter Pushdown Example&lt;a class="headerlink" href="#a-complete-filter-pushdown-example" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h3&gt;
&lt;p&gt;To make filter pushdown concrete, here is an illustrative example. Imagine a
table provider that reads from a set of date-partitioned directories on disk
(e.g., &lt;code&gt;data/2026-03-01/&lt;/code&gt;, &lt;code&gt;data/2026-03-02/&lt;/code&gt;, ...). Each directory contains
one or more Parquet files for that date. By pushing down a filter on the &lt;code&gt;date&lt;/code&gt;
column, the provider can skip entire directories -- avoiding the I/O of listing
and reading files that cannot possibly match the query.&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;/// A table provider backed by date-partitioned directories.
/// Each date directory contains data files; by filtering on the
/// `date` column we can skip entire directories of I/O.
struct DatePartitionedTable {
    schema: SchemaRef,
    /// Maps date strings ("2026-03-01") to directory paths
    partitions: HashMap&amp;lt;String, String&amp;gt;,
}

#[async_trait::async_trait]
impl TableProvider for DatePartitionedTable {
    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }
    fn schema(&amp;amp;self) -&amp;gt; SchemaRef { Arc::clone(&amp;amp;self.schema) }
    fn table_type(&amp;amp;self) -&amp;gt; TableType { TableType::Base }

    fn supports_filters_pushdown(
        &amp;amp;self,
        filters: &amp;amp;[&amp;amp;Expr],
    ) -&amp;gt; Result&amp;lt;Vec&amp;lt;TableProviderFilterPushDown&amp;gt;&amp;gt; {
        Ok(filters.iter().map(|f| {
            if Self::is_date_equality_filter(f) {
                // We can fully evaluate this: we will only read
                // directories matching the date, so no rows with
                // a different date will appear in the output.
                TableProviderFilterPushDown::Exact
            } else {
                TableProviderFilterPushDown::Unsupported
            }
        }).collect())
    }

    async fn scan(
        &amp;amp;self,
        _state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        // Determine which date partitions to read by inspecting
        // the pushed-down filters. This is the key optimization:
        // we decide *during planning* which directories to scan,
        // so that execution never touches irrelevant data.
        let dates_to_read: Vec&amp;lt;String&amp;gt; = self
            .extract_date_values(filters)
            .unwrap_or_else(||
                self.partitions.keys().cloned().collect()
            );

        let dirs: Vec&amp;lt;String&amp;gt; = dates_to_read
            .iter()
            .filter_map(|d| self.partitions.get(d).cloned())
            .collect();
        let num_dirs = dirs.len();

        Ok(Arc::new(DatePartitionedExec {
            schema: Arc::clone(&amp;amp;self.schema),
            directories: dirs,
            properties: PlanProperties::new(
                EquivalenceProperties::new(
                    Arc::clone(&amp;amp;self.schema),
                ),
                // One partition per date directory -- these
                // will be read in parallel.
                Partitioning::UnknownPartitioning(num_dirs),
                EmissionType::Incremental,
                Boundedness::Bounded,
            ),
        }))
    }
}

impl DatePartitionedTable {
    /// Check if a filter is an equality comparison on the `date` column.
    fn is_date_equality_filter(expr: &amp;amp;Expr) -&amp;gt; bool {
        // In practice, match on BinaryExpr { left, op: Eq, right }
        // and check if either side references the "date" column.
        // Simplified here for clarity.
        todo!("match on date equality expressions")
    }

    /// Extract date literal values from pushed-down equality filters.
    fn extract_date_values(&amp;amp;self, filters: &amp;amp;[Expr]) -&amp;gt; Option&amp;lt;Vec&amp;lt;String&amp;gt;&amp;gt; {
        // Parse filters like `date = '2026-03-01'` and return
        // the literal date strings. Returns None if no date
        // filters are present (meaning: read all partitions).
        todo!("extract date literals from filter expressions")
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;p&gt;The key insight is that the filter pushdown decision (&lt;code&gt;supports_filters_pushdown&lt;/code&gt;)
and the partition pruning (&lt;code&gt;scan()&lt;/code&gt;) work together: the first tells DataFusion
that a &lt;code&gt;FilterExec&lt;/code&gt; is unnecessary for the &lt;code&gt;date&lt;/code&gt; predicate, and the second
ensures that only the relevant directories are scanned. The actual file reading
happens later, in the stream produced by &lt;code&gt;execute()&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id="putting-it-all-together"&gt;Putting It All Together&lt;a class="headerlink" href="#putting-it-all-together" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;p&gt;Here is a minimal but complete example of a custom table provider that generates
data lazily during streaming:&lt;/p&gt;
&lt;pre&gt;&lt;code class="language-rust"&gt;use std::any::Any;
use std::sync::Arc;

use arrow::array::Int64Array;
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use datafusion::catalog::TableProvider;
use datafusion::common::Result;
use datafusion::datasource::TableType;
use datafusion::catalog::Session;
use datafusion::execution::SendableRecordBatchStream;
use datafusion::logical_expr::Expr;
use datafusion::physical_expr::EquivalenceProperties;
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
use datafusion::physical_plan::{
    ExecutionPlan, Partitioning, PlanProperties,
};
use futures::stream;

/// A table provider that generates sequential numbers on demand.
struct CountingTable {
    schema: SchemaRef,
    num_partitions: usize,
    rows_per_partition: usize,
}

impl CountingTable {
    fn new(num_partitions: usize, rows_per_partition: usize) -&amp;gt; Self {
        let schema = Arc::new(Schema::new(vec![
            Field::new("partition", DataType::Int64, false),
            Field::new("value", DataType::Int64, false),
        ]));
        Self { schema, num_partitions, rows_per_partition }
    }
}

#[async_trait::async_trait]
impl TableProvider for CountingTable {
    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }
    fn schema(&amp;amp;self) -&amp;gt; SchemaRef { Arc::clone(&amp;amp;self.schema) }
    fn table_type(&amp;amp;self) -&amp;gt; TableType { TableType::Base }

    async fn scan(
        &amp;amp;self,
        _state: &amp;amp;dyn Session,
        projection: Option&amp;lt;&amp;amp;Vec&amp;lt;usize&amp;gt;&amp;gt;,
        _filters: &amp;amp;[Expr],
        limit: Option&amp;lt;usize&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        // Light work only: build the plan with metadata
        Ok(Arc::new(CountingExec {
            schema: Arc::clone(&amp;amp;self.schema),
            num_partitions: self.num_partitions,
            rows_per_partition: limit
                .unwrap_or(self.rows_per_partition)
                .min(self.rows_per_partition),
            properties: PlanProperties::new(
                EquivalenceProperties::new(Arc::clone(&amp;amp;self.schema)),
                Partitioning::UnknownPartitioning(self.num_partitions),
                EmissionType::Incremental,
                Boundedness::Bounded,
            ),
        }))
    }
}

struct CountingExec {
    schema: SchemaRef,
    num_partitions: usize,
    rows_per_partition: usize,
    properties: PlanProperties,
}

impl ExecutionPlan for CountingExec {
    fn name(&amp;amp;self) -&amp;gt; &amp;amp;str { "CountingExec" }
    fn as_any(&amp;amp;self) -&amp;gt; &amp;amp;dyn Any { self }
    fn properties(&amp;amp;self) -&amp;gt; &amp;amp;PlanProperties { &amp;amp;self.properties }
    fn children(&amp;amp;self) -&amp;gt; Vec&amp;lt;&amp;amp;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; { vec![] }

    fn with_new_children(
        self: Arc&amp;lt;Self&amp;gt;,
        _children: Vec&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt;,
    ) -&amp;gt; Result&amp;lt;Arc&amp;lt;dyn ExecutionPlan&amp;gt;&amp;gt; {
        Ok(self)
    }

    fn execute(
        &amp;amp;self,
        partition: usize,
        _context: Arc&amp;lt;TaskContext&amp;gt;,
    ) -&amp;gt; Result&amp;lt;SendableRecordBatchStream&amp;gt; {
        let schema = Arc::clone(&amp;amp;self.schema);
        let rows = self.rows_per_partition;

        // The heavy work (data generation) happens inside the stream,
        // not here in execute().
        let batch_stream = stream::once(async move {
            let partitions = Int64Array::from(
                vec![partition as i64; rows],
            );
            let values = Int64Array::from(
                (0..rows as i64).collect::&amp;lt;Vec&amp;lt;_&amp;gt;&amp;gt;(),
            );
            let batch = RecordBatch::try_new(
                Arc::clone(&amp;amp;schema),
                vec![Arc::new(partitions), Arc::new(values)],
            )?;
            Ok(batch)
        });

        Ok(Box::pin(RecordBatchStreamAdapter::new(
            Arc::clone(&amp;amp;self.schema),
            batch_stream,
        )))
    }
}
&lt;/code&gt;&lt;/pre&gt;
&lt;h2 id="acknowledgements"&gt;Acknowledgements&lt;a class="headerlink" href="#acknowledgements" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;I would like to thank &lt;a href="https://rerun.io"&gt;Rerun.io&lt;/a&gt; for sponsoring the development of this work. Rerun is
building a data visualization system for Physical AI and makes heavy use of DataFusion
table providers for working with data analytics.&lt;/p&gt;
&lt;p&gt;I would also like to thank the reviewers of this post for their helpful feedback and
suggestions: &lt;a href="https://github.com/2010YOUY01"&gt;@2010YOUY01&lt;/a&gt;, &lt;a href="https://github.com/adriangb"&gt;@adriangb&lt;/a&gt;, &lt;a href="https://github.com/alamb"&gt;@alamb&lt;/a&gt;, &lt;a href="https://github.com/kevinjqliu"&gt;@kevinjqliu&lt;/a&gt;, &lt;a href="https://github.com/Omega359"&gt;@Omega359&lt;/a&gt;,
&lt;a href="https://github.com/pgwhalen"&gt;@pgwhalen&lt;/a&gt;, and &lt;a href="https://github.com/stuhood"&gt;@stuhood&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="get-involved"&gt;Get Involved&lt;a class="headerlink" href="#get-involved" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;p&gt;DataFusion is not a project built or driven by a single person, company, or
foundation. Our community of users and contributors works together to build a
shared technology that none of us could have built alone.&lt;/p&gt;
&lt;p&gt;If you are interested in joining us, we would love to have you. You can try out
DataFusion on some of your own data and projects and let us know how it goes,
contribute suggestions, documentation, bug reports, or a PR with documentation,
tests, or code. A list of open issues suitable for beginners is &lt;a href="https://github.com/apache/datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22"&gt;here&lt;/a&gt;, and you
can find out how to reach us on the &lt;a href="https://datafusion.apache.org/contributor-guide/communication.html"&gt;communication doc&lt;/a&gt;.&lt;/p&gt;
&lt;h2 id="further-reading"&gt;Further Reading&lt;a class="headerlink" href="#further-reading" title="Permanent link"&gt;¶&lt;/a&gt;&lt;/h2&gt;
&lt;hr/&gt;
&lt;ul&gt;
&lt;li&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html"&gt;&lt;code&gt;TableProvider&lt;/code&gt; API docs&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html"&gt;&lt;code&gt;ExecutionPlan&lt;/code&gt; API docs&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://docs.rs/datafusion/latest/datafusion/execution/type.SendableRecordBatchStream.html"&gt;&lt;code&gt;SendableRecordBatchStream&lt;/code&gt; API docs&lt;/a&gt;&lt;/li&gt;
&lt;li&gt;&lt;a href="https://github.com/apache/datafusion/tree/main/datafusion-examples/examples"&gt;DataFusion examples directory&lt;/a&gt; --
  contains working examples including custom table providers&lt;/li&gt;
&lt;/ul&gt;
&lt;hr/&gt;
&lt;p&gt;&lt;em&gt;Note: Portions of this blog post were written with the assistance of an AI agent.&lt;/em&gt;&lt;/p&gt;</content><category term="blog"/></entry></feed>