Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Added to_kafka directly from a Dask worker #279

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
51 changes: 48 additions & 3 deletions streamz/dask.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,13 +2,13 @@

from operator import getitem

from tornado import gen

from dask.compatibility import apply
from dask.distributed import get_worker
from distributed.client import default_client
from tornado import gen

from .core import Stream
from . import core, sources
from .core import Stream


class DaskStream(Stream):
Expand Down Expand Up @@ -198,3 +198,48 @@ class filenames(DaskStream, sources.filenames):
@DaskStream.register_api(staticmethod)
class from_textfile(DaskStream, sources.from_textfile):
pass


@DaskStream.register_api()
class to_kafka(DaskStream):
""" Write values to Kafka directly from the Dask worker

Parameters
----------
topic : string
The topic which to write
producer_config : dict
Settings to set up the stream, see
https://docs.confluent.io/current/clients/confluent-kafka-python/#configuration
https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md
Examples:
bootstrap.servers: Connection string (host:port) to Kafka

"""
def __init__(self, upstream, topic, producer_config, **kwargs):
self.topic = topic
self.producer_config = producer_config

stream_name = kwargs.pop('stream_name', None)
DaskStream.__init__(self, upstream, stream_name=stream_name)

def update(self, x, who=None):

def get_producer(config):
w = get_worker()
if hasattr(w, 'producer'):
return w.producer

import confluent_kafka as ck
w.producer = getattr(ck, 'Producer')(config)
return w.producer

def produce(topic, value, producer_config):
producer = get_producer(producer_config)
producer.produce(topic, value)
producer.flush()
return value

client = default_client()
result = client.submit(produce, self.topic, x, self.producer_config)
self._emit(result)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this call self.emit? That way it integrates with the async support?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed. Anything else that comes to mind that needs change, @CJ-Wright?