@@ -50,6 +50,9 @@ class from_textfile(Source):
5050 start: bool (False)
5151 Whether to start running immediately; otherwise call stream.start()
5252 explicitly.
53+ from_end: bool (False)
54+ Whether to begin streaming from the end of the file (i.e., only emit
55+ lines appended after the stream starts).
5356
5457 Example
5558 -------
@@ -63,26 +66,33 @@ class from_textfile(Source):
6366 Stream
6467 """
6568 def __init__ (self , f , poll_interval = 0.100 , delimiter = '\n ' , start = False ,
66- ** kwargs ):
69+ from_end = False , ** kwargs ):
6770 if isinstance (f , str ):
6871 f = open (f )
6972 self .file = f
73+ self .from_end = from_end
7074 self .delimiter = delimiter
7175
7276 self .poll_interval = poll_interval
7377 super (from_textfile , self ).__init__ (ensure_io_loop = True , ** kwargs )
7478 self .stopped = True
79+ self .started = False
7580 if start :
7681 self .start ()
7782
7883 def start (self ):
7984 self .stopped = False
85+ self .started = False
8086 self .loop .add_callback (self .do_poll )
8187
8288 @gen .coroutine
8389 def do_poll (self ):
8490 buffer = ''
85- while True :
91+ if self .from_end :
92+ # this only happens when we are ready to read
93+ self .file .seek (0 , 2 )
94+ while not self .stopped :
95+ self .started = True
8696 line = self .file .read ()
8797 if line :
8898 buffer = buffer + line
@@ -93,8 +103,6 @@ def do_poll(self):
93103 yield self ._emit (part + self .delimiter )
94104 else :
95105 yield gen .sleep (self .poll_interval )
96- if self .stopped :
97- break
98106
99107
100108@Stream .register_api (staticmethod )
@@ -147,6 +155,142 @@ def do_poll(self):
147155 break
148156
149157
158+ @Stream .register_api (staticmethod )
159+ class from_tcp (Source ):
160+ """
161+ Creates events by reading from a socket using tornado TCPServer
162+
163+ The stream of incoming bytes is split on a given delimiter, and the parts
164+ become the emitted events.
165+
166+ Parameters
167+ ----------
168+ port : int
169+ The port to open and listen on. It only gets opened when the source
170+ is started, and closed upon ``stop()``
171+ delimiter : bytes
172+ The incoming data will be split on this value. The resulting events
173+ will still have the delimiter at the end.
174+ start : bool
175+ Whether to immediately initiate the source. You probably want to
176+ set up downstream nodes first.
177+ server_kwargs : dict or None
178+ If given, additional arguments to pass to TCPServer
179+
180+ Example
181+ -------
182+
183+ >>> source = Source.from_tcp(4567) # doctest: +SKIP
184+ """
185+ def __init__ (self , port , delimiter = b'\n ' , start = False ,
186+ server_kwargs = None ):
187+ super (from_tcp , self ).__init__ (ensure_io_loop = True )
188+ self .stopped = True
189+ self .server_kwargs = server_kwargs or {}
190+ self .port = port
191+ self .server = None
192+ self .delimiter = delimiter
193+ if start : # pragma: no cover
194+ self .start ()
195+
196+ @gen .coroutine
197+ def _start_server (self ):
198+ from tornado .tcpserver import TCPServer
199+ from tornado .iostream import StreamClosedError
200+
201+ class EmitServer (TCPServer ):
202+ source = self
203+
204+ @gen .coroutine
205+ def handle_stream (self , stream , address ):
206+ while True :
207+ try :
208+ data = yield stream .read_until (self .source .delimiter )
209+ yield self .source ._emit (data )
210+ except StreamClosedError :
211+ break
212+
213+ self .server = EmitServer (** self .server_kwargs )
214+ self .server .listen (self .port )
215+
216+ def start (self ):
217+ if self .stopped :
218+ self .loop .add_callback (self ._start_server )
219+ self .stopped = False
220+
221+ def stop (self ):
222+ if not self .stopped :
223+ self .server .stop ()
224+ self .server = None
225+ self .stopped = True
226+
227+
228+ @Stream .register_api (staticmethod )
229+ class from_http_server (Source ):
230+ """Listen for HTTP POSTs on given port
231+
232+ Each connection will emit one event, containing the body data of
233+ the request
234+
235+ Parameters
236+ ----------
237+ port : int
238+ The port to listen on
239+ path : str
240+ Specific path to listen on. Can be regex, but content is not used.
241+ start : bool
242+ Whether to immediately startup the server. Usually you want to connect
243+ downstream nodes first, and then call ``.start()``.
244+ server_kwargs : dict or None
245+ If given, set of further parameters to pass on to HTTPServer
246+
247+ Example
248+ -------
249+ >>> source = Source.from_http_server(4567) # doctest: +SKIP
250+ """
251+
252+ def __init__ (self , port , path = '/.*' , start = False , server_kwargs = None ):
253+ self .port = port
254+ self .path = path
255+ self .server_kwargs = server_kwargs or {}
256+ super (from_http_server , self ).__init__ (ensure_io_loop = True )
257+ self .stopped = True
258+ self .server = None
259+ if start : # pragma: no cover
260+ self .start ()
261+
262+ def _start_server (self ):
263+ from tornado .web import Application , RequestHandler
264+ from tornado .httpserver import HTTPServer
265+
266+ class Handler (RequestHandler ):
267+ source = self
268+
269+ @gen .coroutine
270+ def post (self ):
271+ yield self .source ._emit (self .request .body )
272+ self .write ('OK' )
273+
274+ application = Application ([
275+ (self .path , Handler ),
276+ ])
277+ self .server = HTTPServer (application , ** self .server_kwargs )
278+ self .server .listen (self .port )
279+
280+ def start (self ):
281+ """Start HTTP server and listen"""
282+ if self .stopped :
283+ self .loop .add_callback (self ._start_server )
284+ self .stopped = False
285+
286+ def stop (self ):
287+ """Shutdown HTTP server"""
288+ if not self .stopped :
289+ self .server .stop ()
290+ self .server = None
291+ self .stopped = True
292+
293+
150294@Stream .register_api (staticmethod )
151295class from_kafka (Source ):
152296 """ Accepts messages from Kafka
0 commit comments