You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@ignite.apache.org by Koitoer <ko...@gmail.com> on 2019/03/13 18:59:33 UTC

Primary partitions return zero partitions before rebalance.

Hi All.

I'm trying to follow the rebalance events of my ignite cluster so I'm able
to track which partitions are assigned to each node at any point in time. I
am listening to the `EVT_CACHE_REBALANCE_STARTED` and
`EVT_CACHE_REBALANCE_STOPPED`
events from Ignite and that is working well, except in the case one node
crash and another take its place.

My cluster is 5 nodes.
Ex. Node 1 has let's say 100 partitions, after I kill this node the
partitions that were assigned to it, got rebalance across the entire
cluster, I'm able to track that done with the STOPPED event and checking
the affinity function in each one of them using the `primaryPartitions`
method gives me that, if I add all those numbers I get 1024 partitions,
which is why I was expected.

However when a new node replaces the previous one, I see a rebalance
process occurs and now I'm getting that some of the partitions `disappear`
from the already existing nodes (which is expected as well as new node will
take some partitions from them) but when the STOPPED event is listened by
this new node if I call the `primaryPartitions` that one returns an empty
list, but if I used the  `allPartitions` method that one give me a list (I
think at this point is primary + backups).

If I let pass some time and I execute the `primaryPartitions` method again
I am able to retrieve the partitions that I was expecting to see after the
STOPPED event comes. I read here
https://cwiki.apache.org/confluence/display/IGNITE/%28Partition+Map%29+Exchange+-+under+the+hood#id-(PartitionMap)Exchange-under
the hood-LateAffinityAssignment that it could be a late assignment, that
after the cache rebalance the new node needs to bring all the entries to
fill-out the cache and after that, the `primaryPartitions` will return
something.
Will be great to know if this actually what is happening.

My question is if there is any kind of event that I should listen so I can
be aware that this process (if this is what is happening) already finish. I
would like to said, "After you bring this node into the cluster the
partitions assigned to that node are the following: XXX, XXX".

Also, I'm aware of the event `EVT_CACHE_REBALANCE_PART_LOADED` but I'm
seeing a ton of them and at this point, I would be able to know when the
last one arrives and say that are now my primary partitions.

Thanks in advance.

Re: Primary partitions return zero partitions before rebalance.

Posted by Павлухин Иван <vo...@gmail.com>.
Ah, sorry for that. PME = Partition Map Exchange. It is described
along with late affinity assignment in article you referenced earlier
[1].

[1] https://cwiki.apache.org/confluence/display/IGNITE/%28Partition+Map%29+Exchange+-+under+the+hood#id-(PartitionMap)Exchange-under

вт, 2 апр. 2019 г. в 20:22, Koitoer <ko...@gmail.com>:
>
> Sorry but what is exactly the PME ?
>
> On Mon, Apr 1, 2019 at 1:55 AM Павлухин Иван <vo...@gmail.com> wrote:
>>
>> Hi,
>>
>> Sorry for the late answer. An observed result seems expected to me. I
>> suppose following:
>> 1. EVT_CACHE_REBALANCE_STOPPED is fired when a particular node loaded
>> all partitions which it will be responsible for.
>> 2. All nodes it the cluster must become aware that partition
>> assignment was changed. So, PME will happen to make all nodes aware of
>> new assignment.
>> 3. Once PME completes all nodes will consistently treat just entered
>> node as primary for a corresponding set of partitions.
>>
>> Do not hesitate to write back if you feel that something is going wrong.
>>
>> вт, 19 мар. 2019 г. в 19:30, Koitoer <ko...@gmail.com>:
>> >
>> > Hello Igniters
>> >
>> > The version of Ignite that we are using is 2.7.0. I'm adding the events that I want to hear via the IgniteConfiguration using the `setIncludeEventTypes`
>> > Then using ignite.event().localListen(listenerPredicate, eventTypes);
>> >
>> > EVT_CACHE_REBALANCE_STARTED,
>> > EVT_CACHE_REBALANCE_STOPPED,
>> > EVT_CACHE_REBALANCE_PART_LOADED,
>> > EVT_CACHE_REBALANCE_PART_UNLOADED,
>> > EVT_CACHE_REBALANCE_PART_DATA_LOST
>> >
>> > Once I listen any of the events above, I used `ignite.affinity(cacheName.name())`  to retrieve the Affinity function in which I'm calling the `primaryPartitions` method or `allPartitions` using the ClusterNode instance that represents `this` node.
>> >
>> > Once I hear the rebalance process stop event I created a thread in charge of checking the partition assignment as follows.
>> >
>> > new Thread(() -> {
>> >     for (int attempt = 0; attempt <= attempts; attempt++) {
>> >         log.info("event=partitionAssignmentRetryLogic attempt={}, before={}, now={}", attempt, assignedPartitions,
>> >             affinity.primaryPartitions(clusterNode));
>> >
>> >         try {
>> >             if (affinity.primaryPartitions(clusterNode).length != 0) {
>> >                 log.info("event=partitionAssignmentRetryLogicSuccess");
>> >             }
>> >             TimeUnit.SECONDS.sleep(delay);
>> >         } catch (Exception e) {
>> >             log.error("event=ErrorOnTimerWait message={}", e.getMessage(), e);
>> >         }
>> >     }
>> > }).start();
>> >
>> >
>> > After a couple of attempts (some seconds), the `primaryPartitions` is returning the correct set of partitions assigned to a node.  I will check the AffinityAssignment for trying to do this in a cleaner way as you suggest.
>> >
>> >
>> > On Fri, Mar 15, 2019 at 12:11 PM Павлухин Иван <vo...@gmail.com> wrote:
>> >>
>> >> Hi,
>> >>
>> >> What Ignite version do you use?
>> >> How do you register your listener?
>> >> On what object do you call primaryPartitions/allPartitions?
>> >>
>> >> It is true that Ignite uses late affinitly assignment. And it means
>> >> that for each topology change (node enter or node leave) parttion
>> >> assigment changes twice. First time temporay backups are created which
>> >> should be rebalanced from other nodes (EVT_CACHE_REBALANCE_STARTED
>> >> takes place here). Second time redundant partition replicas should be
>> >> marked as unusable (and unloaded after that)
>> >> (EVT_CACHE_REBALANCE_STOPPED). And it is useful to understand that
>> >> Affinity interface calculates partition distribution using affinity
>> >> function and such distribution might differ from real partitoin
>> >> assignment. And it differes when rebalance is in progress. See
>> >> AffinityAssignment interface.
>> >>
>> >> ср, 13 мар. 2019 г. в 21:59, Koitoer <ko...@gmail.com>:
>> >> >
>> >> > Hi All.
>> >> >
>> >> > I'm trying to follow the rebalance events of my ignite cluster so I'm able to track which partitions are assigned to each node at any point in time. I am listening to the `EVT_CACHE_REBALANCE_STARTED` and `EVT_CACHE_REBALANCE_STOPPED`
>> >> > events from Ignite and that is working well, except in the case one node crash and another take its place.
>> >> >
>> >> > My cluster is 5 nodes.
>> >> > Ex. Node 1 has let's say 100 partitions, after I kill this node the partitions that were assigned to it, got rebalance across the entire cluster, I'm able to track that done with the STOPPED event and checking the affinity function in each one of them using the `primaryPartitions` method gives me that, if I add all those numbers I get 1024 partitions, which is why I was expected.
>> >> >
>> >> > However when a new node replaces the previous one, I see a rebalance process occurs and now I'm getting that some of the partitions `disappear` from the already existing nodes (which is expected as well as new node will take some partitions from them) but when the STOPPED event is listened by this new node if I call the `primaryPartitions` that one returns an empty list, but if I used the  `allPartitions` method that one give me a list (I think at this point is primary + backups).
>> >> >
>> >> > If I let pass some time and I execute the `primaryPartitions` method again I am able to retrieve the partitions that I was expecting to see after the STOPPED event comes. I read here https://cwiki.apache.org/confluence/display/IGNITE/%28Partition+Map%29+Exchange+-+under+the+hood#id-(PartitionMap)Exchange-under the hood-LateAffinityAssignment that it could be a late assignment, that after the cache rebalance the new node needs to bring all the entries to fill-out the cache and after that, the `primaryPartitions` will return something.
>> >> > Will be great to know if this actually what is happening.
>> >> >
>> >> > My question is if there is any kind of event that I should listen so I can be aware that this process (if this is what is happening) already finish. I would like to said, "After you bring this node into the cluster the partitions assigned to that node are the following: XXX, XXX".
>> >> >
>> >> > Also, I'm aware of the event `EVT_CACHE_REBALANCE_PART_LOADED` but I'm seeing a ton of them and at this point, I would be able to know when the last one arrives and say that are now my primary partitions.
>> >> >
>> >> > Thanks in advance.
>> >>
>> >>
>> >>
>> >> --
>> >> Best regards,
>> >> Ivan Pavlukhin
>> >
>> >
>> >
>> > --
>> > koitoer ....
>>
>>
>>
>> --
>> Best regards,
>> Ivan Pavlukhin
>
>
>
> --
> koitoer ....



-- 
Best regards,
Ivan Pavlukhin

Re: Primary partitions return zero partitions before rebalance.

Posted by Koitoer <ko...@gmail.com>.
Sorry but what is exactly the PME ?

On Mon, Apr 1, 2019 at 1:55 AM Павлухин Иван <vo...@gmail.com> wrote:

> Hi,
>
> Sorry for the late answer. An observed result seems expected to me. I
> suppose following:
> 1. EVT_CACHE_REBALANCE_STOPPED is fired when a particular node loaded
> all partitions which it will be responsible for.
> 2. All nodes it the cluster must become aware that partition
> assignment was changed. So, PME will happen to make all nodes aware of
> new assignment.
> 3. Once PME completes all nodes will consistently treat just entered
> node as primary for a corresponding set of partitions.
>
> Do not hesitate to write back if you feel that something is going wrong.
>
> вт, 19 мар. 2019 г. в 19:30, Koitoer <ko...@gmail.com>:
> >
> > Hello Igniters
> >
> > The version of Ignite that we are using is 2.7.0. I'm adding the events
> that I want to hear via the IgniteConfiguration using the
> `setIncludeEventTypes`
> > Then using ignite.event().localListen(listenerPredicate, eventTypes);
> >
> > EVT_CACHE_REBALANCE_STARTED,
> > EVT_CACHE_REBALANCE_STOPPED,
> > EVT_CACHE_REBALANCE_PART_LOADED,
> > EVT_CACHE_REBALANCE_PART_UNLOADED,
> > EVT_CACHE_REBALANCE_PART_DATA_LOST
> >
> > Once I listen any of the events above, I used
> `ignite.affinity(cacheName.name())`  to retrieve the Affinity function in
> which I'm calling the `primaryPartitions` method or `allPartitions` using
> the ClusterNode instance that represents `this` node.
> >
> > Once I hear the rebalance process stop event I created a thread in
> charge of checking the partition assignment as follows.
> >
> > new Thread(() -> {
> >     for (int attempt = 0; attempt <= attempts; attempt++) {
> >         log.info("event=partitionAssignmentRetryLogic attempt={},
> before={}, now={}", attempt, assignedPartitions,
> >             affinity.primaryPartitions(clusterNode));
> >
> >         try {
> >             if (affinity.primaryPartitions(clusterNode).length != 0) {
> >                 log.info("event=partitionAssignmentRetryLogicSuccess");
> >             }
> >             TimeUnit.SECONDS.sleep(delay);
> >         } catch (Exception e) {
> >             log.error("event=ErrorOnTimerWait message={}",
> e.getMessage(), e);
> >         }
> >     }
> > }).start();
> >
> >
> > After a couple of attempts (some seconds), the `primaryPartitions` is
> returning the correct set of partitions assigned to a node.  I will check
> the AffinityAssignment for trying to do this in a cleaner way as you
> suggest.
> >
> >
> > On Fri, Mar 15, 2019 at 12:11 PM Павлухин Иван <vo...@gmail.com>
> wrote:
> >>
> >> Hi,
> >>
> >> What Ignite version do you use?
> >> How do you register your listener?
> >> On what object do you call primaryPartitions/allPartitions?
> >>
> >> It is true that Ignite uses late affinitly assignment. And it means
> >> that for each topology change (node enter or node leave) parttion
> >> assigment changes twice. First time temporay backups are created which
> >> should be rebalanced from other nodes (EVT_CACHE_REBALANCE_STARTED
> >> takes place here). Second time redundant partition replicas should be
> >> marked as unusable (and unloaded after that)
> >> (EVT_CACHE_REBALANCE_STOPPED). And it is useful to understand that
> >> Affinity interface calculates partition distribution using affinity
> >> function and such distribution might differ from real partitoin
> >> assignment. And it differes when rebalance is in progress. See
> >> AffinityAssignment interface.
> >>
> >> ср, 13 мар. 2019 г. в 21:59, Koitoer <ko...@gmail.com>:
> >> >
> >> > Hi All.
> >> >
> >> > I'm trying to follow the rebalance events of my ignite cluster so I'm
> able to track which partitions are assigned to each node at any point in
> time. I am listening to the `EVT_CACHE_REBALANCE_STARTED` and
> `EVT_CACHE_REBALANCE_STOPPED`
> >> > events from Ignite and that is working well, except in the case one
> node crash and another take its place.
> >> >
> >> > My cluster is 5 nodes.
> >> > Ex. Node 1 has let's say 100 partitions, after I kill this node the
> partitions that were assigned to it, got rebalance across the entire
> cluster, I'm able to track that done with the STOPPED event and checking
> the affinity function in each one of them using the `primaryPartitions`
> method gives me that, if I add all those numbers I get 1024 partitions,
> which is why I was expected.
> >> >
> >> > However when a new node replaces the previous one, I see a rebalance
> process occurs and now I'm getting that some of the partitions `disappear`
> from the already existing nodes (which is expected as well as new node will
> take some partitions from them) but when the STOPPED event is listened by
> this new node if I call the `primaryPartitions` that one returns an empty
> list, but if I used the  `allPartitions` method that one give me a list (I
> think at this point is primary + backups).
> >> >
> >> > If I let pass some time and I execute the `primaryPartitions` method
> again I am able to retrieve the partitions that I was expecting to see
> after the STOPPED event comes. I read here
> https://cwiki.apache.org/confluence/display/IGNITE/%28Partition+Map%29+Exchange+-+under+the+hood#id-(PartitionMap)Exchange-under
> the hood-LateAffinityAssignment that it could be a late assignment, that
> after the cache rebalance the new node needs to bring all the entries to
> fill-out the cache and after that, the `primaryPartitions` will return
> something.
> >> > Will be great to know if this actually what is happening.
> >> >
> >> > My question is if there is any kind of event that I should listen so
> I can be aware that this process (if this is what is happening) already
> finish. I would like to said, "After you bring this node into the cluster
> the partitions assigned to that node are the following: XXX, XXX".
> >> >
> >> > Also, I'm aware of the event `EVT_CACHE_REBALANCE_PART_LOADED` but
> I'm seeing a ton of them and at this point, I would be able to know when
> the last one arrives and say that are now my primary partitions.
> >> >
> >> > Thanks in advance.
> >>
> >>
> >>
> >> --
> >> Best regards,
> >> Ivan Pavlukhin
> >
> >
> >
> > --
> > koitoer ....
>
>
>
> --
> Best regards,
> Ivan Pavlukhin
>


-- 
koitoer ....

Re: Primary partitions return zero partitions before rebalance.

Posted by Павлухин Иван <vo...@gmail.com>.
Hi,

Sorry for the late answer. An observed result seems expected to me. I
suppose following:
1. EVT_CACHE_REBALANCE_STOPPED is fired when a particular node loaded
all partitions which it will be responsible for.
2. All nodes it the cluster must become aware that partition
assignment was changed. So, PME will happen to make all nodes aware of
new assignment.
3. Once PME completes all nodes will consistently treat just entered
node as primary for a corresponding set of partitions.

Do not hesitate to write back if you feel that something is going wrong.

вт, 19 мар. 2019 г. в 19:30, Koitoer <ko...@gmail.com>:
>
> Hello Igniters
>
> The version of Ignite that we are using is 2.7.0. I'm adding the events that I want to hear via the IgniteConfiguration using the `setIncludeEventTypes`
> Then using ignite.event().localListen(listenerPredicate, eventTypes);
>
> EVT_CACHE_REBALANCE_STARTED,
> EVT_CACHE_REBALANCE_STOPPED,
> EVT_CACHE_REBALANCE_PART_LOADED,
> EVT_CACHE_REBALANCE_PART_UNLOADED,
> EVT_CACHE_REBALANCE_PART_DATA_LOST
>
> Once I listen any of the events above, I used `ignite.affinity(cacheName.name())`  to retrieve the Affinity function in which I'm calling the `primaryPartitions` method or `allPartitions` using the ClusterNode instance that represents `this` node.
>
> Once I hear the rebalance process stop event I created a thread in charge of checking the partition assignment as follows.
>
> new Thread(() -> {
>     for (int attempt = 0; attempt <= attempts; attempt++) {
>         log.info("event=partitionAssignmentRetryLogic attempt={}, before={}, now={}", attempt, assignedPartitions,
>             affinity.primaryPartitions(clusterNode));
>
>         try {
>             if (affinity.primaryPartitions(clusterNode).length != 0) {
>                 log.info("event=partitionAssignmentRetryLogicSuccess");
>             }
>             TimeUnit.SECONDS.sleep(delay);
>         } catch (Exception e) {
>             log.error("event=ErrorOnTimerWait message={}", e.getMessage(), e);
>         }
>     }
> }).start();
>
>
> After a couple of attempts (some seconds), the `primaryPartitions` is returning the correct set of partitions assigned to a node.  I will check the AffinityAssignment for trying to do this in a cleaner way as you suggest.
>
>
> On Fri, Mar 15, 2019 at 12:11 PM Павлухин Иван <vo...@gmail.com> wrote:
>>
>> Hi,
>>
>> What Ignite version do you use?
>> How do you register your listener?
>> On what object do you call primaryPartitions/allPartitions?
>>
>> It is true that Ignite uses late affinitly assignment. And it means
>> that for each topology change (node enter or node leave) parttion
>> assigment changes twice. First time temporay backups are created which
>> should be rebalanced from other nodes (EVT_CACHE_REBALANCE_STARTED
>> takes place here). Second time redundant partition replicas should be
>> marked as unusable (and unloaded after that)
>> (EVT_CACHE_REBALANCE_STOPPED). And it is useful to understand that
>> Affinity interface calculates partition distribution using affinity
>> function and such distribution might differ from real partitoin
>> assignment. And it differes when rebalance is in progress. See
>> AffinityAssignment interface.
>>
>> ср, 13 мар. 2019 г. в 21:59, Koitoer <ko...@gmail.com>:
>> >
>> > Hi All.
>> >
>> > I'm trying to follow the rebalance events of my ignite cluster so I'm able to track which partitions are assigned to each node at any point in time. I am listening to the `EVT_CACHE_REBALANCE_STARTED` and `EVT_CACHE_REBALANCE_STOPPED`
>> > events from Ignite and that is working well, except in the case one node crash and another take its place.
>> >
>> > My cluster is 5 nodes.
>> > Ex. Node 1 has let's say 100 partitions, after I kill this node the partitions that were assigned to it, got rebalance across the entire cluster, I'm able to track that done with the STOPPED event and checking the affinity function in each one of them using the `primaryPartitions` method gives me that, if I add all those numbers I get 1024 partitions, which is why I was expected.
>> >
>> > However when a new node replaces the previous one, I see a rebalance process occurs and now I'm getting that some of the partitions `disappear` from the already existing nodes (which is expected as well as new node will take some partitions from them) but when the STOPPED event is listened by this new node if I call the `primaryPartitions` that one returns an empty list, but if I used the  `allPartitions` method that one give me a list (I think at this point is primary + backups).
>> >
>> > If I let pass some time and I execute the `primaryPartitions` method again I am able to retrieve the partitions that I was expecting to see after the STOPPED event comes. I read here https://cwiki.apache.org/confluence/display/IGNITE/%28Partition+Map%29+Exchange+-+under+the+hood#id-(PartitionMap)Exchange-under the hood-LateAffinityAssignment that it could be a late assignment, that after the cache rebalance the new node needs to bring all the entries to fill-out the cache and after that, the `primaryPartitions` will return something.
>> > Will be great to know if this actually what is happening.
>> >
>> > My question is if there is any kind of event that I should listen so I can be aware that this process (if this is what is happening) already finish. I would like to said, "After you bring this node into the cluster the partitions assigned to that node are the following: XXX, XXX".
>> >
>> > Also, I'm aware of the event `EVT_CACHE_REBALANCE_PART_LOADED` but I'm seeing a ton of them and at this point, I would be able to know when the last one arrives and say that are now my primary partitions.
>> >
>> > Thanks in advance.
>>
>>
>>
>> --
>> Best regards,
>> Ivan Pavlukhin
>
>
>
> --
> koitoer ....



-- 
Best regards,
Ivan Pavlukhin

Re: Primary partitions return zero partitions before rebalance.

Posted by Koitoer <ko...@gmail.com>.
Hello Igniters

The version of Ignite that we are using is 2.7.0. I'm adding the events
that I want to hear via the IgniteConfiguration using the
`setIncludeEventTypes`
Then using ignite.event().localListen(listenerPredicate, eventTypes);

EVT_CACHE_REBALANCE_STARTED,
EVT_CACHE_REBALANCE_STOPPED,
EVT_CACHE_REBALANCE_PART_LOADED,
EVT_CACHE_REBALANCE_PART_UNLOADED,
EVT_CACHE_REBALANCE_PART_DATA_LOST

Once I listen any of the events above, I used
`ignite.affinity(cacheName.name())`  to retrieve the Affinity function in
which I'm calling the `primaryPartitions` method or `allPartitions` using
the ClusterNode instance that represents `this` node.

Once I hear the rebalance process stop event I created a thread in charge
of checking the partition assignment as follows.

new Thread(() -> {
    for (int attempt = 0; attempt <= attempts; attempt++) {
        log.info("event=partitionAssignmentRetryLogic attempt={},
before={}, now={}", attempt, assignedPartitions,
            affinity.primaryPartitions(clusterNode));

        try {
            if (affinity.primaryPartitions(clusterNode).length != 0) {
                log.info("event=partitionAssignmentRetryLogicSuccess");
            }
            TimeUnit.SECONDS.sleep(delay);
        } catch (Exception e) {
            log.error("event=ErrorOnTimerWait message={}", e.getMessage(), e);
        }
    }
}).start();


After a couple of attempts (some seconds), the `primaryPartitions` is
returning the correct set of partitions assigned to a node.  I will check
the AffinityAssignment for trying to do this in a cleaner way as you
suggest.


On Fri, Mar 15, 2019 at 12:11 PM Павлухин Иван <vo...@gmail.com> wrote:

> Hi,
>
> What Ignite version do you use?
> How do you register your listener?
> On what object do you call primaryPartitions/allPartitions?
>
> It is true that Ignite uses late affinitly assignment. And it means
> that for each topology change (node enter or node leave) parttion
> assigment changes twice. First time temporay backups are created which
> should be rebalanced from other nodes (EVT_CACHE_REBALANCE_STARTED
> takes place here). Second time redundant partition replicas should be
> marked as unusable (and unloaded after that)
> (EVT_CACHE_REBALANCE_STOPPED). And it is useful to understand that
> Affinity interface calculates partition distribution using affinity
> function and such distribution might differ from real partitoin
> assignment. And it differes when rebalance is in progress. See
> AffinityAssignment interface.
>
> ср, 13 мар. 2019 г. в 21:59, Koitoer <ko...@gmail.com>:
> >
> > Hi All.
> >
> > I'm trying to follow the rebalance events of my ignite cluster so I'm
> able to track which partitions are assigned to each node at any point in
> time. I am listening to the `EVT_CACHE_REBALANCE_STARTED` and
> `EVT_CACHE_REBALANCE_STOPPED`
> > events from Ignite and that is working well, except in the case one node
> crash and another take its place.
> >
> > My cluster is 5 nodes.
> > Ex. Node 1 has let's say 100 partitions, after I kill this node the
> partitions that were assigned to it, got rebalance across the entire
> cluster, I'm able to track that done with the STOPPED event and checking
> the affinity function in each one of them using the `primaryPartitions`
> method gives me that, if I add all those numbers I get 1024 partitions,
> which is why I was expected.
> >
> > However when a new node replaces the previous one, I see a rebalance
> process occurs and now I'm getting that some of the partitions `disappear`
> from the already existing nodes (which is expected as well as new node will
> take some partitions from them) but when the STOPPED event is listened by
> this new node if I call the `primaryPartitions` that one returns an empty
> list, but if I used the  `allPartitions` method that one give me a list (I
> think at this point is primary + backups).
> >
> > If I let pass some time and I execute the `primaryPartitions` method
> again I am able to retrieve the partitions that I was expecting to see
> after the STOPPED event comes. I read here
> https://cwiki.apache.org/confluence/display/IGNITE/%28Partition+Map%29+Exchange+-+under+the+hood#id-(PartitionMap)Exchange-under
> the hood-LateAffinityAssignment that it could be a late assignment, that
> after the cache rebalance the new node needs to bring all the entries to
> fill-out the cache and after that, the `primaryPartitions` will return
> something.
> > Will be great to know if this actually what is happening.
> >
> > My question is if there is any kind of event that I should listen so I
> can be aware that this process (if this is what is happening) already
> finish. I would like to said, "After you bring this node into the cluster
> the partitions assigned to that node are the following: XXX, XXX".
> >
> > Also, I'm aware of the event `EVT_CACHE_REBALANCE_PART_LOADED` but I'm
> seeing a ton of them and at this point, I would be able to know when the
> last one arrives and say that are now my primary partitions.
> >
> > Thanks in advance.
>
>
>
> --
> Best regards,
> Ivan Pavlukhin
>


-- 
koitoer ....

Re: Primary partitions return zero partitions before rebalance.

Posted by Павлухин Иван <vo...@gmail.com>.
Hi,

What Ignite version do you use?
How do you register your listener?
On what object do you call primaryPartitions/allPartitions?

It is true that Ignite uses late affinitly assignment. And it means
that for each topology change (node enter or node leave) parttion
assigment changes twice. First time temporay backups are created which
should be rebalanced from other nodes (EVT_CACHE_REBALANCE_STARTED
takes place here). Second time redundant partition replicas should be
marked as unusable (and unloaded after that)
(EVT_CACHE_REBALANCE_STOPPED). And it is useful to understand that
Affinity interface calculates partition distribution using affinity
function and such distribution might differ from real partitoin
assignment. And it differes when rebalance is in progress. See
AffinityAssignment interface.

ср, 13 мар. 2019 г. в 21:59, Koitoer <ko...@gmail.com>:
>
> Hi All.
>
> I'm trying to follow the rebalance events of my ignite cluster so I'm able to track which partitions are assigned to each node at any point in time. I am listening to the `EVT_CACHE_REBALANCE_STARTED` and `EVT_CACHE_REBALANCE_STOPPED`
> events from Ignite and that is working well, except in the case one node crash and another take its place.
>
> My cluster is 5 nodes.
> Ex. Node 1 has let's say 100 partitions, after I kill this node the partitions that were assigned to it, got rebalance across the entire cluster, I'm able to track that done with the STOPPED event and checking the affinity function in each one of them using the `primaryPartitions` method gives me that, if I add all those numbers I get 1024 partitions, which is why I was expected.
>
> However when a new node replaces the previous one, I see a rebalance process occurs and now I'm getting that some of the partitions `disappear` from the already existing nodes (which is expected as well as new node will take some partitions from them) but when the STOPPED event is listened by this new node if I call the `primaryPartitions` that one returns an empty list, but if I used the  `allPartitions` method that one give me a list (I think at this point is primary + backups).
>
> If I let pass some time and I execute the `primaryPartitions` method again I am able to retrieve the partitions that I was expecting to see after the STOPPED event comes. I read here https://cwiki.apache.org/confluence/display/IGNITE/%28Partition+Map%29+Exchange+-+under+the+hood#id-(PartitionMap)Exchange-under the hood-LateAffinityAssignment that it could be a late assignment, that after the cache rebalance the new node needs to bring all the entries to fill-out the cache and after that, the `primaryPartitions` will return something.
> Will be great to know if this actually what is happening.
>
> My question is if there is any kind of event that I should listen so I can be aware that this process (if this is what is happening) already finish. I would like to said, "After you bring this node into the cluster the partitions assigned to that node are the following: XXX, XXX".
>
> Also, I'm aware of the event `EVT_CACHE_REBALANCE_PART_LOADED` but I'm seeing a ton of them and at this point, I would be able to know when the last one arrives and say that are now my primary partitions.
>
> Thanks in advance.



-- 
Best regards,
Ivan Pavlukhin