Hi Eugene,

Sorry, but I didn't catch the meaning of your question about Zookeeper
Discovery. Could you please re-phrase it?

ср, 12 сент. 2018 г. в 17:54, Ilya Lantukh <[email protected]>:

> Pavel K., can you please answer about Zookeeper discovery?
>
> On Wed, Sep 12, 2018 at 5:49 PM, eugene miretsky <
> [email protected]> wrote:
>
>> Thanks for the patience with my questions - just trying to understand the
>> system better.
>>
>> 3) I was referring to
>> https://apacheignite.readme.io/docs/zookeeper-discovery#section-failures-and-split-brain-handling.
>> How come it doesn't get the node to shut down?
>> 4) Are there any docs/JIRAs that explain how counters are used, and why
>> they are required in the state?
>>
>> Cheers,
>> Eugene
>>
>>
>> On Wed, Sep 12, 2018 at 10:04 AM Ilya Lantukh <[email protected]>
>> wrote:
>>
>>> 3) Such mechanics will be implemented in IEP-25 (linked above).
>>> 4) Partition map states include update counters, which are incremented
>>> on every cache update and play important role in new state calculation. So,
>>> technically, every cache operation can lead to partition map change, and
>>> for obvious reasons we can't route them through coordinator. Ignite is a
>>> more complex system than Akka or Kafka and such simple solutions won't work
>>> here (in general case). However, it is true that PME could be simplified or
>>> completely avoid for certain cases and the community is currently working
>>> on such optimizations (https://issues.apache.org/jira/browse/IGNITE-9558
>>> for example).
>>>
>>> On Wed, Sep 12, 2018 at 9:08 AM, eugene miretsky <
>>> [email protected]> wrote:
>>>
>>>> 2b) I had a few situations where the cluster went into a state where
>>>> PME constantly failed, and could never recover. I think the root cause was
>>>> that a transaction got stuck and didn't timeout/rollback.  I will try to
>>>> reproduce it again and get back to you
>>>> 3) If a node is down, I would expect it to get detected and the node to
>>>> get removed from the cluster. In such case, PME should not even be
>>>> attempted with that node. Hence you would expect PME to fail very rarely
>>>> (any faulty node will be removed before it has a chance to fail PME)
>>>> 4) Don't all partition map changes go through the coordinator? I
>>>> believe a lot of distributed systems work in this way (all decisions are
>>>> made by the coordinator/leader) - In Akka the leader is responsible for
>>>> making all cluster membership changes, in Kafka the controller does the
>>>> leader election.
>>>>
>>>> On Tue, Sep 11, 2018 at 11:11 AM Ilya Lantukh <[email protected]>
>>>> wrote:
>>>>
>>>>> 1) It is.
>>>>> 2a) Ignite has retry mechanics for all messages, including PME-related
>>>>> ones.
>>>>> 2b) In this situation PME will hang, but it isn't a "deadlock".
>>>>> 3) Sorry, I didn't understand your question. If a node is down, but
>>>>> DiscoverySpi doesn't detect it, it isn't PME-related problem.
>>>>> 4) How can you ensure that partition maps on coordinator are *latest 
>>>>> *without
>>>>> "freezing" cluster state for some time?
>>>>>
>>>>> On Sat, Sep 8, 2018 at 3:21 AM, eugene miretsky <
>>>>> [email protected]> wrote:
>>>>>
>>>>>> Thanks!
>>>>>>
>>>>>> We are using persistence, so I am not sure if shutting down nodes
>>>>>> will be the desired outcome for us since we would need to modify the
>>>>>> baseline topolgy.
>>>>>>
>>>>>> A couple more follow up questions
>>>>>>
>>>>>> 1) Is PME triggered when client nodes join us well? We are using
>>>>>> Spark client, so new nodes are created/destroy every time.
>>>>>> 2) It sounds to me like there is a pontential for the cluster to get
>>>>>> into a deadlock if
>>>>>>    a) single PME message is lost (PME never finishes, there are no
>>>>>> retries, and all future operations are blocked on the pending PME)
>>>>>>    b) one of the nodes has a  long running/stuck pending operation
>>>>>> 3) Under what circumastance can PME fail, while DiscoverySpi fails to
>>>>>> detect the node being down? We are using ZookeeperSpi so I would expect 
>>>>>> the
>>>>>> split brain resolver to shut down the node.
>>>>>> 4) Why is PME needed? Doesn't the coordinator know the altest
>>>>>> toplogy/pertition map of the cluster through regualr gossip?
>>>>>>
>>>>>> Cheers,
>>>>>> Eugene
>>>>>>
>>>>>> On Fri, Sep 7, 2018 at 5:18 PM Ilya Lantukh <[email protected]>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi Eugene,
>>>>>>>
>>>>>>> 1) PME happens when topology is modified (TopologyVersion is
>>>>>>> incremented). The most common events that trigger it are: node
>>>>>>> start/stop/fail, cluster activation/deactivation, dynamic cache 
>>>>>>> start/stop.
>>>>>>> 2) It is done by a separate ExchangeWorker. Events that trigger PME
>>>>>>> are transferred using DiscoverySpi instead of CommunicationSpi.
>>>>>>> 3) All nodes wait for all pending cache operations to finish and
>>>>>>> then send their local partition maps to the coordinator (oldest node). 
>>>>>>> Then
>>>>>>> coordinator calculates new global partition maps and sends them to every
>>>>>>> node.
>>>>>>> 4) All cache operations.
>>>>>>> 5) Exchange is never retried. Ignite community is currently working
>>>>>>> on PME failure handling that should kick all problematic nodes after
>>>>>>> timeout is reached (see
>>>>>>> https://cwiki.apache.org/confluence/display/IGNITE/IEP-25%3A+Partition+Map+Exchange+hangs+resolving
>>>>>>> for details), but it isn't done yet.
>>>>>>> 6) You shouldn't consider PME failure as a error by itself, but
>>>>>>> rather as a result of some other error. The most common reason of PME
>>>>>>> hang-up is pending cache operation that couldn't finish. Check your 
>>>>>>> logs -
>>>>>>> it should list pending transactions and atomic updates. Search for 
>>>>>>> "Found
>>>>>>> long running" substring.
>>>>>>>
>>>>>>> Hope this helps.
>>>>>>>
>>>>>>> On Fri, Sep 7, 2018 at 11:45 PM, eugene miretsky <
>>>>>>> [email protected]> wrote:
>>>>>>>
>>>>>>>> Hello,
>>>>>>>>
>>>>>>>> Out cluster occasionally fails with "partition map exchange
>>>>>>>> failure" errors, I have searched around and it seems that a lot of 
>>>>>>>> people
>>>>>>>> have had a similar issue in the past. My high-level understanding is 
>>>>>>>> that
>>>>>>>> when one of the nodes fails (out of memory, exception, GC etc.) nodes 
>>>>>>>> fail
>>>>>>>> to exchange partition maps. However, I have a few questions
>>>>>>>> 1) When does partition map exchange happen? Periodically, when a
>>>>>>>> node joins, etc.
>>>>>>>> 2) Is it done in the same thread as communication SPI, or is a
>>>>>>>> separate worker?
>>>>>>>> 3) How does the exchange happen? Via a coordinator, peer to peer,
>>>>>>>> etc?
>>>>>>>> 4) What does the exchange block?
>>>>>>>> 5) When is the exchange retried?
>>>>>>>> 5) How to resolve the error? The only thing I have seen online is
>>>>>>>> to decrease failureDetectionTimeout
>>>>>>>>
>>>>>>>> Our settings are
>>>>>>>> - Zookeeper SPI
>>>>>>>> - Persistence enabled
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Eugene
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> Best regards,
>>>>>>> Ilya
>>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> Best regards,
>>>>> Ilya
>>>>>
>>>>
>>>
>>>
>>> --
>>> Best regards,
>>> Ilya
>>>
>>
>
>
> --
> Best regards,
> Ilya
>

Reply via email to