Hello, Igniters!
I have implemented lightweight version of partitions map exchange for the case when the baseline node leaves topology. [1] If partitions are assigned according to the baseline topology and server node leaves there's no actual need to perform distributed PME. Every cluster will recalculate new affinity assignments and partition states locally. There is no need to wait for partitions released and PME will be started immediately. I have benchmarked duration of PME under yardstick load. PME duration was decreased up to 10 times and the maximum latency of transactions was decreased up to 4-5 times. See details in Jira issue comments. [1] Could some expert of PME take a look at my changes? [2] 1. https://issues.apache.org/jira/browse/IGNITE-9913 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 -- Best wishes, Amelchev Nikita |
Hi Nikita,
Thank you for your work. This is great improvement. I'll take look on it in next couple of days. Could you please run TC and provide MTCGA bot status about this change? чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: > Hello, Igniters! > > I have implemented lightweight version of partitions map exchange for > the case when the baseline node leaves topology. [1] > > If partitions are assigned according to the baseline topology and > server node leaves there's no actual need to perform distributed PME. > Every cluster will recalculate new affinity assignments and partition > states locally. There is no need to wait for partitions released and > PME will be started immediately. > > I have benchmarked duration of PME under yardstick load. PME duration > was decreased up to 10 times and the maximum latency of transactions > was decreased up to 4-5 times. See details in Jira issue comments. [1] > > Could some expert of PME take a look at my changes? [2] > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 > > -- > Best wishes, > Amelchev Nikita > |
Nikita,
It sounds cool. But I didn't get about in-memory caches. The baseline is not used for their affinity calculation. So, this improvement would be switched off for them or completely (when such caches are presented), wouldn't it? On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> wrote: > Hi Nikita, > > Thank you for your work. This is great improvement. I'll take look on it in > next couple of days. Could you please run TC and provide MTCGA bot status > about this change? > > чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: > > > Hello, Igniters! > > > > I have implemented lightweight version of partitions map exchange for > > the case when the baseline node leaves topology. [1] > > > > If partitions are assigned according to the baseline topology and > > server node leaves there's no actual need to perform distributed PME. > > Every cluster will recalculate new affinity assignments and partition > > states locally. There is no need to wait for partitions released and > > PME will be started immediately. > > > > I have benchmarked duration of PME under yardstick load. PME duration > > was decreased up to 10 times and the maximum latency of transactions > > was decreased up to 4-5 times. See details in Jira issue comments. [1] > > > > Could some expert of PME take a look at my changes? [2] > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 > > > > -- > > Best wishes, > > Amelchev Nikita > > > |
Pavel,
I have provided MTCGA bot status in Jira issue comments. [1] Eduard, Yes, for current implementation it will be distributed PME if in-memory caches configured. 1. https://issues.apache.org/jira/browse/IGNITE-9913 пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev <[hidden email]>: > > Nikita, > > It sounds cool. But I didn't get about in-memory caches. The baseline is > not used for their affinity calculation. > So, this improvement would be switched off for them or completely (when > such caches are presented), wouldn't it? > > On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> wrote: > > > Hi Nikita, > > > > Thank you for your work. This is great improvement. I'll take look on it in > > next couple of days. Could you please run TC and provide MTCGA bot status > > about this change? > > > > чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: > > > > > Hello, Igniters! > > > > > > I have implemented lightweight version of partitions map exchange for > > > the case when the baseline node leaves topology. [1] > > > > > > If partitions are assigned according to the baseline topology and > > > server node leaves there's no actual need to perform distributed PME. > > > Every cluster will recalculate new affinity assignments and partition > > > states locally. There is no need to wait for partitions released and > > > PME will be started immediately. > > > > > > I have benchmarked duration of PME under yardstick load. PME duration > > > was decreased up to 10 times and the maximum latency of transactions > > > was decreased up to 4-5 times. See details in Jira issue comments. [1] > > > > > > Could some expert of PME take a look at my changes? [2] > > > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 > > > > > > -- > > > Best wishes, > > > Amelchev Nikita > > > > > -- Best wishes, Amelchev Nikita |
Hello, Igniters!
I am working on the implementation of lightweight PME for the case of a BLT node leave. [1] There is a question: whether to allow lightweight PME if the cluster has MOVING partitions? The problems that may happen if allow: - Nodes can differently select the primary node from current OWNING backups. - One part of nodes can mark a partition as LOST and another one as OWNING. We can take states of the partitions from the node2part map. The root cause of those problems is that when rebalancing ends (get the last message), it updates partition state of the local node to OWNING (and schedules partitions resend). This may lead to different affinity re-calculations on nodes. I see two solutions: 1. Nodes will store “moving-owning” transition of partitions state until the rebalancing ends. Each node will locally recalculate the affinity on node left event. 2. The coordinator will calculate affinity and send "full map" to nodes. In this case, nodes still should wait for topology change event (to get correct topology in discovery). If disallow lightweight PME when the cluster has MOVING partitions - there are no problems and it works fine. Any thoughts? 1. https://issues.apache.org/jira/browse/IGNITE-9913 пт, 29 мар. 2019 г. в 15:00, Nikita Amelchev <[hidden email]>: > > Pavel, > I have provided MTCGA bot status in Jira issue comments. [1] > > Eduard, > Yes, for current implementation it will be distributed PME if > in-memory caches configured. > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev <[hidden email]>: > > > > Nikita, > > > > It sounds cool. But I didn't get about in-memory caches. The baseline is > > not used for their affinity calculation. > > So, this improvement would be switched off for them or completely (when > > such caches are presented), wouldn't it? > > > > On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> wrote: > > > > > Hi Nikita, > > > > > > Thank you for your work. This is great improvement. I'll take look on it in > > > next couple of days. Could you please run TC and provide MTCGA bot status > > > about this change? > > > > > > чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: > > > > > > > Hello, Igniters! > > > > > > > > I have implemented lightweight version of partitions map exchange for > > > > the case when the baseline node leaves topology. [1] > > > > > > > > If partitions are assigned according to the baseline topology and > > > > server node leaves there's no actual need to perform distributed PME. > > > > Every cluster will recalculate new affinity assignments and partition > > > > states locally. There is no need to wait for partitions released and > > > > PME will be started immediately. > > > > > > > > I have benchmarked duration of PME under yardstick load. PME duration > > > > was decreased up to 10 times and the maximum latency of transactions > > > > was decreased up to 4-5 times. See details in Jira issue comments. [1] > > > > > > > > Could some expert of PME take a look at my changes? [2] > > > > > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 > > > > > > > > -- > > > > Best wishes, > > > > Amelchev Nikita > > > > > > > > > > > -- > Best wishes, > Amelchev Nikita -- Best wishes, Amelchev Nikita |
Igniters,
I've looked through Nikita's changes and I think for the current issue [1] we should not allow the existence of MOVING partitions in the cluster (it must be stable) to run the lightweight PME on BLT node leave event occurred to achieve truly unlocked operations and here are my thoughts why. In general, as Nikita mentioned above, the existence of MOVING partitions in the cluster means that the rebalance procedure is currently running. It owns cache partitions locally and sends in the background (with additional timeout) the actual statuses of his local partitions to the coordinator node. So, we will always have a lag between local node partition states and all other cluster nodes partitions states. This lag can be very huge since previous #scheduleResendPartitions() is cancelled when a new cache group rebalance finished. Without the fair partition states synchronization (without full PME) and in case of local affinity recalculation on BLT node leave event, other nodes will mark such partitions LOST in most of the cases, which in fact are present in the cluster and saved on some node under checkpoint. I see that it cannot be solved by saving transition states of such partitions on each node. As for the case when the coordinator will calculate affinity and send "full map" to other nodes, I think it is better here to focus on designing a new lightweight PME when the rebalancing process finishes. Сurrently full distributed PME will occur anyway by the coordinator by sending CacheAffinityChaneMessage, but I think we can avoid it here, since no new MOVING or OWNING node partition states are introduced and all the previous mappings are still valid. We don't need a distributed PME if we will leave partition primaries on those nodes where they were, just set correct partition statuses via a light discovery message. So, my plan here can be: Phase 1. Lightweight PME on BLT node leave on a stable cluster (no MOVING partitions); Phase 2. Lightweight PME on BLT node finishes its rebalance procedure. Folks, Nikita, WDYT? [1] https://issues.apache.org/jira/browse/IGNITE-9913 On Fri, 24 May 2019 at 13:31, Nikita Amelchev <[hidden email]> wrote: > > Hello, Igniters! > > I am working on the implementation of lightweight PME for the case of > a BLT node leave. [1] > > There is a question: whether to allow lightweight PME if the cluster > has MOVING partitions? > > The problems that may happen if allow: > - Nodes can differently select the primary node from current OWNING backups. > - One part of nodes can mark a partition as LOST and another one as OWNING. > > We can take states of the partitions from the node2part map. The root > cause of those problems is that when rebalancing ends (get the last > message), it updates partition state of the local node to OWNING (and > schedules partitions resend). This may lead to different affinity > re-calculations on nodes. > > I see two solutions: > > 1. Nodes will store “moving-owning” transition of partitions state > until the rebalancing ends. Each node will locally recalculate the > affinity on node left event. > 2. The coordinator will calculate affinity and send "full map" to > nodes. In this case, nodes still should wait for topology change event > (to get correct topology in discovery). > > If disallow lightweight PME when the cluster has MOVING partitions - > there are no problems and it works fine. > > Any thoughts? > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > пт, 29 мар. 2019 г. в 15:00, Nikita Amelchev <[hidden email]>: > > > > Pavel, > > I have provided MTCGA bot status in Jira issue comments. [1] > > > > Eduard, > > Yes, for current implementation it will be distributed PME if > > in-memory caches configured. > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev <[hidden email]>: > > > > > > Nikita, > > > > > > It sounds cool. But I didn't get about in-memory caches. The baseline is > > > not used for their affinity calculation. > > > So, this improvement would be switched off for them or completely (when > > > such caches are presented), wouldn't it? > > > > > > On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> wrote: > > > > > > > Hi Nikita, > > > > > > > > Thank you for your work. This is great improvement. I'll take look on it in > > > > next couple of days. Could you please run TC and provide MTCGA bot status > > > > about this change? > > > > > > > > чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: > > > > > > > > > Hello, Igniters! > > > > > > > > > > I have implemented lightweight version of partitions map exchange for > > > > > the case when the baseline node leaves topology. [1] > > > > > > > > > > If partitions are assigned according to the baseline topology and > > > > > server node leaves there's no actual need to perform distributed PME. > > > > > Every cluster will recalculate new affinity assignments and partition > > > > > states locally. There is no need to wait for partitions released and > > > > > PME will be started immediately. > > > > > > > > > > I have benchmarked duration of PME under yardstick load. PME duration > > > > > was decreased up to 10 times and the maximum latency of transactions > > > > > was decreased up to 4-5 times. See details in Jira issue comments. [1] > > > > > > > > > > Could some expert of PME take a look at my changes? [2] > > > > > > > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > > 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 > > > > > > > > > > -- > > > > > Best wishes, > > > > > Amelchev Nikita > > > > > > > > > > > > > > > > > -- > > Best wishes, > > Amelchev Nikita > > > > -- > Best wishes, > Amelchev Nikita |
Maksim,
I agree with you that we should implement current issue and do not allow lightweight PME if there are MOVING partitions in the cluster. But now I'm investigating issue about finalizing update counters cause it assumes that finalizing happens on exchange and all cache updates are completed. Here we can wrong process update counters gaps and break recently merged IGNITE-10078. And about phase 2, correct me if I misunderstood you. You suggest do not move primary partitions on rebalancing completing (do not change affinity assignment)? In this case, nodes recently join to cluster will not have primary partitions and won't get a load after rebalancing. чт, 30 мая 2019 г. в 19:55, Maxim Muzafarov <[hidden email]>: > > Igniters, > > > I've looked through Nikita's changes and I think for the current issue > [1] we should not allow the existence of MOVING partitions in the > cluster (it must be stable) to run the lightweight PME on BLT node > leave event occurred to achieve truly unlocked operations and here are > my thoughts why. > > In general, as Nikita mentioned above, the existence of MOVING > partitions in the cluster means that the rebalance procedure is > currently running. It owns cache partitions locally and sends in the > background (with additional timeout) the actual statuses of his local > partitions to the coordinator node. So, we will always have a lag > between local node partition states and all other cluster nodes > partitions states. This lag can be very huge since previous > #scheduleResendPartitions() is cancelled when a new cache group > rebalance finished. Without the fair partition states synchronization > (without full PME) and in case of local affinity recalculation on BLT > node leave event, other nodes will mark such partitions LOST in most > of the cases, which in fact are present in the cluster and saved on > some node under checkpoint. I see that it cannot be solved by saving > transition states of such partitions on each node. > > As for the case when the coordinator will calculate affinity and send > "full map" to other nodes, I think it is better here to focus on > designing a new lightweight PME when the rebalancing process finishes. > Сurrently full distributed PME will occur anyway by the coordinator by > sending CacheAffinityChaneMessage, but I think we can avoid it here, > since no new MOVING or OWNING node partition states are introduced and > all the previous mappings are still valid. We don't need a distributed > PME if we will leave partition primaries on those nodes where they > were, just set correct partition statuses via a light discovery > message. > > So, my plan here can be: > Phase 1. Lightweight PME on BLT node leave on a stable cluster (no > MOVING partitions); > Phase 2. Lightweight PME on BLT node finishes its rebalance procedure. > > Folks, Nikita, > WDYT? > > [1] https://issues.apache.org/jira/browse/IGNITE-9913 > > On Fri, 24 May 2019 at 13:31, Nikita Amelchev <[hidden email]> wrote: > > > > Hello, Igniters! > > > > I am working on the implementation of lightweight PME for the case of > > a BLT node leave. [1] > > > > There is a question: whether to allow lightweight PME if the cluster > > has MOVING partitions? > > > > The problems that may happen if allow: > > - Nodes can differently select the primary node from current OWNING backups. > > - One part of nodes can mark a partition as LOST and another one as OWNING. > > > > We can take states of the partitions from the node2part map. The root > > cause of those problems is that when rebalancing ends (get the last > > message), it updates partition state of the local node to OWNING (and > > schedules partitions resend). This may lead to different affinity > > re-calculations on nodes. > > > > I see two solutions: > > > > 1. Nodes will store “moving-owning” transition of partitions state > > until the rebalancing ends. Each node will locally recalculate the > > affinity on node left event. > > 2. The coordinator will calculate affinity and send "full map" to > > nodes. In this case, nodes still should wait for topology change event > > (to get correct topology in discovery). > > > > If disallow lightweight PME when the cluster has MOVING partitions - > > there are no problems and it works fine. > > > > Any thoughts? > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > пт, 29 мар. 2019 г. в 15:00, Nikita Amelchev <[hidden email]>: > > > > > > Pavel, > > > I have provided MTCGA bot status in Jira issue comments. [1] > > > > > > Eduard, > > > Yes, for current implementation it will be distributed PME if > > > in-memory caches configured. > > > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > > > пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev <[hidden email]>: > > > > > > > > Nikita, > > > > > > > > It sounds cool. But I didn't get about in-memory caches. The baseline is > > > > not used for their affinity calculation. > > > > So, this improvement would be switched off for them or completely (when > > > > such caches are presented), wouldn't it? > > > > > > > > On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> wrote: > > > > > > > > > Hi Nikita, > > > > > > > > > > Thank you for your work. This is great improvement. I'll take look on it in > > > > > next couple of days. Could you please run TC and provide MTCGA bot status > > > > > about this change? > > > > > > > > > > чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: > > > > > > > > > > > Hello, Igniters! > > > > > > > > > > > > I have implemented lightweight version of partitions map exchange for > > > > > > the case when the baseline node leaves topology. [1] > > > > > > > > > > > > If partitions are assigned according to the baseline topology and > > > > > > server node leaves there's no actual need to perform distributed PME. > > > > > > Every cluster will recalculate new affinity assignments and partition > > > > > > states locally. There is no need to wait for partitions released and > > > > > > PME will be started immediately. > > > > > > > > > > > > I have benchmarked duration of PME under yardstick load. PME duration > > > > > > was decreased up to 10 times and the maximum latency of transactions > > > > > > was decreased up to 4-5 times. See details in Jira issue comments. [1] > > > > > > > > > > > > Could some expert of PME take a look at my changes? [2] > > > > > > > > > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > > > 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 > > > > > > > > > > > > -- > > > > > > Best wishes, > > > > > > Amelchev Nikita > > > > > > > > > > > > > > > > > > > > > > > -- > > > Best wishes, > > > Amelchev Nikita > > > > > > > > -- > > Best wishes, > > Amelchev Nikita -- Best wishes, Amelchev Nikita |
Hi, Igniters.
I'm working on the implementation of lightweight PME for a baseline node leave case. [1] In my implementation, each node recalculates a new affinity and completes PME locally without distributed communication. This is possible because there are all partitions are distributed according to the baseline topology. And I found two possible blockers to do it without blocking updates: 1. Finalize partitions counter. It seems that we can't correctly collect gaps and process them without completing all txs. See the GridDhtPartitionTopologyImpl#finalizeUpdateCounters method. 2. Apply update counters. We can't correctly set HWM counter if primary left the cluster and sent updates to part of backups. Such updates can be processed later and break guarantee that LWM<=HWM. Is it impossible to leave a baseline node without waiting for all txs completed? 1. https://issues.apache.org/jira/browse/IGNITE-9913 ср, 5 июн. 2019 г. в 12:15, Nikita Amelchev <[hidden email]>: > > Maksim, > > I agree with you that we should implement current issue and do not > allow lightweight PME if there are MOVING partitions in the cluster. > > But now I'm investigating issue about finalizing update counters cause > it assumes that finalizing happens on exchange and all cache updates > are completed. Here we can wrong process update counters gaps and > break recently merged IGNITE-10078. > > And about phase 2, correct me if I misunderstood you. > You suggest do not move primary partitions on rebalancing completing > (do not change affinity assignment)? In this case, nodes recently join > to cluster will not have primary partitions and won't get a load after > rebalancing. > > чт, 30 мая 2019 г. в 19:55, Maxim Muzafarov <[hidden email]>: > > > > Igniters, > > > > > > I've looked through Nikita's changes and I think for the current issue > > [1] we should not allow the existence of MOVING partitions in the > > cluster (it must be stable) to run the lightweight PME on BLT node > > leave event occurred to achieve truly unlocked operations and here are > > my thoughts why. > > > > In general, as Nikita mentioned above, the existence of MOVING > > partitions in the cluster means that the rebalance procedure is > > currently running. It owns cache partitions locally and sends in the > > background (with additional timeout) the actual statuses of his local > > partitions to the coordinator node. So, we will always have a lag > > between local node partition states and all other cluster nodes > > partitions states. This lag can be very huge since previous > > #scheduleResendPartitions() is cancelled when a new cache group > > rebalance finished. Without the fair partition states synchronization > > (without full PME) and in case of local affinity recalculation on BLT > > node leave event, other nodes will mark such partitions LOST in most > > of the cases, which in fact are present in the cluster and saved on > > some node under checkpoint. I see that it cannot be solved by saving > > transition states of such partitions on each node. > > > > As for the case when the coordinator will calculate affinity and send > > "full map" to other nodes, I think it is better here to focus on > > designing a new lightweight PME when the rebalancing process finishes. > > Сurrently full distributed PME will occur anyway by the coordinator by > > sending CacheAffinityChaneMessage, but I think we can avoid it here, > > since no new MOVING or OWNING node partition states are introduced and > > all the previous mappings are still valid. We don't need a distributed > > PME if we will leave partition primaries on those nodes where they > > were, just set correct partition statuses via a light discovery > > message. > > > > So, my plan here can be: > > Phase 1. Lightweight PME on BLT node leave on a stable cluster (no > > MOVING partitions); > > Phase 2. Lightweight PME on BLT node finishes its rebalance procedure. > > > > Folks, Nikita, > > WDYT? > > > > [1] https://issues.apache.org/jira/browse/IGNITE-9913 > > > > On Fri, 24 May 2019 at 13:31, Nikita Amelchev <[hidden email]> wrote: > > > > > > Hello, Igniters! > > > > > > I am working on the implementation of lightweight PME for the case of > > > a BLT node leave. [1] > > > > > > There is a question: whether to allow lightweight PME if the cluster > > > has MOVING partitions? > > > > > > The problems that may happen if allow: > > > - Nodes can differently select the primary node from current OWNING backups. > > > - One part of nodes can mark a partition as LOST and another one as OWNING. > > > > > > We can take states of the partitions from the node2part map. The root > > > cause of those problems is that when rebalancing ends (get the last > > > message), it updates partition state of the local node to OWNING (and > > > schedules partitions resend). This may lead to different affinity > > > re-calculations on nodes. > > > > > > I see two solutions: > > > > > > 1. Nodes will store “moving-owning” transition of partitions state > > > until the rebalancing ends. Each node will locally recalculate the > > > affinity on node left event. > > > 2. The coordinator will calculate affinity and send "full map" to > > > nodes. In this case, nodes still should wait for topology change event > > > (to get correct topology in discovery). > > > > > > If disallow lightweight PME when the cluster has MOVING partitions - > > > there are no problems and it works fine. > > > > > > Any thoughts? > > > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > > > пт, 29 мар. 2019 г. в 15:00, Nikita Amelchev <[hidden email]>: > > > > > > > > Pavel, > > > > I have provided MTCGA bot status in Jira issue comments. [1] > > > > > > > > Eduard, > > > > Yes, for current implementation it will be distributed PME if > > > > in-memory caches configured. > > > > > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > > > > > пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev <[hidden email]>: > > > > > > > > > > Nikita, > > > > > > > > > > It sounds cool. But I didn't get about in-memory caches. The baseline is > > > > > not used for their affinity calculation. > > > > > So, this improvement would be switched off for them or completely (when > > > > > such caches are presented), wouldn't it? > > > > > > > > > > On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> wrote: > > > > > > > > > > > Hi Nikita, > > > > > > > > > > > > Thank you for your work. This is great improvement. I'll take look on it in > > > > > > next couple of days. Could you please run TC and provide MTCGA bot status > > > > > > about this change? > > > > > > > > > > > > чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: > > > > > > > > > > > > > Hello, Igniters! > > > > > > > > > > > > > > I have implemented lightweight version of partitions map exchange for > > > > > > > the case when the baseline node leaves topology. [1] > > > > > > > > > > > > > > If partitions are assigned according to the baseline topology and > > > > > > > server node leaves there's no actual need to perform distributed PME. > > > > > > > Every cluster will recalculate new affinity assignments and partition > > > > > > > states locally. There is no need to wait for partitions released and > > > > > > > PME will be started immediately. > > > > > > > > > > > > > > I have benchmarked duration of PME under yardstick load. PME duration > > > > > > > was decreased up to 10 times and the maximum latency of transactions > > > > > > > was decreased up to 4-5 times. See details in Jira issue comments. [1] > > > > > > > > > > > > > > Could some expert of PME take a look at my changes? [2] > > > > > > > > > > > > > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > > > > > > 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 > > > > > > > > > > > > > > -- > > > > > > > Best wishes, > > > > > > > Amelchev Nikita > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > Best wishes, > > > > Amelchev Nikita > > > > > > > > > > > > -- > > > Best wishes, > > > Amelchev Nikita > > > > -- > Best wishes, > Amelchev Nikita -- Best wishes, Amelchev Nikita |
Hi Nikita,
I've checked out your branch, looked through the changes and run IgniteBaselineNodeLeaveExchangeTest. Some thoughts: 1. First of all, there's fundamental issue that backup and primary partitions behave differently: - On primary, updating transaction needs to own exclusive lock (be on top of GridCacheMapEntry#localCandidates queue) on key object for the whole prepare-commit cycle. That's how two-phase commit works in Ignite. - Primary node generates update counter via PartitionTxUpdateCounterImpl#reserve, while backup receives update and just applies it with provided counter. So, if we'll perform PME in non-distributed way, we'll lose happen-before guarantees between updates of transactions mapped on previous topology and updates of transactions that are mapped to new topology. This may cause the following issues: - New primary node may start behaving as primary (spawn DHT transaction instances and acquire exclusive locks) but still may receive updates from previous primary. I don't know how to handle these updates correctly as they may conflict with new updates and locks. - New primary node should start generating update counters, but it actually doesn't know last update counter in cluster. If it optimistically will start from last known counter, partition consistency may break in case updates with actual last update counter will arrive (I guess, this issue should be reproduced as LWM > HWM assertion error). 2. According to current state of your test, testBltServerLeaveUnderLoad is called only with PickKeyOption#NO_DATA_ON_LEAVING_NODE (which means backups that are promoted to primaries without global synchronization are not affected by transactional load). However, it still fails with LWM > HWM assertion. I guess, there are another details in new partition counters implementation that require global happen-before between updates of transactions that are mapped to different topology versions. Alex S, > backups that are promoted to primaries without global synchronization > are not affected by transactional load Best Regards, Ivan Rakov On 01.07.2019 11:13, Nikita Amelchev wrote: > Hi, Igniters. > > I'm working on the implementation of lightweight PME for a baseline > node leave case. [1] In my implementation, each node recalculates a > new affinity and completes PME locally without distributed > communication. This is possible because there are all partitions are > distributed according to the baseline topology. And I found two > possible blockers to do it without blocking updates: > > 1. Finalize partitions counter. It seems that we can't correctly > collect gaps and process them without completing all txs. See the > GridDhtPartitionTopologyImpl#finalizeUpdateCounters method. > > 2. Apply update counters. We can't correctly set HWM counter if > primary left the cluster and sent updates to part of backups. Such > updates can be processed later and break guarantee that LWM<=HWM. > > Is it impossible to leave a baseline node without waiting for all txs completed? > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > ср, 5 июн. 2019 г. в 12:15, Nikita Amelchev <[hidden email]>: >> Maksim, >> >> I agree with you that we should implement current issue and do not >> allow lightweight PME if there are MOVING partitions in the cluster. >> >> But now I'm investigating issue about finalizing update counters cause >> it assumes that finalizing happens on exchange and all cache updates >> are completed. Here we can wrong process update counters gaps and >> break recently merged IGNITE-10078. >> >> And about phase 2, correct me if I misunderstood you. >> You suggest do not move primary partitions on rebalancing completing >> (do not change affinity assignment)? In this case, nodes recently join >> to cluster will not have primary partitions and won't get a load after >> rebalancing. >> >> чт, 30 мая 2019 г. в 19:55, Maxim Muzafarov <[hidden email]>: >>> Igniters, >>> >>> >>> I've looked through Nikita's changes and I think for the current issue >>> [1] we should not allow the existence of MOVING partitions in the >>> cluster (it must be stable) to run the lightweight PME on BLT node >>> leave event occurred to achieve truly unlocked operations and here are >>> my thoughts why. >>> >>> In general, as Nikita mentioned above, the existence of MOVING >>> partitions in the cluster means that the rebalance procedure is >>> currently running. It owns cache partitions locally and sends in the >>> background (with additional timeout) the actual statuses of his local >>> partitions to the coordinator node. So, we will always have a lag >>> between local node partition states and all other cluster nodes >>> partitions states. This lag can be very huge since previous >>> #scheduleResendPartitions() is cancelled when a new cache group >>> rebalance finished. Without the fair partition states synchronization >>> (without full PME) and in case of local affinity recalculation on BLT >>> node leave event, other nodes will mark such partitions LOST in most >>> of the cases, which in fact are present in the cluster and saved on >>> some node under checkpoint. I see that it cannot be solved by saving >>> transition states of such partitions on each node. >>> >>> As for the case when the coordinator will calculate affinity and send >>> "full map" to other nodes, I think it is better here to focus on >>> designing a new lightweight PME when the rebalancing process finishes. >>> Сurrently full distributed PME will occur anyway by the coordinator by >>> sending CacheAffinityChaneMessage, but I think we can avoid it here, >>> since no new MOVING or OWNING node partition states are introduced and >>> all the previous mappings are still valid. We don't need a distributed >>> PME if we will leave partition primaries on those nodes where they >>> were, just set correct partition statuses via a light discovery >>> message. >>> >>> So, my plan here can be: >>> Phase 1. Lightweight PME on BLT node leave on a stable cluster (no >>> MOVING partitions); >>> Phase 2. Lightweight PME on BLT node finishes its rebalance procedure. >>> >>> Folks, Nikita, >>> WDYT? >>> >>> [1] https://issues.apache.org/jira/browse/IGNITE-9913 >>> >>> On Fri, 24 May 2019 at 13:31, Nikita Amelchev <[hidden email]> wrote: >>>> Hello, Igniters! >>>> >>>> I am working on the implementation of lightweight PME for the case of >>>> a BLT node leave. [1] >>>> >>>> There is a question: whether to allow lightweight PME if the cluster >>>> has MOVING partitions? >>>> >>>> The problems that may happen if allow: >>>> - Nodes can differently select the primary node from current OWNING backups. >>>> - One part of nodes can mark a partition as LOST and another one as OWNING. >>>> >>>> We can take states of the partitions from the node2part map. The root >>>> cause of those problems is that when rebalancing ends (get the last >>>> message), it updates partition state of the local node to OWNING (and >>>> schedules partitions resend). This may lead to different affinity >>>> re-calculations on nodes. >>>> >>>> I see two solutions: >>>> >>>> 1. Nodes will store “moving-owning” transition of partitions state >>>> until the rebalancing ends. Each node will locally recalculate the >>>> affinity on node left event. >>>> 2. The coordinator will calculate affinity and send "full map" to >>>> nodes. In this case, nodes still should wait for topology change event >>>> (to get correct topology in discovery). >>>> >>>> If disallow lightweight PME when the cluster has MOVING partitions - >>>> there are no problems and it works fine. >>>> >>>> Any thoughts? >>>> >>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>> >>>> пт, 29 мар. 2019 г. в 15:00, Nikita Amelchev <[hidden email]>: >>>>> Pavel, >>>>> I have provided MTCGA bot status in Jira issue comments. [1] >>>>> >>>>> Eduard, >>>>> Yes, for current implementation it will be distributed PME if >>>>> in-memory caches configured. >>>>> >>>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>>> >>>>> пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev <[hidden email]>: >>>>>> Nikita, >>>>>> >>>>>> It sounds cool. But I didn't get about in-memory caches. The baseline is >>>>>> not used for their affinity calculation. >>>>>> So, this improvement would be switched off for them or completely (when >>>>>> such caches are presented), wouldn't it? >>>>>> >>>>>> On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> wrote: >>>>>> >>>>>>> Hi Nikita, >>>>>>> >>>>>>> Thank you for your work. This is great improvement. I'll take look on it in >>>>>>> next couple of days. Could you please run TC and provide MTCGA bot status >>>>>>> about this change? >>>>>>> >>>>>>> чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: >>>>>>> >>>>>>>> Hello, Igniters! >>>>>>>> >>>>>>>> I have implemented lightweight version of partitions map exchange for >>>>>>>> the case when the baseline node leaves topology. [1] >>>>>>>> >>>>>>>> If partitions are assigned according to the baseline topology and >>>>>>>> server node leaves there's no actual need to perform distributed PME. >>>>>>>> Every cluster will recalculate new affinity assignments and partition >>>>>>>> states locally. There is no need to wait for partitions released and >>>>>>>> PME will be started immediately. >>>>>>>> >>>>>>>> I have benchmarked duration of PME under yardstick load. PME duration >>>>>>>> was decreased up to 10 times and the maximum latency of transactions >>>>>>>> was decreased up to 4-5 times. See details in Jira issue comments. [1] >>>>>>>> >>>>>>>> Could some expert of PME take a look at my changes? [2] >>>>>>>> >>>>>>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>>>>>> 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 >>>>>>>> >>>>>>>> -- >>>>>>>> Best wishes, >>>>>>>> Amelchev Nikita >>>>>>>> >>>>> >>>>> >>>>> -- >>>>> Best wishes, >>>>> Amelchev Nikita >>>> >>>> >>>> -- >>>> Best wishes, >>>> Amelchev Nikita >> >> >> -- >> Best wishes, >> Amelchev Nikita > > |
My bad, I've sent the message accidentally. What I wanted to ask:
Alex S, > backups that are promoted to primaries without global synchronization > are not affected by transactional load > test still fails with LWM > HWM assertion Do you have any ideas why this may happen? > New primary node should start generating update counters, but it > actually doesn't know last update counter in cluster. If it > optimistically will start from last known counter, partition > consistency may break in case updates with actual last update counter > will arrive (I guess, this issue should be reproduced as LWM > HWM > assertion error). How do you think, does this problem looks solvable? Alex S and Alex G, > New primary node may start behaving as primary (spawn DHT transaction > instances and acquire exclusive locks) but still may receive updates > from previous primary. I don't know how to handle these updates > correctly as they may conflict with new updates and locks. How do you think, can we overcome this limitation with existing transaction implementation? Best Regards, Ivan Rakov On 10.07.2019 2:25, Ivan Rakov wrote: > Hi Nikita, > > I've checked out your branch, looked through the changes and run > IgniteBaselineNodeLeaveExchangeTest. Some thoughts: > > 1. First of all, there's fundamental issue that backup and primary > partitions behave differently: > - On primary, updating transaction needs to own exclusive lock (be on > top of GridCacheMapEntry#localCandidates queue) on key object for the > whole prepare-commit cycle. That's how two-phase commit works in Ignite. > - Primary node generates update counter via > PartitionTxUpdateCounterImpl#reserve, while backup receives update and > just applies it with provided counter. > So, if we'll perform PME in non-distributed way, we'll lose > happen-before guarantees between updates of transactions mapped on > previous topology and updates of transactions that are mapped to new > topology. This may cause the following issues: > - New primary node may start behaving as primary (spawn DHT > transaction instances and acquire exclusive locks) but still may > receive updates from previous primary. I don't know how to handle > these updates correctly as they may conflict with new updates and locks. > - New primary node should start generating update counters, but it > actually doesn't know last update counter in cluster. If it > optimistically will start from last known counter, partition > consistency may break in case updates with actual last update counter > will arrive (I guess, this issue should be reproduced as LWM > HWM > assertion error). > > 2. According to current state of your test, > testBltServerLeaveUnderLoad is called only with > PickKeyOption#NO_DATA_ON_LEAVING_NODE (which means backups that are > promoted to primaries without global synchronization are not affected > by transactional load). However, it still fails with LWM > HWM > assertion. I guess, there are another details in new partition > counters implementation that require global happen-before between > updates of transactions that are mapped to different topology versions. > > Alex S, > >> backups that are promoted to primaries without global synchronization >> are not affected by transactional load > > > Best Regards, > Ivan Rakov > > On 01.07.2019 11:13, Nikita Amelchev wrote: >> Hi, Igniters. >> >> I'm working on the implementation of lightweight PME for a baseline >> node leave case. [1] In my implementation, each node recalculates a >> new affinity and completes PME locally without distributed >> communication. This is possible because there are all partitions are >> distributed according to the baseline topology. And I found two >> possible blockers to do it without blocking updates: >> >> 1. Finalize partitions counter. It seems that we can't correctly >> collect gaps and process them without completing all txs. See the >> GridDhtPartitionTopologyImpl#finalizeUpdateCounters method. >> >> 2. Apply update counters. We can't correctly set HWM counter if >> primary left the cluster and sent updates to part of backups. Such >> updates can be processed later and break guarantee that LWM<=HWM. >> >> Is it impossible to leave a baseline node without waiting for all txs >> completed? >> >> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >> >> ср, 5 июн. 2019 г. в 12:15, Nikita Amelchev <[hidden email]>: >>> Maksim, >>> >>> I agree with you that we should implement current issue and do not >>> allow lightweight PME if there are MOVING partitions in the cluster. >>> >>> But now I'm investigating issue about finalizing update counters cause >>> it assumes that finalizing happens on exchange and all cache updates >>> are completed. Here we can wrong process update counters gaps and >>> break recently merged IGNITE-10078. >>> >>> And about phase 2, correct me if I misunderstood you. >>> You suggest do not move primary partitions on rebalancing completing >>> (do not change affinity assignment)? In this case, nodes recently join >>> to cluster will not have primary partitions and won't get a load after >>> rebalancing. >>> >>> чт, 30 мая 2019 г. в 19:55, Maxim Muzafarov <[hidden email]>: >>>> Igniters, >>>> >>>> >>>> I've looked through Nikita's changes and I think for the current issue >>>> [1] we should not allow the existence of MOVING partitions in the >>>> cluster (it must be stable) to run the lightweight PME on BLT node >>>> leave event occurred to achieve truly unlocked operations and here are >>>> my thoughts why. >>>> >>>> In general, as Nikita mentioned above, the existence of MOVING >>>> partitions in the cluster means that the rebalance procedure is >>>> currently running. It owns cache partitions locally and sends in the >>>> background (with additional timeout) the actual statuses of his local >>>> partitions to the coordinator node. So, we will always have a lag >>>> between local node partition states and all other cluster nodes >>>> partitions states. This lag can be very huge since previous >>>> #scheduleResendPartitions() is cancelled when a new cache group >>>> rebalance finished. Without the fair partition states synchronization >>>> (without full PME) and in case of local affinity recalculation on BLT >>>> node leave event, other nodes will mark such partitions LOST in most >>>> of the cases, which in fact are present in the cluster and saved on >>>> some node under checkpoint. I see that it cannot be solved by saving >>>> transition states of such partitions on each node. >>>> >>>> As for the case when the coordinator will calculate affinity and send >>>> "full map" to other nodes, I think it is better here to focus on >>>> designing a new lightweight PME when the rebalancing process finishes. >>>> Сurrently full distributed PME will occur anyway by the coordinator by >>>> sending CacheAffinityChaneMessage, but I think we can avoid it here, >>>> since no new MOVING or OWNING node partition states are introduced and >>>> all the previous mappings are still valid. We don't need a distributed >>>> PME if we will leave partition primaries on those nodes where they >>>> were, just set correct partition statuses via a light discovery >>>> message. >>>> >>>> So, my plan here can be: >>>> Phase 1. Lightweight PME on BLT node leave on a stable cluster (no >>>> MOVING partitions); >>>> Phase 2. Lightweight PME on BLT node finishes its rebalance procedure. >>>> >>>> Folks, Nikita, >>>> WDYT? >>>> >>>> [1] https://issues.apache.org/jira/browse/IGNITE-9913 >>>> >>>> On Fri, 24 May 2019 at 13:31, Nikita Amelchev >>>> <[hidden email]> wrote: >>>>> Hello, Igniters! >>>>> >>>>> I am working on the implementation of lightweight PME for the case of >>>>> a BLT node leave. [1] >>>>> >>>>> There is a question: whether to allow lightweight PME if the cluster >>>>> has MOVING partitions? >>>>> >>>>> The problems that may happen if allow: >>>>> - Nodes can differently select the primary node from current >>>>> OWNING backups. >>>>> - One part of nodes can mark a partition as LOST and another one >>>>> as OWNING. >>>>> >>>>> We can take states of the partitions from the node2part map. The root >>>>> cause of those problems is that when rebalancing ends (get the last >>>>> message), it updates partition state of the local node to OWNING (and >>>>> schedules partitions resend). This may lead to different affinity >>>>> re-calculations on nodes. >>>>> >>>>> I see two solutions: >>>>> >>>>> 1. Nodes will store “moving-owning” transition of partitions state >>>>> until the rebalancing ends. Each node will locally recalculate the >>>>> affinity on node left event. >>>>> 2. The coordinator will calculate affinity and send "full map" to >>>>> nodes. In this case, nodes still should wait for topology change >>>>> event >>>>> (to get correct topology in discovery). >>>>> >>>>> If disallow lightweight PME when the cluster has MOVING partitions - >>>>> there are no problems and it works fine. >>>>> >>>>> Any thoughts? >>>>> >>>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>>> >>>>> пт, 29 мар. 2019 г. в 15:00, Nikita Amelchev <[hidden email]>: >>>>>> Pavel, >>>>>> I have provided MTCGA bot status in Jira issue comments. [1] >>>>>> >>>>>> Eduard, >>>>>> Yes, for current implementation it will be distributed PME if >>>>>> in-memory caches configured. >>>>>> >>>>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>>>> >>>>>> пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev >>>>>> <[hidden email]>: >>>>>>> Nikita, >>>>>>> >>>>>>> It sounds cool. But I didn't get about in-memory caches. The >>>>>>> baseline is >>>>>>> not used for their affinity calculation. >>>>>>> So, this improvement would be switched off for them or >>>>>>> completely (when >>>>>>> such caches are presented), wouldn't it? >>>>>>> >>>>>>> On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko >>>>>>> <[hidden email]> wrote: >>>>>>> >>>>>>>> Hi Nikita, >>>>>>>> >>>>>>>> Thank you for your work. This is great improvement. I'll take >>>>>>>> look on it in >>>>>>>> next couple of days. Could you please run TC and provide MTCGA >>>>>>>> bot status >>>>>>>> about this change? >>>>>>>> >>>>>>>> чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev >>>>>>>> <[hidden email]>: >>>>>>>> >>>>>>>>> Hello, Igniters! >>>>>>>>> >>>>>>>>> I have implemented lightweight version of partitions map >>>>>>>>> exchange for >>>>>>>>> the case when the baseline node leaves topology. [1] >>>>>>>>> >>>>>>>>> If partitions are assigned according to the baseline topology and >>>>>>>>> server node leaves there's no actual need to perform >>>>>>>>> distributed PME. >>>>>>>>> Every cluster will recalculate new affinity assignments and >>>>>>>>> partition >>>>>>>>> states locally. There is no need to wait for partitions >>>>>>>>> released and >>>>>>>>> PME will be started immediately. >>>>>>>>> >>>>>>>>> I have benchmarked duration of PME under yardstick load. PME >>>>>>>>> duration >>>>>>>>> was decreased up to 10 times and the maximum latency of >>>>>>>>> transactions >>>>>>>>> was decreased up to 4-5 times. See details in Jira issue >>>>>>>>> comments. [1] >>>>>>>>> >>>>>>>>> Could some expert of PME take a look at my changes? [2] >>>>>>>>> >>>>>>>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>>>>>>> 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 >>>>>>>>> >>>>>>>>> -- >>>>>>>>> Best wishes, >>>>>>>>> Amelchev Nikita >>>>>>>>> >>>>>> >>>>>> >>>>>> -- >>>>>> Best wishes, >>>>>> Amelchev Nikita >>>>> >>>>> >>>>> -- >>>>> Best wishes, >>>>> Amelchev Nikita >>> >>> >>> -- >>> Best wishes, >>> Amelchev Nikita >> >> |
In reply to this post by Nikita Amelchev
Hi Nikita,
I've checked out your branch, looked through the changes and run IgniteBaselineNodeLeaveExchangeTest. Some thoughts: 1. First of all, there's fundamental issue that backup and primary partitions behave differently: - On primary, updating transaction needs to own exclusive lock (be on top of GridCacheMapEntry#localCandidates queue) on key object for the whole prepare-commit cycle. That's how two-phase commit works in Ignite. - Primary node generates update counter via PartitionTxUpdateCounterImpl#reserve, while backup receives update and just applies it with provided counter. So, if we'll perform PME in non-distributed way, we'll lose happen-before guarantees between updates of transactions mapped on previous topology and ones that are mapped to new topology. This may cause the following issues: - New primary node may start behaving as primary (spawn DHT transaction instances and acquire exclusive locks) but still may receive updates from previous primary. I don't know how to handle these updates correctly as they may conflict with new updates and locks. - New primary node should start generating update counters, but it actually doesn't know last update counter in cluster. If it optimistically will start from last known counter, partition consistency may break in case updates with actual last update counter will arrive (I guess, this issue should be reproduced as LWM > HWM assertion error). 2. According to current state of your test, testBltServerLeaveUnderLoad is called only with PickKeyOption#NO_DATA_ON_LEAVING_NODE (which means backups that are promoted to primaries without global synchronization are not affected by transactional load). However, it still fails with LWM > HWM assertion. I guess, there are another details in new partition counters implementation that require global happen-before between updates of transactions that are mapped to different topology versions. Alex S, > backups that are promoted to primaries without global synchronization > are not affected by transactional load > test still fails with LWM > HWM assertion Do you have any ideas why this may happen? > New primary node should start generating update counters, but it > actually doesn't know last update counter in cluster. If it > optimistically will start from last known counter, partition > consistency may break in case updates with actual last update counter > will arrive (I guess, this issue should be reproduced as LWM > HWM > assertion error). How do you think, does this problem looks solvable? Alex S and Alex G, > New primary node may start behaving as primary (spawn DHT transaction > instances and acquire exclusive locks) but still may receive updates > from previous primary. I don't know how to handle these updates > correctly as they may conflict with new updates and locks. How do you think, can we overcome this limitation with our existing implementation of transactions? Best Regards, Ivan Rakov On 01.07.2019 11:13, Nikita Amelchev wrote: > Hi, Igniters. > > I'm working on the implementation of lightweight PME for a baseline > node leave case. [1] In my implementation, each node recalculates a > new affinity and completes PME locally without distributed > communication. This is possible because there are all partitions are > distributed according to the baseline topology. And I found two > possible blockers to do it without blocking updates: > > 1. Finalize partitions counter. It seems that we can't correctly > collect gaps and process them without completing all txs. See the > GridDhtPartitionTopologyImpl#finalizeUpdateCounters method. > > 2. Apply update counters. We can't correctly set HWM counter if > primary left the cluster and sent updates to part of backups. Such > updates can be processed later and break guarantee that LWM<=HWM. > > Is it impossible to leave a baseline node without waiting for all txs completed? > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > ср, 5 июн. 2019 г. в 12:15, Nikita Amelchev <[hidden email]>: >> Maksim, >> >> I agree with you that we should implement current issue and do not >> allow lightweight PME if there are MOVING partitions in the cluster. >> >> But now I'm investigating issue about finalizing update counters cause >> it assumes that finalizing happens on exchange and all cache updates >> are completed. Here we can wrong process update counters gaps and >> break recently merged IGNITE-10078. >> >> And about phase 2, correct me if I misunderstood you. >> You suggest do not move primary partitions on rebalancing completing >> (do not change affinity assignment)? In this case, nodes recently join >> to cluster will not have primary partitions and won't get a load after >> rebalancing. >> >> чт, 30 мая 2019 г. в 19:55, Maxim Muzafarov <[hidden email]>: >>> Igniters, >>> >>> >>> I've looked through Nikita's changes and I think for the current issue >>> [1] we should not allow the existence of MOVING partitions in the >>> cluster (it must be stable) to run the lightweight PME on BLT node >>> leave event occurred to achieve truly unlocked operations and here are >>> my thoughts why. >>> >>> In general, as Nikita mentioned above, the existence of MOVING >>> partitions in the cluster means that the rebalance procedure is >>> currently running. It owns cache partitions locally and sends in the >>> background (with additional timeout) the actual statuses of his local >>> partitions to the coordinator node. So, we will always have a lag >>> between local node partition states and all other cluster nodes >>> partitions states. This lag can be very huge since previous >>> #scheduleResendPartitions() is cancelled when a new cache group >>> rebalance finished. Without the fair partition states synchronization >>> (without full PME) and in case of local affinity recalculation on BLT >>> node leave event, other nodes will mark such partitions LOST in most >>> of the cases, which in fact are present in the cluster and saved on >>> some node under checkpoint. I see that it cannot be solved by saving >>> transition states of such partitions on each node. >>> >>> As for the case when the coordinator will calculate affinity and send >>> "full map" to other nodes, I think it is better here to focus on >>> designing a new lightweight PME when the rebalancing process finishes. >>> Сurrently full distributed PME will occur anyway by the coordinator by >>> sending CacheAffinityChaneMessage, but I think we can avoid it here, >>> since no new MOVING or OWNING node partition states are introduced and >>> all the previous mappings are still valid. We don't need a distributed >>> PME if we will leave partition primaries on those nodes where they >>> were, just set correct partition statuses via a light discovery >>> message. >>> >>> So, my plan here can be: >>> Phase 1. Lightweight PME on BLT node leave on a stable cluster (no >>> MOVING partitions); >>> Phase 2. Lightweight PME on BLT node finishes its rebalance procedure. >>> >>> Folks, Nikita, >>> WDYT? >>> >>> [1] https://issues.apache.org/jira/browse/IGNITE-9913 >>> >>> On Fri, 24 May 2019 at 13:31, Nikita Amelchev <[hidden email]> wrote: >>>> Hello, Igniters! >>>> >>>> I am working on the implementation of lightweight PME for the case of >>>> a BLT node leave. [1] >>>> >>>> There is a question: whether to allow lightweight PME if the cluster >>>> has MOVING partitions? >>>> >>>> The problems that may happen if allow: >>>> - Nodes can differently select the primary node from current OWNING backups. >>>> - One part of nodes can mark a partition as LOST and another one as OWNING. >>>> >>>> We can take states of the partitions from the node2part map. The root >>>> cause of those problems is that when rebalancing ends (get the last >>>> message), it updates partition state of the local node to OWNING (and >>>> schedules partitions resend). This may lead to different affinity >>>> re-calculations on nodes. >>>> >>>> I see two solutions: >>>> >>>> 1. Nodes will store “moving-owning” transition of partitions state >>>> until the rebalancing ends. Each node will locally recalculate the >>>> affinity on node left event. >>>> 2. The coordinator will calculate affinity and send "full map" to >>>> nodes. In this case, nodes still should wait for topology change event >>>> (to get correct topology in discovery). >>>> >>>> If disallow lightweight PME when the cluster has MOVING partitions - >>>> there are no problems and it works fine. >>>> >>>> Any thoughts? >>>> >>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>> >>>> пт, 29 мар. 2019 г. в 15:00, Nikita Amelchev <[hidden email]>: >>>>> Pavel, >>>>> I have provided MTCGA bot status in Jira issue comments. [1] >>>>> >>>>> Eduard, >>>>> Yes, for current implementation it will be distributed PME if >>>>> in-memory caches configured. >>>>> >>>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>>> >>>>> пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev <[hidden email]>: >>>>>> Nikita, >>>>>> >>>>>> It sounds cool. But I didn't get about in-memory caches. The baseline is >>>>>> not used for their affinity calculation. >>>>>> So, this improvement would be switched off for them or completely (when >>>>>> such caches are presented), wouldn't it? >>>>>> >>>>>> On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> wrote: >>>>>> >>>>>>> Hi Nikita, >>>>>>> >>>>>>> Thank you for your work. This is great improvement. I'll take look on it in >>>>>>> next couple of days. Could you please run TC and provide MTCGA bot status >>>>>>> about this change? >>>>>>> >>>>>>> чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]>: >>>>>>> >>>>>>>> Hello, Igniters! >>>>>>>> >>>>>>>> I have implemented lightweight version of partitions map exchange for >>>>>>>> the case when the baseline node leaves topology. [1] >>>>>>>> >>>>>>>> If partitions are assigned according to the baseline topology and >>>>>>>> server node leaves there's no actual need to perform distributed PME. >>>>>>>> Every cluster will recalculate new affinity assignments and partition >>>>>>>> states locally. There is no need to wait for partitions released and >>>>>>>> PME will be started immediately. >>>>>>>> >>>>>>>> I have benchmarked duration of PME under yardstick load. PME duration >>>>>>>> was decreased up to 10 times and the maximum latency of transactions >>>>>>>> was decreased up to 4-5 times. See details in Jira issue comments. [1] >>>>>>>> >>>>>>>> Could some expert of PME take a look at my changes? [2] >>>>>>>> >>>>>>>> 1. https://issues.apache.org/jira/browse/IGNITE-9913 >>>>>>>> 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 >>>>>>>> >>>>>>>> -- >>>>>>>> Best wishes, >>>>>>>> Amelchev Nikita >>>>>>>> >>>>> >>>>> >>>>> -- >>>>> Best wishes, >>>>> Amelchev Nikita >>>> >>>> >>>> -- >>>> Best wishes, >>>> Amelchev Nikita >> >> >> -- >> Best wishes, >> Amelchev Nikita > > |
Ivan, NIkita
"test still fails with LWM > HWM assertion" I recently did some fixed on this in GridGain CE but still not donated to Apache. Currently I'm on vacation and have plans to donate in the start of August. "How do you think, does this problem looks solvable? " Looks like it's impossible to avoid situation when we have two primary nodes for the same partition during topology change without some kind of distributed synchronization. At least I'm not having synchronization free solution right now. This is because nodes are receiving topology events "eventually". TX for the same partition may start for example on two client nodes having different ready topology version, causing mapping on different primaries. But we can try to make it as small and lightweight as possible. For example, for BLT node left event it's enough to wait until all active transactions which are mapped as primary on left node will be finished by recovery (committed or rolled back), before new transactions are allowed to map on new primaries. In general we must hold the invariant: all transaction to affected partitions are finished before new update are allowed on new primary nodes. All other transactions are free to proceed. My current very high level view on lightweigh PME algorithm (for simplicity considering BLT node left): 1. Instead of full PME we should only wait for distributed latch event when all affected transactions are finished by recovery. No other tx waiting/blocking is required. 2. Exchange future is finished when all affected txs are finished. 3. All non-affected transactions are mapped on previous topology version. 4. Affected transactions are waiting for new affinity version to avoid collisions with counters (HWM < LWM) due to mutiple primary nodes. Of course, I may miss some details, but in general such approach should work. Alex G, do you have objections ? ср, 10 июл. 2019 г. в 02:40, Ivan Rakov <[hidden email]>: > Hi Nikita, > > I've checked out your branch, looked through the changes and run > IgniteBaselineNodeLeaveExchangeTest. Some thoughts: > > 1. First of all, there's fundamental issue that backup and primary > partitions behave differently: > - On primary, updating transaction needs to own exclusive lock (be on top > of GridCacheMapEntry#localCandidates queue) on key object for the whole > prepare-commit cycle. That's how two-phase commit works in Ignite. > - Primary node generates update counter via > PartitionTxUpdateCounterImpl#reserve, while backup receives update and just > applies it with provided counter. > So, if we'll perform PME in non-distributed way, we'll lose happen-before > guarantees between updates of transactions mapped on previous topology and > ones that are mapped to new topology. This may cause the following issues: > - New primary node may start behaving as primary (spawn DHT transaction > instances and acquire exclusive locks) but still may receive updates from > previous primary. I don't know how to handle these updates correctly as > they may conflict with new updates and locks. > - New primary node should start generating update counters, but it > actually doesn't know last update counter in cluster. If it optimistically > will start from last known counter, partition consistency may break in case > updates with actual last update counter will arrive (I guess, this issue > should be reproduced as LWM > HWM assertion error). > > 2. According to current state of your test, testBltServerLeaveUnderLoad is > called only with PickKeyOption#NO_DATA_ON_LEAVING_NODE (which means backups > that are promoted to primaries without global synchronization are not > affected by transactional load). However, it still fails with LWM > HWM > assertion. I guess, there are another details in new partition counters > implementation that require global happen-before between updates of > transactions that are mapped to different topology versions. > > Alex S, > > backups that are promoted to primaries without global synchronization are > not affected by transactional load > > test still fails with LWM > HWM assertion > > Do you have any ideas why this may happen? > > New primary node should start generating update counters, but it actually > doesn't know last update counter in cluster. If it optimistically will > start from last known counter, partition consistency may break in case > updates with actual last update counter will arrive (I guess, this issue > should be reproduced as LWM > HWM assertion error). > > How do you think, does this problem looks solvable? > > Alex S and Alex G, > > New primary node may start behaving as primary (spawn DHT transaction > instances and acquire exclusive locks) but still may receive updates from > previous primary. I don't know how to handle these updates correctly as > they may conflict with new updates and locks. > > How do you think, can we overcome this limitation with our existing > implementation of transactions? > > Best Regards, > Ivan Rakov > > On 01.07.2019 11:13, Nikita Amelchev wrote: > > Hi, Igniters. > > I'm working on the implementation of lightweight PME for a baseline > node leave case. [1] In my implementation, each node recalculates a > new affinity and completes PME locally without distributed > communication. This is possible because there are all partitions are > distributed according to the baseline topology. And I found two > possible blockers to do it without blocking updates: > > 1. Finalize partitions counter. It seems that we can't correctly > collect gaps and process them without completing all txs. See the > GridDhtPartitionTopologyImpl#finalizeUpdateCounters method. > > 2. Apply update counters. We can't correctly set HWM counter if > primary left the cluster and sent updates to part of backups. Such > updates can be processed later and break guarantee that LWM<=HWM. > > Is it impossible to leave a baseline node without waiting for all txs completed? > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > ср, 5 июн. 2019 г. в 12:15, Nikita Amelchev <[hidden email]> <[hidden email]>: > > Maksim, > > I agree with you that we should implement current issue and do not > allow lightweight PME if there are MOVING partitions in the cluster. > > But now I'm investigating issue about finalizing update counters cause > it assumes that finalizing happens on exchange and all cache updates > are completed. Here we can wrong process update counters gaps and > break recently merged IGNITE-10078. > > And about phase 2, correct me if I misunderstood you. > You suggest do not move primary partitions on rebalancing completing > (do not change affinity assignment)? In this case, nodes recently join > to cluster will not have primary partitions and won't get a load after > rebalancing. > > чт, 30 мая 2019 г. в 19:55, Maxim Muzafarov <[hidden email]> <[hidden email]>: > > Igniters, > > > I've looked through Nikita's changes and I think for the current issue > [1] we should not allow the existence of MOVING partitions in the > cluster (it must be stable) to run the lightweight PME on BLT node > leave event occurred to achieve truly unlocked operations and here are > my thoughts why. > > In general, as Nikita mentioned above, the existence of MOVING > partitions in the cluster means that the rebalance procedure is > currently running. It owns cache partitions locally and sends in the > background (with additional timeout) the actual statuses of his local > partitions to the coordinator node. So, we will always have a lag > between local node partition states and all other cluster nodes > partitions states. This lag can be very huge since previous > #scheduleResendPartitions() is cancelled when a new cache group > rebalance finished. Without the fair partition states synchronization > (without full PME) and in case of local affinity recalculation on BLT > node leave event, other nodes will mark such partitions LOST in most > of the cases, which in fact are present in the cluster and saved on > some node under checkpoint. I see that it cannot be solved by saving > transition states of such partitions on each node. > > As for the case when the coordinator will calculate affinity and send > "full map" to other nodes, I think it is better here to focus on > designing a new lightweight PME when the rebalancing process finishes. > Сurrently full distributed PME will occur anyway by the coordinator by > sending CacheAffinityChaneMessage, but I think we can avoid it here, > since no new MOVING or OWNING node partition states are introduced and > all the previous mappings are still valid. We don't need a distributed > PME if we will leave partition primaries on those nodes where they > were, just set correct partition statuses via a light discovery > message. > > So, my plan here can be: > Phase 1. Lightweight PME on BLT node leave on a stable cluster (no > MOVING partitions); > Phase 2. Lightweight PME on BLT node finishes its rebalance procedure. > > Folks, Nikita, > WDYT? > > [1] https://issues.apache.org/jira/browse/IGNITE-9913 > > On Fri, 24 May 2019 at 13:31, Nikita Amelchev <[hidden email]> <[hidden email]> wrote: > > Hello, Igniters! > > I am working on the implementation of lightweight PME for the case of > a BLT node leave. [1] > > There is a question: whether to allow lightweight PME if the cluster > has MOVING partitions? > > The problems that may happen if allow: > - Nodes can differently select the primary node from current OWNING backups. > - One part of nodes can mark a partition as LOST and another one as OWNING. > > We can take states of the partitions from the node2part map. The root > cause of those problems is that when rebalancing ends (get the last > message), it updates partition state of the local node to OWNING (and > schedules partitions resend). This may lead to different affinity > re-calculations on nodes. > > I see two solutions: > > 1. Nodes will store “moving-owning” transition of partitions state > until the rebalancing ends. Each node will locally recalculate the > affinity on node left event. > 2. The coordinator will calculate affinity and send "full map" to > nodes. In this case, nodes still should wait for topology change event > (to get correct topology in discovery). > > If disallow lightweight PME when the cluster has MOVING partitions - > there are no problems and it works fine. > > Any thoughts? > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > пт, 29 мар. 2019 г. в 15:00, Nikita Amelchev <[hidden email]> <[hidden email]>: > > Pavel, > I have provided MTCGA bot status in Jira issue comments. [1] > > Eduard, > Yes, for current implementation it will be distributed PME if > in-memory caches configured. > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > > пт, 29 мар. 2019 г. в 14:49, Eduard Shangareev <[hidden email]> <[hidden email]>: > > Nikita, > > It sounds cool. But I didn't get about in-memory caches. The baseline is > not used for their affinity calculation. > So, this improvement would be switched off for them or completely (when > such caches are presented), wouldn't it? > > On Thu, Mar 28, 2019 at 3:14 PM Pavel Kovalenko <[hidden email]> <[hidden email]> wrote: > > > Hi Nikita, > > Thank you for your work. This is great improvement. I'll take look on it in > next couple of days. Could you please run TC and provide MTCGA bot status > about this change? > > чт, 28 мар. 2019 г. в 14:29, Nikita Amelchev <[hidden email]> <[hidden email]>: > > > Hello, Igniters! > > I have implemented lightweight version of partitions map exchange for > the case when the baseline node leaves topology. [1] > > If partitions are assigned according to the baseline topology and > server node leaves there's no actual need to perform distributed PME. > Every cluster will recalculate new affinity assignments and partition > states locally. There is no need to wait for partitions released and > PME will be started immediately. > > I have benchmarked duration of PME under yardstick load. PME duration > was decreased up to 10 times and the maximum latency of transactions > was decreased up to 4-5 times. See details in Jira issue comments. [1] > > Could some expert of PME take a look at my changes? [2] > > 1. https://issues.apache.org/jira/browse/IGNITE-9913 > 2. https://reviews.ignite.apache.org/ignite/review/IGNT-CR-1027 > > -- > Best wishes, > Amelchev Nikita > > > -- > Best wishes, > Amelchev Nikita > > -- > Best wishes, > Amelchev Nikita > > -- > Best wishes, > Amelchev Nikita > > -- Best regards, Alexei Scherbakov |
Free forum by Nabble | Edit this page |