This commit is contained in:
Runji Wang
2025-02-25 18:16:31 +08:00
commit 770aa417d5
77 changed files with 18785 additions and 0 deletions

81
docs/source/api.rst Normal file
View File

@@ -0,0 +1,81 @@
API Reference
=============
Smallpond provides both high-level and low-level APIs.
.. note::
Currently, smallpond provides two different APIs, supporting dynamic and static construction of data flow graphs respectively. Due to historical reasons, these two APIs use different scheduler backends and support different configuration options.
- The High-level API currently uses Ray as the backend, supporting dynamic construction and execution of data flow graphs.
- The Low-level API uses a built-in scheduler and only supports one-time execution of static data flow graphs. However, it offers more performance optimizations and richer configuration options.
We are working to merge them so that in the future, you can use a unified high-level API and freely choose between Ray or the built-in scheduler.
High-level API
--------------
The high-level API is centered around :ref:`dataframe`. It allows dynamic construction of data flow graphs, execution, and result retrieval.
A typical workflow looks like this:
.. code-block:: python
import smallpond
sp = smallpond.init()
df = sp.read_parquet("path/to/dataset/*.parquet")
df = df.repartition(10)
df = df.map("x + 1")
df.write_parquet("path/to/output")
.. toctree::
:maxdepth: 2
api/dataframe
It is recommended to use the DataFrame API.
Low-level API
-------------
In the low-level API, users manually create :ref:`nodes` to construct static data flow graphs, then submit them to smallpond to generate :ref:`tasks` and wait for all tasks to complete.
A complete example is shown below.
.. code-block:: python
from smallpond.logical.dataset import ParquetDataSet
from smallpond.logical.node import Context, DataSourceNode, DataSetPartitionNode, SqlEngineNode, LogicalPlan
from smallpond.execution.driver import Driver
def my_pipeline(input_paths: List[str], npartitions: int):
ctx = Context()
dataset = ParquetDataSet(input_paths)
node = DataSourceNode(ctx, dataset)
node = DataSetPartitionNode(ctx, (node,), npartitions=npartitions)
node = SqlEngineNode(ctx, (node,), "SELECT * FROM {0}")
return LogicalPlan(ctx, node)
if __name__ == "__main__":
driver = Driver()
driver.add_argument("-i", "--input_paths", nargs="+")
driver.add_argument("-n", "--npartitions", type=int, default=10)
plan = my_pipeline(**driver.get_arguments())
driver.run(plan)
To run this script:
.. code-block:: bash
python script.py -i "path/to/*.parquet" -n 10
.. toctree::
:maxdepth: 2
api/dataset
api/nodes
api/tasks
api/execution

View File

@@ -0,0 +1,104 @@
.. _dataframe:
DataFrame
=========
DataFrame is the main class in smallpond. It represents a lazily computed, partitioned data set.
A typical workflow looks like this:
.. code-block:: python
import smallpond
sp = smallpond.init()
df = sp.read_parquet("path/to/dataset/*.parquet")
df = df.repartition(10)
df = df.map("x + 1")
df.write_parquet("path/to/output")
Initialization
--------------
.. autosummary::
:toctree: ../generated
smallpond.init
.. currentmodule:: smallpond.dataframe
.. _loading_data:
Loading Data
------------
.. autosummary::
:toctree: ../generated
Session.from_items
Session.from_arrow
Session.from_pandas
Session.read_csv
Session.read_json
Session.read_parquet
.. _partitioning_data:
Partitioning Data
-----------------
.. autosummary::
:toctree: ../generated
DataFrame.repartition
.. _transformations:
Transformations
---------------
Apply transformations and return a new DataFrame.
.. autosummary::
:toctree: ../generated
Session.partial_sql
DataFrame.map
DataFrame.map_batches
DataFrame.flat_map
DataFrame.filter
DataFrame.limit
DataFrame.partial_sort
DataFrame.random_shuffle
.. _consuming_data:
Consuming Data
--------------
These operations will trigger execution of the lazy transformations performed on this DataFrame.
.. autosummary::
:toctree: ../generated
DataFrame.count
DataFrame.take
DataFrame.take_all
DataFrame.to_arrow
DataFrame.to_pandas
DataFrame.write_parquet
DataFrame.write_parquet_lazy
Execution
---------
DataFrames are lazily computed. You can use these methods to manually trigger computation.
.. autosummary::
:toctree: ../generated
DataFrame.compute
DataFrame.is_computed
DataFrame.recompute
Session.wait

View File

@@ -0,0 +1,28 @@
.. currentmodule:: smallpond.logical.dataset
Dataset
=======
Dataset represents a collection of files.
To create a dataset:
.. code-block:: python
dataset = ParquetDataSet("path/to/dataset/*.parquet")
DataSets
--------
.. autosummary::
:toctree: ../generated
DataSet
FileSet
ParquetDataSet
CsvDataSet
JsonDataSet
ArrowTableDataSet
PandasDataSet
PartitionedDataSet
SqlQueryDataSet

View File

@@ -0,0 +1,85 @@
.. currentmodule:: smallpond.execution
.. _execution:
Execution
=========
Submit a Job
------------
After constructing the LogicalPlan, you can use the JobManager to create a Job in the cluster to execute it. However, in most cases, you only need to use the Driver as the entry point of the entire script and then submit the plan. The Driver is a simple wrapper around the JobManager. It reads the configuration from the command line arguments and passes it to the JobManager.
.. code-block:: python
from smallpond.execution.driver import Driver
if __name__ == "__main__":
driver = Driver()
# add your own arguments
driver.add_argument("-i", "--input_paths", nargs="+")
driver.add_argument("-n", "--npartitions", type=int, default=10)
# build and run logical plan
plan = my_pipeline(**driver.get_arguments())
driver.run(plan)
.. autosummary::
:toctree: ../generated
~driver.Driver
~manager.JobManager
Scheduler and Executor
----------------------
Scheduler and Executor are lower-level APIs. They are directly responsible for scheduling and executing tasks, respectively. Generally, users do not need to use them directly.
.. autosummary::
:toctree: ../generated
~scheduler.Scheduler
~executor.Executor
.. _platform:
Customize Platform
------------------
Smallpond supports user-defined task execution platforms. A Platform includes methods for submitting jobs and a series of default configurations. By default, smallpond automatically detects the current environment and selects the most suitable platform. If it cannot detect one, it uses the default platform.
You can specify a built-in platform via parameters:
.. code-block:: bash
# run with your platform
python script.py --platform mpi
Or implement your own Platform class:
.. code-block:: python
# path/to/my/platform.py
from smallpond.platform import Platform
class MyPlatform(Platform):
def start_job(self, ...) -> List[str]:
...
.. code-block:: bash
# run with your platform
# if using Driver
python script.py --platform path.to.my.platform
# if using smallpond.init
SP_PLATFORM=path.to.my.platform python script.py
.. currentmodule:: smallpond
.. autosummary::
:toctree: ../generated
~platform.Platform
~platform.MPI

89
docs/source/api/nodes.rst Normal file
View File

@@ -0,0 +1,89 @@
.. currentmodule:: smallpond.logical.node
.. _nodes:
Nodes
=====
Nodes represent the fundamental building blocks of a data processing pipeline. Each node encapsulates a specific operation or transformation that can be applied to a dataset.
Nodes can be chained together to form a logical plan, which is a directed acyclic graph (DAG) of nodes that represent the overall data processing workflow.
A typical workflow to create a logical plan is as follows:
.. code-block:: python
# Create a global context
ctx = Context()
# Create a dataset
dataset = ParquetDataSet("path/to/dataset/*.parquet")
# Create a data source node
node = DataSourceNode(ctx, dataset)
# Partition the data
node = DataSetPartitionNode(ctx, (node,), npartitions=2)
# Create a SQL engine node to transform the data
node = SqlEngineNode(ctx, (node,), "SELECT * FROM {0}")
# Create a logical plan from the root node
plan = LogicalPlan(ctx, node)
You can then create tasks from the logical plan, see :ref:`tasks`.
Notable properties of Node:
1. Nodes are partitioned. Each Node generates a series of tasks, with each task processing one partition of data.
2. The input and output of a Node are a series of partitioned Datasets. A Node may write data to shared storage and return a new Dataset, or it may simply recombine the input Datasets.
Context
-------
.. autosummary::
:toctree: ../generated
Context
NodeId
LogicalPlan
-----------
.. autosummary::
:toctree: ../generated
LogicalPlan
LogicalPlanVisitor
.. Planner
Nodes
-----
.. autosummary::
:toctree: ../generated
Node
DataSetPartitionNode
ArrowBatchNode
ArrowComputeNode
ArrowStreamNode
ConsolidateNode
DataSinkNode
DataSourceNode
EvenlyDistributedPartitionNode
HashPartitionNode
LimitNode
LoadPartitionedDataSetNode
PandasBatchNode
PandasComputeNode
PartitionNode
ProjectionNode
PythonScriptNode
RangePartitionNode
RepeatPartitionNode
RootNode
ShuffleNode
SqlEngineNode
UnionNode
UserDefinedPartitionNode
UserPartitionedDataSourceNode

76
docs/source/api/tasks.rst Normal file
View File

@@ -0,0 +1,76 @@
.. currentmodule:: smallpond.execution.task
.. _tasks:
Tasks
=====
.. code-block:: python
# create a runtime context
runtime_ctx = RuntimeContext(JobId.new(), data_root)
runtime_ctx.initialize(socket.gethostname(), cleanup_root=True)
# create a logical plan
plan = create_logical_plan()
# create an execution plan
planner = Planner(runtime_ctx)
exec_plan = planner.create_exec_plan(plan)
You can then execute the tasks in a scheduler, see :ref:`execution`.
RuntimeContext
--------------
.. autosummary::
:toctree: ../generated
RuntimeContext
JobId
TaskId
TaskRuntimeId
PartitionInfo
PerfStats
ExecutionPlan
-------------
.. autosummary::
:toctree: ../generated
ExecutionPlan
Tasks
-----
.. autosummary::
:toctree: ../generated
Task
ArrowBatchTask
ArrowComputeTask
ArrowStreamTask
DataSinkTask
DataSourceTask
EvenlyDistributedPartitionProducerTask
HashPartitionArrowTask
HashPartitionDuckDbTask
HashPartitionTask
LoadPartitionedDataSetProducerTask
MergeDataSetsTask
PandasBatchTask
PandasComputeTask
PartitionConsumerTask
PartitionProducerTask
ProjectionTask
PythonScriptTask
RangePartitionTask
RepeatPartitionProducerTask
RootTask
SplitDataSetTask
SqlEngineTask
UserDefinedPartitionProducerTask

44
docs/source/conf.py Normal file
View File

@@ -0,0 +1,44 @@
# Configuration file for the Sphinx documentation builder.
#
# For the full list of built-in configuration values, see the documentation:
# https://www.sphinx-doc.org/en/master/usage/configuration.html
# -- Project information -----------------------------------------------------
# https://www.sphinx-doc.org/en/master/usage/configuration.html#project-information
project = "smallpond"
copyright = "2025, deepseek"
author = "deepseek"
# -- General configuration ---------------------------------------------------
# https://www.sphinx-doc.org/en/master/usage/configuration.html#general-configuration
extensions = [
"sphinx.ext.autodoc",
"sphinx.ext.autosummary",
]
templates_path = ["_templates"]
exclude_patterns = []
# -- Options for HTML output -------------------------------------------------
# https://www.sphinx-doc.org/en/master/usage/configuration.html#options-for-html-output
html_theme = "pydata_sphinx_theme"
html_static_path = ["_static"]
html_theme_options = {
"icon_links": [
{
"name": "GitHub",
"url": "https://github.com/deepseek-ai/smallpond",
"icon": "fa-brands fa-square-github",
}
]
}
import os
import sys
sys.path.insert(0, os.path.abspath("../.."))

View File

@@ -0,0 +1,83 @@
Getting Started
===============
Installation
------------
Python 3.8+ is required.
.. code-block:: bash
pip install smallpond
Initialization
--------------
The first step is to initialize the smallpond session:
.. code-block:: python
import smallpond
sp = smallpond.init()
Loading Data
------------
Create a DataFrame from a set of files:
.. code-block:: python
df = sp.read_parquet("path/to/dataset/*.parquet")
To learn more about loading data, please refer to :ref:`loading_data`.
Partitioning Data
-----------------
Smallpond requires users to manually specify data partitions for now.
.. code-block:: python
df = df.repartition(3) # repartition by files
df = df.repartition(3, by_row=True) # repartition by rows
df = df.repartition(3, hash_by="host") # repartition by hash of column
To learn more about partitioning data, please refer to :ref:`partitioning_data`.
Transforming Data
-----------------
Apply python functions or SQL expressions to transform data.
.. code-block:: python
df = df.map('a + b as c')
df = df.map(lambda row: {'c': row['a'] + row['b']})
To learn more about transforming data, please refer to :ref:`transformations`.
Saving Data
-----------
Save the transformed data to a set of files:
.. code-block:: python
df.write_parquet("path/to/output")
To learn more about saving data, please refer to :ref:`consuming_data`.
Monitoring
----------
Smallpond uses `Ray Core`_ as the task scheduler. You can use `Ray Dashboard`_ to monitor the task execution.
.. _Ray Core: https://docs.ray.io/en/latest/ray-core/walkthrough.html
.. _Ray Dashboard: https://docs.ray.io/en/latest/ray-observability/getting-started.html
When smallpond starts, it will print the Ray Dashboard URL:
.. code-block:: bash
... Started a local Ray instance. View the dashboard at http://127.0.0.1:8008

27
docs/source/index.rst Normal file
View File

@@ -0,0 +1,27 @@
smallpond
=========
Smallpond is a lightweight distributed data processing framework.
It uses `duckdb`_ as the compute engine and stores data in `parquet`_ format on a distributed file system (e.g. `3FS`_).
.. _duckdb: https://duckdb.org/
.. _parquet: https://parquet.apache.org/
.. _3FS: https://github.com/deepseek-ai/3fs
Why smallpond?
--------------
- **Performance**: Smallpond uses DuckDB to deliver native-level performance for efficient data processing.
- **Scalability**: Leverages high-performance distributed file systems for intermediate storage, enabling PB-scale data handling without memory bottlenecks.
- **Simplicity**: No long-running services or complex dependencies, making it easy to deploy and maintain.
.. toctree::
:maxdepth: 1
getstarted
internals
.. toctree::
:maxdepth: 3
api

37
docs/source/internals.rst Normal file
View File

@@ -0,0 +1,37 @@
Internals
=========
Data Root
---------
Smallpond stores all data in a single directory called data root.
This directory has the following structure:
.. code-block:: bash
data_root
└── 2024-12-11-12-00-28.2cc39990-296f-48a3-8063-78cf6dca460b # job_time.job_id
├── config # configuration and state
│ ├── exec_plan.pickle
│ ├── logical_plan.pickle
│ └── runtime_ctx.pickle
├── log # logs
│ ├── graph.png
│ └── scheduler.log
├── queue # message queue between scheduler and workers
├── output # output data
├── staging # intermediate data
│ ├── DataSourceTask.000001
│ ├── EvenlyDistributedPartitionProducerTask.000002
│ ├── completed_tasks # output dataset of completed tasks
│ └── started_tasks # used for checkpoint
└── temp # temporary data
├── DataSourceTask.000001
└── EvenlyDistributedPartitionProducerTask.000002
Failure Recovery
----------------
Smallpond can recover from failure and resume execution from the last checkpoint.
Checkpoint is task-level. A few tasks, such as `ArrowBatchTask`, support checkpointing at the batch level.