Proposal: Publishing and Retrieving SDFs for SDF Checkpointing
See original GitHub issue[Copy-pasting part of the description of #340]
To checkpoint the state of an SDF, we need to do 3 things:
- When a job starts, retrieve (either from Dask datasets/S3/disk) the last known SDF state (if any) from previous runs. This can just be a one-time function call to a UDF outside of streamz.
- Pass this retrieved state into an SDF so that the new run of this streamz job can pick up from where it left off — basically, allow SDFs to accept an initial state.
- Publish (either to Dask/S3/disk) the updated SDF state at the end of every batch.
The code for step 2, i.e., SDFs to accept an initial state, was merged by the #340 above. This issue is to invite ideas for how steps 1 and 3 (mainly 3) above should be implemented.
I was able to “manually” publish SDFs at the end of every batch using Dask’s published datasets. Here’s a simple example of a streamz job which retrieves the SDF checkpoint stored as a Dask dataset, uses it to initialize an SDF operation, and then updates this dataset after every batch. In case the job crashes, the stream restarts using the last known SDF checkpoint dataset published to Dask.
from distributed import Client, LocalCluster
from streamz import Stream
from streamz.dataframe import DataFrame
import pandas as pd
# Dask Client
client = Client("localhost:8786")
client.get_versions(check=True)
client
def get_SDF_checkpoint(key):
client_x = Client("localhost:8786")
datasets = client_x.list_datasets()
res = None
if key in datasets:
res = client_x.get_dataset(key)
client_x.close()
return res
# Retrieve last known SDF checkpoint, if any
start_state = get_SDF_checkpoint("sdf_checkpoint")
# UDF to publish SDF state onto Dask (we would need a generic interface for other sinks like S3)
def publish_SDF_checkpoint(res, key):
client_x = Client("localhost:8786")
datasets = client_x.list_datasets()
# For some groupby aggs
if type(res) is not tuple:
res = (res, res)
if res[0] is None:
return res[1]
if key in datasets:
client_x.unpublish_dataset(key)
client_x.publish_dataset(res[0], name=key)
client_x.close()
return res[1]
# Kafka consumer Configs
topic = "custreamz-test"
bootstrap_servers = 'localhost:9092'
consumer_conf = {
"bootstrap.servers": bootstrap_servers,
"enable.partition.eof": "true",
"group.id": "custreamz-test",
"auto.offset.reset":"latest"
}
# Start a stream from Kafka
source = Stream.from_kafka_batched(topic, consumer_conf, poll_interval='20s',
npartitions=4, max_batch_size=10,
asynchronous=True, dask=True)
def preprocess(messages):
json_input_string = "\n".join([msg.decode('utf-8') for msg in messages])
df = pd.read_json(json_input_string, lines=True)
return df
# Preprocess data
stream = source.map(preprocess)
# Create an SDF
example = pd.DataFrame({'Name':[], 'Amount':[]})
sdf = DataFrame(stream, example=example)
# Window Function for SDFs
def window_func(window_gdf):
# Handle state downstream
state = None
if isinstance(window_gdf, tuple):
state = window_gdf[0]
window_gdf = window_gdf[1]
aggs = window_gdf.groupby(["Name"]).agg({"Amount":"sum"}).reset_index()
return state, aggs
# Apply window function on SDF. Uses the retrieved state to initialize the stream.
postproc_stream = sdf.window(5, with_state=True, start=start_state).apply(window_func).stream
# Publish updated SDF at the end of every batch and gather output
output = postproc_stream.map(publish_SDF_checkpoint, "sdf_checkpoint").gather().sink_to_list()
# Start the stream
source.start()
Simple Producer script:
import confluent_kafka as ck
producer_conf = {'bootstrap.servers': bootstrap_servers, 'compression.type':'snappy'}
producer = ck.Producer(producer_conf)
producer.produce(topic, '{"Name":"Alice", "Amount":100}')
producer.produce(topic, '{"Name":"Tom", "Amount":200}')
producer.produce(topic, '{"Name":"Linda", "Amount":300}')
producer.produce(topic, '{"Name":"Bob", "Amount":50}')
producer.produce(topic, '{"Name":"Alice", "Amount":400}')
producer.flush()
Every time a batch is processed, we can see the final result of the streamz job using output[-1]
and the last saved SDF checkpoint Dask dataset using get_SDF_checkpoint("sdf_checkpoint")
.
Everything works perfectly functionality-wise, other than the fact that in the above script, the SDF checkpoint and the Kafka checkpoint are out-of-sync. The SDF publishing happens before the Kafka checkpointing, which is not how it should ideally be. The easiest way to tie them up would be to add the SDF state to the metadata
passed down the stream so that when the Kafka checkpointing happens, the callback would also trigger publishing the SDF onto Dask/S3.
I would like to hear about whether this would be the correct way to implement this feature? If yes, any suggestions on how this should be done cleanly and efficiently?
Issue Analytics
- State:
- Created 3 years ago
- Comments:33 (32 by maintainers)
I’d like to write up the code and send over a PR. I just want to make sure that no one and any objections here.
I am not convinced that the current internal state can be considered as part of the metadata, it feels separate; but I can see why you would want to reduce the number of output types.