[ 
https://issues.apache.org/jira/browse/IGNITE-17084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ivan Bessonov updated IGNITE-17084:
-----------------------------------
    Description: 
General idea of full rebalance is described in 
https://issues.apache.org/jira/browse/IGNITE-17083

For persistent storages, there's an option to avoid copy-on-write rebalance 
algorithms if it's desired. Intuitively, it's a preferable option. Each storage 
chooses its own format.

In this case, RocksDB allows consistent db iteration using a "Snapshot" 
feature. Idea is very simple:
 * Take a RoackDB snapshot.
 * Iterate through partition data.
 * Iterate through indexes.
 * Relese the snapshot.

There must be a common "infrastructure" or a framework to stream native 
rebalance snapshots. Data format should be as simple as possible.

NOTE: of course, it has to be mentioned that this approach might lead to 
ineffective storage space usage. What I mean is that "previous" versions of 
values, in terms of RocksDB, must be stored on the device if they're visible 
from any of snapshots. It can be a problem in theory, but in practice full 
rebalance isn't expected to occur often, and event then we don't expect that 
users will rewrite the entire partition data in a span of a single rebalance.
h2. Possible problems

Given that "raw" data is sent, including sql indexes, all incompleted indexes 
will be sent incompleted. Maybe we should also send a build state for each 
index so that the receiving side could continue from the right place, not from 
the beginning.

This problem will be resolved in the future. Currently we don't have indexes 
implemented.

  was:
General idea of full rebalance is described in 
https://issues.apache.org/jira/browse/IGNITE-17083

For persistent storages, there's an option to avoid copy-on-write rebalance 
algorithms if it's desired. Intuitively, it's a preferable option. Each storage 
chooses its own format.

In this case, RocksDB allows consistent db iteration using a "Snapshot" 
feature. Idea is very simple:
 * Take a RoackDB snapshot.
 * Iterate through partition data.
 * Iterate through indexes.
 * Relese the snapshot.

There must be a common "infrastructure" or a framework to stream native 
rebalance snapshots. Data format should be as simple as possible.
h2. Possible problems

Given that "raw" data is sent, including sql indexes, all incompleted indexes 
will be sent incompleted. Maybe we should also send a build state for each 
index so that the receiving side could continue from the right place, not from 
the beginning.

This problem will be resolved in the future. Currently we don't have indexes 
implemented.


> Native rebalance for RocksDB partitions
> ---------------------------------------
>
>                 Key: IGNITE-17084
>                 URL: https://issues.apache.org/jira/browse/IGNITE-17084
>             Project: Ignite
>          Issue Type: Improvement
>            Reporter: Ivan Bessonov
>            Priority: Major
>              Labels: ignite-3
>
> General idea of full rebalance is described in 
> https://issues.apache.org/jira/browse/IGNITE-17083
> For persistent storages, there's an option to avoid copy-on-write rebalance 
> algorithms if it's desired. Intuitively, it's a preferable option. Each 
> storage chooses its own format.
> In this case, RocksDB allows consistent db iteration using a "Snapshot" 
> feature. Idea is very simple:
>  * Take a RoackDB snapshot.
>  * Iterate through partition data.
>  * Iterate through indexes.
>  * Relese the snapshot.
> There must be a common "infrastructure" or a framework to stream native 
> rebalance snapshots. Data format should be as simple as possible.
> NOTE: of course, it has to be mentioned that this approach might lead to 
> ineffective storage space usage. What I mean is that "previous" versions of 
> values, in terms of RocksDB, must be stored on the device if they're visible 
> from any of snapshots. It can be a problem in theory, but in practice full 
> rebalance isn't expected to occur often, and event then we don't expect that 
> users will rewrite the entire partition data in a span of a single rebalance.
> h2. Possible problems
> Given that "raw" data is sent, including sql indexes, all incompleted indexes 
> will be sent incompleted. Maybe we should also send a build state for each 
> index so that the receiving side could continue from the right place, not 
> from the beginning.
> This problem will be resolved in the future. Currently we don't have indexes 
> implemented.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to