Hi all,
Thanks for all the support!

I put together a rough working version of this already and it was quite
easy, even for a Java newb.

After playing with it a little I was surprised to find that:

A) completed jobs are not cleared from the job service
B) job info is not persisted between restarts of the job service

It seems like this adds up to a memory leak which can only be resolved by
restarting the service and thereby losing information about jobs which may
be actively running.  How are people dealing with this currently?

Note, I'm referring to the InMemoryJobService that's started like this:

./gradlew :beam-runners-flink-1.8-job-server:runShadow
-PflinkMasterUrl=localhost:8081

Thanks for the tip on the .dot graph exporter!  That will come in handy.

-chad


On Wed, Jun 26, 2019 at 6:39 AM Tim Robertson <[email protected]>
wrote:

> Another +1 to support your research into this Chad. Thank you.
>
> Trying to understand where a beam process is in the Spark DAG is... not
> easy. A UI that helped would be a great addition.
>
>
>
> On Wed, Jun 26, 2019 at 3:30 PM Ismaël Mejía <[email protected]> wrote:
>
>> +1 don't hesitate to create a JIRA + PR. You may be interested in [1].
>> This is a simple util class that takes a proto pipeline object and
>> converts it into its graph representation in .dot format. You can
>> easily reuse the code or the idea as a first approach to show what the
>> pipeline is about.
>>
>> [1]
>> https://github.com/apache/beam/blob/2df702a1448fa6cbd22cd225bf16e9ffc4c82595/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRenderer.java#L29
>>
>> On Wed, Jun 26, 2019 at 10:27 AM Robert Bradshaw <[email protected]>
>> wrote:
>> >
>> > Yes, offering a way to get a pipeline from the job service directly
>> > would be a completely reasonable thing to do (and likely not hard at
>> > all). We welcome pull requests.
>> >
>> > Alternative UIs built on top of this abstraction would be an
>> > interesting project to explore.
>> >
>> > On Wed, Jun 26, 2019 at 8:44 AM Chad Dombrova <[email protected]>
>> wrote:
>> > >
>> > > Hi all,
>> > > I've been poking around the beam source code trying to determine
>> whether it's possible to get the definition of a pipeline via beam's
>> gPRC-based services.   It looks like the message types are there for
>> describing a Pipeline but as far as I can tell, they're only used by
>> JobService.Prepare() for submitting a new job.
>> > >
>> > > If I were to create a PR to add support for a
>> JobService.GetPipeline() method, would that be interesting to others?  Is
>> it technically feasible?  i.e. is the pipeline definition readily available
>> to the job service after the job has been prepared and sent to the runner?
>> > >
>> > > Bigger picture, what I'm thinking about is writing a UI that's
>> designed to view and monitor Beam pipelines via the portability
>> abstraction, rather than using the (rather clunky) UIs that come with
>> runners like Flink and Dataflow.  My thinking is that using beam's
>> abstractions would future proof the UI by allowing it to work with any
>> portable runner.  Right now it's just an idea, so I'd love to know what
>> others think of this.
>> > >
>> > > thanks!
>> > > -chad
>> > >
>>
>

Reply via email to