[ 
https://issues.apache.org/jira/browse/SAMZA-375?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14098762#comment-14098762
 ] 

Chris Riccomini commented on SAMZA-375:
---------------------------------------

bq. By keeping the state in zookeeper, the scheduler should be able to fail and 
reregister() without requiring the samza tasks to restart.

Do we ever run into a race condition where the Mesos container has been 
allocated and started, but the ZK state has not yet been saved?

bq. Any thoughts on the overall design?

Longer term, we're going to be providing a state-management facility called the 
ConfigLog (SAMZA-348). It might be possible to store this state in the 
ConfigLog, rather than ZooKeeper. Thus far, we've been able to avoid all direct 
dependencies on ZooKeeper in Samza (we still use it transitively through Kafka, 
though that is going away too). Not something that you have to worry about now, 
but we should consider this use case as part of SAMZA-348.

Also, we should extract out the shared logic between the YARN AM and the 
SamzaScheduler, but this isn't something you should worry about right now. We 
can open a separate ticket for it.

Naive question: does Mesos provide any state facility itself? If the scheduler 
fails and is restarted, will it tell you what was running? If so, does it give 
you any metadata? Perhaps we could just store enough information in the 
metadata about the already-existing containers.

Another naive comment: In YARN, right now, we just nuke all containers when a 
AM restarts. We then start from scratch and request new resources. This could 
be a simpler (but less efficient) alternative strategy to the ZK-based 
scheduler state.

> Investigate Mesos Job Support
> -----------------------------
>
>                 Key: SAMZA-375
>                 URL: https://issues.apache.org/jira/browse/SAMZA-375
>             Project: Samza
>          Issue Type: Bug
>            Reporter: Jon Bringhurst
>            Assignee: Jon Bringhurst
>
> It would be nice if Samza had support for Mesos (https://mesos.apache.org/).
> The current plan is to create a MesosJob and MesosJobFactory, then look into 
> what it would take to allow the AM code to act as a Mesos scheduler.
> The feasibility of this landing in trunk will be better understood after a 
> rough prototype has been created.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to