flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stephan Ewen <se...@apache.org>
Subject Re: Multiple restarts of Local Cluster
Date Thu, 03 Sep 2015 19:19:58 GMT
Stopping the JVM process clean up all resources, except temp files.

Everything that creates temp files uses a shutdown hook to remove these:
IOManager, BlobManager, LibraryCache, ...

On Wed, Sep 2, 2015 at 7:40 PM, Sachin Goel <sachingoel0101@gmail.com>
wrote:

> I'm not sure what you mean by "Crucial cleanup is in shutdown hooks".
> Could you elaborate?
>
> -- Sachin Goel
> Computer Science, IIT Delhi
> m. +91-9871457685
>
> On Wed, Sep 2, 2015 at 10:25 PM, Stephan Ewen <sewen@apache.org> wrote:
>
>> You can always shut down a cluster manually (via shutdown()) and if the
>> JVM simply exists, all is well as well. Crucial cleanup is in shutdown
>> hooks.
>>
>> On Wed, Sep 2, 2015 at 6:22 PM, Till Rohrmann <till.rohrmann@gmail.com>
>> wrote:
>>
>>> If I'm not mistaken, then the cluster should be properly terminated when
>>> it gets garbage collected. Thus, also when the main method exits.
>>>
>>> On Wed, Sep 2, 2015 at 6:14 PM, Sachin Goel <sachingoel0101@gmail.com>
>>> wrote:
>>>
>>>> If I'm right, all Tests use either the MultipleProgramTestBase or
>>>> JavaProgramTestBase​. Those shut down the cluster explicitly anyway.
>>>> I will make sure if this is the case.
>>>>
>>>> Regards
>>>> Sachin
>>>>
>>>> -- Sachin Goel
>>>> Computer Science, IIT Delhi
>>>> m. +91-9871457685
>>>>
>>>> On Wed, Sep 2, 2015 at 9:40 PM, Till Rohrmann <trohrmann@apache.org>
>>>> wrote:
>>>>
>>>>> Maybe we can create a single PlanExecutor for the LocalEnvironment
>>>>> which is used when calling execute. This of course entails that we
>>>>> don’t call stop on the LocalCluster. For cases where the program
>>>>> exits after calling execute, this should be fine because all resources
will
>>>>> then be released anyway. It might matter for the test execution where
maven
>>>>> reuses the JVMs and where the LocalFlinkMiniCluster won’t be garbage
>>>>> collected right away. You could try it out and see what happens.
>>>>>
>>>>> Cheers,
>>>>> Till
>>>>> ​
>>>>>
>>>>> On Wed, Sep 2, 2015 at 6:03 PM, Till Rohrmann <trohrmann@apache.org>
>>>>> wrote:
>>>>>
>>>>>> Oh sorry, then I got the wrong context. I somehow thought it was
>>>>>> about test cases because I read `MultipleProgramTestBase` etc. Sorry
my bad.
>>>>>>
>>>>>> On Wed, Sep 2, 2015 at 6:00 PM, Sachin Goel <sachingoel0101@gmail.com
>>>>>> > wrote:
>>>>>>
>>>>>>> I was under the impression that the @AfterClass annotation can
only
>>>>>>> be used in test classes.
>>>>>>> Even so, the idea is that a user program running in the IDE should
>>>>>>> not be starting up the cluster several times [my primary concern
is the
>>>>>>> addition of the persist operator], and we certainly cannot ask
the user to
>>>>>>> terminate the cluster after execution, while in local mode.
>>>>>>>
>>>>>>> -- Sachin Goel
>>>>>>> Computer Science, IIT Delhi
>>>>>>> m. +91-9871457685
>>>>>>>
>>>>>>> On Wed, Sep 2, 2015 at 9:19 PM, Till Rohrmann <trohrmann@apache.org>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Why is it not possible to shut down the local cluster? Can’t
you
>>>>>>>> shut it down in the @AfterClass method?
>>>>>>>> ​
>>>>>>>>
>>>>>>>> On Wed, Sep 2, 2015 at 4:56 PM, Sachin Goel <
>>>>>>>> sachingoel0101@gmail.com> wrote:
>>>>>>>>
>>>>>>>>> Yes. That will work too. However, then it isn't possible
to shut
>>>>>>>>> down the local cluster. [Is it necessary to do so or
does it shut down
>>>>>>>>> automatically when the program exists? I'm not entirely
sure.]
>>>>>>>>>
>>>>>>>>> -- Sachin Goel
>>>>>>>>> Computer Science, IIT Delhi
>>>>>>>>> m. +91-9871457685
>>>>>>>>>
>>>>>>>>> On Wed, Sep 2, 2015 at 7:59 PM, Stephan Ewen <sewen@apache.org>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Have a look at some other tests, like the checkpointing
tests.
>>>>>>>>>> They start one cluster manually and keep it running.
They connect against
>>>>>>>>>> it using the remote environment ("localhost",
>>>>>>>>>> miniCluster.getJobManagerRpcPort()).
>>>>>>>>>>
>>>>>>>>>> That works nicely...
>>>>>>>>>>
>>>>>>>>>> On Wed, Sep 2, 2015 at 4:23 PM, Sachin Goel <
>>>>>>>>>> sachingoel0101@gmail.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi all
>>>>>>>>>>> While using LocalEnvironment, in case the program
triggers
>>>>>>>>>>> execution several times, the {{LocalFlinkMiniCluster}}
is started as many
>>>>>>>>>>> times. This can consume a lot of time in setting
up and tearing down the
>>>>>>>>>>> cluster. Further, this hinders with a new functionality
I'm working on
>>>>>>>>>>> based on persisted results.
>>>>>>>>>>> One potential solution could be to follow the
methodology in
>>>>>>>>>>> `MultipleProgramsTestBase`. The user code then
would have to reside in a
>>>>>>>>>>> fixed function name, instead of the main method.
Or is that too cumbersome?
>>>>>>>>>>>
>>>>>>>>>>> Regards
>>>>>>>>>>> Sachin
>>>>>>>>>>> -- Sachin Goel
>>>>>>>>>>> Computer Science, IIT Delhi
>>>>>>>>>>> m. +91-9871457685
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Mime
View raw message