Yes, only the TaskID is required. The SlaveID is optional and will lead to
a better response rate from the Master.

Good to see these questions, I will be working on some reconciliation
documentation before the 0.21.0 release, your feedback on it would be
appreciated!

On Sun, Oct 5, 2014 at 9:43 AM, Whitney Sorenson <[email protected]>
wrote:

> One last question - which fields of the TaskStatus objects are meaningful
> when calling driver.reconcileTasks?
>
> For example, if I want an update for a given Task, can I send a
> self-generated TaskStatus object with just a taskId (and I believe also
> slaveId?) without knowing the current taskState or sending other metadata?
>
> Thanks,
>
> -Whitney
>
>
> On Wed, Oct 1, 2014 at 8:33 PM, Benjamin Mahler <[email protected]
> > wrote:
>
>> Updated timestamps will be generated.
>>
>> On Tue, Sep 30, 2014 at 6:21 AM, Whitney Sorenson <[email protected]>
>> wrote:
>>
>>> That makes sense and I suppose your first sentence was the real
>>> summary/confirmation of the change in 0.20 that I was looking for.
>>>
>>> A quick question - can we trust that the taskStatus objects sent as a
>>> result of calling reconciliation themselves are the exact same object as
>>> was (possibly) sent before, or would they have updated timestamps?
>>>
>>>
>>>
>>>
>>> On Tue, Sep 30, 2014 at 4:14 AM, Benjamin Mahler <
>>> [email protected]> wrote:
>>>
>>>> We want reconciliation to be a process that eventually terminates.
>>>>
>>>> In <= 0.19.0, the following two cases are conflated through no update
>>>> being sent:
>>>>   (1) No state difference.
>>>>   (2) Master temporarily cannot reply / dropped message.
>>>>
>>>> As a result, a scheduler cannot determine when it is finished
>>>> reconciling (is my state correct? or was my message not processed?).
>>>>
>>>>
>>>> The way we want to steer frameworks to use reconciliation is as follows.
>>>>
>>>> (1) You should only need to reconcile with the master after a
>>>> re-registration occurs (either master failed over, or framework failed
>>>> over). Some frameworks may want to be more defensive against both
>>>> themselves and against Mesos, and may reconcile on a periodic basis (e.g.
>>>> hourly or daily).
>>>>
>>>> (2) Reconciliation is a process which terminates when an update has
>>>> been received for each task. Here is some pseudo-code that demonstrates how
>>>> a scheduler would implement reliable reconciliation:
>>>>
>>>>
>>>>     # Reconciles state against the master.
>>>>     # TODO: If you call this twice, it will start two reconciliation
>>>> cycles, instead of starting a new one.
>>>>     def reconcile():
>>>>       start_time = now()
>>>>       remaining_tasks = [all non terminal tasks]
>>>>       driver.reconcileTasks() # Implicit reconciliation, lets you
>>>> discover unknown tasks.
>>>>       delay(Seconds(30), _reconcile, start_time, remaining_tasks)
>>>>
>>>>     def _reconcile(start_time, remaining_tasks):
>>>>       remaining_tasks = [t for t in remaining_tasks if
>>>> t.latest_update_time() < start_time]
>>>>       if not remaining_tasks.empty():
>>>>         driver.reconcileTasks(remaining_tasks)
>>>>         delay(Seconds(30), _reconcile, start_time, remaining_tasks) #
>>>> TODO: Use backoff instead.
>>>>
>>>>
>>>> The idea is that you reconcile a set of tasks until you receive updates
>>>> for each one, this set will converge to become empty. You would call
>>>> reconcile() when a (re-)registration occurs.
>>>>
>>>> This is the model you should use in 0.20.0, there are some edge cases
>>>> that we'll fix for 0.21.0, but you likely will not notice them:
>>>> https://issues.apache.org/jira/browse/MESOS-1407
>>>>
>>>> Appreciate you starting this thread. Let me know if anything is not
>>>> clear.
>>>>
>>>> Ben
>>>>
>>>> On Sun, Sep 28, 2014 at 11:58 AM, Whitney Sorenson <
>>>> [email protected]> wrote:
>>>>
>>>>> I'm trying to understand the changes in
>>>>> https://issues.apache.org/jira/browse/MESOS-1453 and the
>>>>> SchedulerDriver JavaDoc.
>>>>>
>>>>> In the 0.19 behavior, it made sense to me that a framework would hold
>>>>> onto a copy of all the latest task statuses it knew about, and could poll
>>>>> reconcileTasks with these statuses in order to request delivery of any 
>>>>> lost
>>>>> messages (covering the case of both the framework being absent for a while
>>>>> or just a general loss of messages.)
>>>>>
>>>>> Is the idea behind the changes in 0.20 that a framework now need only
>>>>> call reconcileTasks once after registering with a master? In that case,
>>>>> what is the use case for having the API still take a list of taskStatus
>>>>> objects - so frameworks can decide that they don't want to know about
>>>>> unknown tasks [1379]? If frameworks should still routinely ask for missing
>>>>> messages - then why bother sending all updates and causing the framework 
>>>>> to
>>>>> have to handle the work of routinely ignoring duplicate status updates?
>>>>>
>>>>> Thanks,
>>>>>
>>>>> -Whitney
>>>>>
>>>>>
>>>>
>>>
>>
>

Reply via email to