mirror of
https://github.com/deepseek-ai/smallpond
synced 2025-06-26 18:27:45 +00:00
init
This commit is contained in:
81
docs/source/api.rst
Normal file
81
docs/source/api.rst
Normal 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
|
||||
104
docs/source/api/dataframe.rst
Normal file
104
docs/source/api/dataframe.rst
Normal 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
|
||||
28
docs/source/api/dataset.rst
Normal file
28
docs/source/api/dataset.rst
Normal 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
|
||||
85
docs/source/api/execution.rst
Normal file
85
docs/source/api/execution.rst
Normal 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
89
docs/source/api/nodes.rst
Normal 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
76
docs/source/api/tasks.rst
Normal 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
44
docs/source/conf.py
Normal 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("../.."))
|
||||
83
docs/source/getstarted.rst
Normal file
83
docs/source/getstarted.rst
Normal 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
27
docs/source/index.rst
Normal 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
37
docs/source/internals.rst
Normal 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.
|
||||
Reference in New Issue
Block a user