Hi, Not directly connected ( its for java sdk ) but some of the concepts in these materials maybe useful:
https://cloud.google.com/blog/products/gcp/running-external-libraries-with-cloud-dataflow-for-grid-computing-workloads https://github.com/apache/beam/tree/master/examples/java/src/main/java/org/apache/beam/examples/subprocess https://cloud.google.com/solutions/running-external-binaries-beam-grid-computing On 25 October 2018 at 04:23, Jeff Klukas <[email protected]> wrote: > Another option here would be to make the perl script operate on batches. > Your DoFn could then store the records to a buffer rather than outputting > them and then periodically flush the buffer, sending records through the > perl script and sending to output. > > On Wed, Oct 24, 2018 at 3:03 PM Robert Bradshaw <[email protected]> > wrote: > >> While one does want to watch out for expensive per-record operations, >> this may still be preferable to (and cheaper than) setting up a server and >> making RPC requests. It depends on the nature of the operation. If >> executing the perl script is (say) 100ms of "startup" for 1ms of actually >> processing $DATA, then you'll be wasting a lot of cycles and a server may >> be the way to go, but if it's 1ms of startup for 100ms of processing $DATA >> than this startup cost won't matter at all. >> >> If the startup cost is prohibitive, you could also start up a local >> "server" on the worker in startBundle (or even setUp), and shut it down in >> finishBundle, and communicate with it in your processElement. >> >> The other bit is actually shipping your perl script (and, more tricky, >> its dependencies). Currently that's very runner-dependent, and typically >> you end up packing it as data in your jars and then trying to >> unpack/install it on the workers at runtime. One of the goals of >> https://beam.apache.org/contribute/portability/ is to make this easier, >> specifically, you can set up your worker environment as a docker container >> with everything you need and this will get used as the environment in which >> your DoFns are executed. >> >> >> On Wed, Oct 24, 2018 at 6:48 AM Sobhan Badiozamany < >> [email protected]> wrote: >> >>> Hi Nima, >>> >>> I think the answer depends on the use-case, but what you suggest is on >>> the list of practices that hurt scalability of pipelines as it will be an >>> example of “Expensive Per-Record Operations”, look it up here: >>> https://cloud.google.com/blog/products/gcp/writing-dataflow- >>> pipelines-with-scalability-in-mind >>> >>> Cheers, >>> Sobi >>> >>> Sent from my iPhone >>> >>> On Oct 23, 2018, at 23:35, Nima Mousavi <[email protected]> wrote: >>> >>> Hi, >>> >>> We have a dataflow pipeline written in Apache python beam, and are >>> wondering if we can run a third party code (written in perl) in the >>> pipeline. We basically want to run >>> >>> perl myscript.pl $DATA >>> >>> for every DATA in a PCollection passed to a DoFn >>> >>> and write the result back into Bigquery. We could have setup a server >>> for myscript.pl, and send HTTP/RPC request to the server from each >>> worker instead. But we are wondering if it is possible to run the script >>> directly inside the Beam worker? Or even through a docker container >>> packaging our perl script? If yes, how? what do you think of this approach? >>> Any caveat we should be aware of? >>> >>> Thanks! >>> >>> -- This email may be confidential and privileged. If you received this communication by mistake, please don't forward it to anyone else, please erase all copies and attachments, and please let me know that it has gone to the wrong person. The above terms reflect a potential business arrangement, are provided solely as a basis for further discussion, and are not intended to be and do not constitute a legally binding obligation. No legally binding obligations will be created, implied, or inferred until an agreement in final form is executed in writing by all parties involved.
