rpuch commented on code in PR #4987:
URL: https://github.com/apache/ignite-3/pull/4987#discussion_r1920245951


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/GroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage;
+
+import java.util.Map;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+
+/** Persists and retrieves intent to complete storages destruction on node 
start. */

Review Comment:
   ```suggestion
   /** Persists and retrieves intents to destroy Raft group storages. */
   ```



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/impl/VaultGroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage.impl;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.raft.storage.GroupStoragesDestructionIntents;
+import org.apache.ignite.internal.replicator.PartitionGroupId;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+
+/** Uses VaultManager to destroy raft group storages durably, using vault to 
store destruction intents. */

Review Comment:
   ```suggestion
   /** Uses VaultManager to store destruction intents. */
   ```



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/GroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage;
+
+import java.util.Map;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+
+/** Persists and retrieves intent to complete storages destruction on node 
start. */
+public interface GroupStoragesDestructionIntents {
+    /** Add configurer for CMG or metastorage raft storages. */
+    void addGroupOptionsConfigurer(ReplicationGroupId groupId, 
RaftGroupOptionsConfigurer groupOptionsConfigurer);
+
+    /** Add configurer for partitions raft storages. */
+    void addPartitionGroupOptionsConfigurer(RaftGroupOptionsConfigurer 
partitionRaftConfigurer);
+
+    /** Save intent to destroy raft storages. */
+    void saveDestroyStorageIntent(RaftNodeId nodeId, RaftGroupOptions 
groupOptions);
+
+    /** Remove intent to destroy raft storages. */
+    void removeDestroyStorageIntent(String nodeId);
+
+    /** Returns group options needed to destroy raft storages, mapped by node 
id represented by String. */
+    Map<String, RaftGroupOptions> readGroupOptionsByNodeIdForDestruction();

Review Comment:
   Could it happen that the number of groups to destroy is huge? Whould it make 
sense to have a streaming API here?



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/GroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage;
+
+import java.util.Map;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+
+/** Persists and retrieves intent to complete storages destruction on node 
start. */
+public interface GroupStoragesDestructionIntents {
+    /** Add configurer for CMG or metastorage raft storages. */
+    void addGroupOptionsConfigurer(ReplicationGroupId groupId, 
RaftGroupOptionsConfigurer groupOptionsConfigurer);
+
+    /** Add configurer for partitions raft storages. */
+    void addPartitionGroupOptionsConfigurer(RaftGroupOptionsConfigurer 
partitionRaftConfigurer);
+
+    /** Save intent to destroy raft storages. */
+    void saveDestroyStorageIntent(RaftNodeId nodeId, RaftGroupOptions 
groupOptions);

Review Comment:
   From `groupOptions`, the implementation of this method only uses 
`isVolatile` flag, so it seems that the whole object should not be passed in



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/impl/VaultGroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage.impl;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.raft.storage.GroupStoragesDestructionIntents;
+import org.apache.ignite.internal.replicator.PartitionGroupId;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+
+/** Uses VaultManager to destroy raft group storages durably, using vault to 
store destruction intents. */
+public class VaultGroupStoragesDestructionIntents implements 
GroupStoragesDestructionIntents {
+    private static final byte[] GROUP_STORAGE_DESTRUCTION_PREFIX = 
"destroy.group.storages.".getBytes(UTF_8);
+    private static final ByteOrder BYTE_UTILS_BYTE_ORDER = 
ByteOrder.BIG_ENDIAN;
+
+    private static final String PARTITION_GROUP_NAME = "partition";
+
+    private static final int RAFT_GROUPS = 3;
+
+    private final VaultManager vault;
+
+    private final ConcurrentMap<String, RaftGroupOptionsConfigurer> 
configurerByName = new ConcurrentHashMap<>();
+
+    /** Constructor. */
+    public VaultGroupStoragesDestructionIntents(VaultManager vault) {
+        this.vault = vault;
+    }
+
+    @Override
+    public void addGroupOptionsConfigurer(ReplicationGroupId groupId, 
RaftGroupOptionsConfigurer groupOptionsConfigurer) {
+        configurerByName.put(groupId.toString(), groupOptionsConfigurer);
+    }
+
+    @Override
+    public void addPartitionGroupOptionsConfigurer(RaftGroupOptionsConfigurer 
groupOptionsConfigurer) {
+        configurerByName.put(PARTITION_GROUP_NAME, groupOptionsConfigurer);
+    }
+
+    @Override
+    public void saveDestroyStorageIntent(RaftNodeId nodeId, RaftGroupOptions 
groupOptions) {
+        String configurerName = nodeId.groupId() instanceof PartitionGroupId ? 
PARTITION_GROUP_NAME : nodeId.groupId().toString();

Review Comment:
   It seems too much for this class to know a difference between partitions and 
not partitions; it should not know about partitions at all. Probably the method 
could accept an object that would serialize the 'intent' (or just build the 
intent object)



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/impl/VaultGroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage.impl;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.raft.storage.GroupStoragesDestructionIntents;
+import org.apache.ignite.internal.replicator.PartitionGroupId;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+
+/** Uses VaultManager to destroy raft group storages durably, using vault to 
store destruction intents. */
+public class VaultGroupStoragesDestructionIntents implements 
GroupStoragesDestructionIntents {
+    private static final byte[] GROUP_STORAGE_DESTRUCTION_PREFIX = 
"destroy.group.storages.".getBytes(UTF_8);
+    private static final ByteOrder BYTE_UTILS_BYTE_ORDER = 
ByteOrder.BIG_ENDIAN;
+
+    private static final String PARTITION_GROUP_NAME = "partition";
+
+    private static final int RAFT_GROUPS = 3;
+
+    private final VaultManager vault;
+
+    private final ConcurrentMap<String, RaftGroupOptionsConfigurer> 
configurerByName = new ConcurrentHashMap<>();
+
+    /** Constructor. */
+    public VaultGroupStoragesDestructionIntents(VaultManager vault) {
+        this.vault = vault;
+    }
+
+    @Override
+    public void addGroupOptionsConfigurer(ReplicationGroupId groupId, 
RaftGroupOptionsConfigurer groupOptionsConfigurer) {
+        configurerByName.put(groupId.toString(), groupOptionsConfigurer);
+    }
+
+    @Override
+    public void addPartitionGroupOptionsConfigurer(RaftGroupOptionsConfigurer 
groupOptionsConfigurer) {
+        configurerByName.put(PARTITION_GROUP_NAME, groupOptionsConfigurer);
+    }
+
+    @Override
+    public void saveDestroyStorageIntent(RaftNodeId nodeId, RaftGroupOptions 
groupOptions) {
+        String configurerName = nodeId.groupId() instanceof PartitionGroupId ? 
PARTITION_GROUP_NAME : nodeId.groupId().toString();
+
+        vault.put(
+                buildKey(nodeId.nodeIdStringForStorage()),
+                toBytes(new DestroyStorageIntent(configurerName, 
groupOptions.volatileStores()))
+        );
+    }
+
+    @Override
+    public void removeDestroyStorageIntent(String nodeId) {
+        vault.remove(buildKey(nodeId));
+    }
+
+    private static ByteArray buildKey(String nodeId) {
+        byte[] nodeIdBytes = nodeId.getBytes(UTF_8);
+
+        byte[] key = 
ByteBuffer.allocate(GROUP_STORAGE_DESTRUCTION_PREFIX.length + 
nodeIdBytes.length)
+                .order(BYTE_UTILS_BYTE_ORDER)
+                .put(GROUP_STORAGE_DESTRUCTION_PREFIX)
+                .put(nodeIdBytes)
+                .array();
+
+        return new ByteArray(key);
+    }
+
+    private static String raftNodeIdFromKey(byte[] key) {
+        return new String(key, GROUP_STORAGE_DESTRUCTION_PREFIX.length, 
key.length - GROUP_STORAGE_DESTRUCTION_PREFIX.length, UTF_8);
+    }
+
+    @Override
+    public Map<String, RaftGroupOptions> 
readGroupOptionsByNodeIdForDestruction() {
+        assert configurerByName.size() == RAFT_GROUPS
+                : "Configurers for CMG, metastorage and partitions must be 
added, got: " + configurerByName.keySet();
+
+        try (Cursor<VaultEntry> cursor = vault.prefix(new 
ByteArray(GROUP_STORAGE_DESTRUCTION_PREFIX))) {
+            Map<String, RaftGroupOptions> result = new HashMap<>();
+
+            while (cursor.hasNext()) {
+                VaultEntry next = cursor.next();
+
+                String nodeId = raftNodeIdFromKey(next.key().bytes());
+
+                // todo add serializer

Review Comment:
   Unresolved TODO



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/impl/VaultGroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage.impl;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.raft.storage.GroupStoragesDestructionIntents;
+import org.apache.ignite.internal.replicator.PartitionGroupId;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+
+/** Uses VaultManager to destroy raft group storages durably, using vault to 
store destruction intents. */
+public class VaultGroupStoragesDestructionIntents implements 
GroupStoragesDestructionIntents {
+    private static final byte[] GROUP_STORAGE_DESTRUCTION_PREFIX = 
"destroy.group.storages.".getBytes(UTF_8);
+    private static final ByteOrder BYTE_UTILS_BYTE_ORDER = 
ByteOrder.BIG_ENDIAN;
+
+    private static final String PARTITION_GROUP_NAME = "partition";
+
+    private static final int RAFT_GROUPS = 3;

Review Comment:
   These are not groups but something like spaces



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/GroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage;
+
+import java.util.Map;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+
+/** Persists and retrieves intent to complete storages destruction on node 
start. */
+public interface GroupStoragesDestructionIntents {
+    /** Add configurer for CMG or metastorage raft storages. */
+    void addGroupOptionsConfigurer(ReplicationGroupId groupId, 
RaftGroupOptionsConfigurer groupOptionsConfigurer);
+
+    /** Add configurer for partitions raft storages. */
+    void addPartitionGroupOptionsConfigurer(RaftGroupOptionsConfigurer 
partitionRaftConfigurer);

Review Comment:
   These methods should not be on the interface. Ideally, the 3 configurers 
should be passed via constructor. If it's not possible, then just the 
implementation based on Vault should have some way to pass the configurers.



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/impl/VaultGroupStoragesDestructionIntents.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft.storage.impl;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.util.ByteUtils.toBytes;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.internal.raft.RaftGroupOptionsConfigurer;
+import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftGroupOptions;
+import org.apache.ignite.internal.raft.storage.GroupStoragesDestructionIntents;
+import org.apache.ignite.internal.replicator.PartitionGroupId;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.vault.VaultEntry;
+import org.apache.ignite.internal.vault.VaultManager;
+
+/** Uses VaultManager to destroy raft group storages durably, using vault to 
store destruction intents. */
+public class VaultGroupStoragesDestructionIntents implements 
GroupStoragesDestructionIntents {
+    private static final byte[] GROUP_STORAGE_DESTRUCTION_PREFIX = 
"destroy.group.storages.".getBytes(UTF_8);
+    private static final ByteOrder BYTE_UTILS_BYTE_ORDER = 
ByteOrder.BIG_ENDIAN;
+
+    private static final String PARTITION_GROUP_NAME = "partition";
+
+    private static final int RAFT_GROUPS = 3;
+
+    private final VaultManager vault;
+
+    private final ConcurrentMap<String, RaftGroupOptionsConfigurer> 
configurerByName = new ConcurrentHashMap<>();
+
+    /** Constructor. */
+    public VaultGroupStoragesDestructionIntents(VaultManager vault) {
+        this.vault = vault;
+    }
+
+    @Override
+    public void addGroupOptionsConfigurer(ReplicationGroupId groupId, 
RaftGroupOptionsConfigurer groupOptionsConfigurer) {
+        configurerByName.put(groupId.toString(), groupOptionsConfigurer);
+    }
+
+    @Override
+    public void addPartitionGroupOptionsConfigurer(RaftGroupOptionsConfigurer 
groupOptionsConfigurer) {
+        configurerByName.put(PARTITION_GROUP_NAME, groupOptionsConfigurer);
+    }
+
+    @Override
+    public void saveDestroyStorageIntent(RaftNodeId nodeId, RaftGroupOptions 
groupOptions) {
+        String configurerName = nodeId.groupId() instanceof PartitionGroupId ? 
PARTITION_GROUP_NAME : nodeId.groupId().toString();
+
+        vault.put(
+                buildKey(nodeId.nodeIdStringForStorage()),
+                toBytes(new DestroyStorageIntent(configurerName, 
groupOptions.volatileStores()))
+        );
+    }
+
+    @Override
+    public void removeDestroyStorageIntent(String nodeId) {
+        vault.remove(buildKey(nodeId));
+    }
+
+    private static ByteArray buildKey(String nodeId) {
+        byte[] nodeIdBytes = nodeId.getBytes(UTF_8);
+
+        byte[] key = 
ByteBuffer.allocate(GROUP_STORAGE_DESTRUCTION_PREFIX.length + 
nodeIdBytes.length)
+                .order(BYTE_UTILS_BYTE_ORDER)
+                .put(GROUP_STORAGE_DESTRUCTION_PREFIX)
+                .put(nodeIdBytes)
+                .array();
+
+        return new ByteArray(key);
+    }
+
+    private static String raftNodeIdFromKey(byte[] key) {
+        return new String(key, GROUP_STORAGE_DESTRUCTION_PREFIX.length, 
key.length - GROUP_STORAGE_DESTRUCTION_PREFIX.length, UTF_8);
+    }
+
+    @Override
+    public Map<String, RaftGroupOptions> 
readGroupOptionsByNodeIdForDestruction() {
+        assert configurerByName.size() == RAFT_GROUPS
+                : "Configurers for CMG, metastorage and partitions must be 
added, got: " + configurerByName.keySet();
+
+        try (Cursor<VaultEntry> cursor = vault.prefix(new 
ByteArray(GROUP_STORAGE_DESTRUCTION_PREFIX))) {
+            Map<String, RaftGroupOptions> result = new HashMap<>();
+
+            while (cursor.hasNext()) {
+                VaultEntry next = cursor.next();
+
+                String nodeId = raftNodeIdFromKey(next.key().bytes());
+
+                // todo add serializer
+                DestroyStorageIntent intent = 
ByteUtils.fromBytes(next.value());
+
+                RaftGroupOptions groupOptions = intent.isVolatile

Review Comment:
   I don't like that we build group options just to conform to the 'group 
options configurer' abstraction, we are making a loop.. But mybe we should 
change this later (probably with revisiting the 'group options configurer' 
abstraction itself)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to