You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ib...@apache.org on 2023/01/20 14:51:33 UTC
[ignite-3] branch main updated: IGNITE-18397 Rework Watches based on Raft Learners (#1490)
This is an automated email from the ASF dual-hosted git repository.
ibessonov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new db8f1e38a5 IGNITE-18397 Rework Watches based on Raft Learners (#1490)
db8f1e38a5 is described below
commit db8f1e38a5e0fab7b2195dec91d8b9913a384b00
Author: Alexander Polovtcev <al...@gmail.com>
AuthorDate: Fri Jan 20 17:51:26 2023 +0300
IGNITE-18397 Rework Watches based on Raft Learners (#1490)
---
modules/distribution-zones/build.gradle | 2 -
.../distributionzones/DistributionZoneManager.java | 147 +++---
...ibutionZoneManagerConfigurationChangesTest.java | 4 +-
...butionZoneManagerLogicalTopologyEventsTest.java | 4 +-
.../DistributionZoneManagerWatchListenerTest.java | 15 +-
.../internal/metastorage/MetaStorageManager.java | 19 +-
.../ignite/internal/metastorage/WatchEvent.java | 30 +-
.../ignite/internal/metastorage/WatchListener.java | 10 +-
modules/metastorage/build.gradle | 1 +
.../impl/ItMetaStorageManagerImplTest.java | 11 +-
.../impl/ItMetaStorageServicePersistenceTest.java | 51 +-
.../metastorage/impl/ItMetaStorageServiceTest.java | 532 +++++++++------------
.../metastorage/impl/ItMetaStorageWatchTest.java | 336 +++++++++++++
.../server/raft/ItMetaStorageRaftGroupTest.java | 77 ++-
.../command/MetastorageCommandsMessageGroup.java | 6 -
.../metastorage/command/WatchExactKeysCommand.java | 82 ----
.../metastorage/command/WatchRangeKeysCommand.java | 54 ---
.../metastorage/impl/MetaStorageManagerImpl.java | 515 ++++++++------------
.../metastorage/impl/MetaStorageService.java | 117 ++---
.../metastorage/impl/MetaStorageServiceImpl.java | 266 +----------
.../metastorage/server/KeyValueStorage.java | 54 ++-
.../ignite/internal/metastorage/server/Watch.java | 89 ++++
.../server/persistence/RocksDbKeyValueStorage.java | 338 +++++++++++--
.../server/persistence/WatchCursor.java | 185 -------
.../server/raft/MetaStorageLearnerListener.java | 73 +++
.../server/raft/MetaStorageListener.java | 334 +------------
.../server/raft/MetaStorageWriteHandler.java | 245 ++++++++++
.../metastorage/watch/AggregatedWatch.java | 68 ---
.../internal/metastorage/watch/KeyCriterion.java | 369 --------------
.../metastorage/watch/WatchAggregator.java | 263 ----------
.../impl/MetaStorageRangeCursorTest.java | 12 +-
.../server/AbstractKeyValueStorageTest.java | 529 ++++++++++----------
.../server/RocksDbKeyValueStorageTest.java | 70 ++-
.../server/SimpleInMemoryKeyValueStorageTest.java | 2 +-
.../metastorage/watch/WatchAggregatorTest.java | 312 ------------
.../server/SimpleInMemoryKeyValueStorage.java | 437 ++++++++---------
.../internal/raft/service/RaftGroupService.java | 4 +-
.../ignite/internal/raft/RaftGroupServiceImpl.java | 2 +-
.../service/ItAbstractListenerSnapshotTest.java | 27 +-
.../ItDistributedConfigurationPropertiesTest.java | 21 +-
.../ItDistributedConfigurationStorageTest.java | 8 +-
.../storage/ItRebalanceDistributedTest.java | 2 +-
.../internal/runner/app/ItDataSchemaSyncTest.java | 2 +-
.../runner/app/ItIgniteNodeRestartTest.java | 119 +++--
.../internal/runner/app/ItTablesApiTest.java | 2 -
.../internal/sql/engine/ItIndexSpoolTest.java | 30 +-
.../internal/sql/engine/ItSecondaryIndexTest.java | 34 +-
.../ignite/internal/table/ItTableScanTest.java | 36 +-
.../internal/test/WatchListenerInhibitor.java | 152 ++----
.../org/apache/ignite/internal/app/IgniteImpl.java | 2 +-
.../storage/DistributedConfigurationStorage.java | 40 +-
.../DistributedConfigurationCatchUpTest.java | 11 +-
.../DistributedConfigurationStorageTest.java | 2 +-
.../ignite/distributed/ItTablePersistenceTest.java | 7 +-
.../internal/table/distributed/TableManager.java | 24 +-
55 files changed, 2604 insertions(+), 3580 deletions(-)
diff --git a/modules/distribution-zones/build.gradle b/modules/distribution-zones/build.gradle
index 5d7b34fb8b..5be2d24f47 100644
--- a/modules/distribution-zones/build.gradle
+++ b/modules/distribution-zones/build.gradle
@@ -32,8 +32,6 @@ dependencies {
implementation project(':ignite-api')
implementation project(':ignite-cluster-management')
implementation project(':ignite-metastorage-api')
- // TODO: remove this dependency, see https://issues.apache.org/jira/browse/IGNITE-18397
- implementation project(':ignite-metastorage')
implementation project(':ignite-vault')
implementation project(':ignite-configuration')
implementation project(':ignite-schema')
diff --git a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
index 5d82f6cfae..cf9abba0c4 100644
--- a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
+++ b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
@@ -70,7 +70,6 @@ import org.apache.ignite.internal.metastorage.dsl.CompoundCondition;
import org.apache.ignite.internal.metastorage.dsl.Condition;
import org.apache.ignite.internal.metastorage.dsl.If;
import org.apache.ignite.internal.metastorage.dsl.Update;
-import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
import org.apache.ignite.internal.schema.configuration.TableChange;
import org.apache.ignite.internal.schema.configuration.TableConfiguration;
import org.apache.ignite.internal.schema.configuration.TableView;
@@ -78,11 +77,11 @@ import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
import org.apache.ignite.internal.util.ByteUtils;
import org.apache.ignite.internal.util.IgniteSpinBusyLock;
import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
import org.apache.ignite.lang.IgniteException;
import org.apache.ignite.lang.IgniteInternalException;
import org.apache.ignite.lang.NodeStoppingException;
import org.apache.ignite.network.ClusterNode;
-import org.jetbrains.annotations.NotNull;
/**
* Distribution zones manager.
@@ -141,8 +140,8 @@ public class DistributionZoneManager implements IgniteComponent {
*/
private volatile Set<String> logicalTopology;
- /** Watch listener id to unregister the watch listener on {@link DistributionZoneManager#stop()}. */
- private volatile Long watchListenerId;
+ /** Watch listener. Needed to unregister it on {@link DistributionZoneManager#stop()}. */
+ private final WatchListener watchListener;
/**
* Creates a new distribution zone manager.
@@ -166,6 +165,8 @@ public class DistributionZoneManager implements IgniteComponent {
this.logicalTopologyService = logicalTopologyService;
this.vaultMgr = vaultMgr;
+ this.watchListener = createMetastorageListener();
+
logicalTopology = Collections.emptySet();
}
@@ -437,9 +438,11 @@ public class DistributionZoneManager implements IgniteComponent {
logicalTopologyService.addEventListener(topologyEventListener);
- registerMetaStorageWatchListener()
- .thenAccept(ignore -> initDataNodesFromVaultManager())
- .thenAccept(ignore -> initMetaStorageKeysOnStart());
+ metaStorageManager.registerExactWatch(zonesLogicalTopologyKey(), watchListener);
+
+ initDataNodesFromVaultManager();
+
+ initMetaStorageKeysOnStart();
} finally {
busyLock.leaveBusy();
}
@@ -456,9 +459,7 @@ public class DistributionZoneManager implements IgniteComponent {
logicalTopologyService.removeEventListener(topologyEventListener);
- if (watchListenerId != null) {
- metaStorageManager.unregisterWatch(watchListenerId);
- }
+ metaStorageManager.unregisterWatch(watchListener);
}
private class ZonesConfigurationListener implements ConfigurationNamedListListener<DistributionZoneView> {
@@ -741,39 +742,27 @@ public class DistributionZoneManager implements IgniteComponent {
}
try {
- // TODO: Remove this call as part of https://issues.apache.org/jira/browse/IGNITE-18397
- vaultMgr.get(MetaStorageManagerImpl.APPLIED_REV)
- .thenApply(appliedRevision -> appliedRevision == null ? 0L : bytesToLong(appliedRevision.value()))
- .thenAccept(vaultAppliedRevision -> {
+ long appliedRevision = metaStorageManager.appliedRevision();
+
+ vaultMgr.get(zonesLogicalTopologyKey())
+ .thenAccept(vaultEntry -> {
if (!busyLock.enterBusy()) {
throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
}
try {
- vaultMgr.get(zonesLogicalTopologyKey())
- .thenAccept(vaultEntry -> {
- if (!busyLock.enterBusy()) {
- throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
- }
-
- try {
- if (vaultEntry != null && vaultEntry.value() != null) {
- logicalTopology = ByteUtils.fromBytes(vaultEntry.value());
-
- zonesConfiguration.distributionZones().value().namedListKeys()
- .forEach(zoneName -> {
- int zoneId = zonesConfiguration.distributionZones().get(zoneName).zoneId()
- .value();
-
- saveDataNodesToMetaStorage(zoneId, vaultEntry.value(), vaultAppliedRevision);
- });
-
- saveDataNodesToMetaStorage(DEFAULT_ZONE_ID, vaultEntry.value(), vaultAppliedRevision);
- }
- } finally {
- busyLock.leaveBusy();
- }
- });
+ if (vaultEntry != null && vaultEntry.value() != null) {
+ logicalTopology = ByteUtils.fromBytes(vaultEntry.value());
+
+ zonesConfiguration.distributionZones().value().namedListKeys()
+ .forEach(zoneName -> {
+ int zoneId = zonesConfiguration.distributionZones().get(zoneName).zoneId().value();
+
+ saveDataNodesToMetaStorage(zoneId, vaultEntry.value(), appliedRevision);
+ });
+
+ saveDataNodesToMetaStorage(DEFAULT_ZONE_ID, vaultEntry.value(), appliedRevision);
+ }
} finally {
busyLock.leaveBusy();
}
@@ -783,65 +772,59 @@ public class DistributionZoneManager implements IgniteComponent {
}
}
- /**
- * Registers {@link WatchListener} which updates data nodes of distribution zones on logical topology changing event.
- *
- * @return Future representing pending completion of the operation.
- */
- private CompletableFuture<?> registerMetaStorageWatchListener() {
- return metaStorageManager.registerExactWatch(zonesLogicalTopologyKey(), new WatchListener() {
- @Override
- public boolean onUpdate(@NotNull WatchEvent evt) {
- if (!busyLock.enterBusy()) {
- throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
- }
+ private WatchListener createMetastorageListener() {
+ return new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent evt) {
+ if (!busyLock.enterBusy()) {
+ throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
+ }
- try {
- assert evt.single() : "Expected an event with one entry but was an event with several entries with keys: "
- + evt.entryEvents().stream().map(entry -> entry.newEntry() == null ? "null" : entry.newEntry().key())
- .collect(toList());
+ try {
+ assert evt.single() : "Expected an event with one entry but was an event with several entries with keys: "
+ + evt.entryEvents().stream().map(entry -> entry.newEntry() == null ? "null" : entry.newEntry().key())
+ .collect(toList());
- Entry newEntry = evt.entryEvent().newEntry();
+ Entry newEntry = evt.entryEvent().newEntry();
- long revision = newEntry.revision();
+ long revision = newEntry.revision();
- byte[] newLogicalTopologyBytes = newEntry.value();
+ byte[] newLogicalTopologyBytes = newEntry.value();
- Set<String> newLogicalTopology = ByteUtils.fromBytes(newLogicalTopologyBytes);
+ vaultMgr.put(new ByteArray(newEntry.key()), newLogicalTopologyBytes).join();
- List<String> removedNodes =
- logicalTopology.stream().filter(node -> !newLogicalTopology.contains(node)).collect(toList());
+ Set<String> newLogicalTopology = ByteUtils.fromBytes(newLogicalTopologyBytes);
- List<String> addedNodes =
- newLogicalTopology.stream().filter(node -> !logicalTopology.contains(node)).collect(toList());
+ List<String> removedNodes =
+ logicalTopology.stream().filter(node -> !newLogicalTopology.contains(node)).collect(toList());
- logicalTopology = newLogicalTopology;
+ List<String> addedNodes =
+ newLogicalTopology.stream().filter(node -> !logicalTopology.contains(node)).collect(toList());
- NamedConfigurationTree<DistributionZoneConfiguration, DistributionZoneView, DistributionZoneChange> zones =
- zonesConfiguration.distributionZones();
+ logicalTopology = newLogicalTopology;
- for (int i = 0; i < zones.value().size(); i++) {
- DistributionZoneView zoneView = zones.value().get(i);
+ NamedConfigurationTree<DistributionZoneConfiguration, DistributionZoneView, DistributionZoneChange> zones =
+ zonesConfiguration.distributionZones();
- scheduleTimers(zoneView, addedNodes, removedNodes, newLogicalTopologyBytes, revision);
- }
+ for (int i = 0; i < zones.value().size(); i++) {
+ DistributionZoneView zoneView = zones.value().get(i);
- DistributionZoneView defaultZoneView = zonesConfiguration.value().defaultDistributionZone();
+ scheduleTimers(zoneView, addedNodes, removedNodes, newLogicalTopologyBytes, revision);
+ }
- scheduleTimers(defaultZoneView, addedNodes, removedNodes, newLogicalTopologyBytes, revision);
+ DistributionZoneView defaultZoneView = zonesConfiguration.value().defaultDistributionZone();
- return true;
- } finally {
- busyLock.leaveBusy();
- }
- }
+ scheduleTimers(defaultZoneView, addedNodes, removedNodes, newLogicalTopologyBytes, revision);
+ } finally {
+ busyLock.leaveBusy();
+ }
+ }
- @Override
- public void onError(@NotNull Throwable e) {
- LOG.warn("Unable to process logical topology event", e);
- }
- })
- .thenAccept(id -> watchListenerId = id);
+ @Override
+ public void onError(Throwable e) {
+ LOG.warn("Unable to process logical topology event", e);
+ }
+ };
}
private void scheduleTimers(
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java
index f2787d6a64..ba1dea44be 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java
@@ -115,8 +115,6 @@ public class DistributionZoneManagerConfigurationChangesTest extends IgniteAbstr
MetaStorageManager metaStorageManager = mock(MetaStorageManager.class);
- when(metaStorageManager.registerExactWatch(any(), any())).thenReturn(completedFuture(null));
-
logicalTopologyService = mock(LogicalTopologyServiceImpl.class);
vaultMgr = mock(VaultManager.class);
@@ -155,7 +153,7 @@ public class DistributionZoneManagerConfigurationChangesTest extends IgniteAbstr
AtomicLong raftIndex = new AtomicLong();
- keyValueStorage = spy(new SimpleInMemoryKeyValueStorage());
+ keyValueStorage = spy(new SimpleInMemoryKeyValueStorage("test"));
MetaStorageListener metaStorageListener = new MetaStorageListener(keyValueStorage);
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerLogicalTopologyEventsTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerLogicalTopologyEventsTest.java
index d9b50ec8a4..de897b0708 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerLogicalTopologyEventsTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerLogicalTopologyEventsTest.java
@@ -129,8 +129,6 @@ public class DistributionZoneManagerLogicalTopologyEventsTest {
when(vaultMgr.get(any())).thenReturn(completedFuture(null));
- when(metaStorageManager.registerExactWatch(any(), any())).then(invocation -> completedFuture(null));
-
TablesConfiguration tablesConfiguration = mock(TablesConfiguration.class);
NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = mock(NamedConfigurationTree.class);
@@ -155,7 +153,7 @@ public class DistributionZoneManagerLogicalTopologyEventsTest {
AtomicLong raftIndex = new AtomicLong();
- keyValueStorage = spy(new SimpleInMemoryKeyValueStorage());
+ keyValueStorage = spy(new SimpleInMemoryKeyValueStorage("test"));
MetaStorageListener metaStorageListener = new MetaStorageListener(keyValueStorage);
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java
index 7e8480f9da..e36870e966 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java
@@ -34,6 +34,7 @@ import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.after;
+import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.lenient;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
@@ -71,7 +72,6 @@ import org.apache.ignite.internal.metastorage.command.info.StatementResultInfo;
import org.apache.ignite.internal.metastorage.dsl.If;
import org.apache.ignite.internal.metastorage.dsl.StatementResult;
import org.apache.ignite.internal.metastorage.impl.EntryImpl;
-import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
import org.apache.ignite.internal.metastorage.server.SimpleInMemoryKeyValueStorage;
import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
import org.apache.ignite.internal.raft.Command;
@@ -162,12 +162,13 @@ public class DistributionZoneManagerWatchListenerTest extends IgniteAbstractTest
mockVaultAppliedRevision(1);
when(vaultMgr.get(zonesLogicalTopologyKey())).thenReturn(completedFuture(new VaultEntry(zonesLogicalTopologyKey(), null)));
+ when(vaultMgr.put(any(), any())).thenReturn(completedFuture(null));
- when(metaStorageManager.registerExactWatch(any(), any())).then(invocation -> {
+ doAnswer(invocation -> {
watchListener = invocation.getArgument(1);
- return completedFuture(null);
- });
+ return null;
+ }).when(metaStorageManager).registerExactWatch(any(), any());
mockDefaultZoneConfiguration();
mockDefaultZoneView();
@@ -175,7 +176,7 @@ public class DistributionZoneManagerWatchListenerTest extends IgniteAbstractTest
AtomicLong raftIndex = new AtomicLong();
- keyValueStorage = spy(new SimpleInMemoryKeyValueStorage());
+ keyValueStorage = spy(new SimpleInMemoryKeyValueStorage("test"));
MetaStorageListener metaStorageListener = new MetaStorageListener(keyValueStorage);
@@ -496,8 +497,6 @@ public class DistributionZoneManagerWatchListenerTest extends IgniteAbstractTest
}
private void mockVaultAppliedRevision(long revision) {
- // TODO: remove this as part of https://issues.apache.org/jira/browse/IGNITE-18397
- when(vaultMgr.get(MetaStorageManagerImpl.APPLIED_REV))
- .thenReturn(completedFuture(new VaultEntry(MetaStorageManagerImpl.APPLIED_REV, longToBytes(revision))));
+ when(metaStorageManager.appliedRevision()).thenReturn(revision);
}
}
diff --git a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
index 14083fed82..d42203dcf2 100644
--- a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
+++ b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/MetaStorageManager.java
@@ -37,6 +37,12 @@ import org.jetbrains.annotations.Nullable;
* Manager that provides operations on the Meta Storage.
*/
public interface MetaStorageManager extends IgniteComponent {
+ /**
+ * Returns the current <em>applied revision</em> of the Meta Storage, that is, the most recent revision of updates that have been
+ * applied on this node.
+ */
+ long appliedRevision();
+
/**
* Retrieves an entry for the given key.
*/
@@ -69,8 +75,6 @@ public interface MetaStorageManager extends IgniteComponent {
* Retrieves entries for the given key prefix in lexicographic order. Entries will be filtered out by upper bound of given revision
* number.
*
- * <p>Prefix query is a synonym of the range query {@code range(prefixKey, nextKey(prefixKey))}.
- *
* @param keyPrefix Prefix of the key to retrieve the entries. Couldn't be {@code null}.
* @param revUpperBound The upper bound for entry revision. {@code -1} means latest revision.
* @return Cursor built upon entries corresponding to the given range and revision.
@@ -106,18 +110,16 @@ public interface MetaStorageManager extends IgniteComponent {
*
* @param key Prefix to listen to.
* @param lsnr Listener which will be notified for each update.
- * @return Subscription identifier. Could be used in {@link #unregisterWatch} method in order to cancel subscription.
*/
- CompletableFuture<Long> registerPrefixWatch(ByteArray key, WatchListener lsnr);
+ void registerPrefixWatch(ByteArray key, WatchListener lsnr);
/**
* Registers a watch listener for the provided key.
*
* @param key Meta Storage key.
* @param listener Listener which will be notified for each update.
- * @return Subscription identifier. Could be used in {@link #unregisterWatch} method in order to cancel subscription.
*/
- CompletableFuture<Long> registerExactWatch(ByteArray key, WatchListener listener);
+ void registerExactWatch(ByteArray key, WatchListener listener);
/**
* Registers a watch listener by a key range.
@@ -125,14 +127,13 @@ public interface MetaStorageManager extends IgniteComponent {
* @param keyFrom Start of the range (inclusive).
* @param keyTo End of the range (exclusive) or {@code null} if the range doesn't have an upper bound.
* @param listener Listener which will be notified for each update.
- * @return Subscription identifier. Could be used in {@link #unregisterWatch} method in order to cancel subscription.
*/
- CompletableFuture<Long> registerRangeWatch(ByteArray keyFrom, ByteArray keyTo, WatchListener listener);
+ void registerRangeWatch(ByteArray keyFrom, @Nullable ByteArray keyTo, WatchListener listener);
/**
* Unregisters a watch listener.
*/
- CompletableFuture<Void> unregisterWatch(long id);
+ void unregisterWatch(WatchListener lsnr);
/**
* Starts all registered watches.
diff --git a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchEvent.java b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchEvent.java
index 512882fe9f..6a2b19d592 100644
--- a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchEvent.java
+++ b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchEvent.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.metastorage;
import java.util.Collection;
import java.util.List;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.internal.tostring.S;
/**
* Watch event contains all entry updates done under one revision. Each particular entry update in this revision is represented by {@link
@@ -29,19 +29,19 @@ public class WatchEvent {
/** Events about each entry update in the revision. */
private final List<EntryEvent> entryEvts;
- /** Designates that watch event contains only one update revision. */
- private final boolean single;
+ private final long revision;
/**
* Constructs an watch event with given entry events collection.
*
* @param entryEvts Events for entries corresponding to an update under one revision.
+ * @param revision Revision of the updated entries.
*/
- public WatchEvent(List<EntryEvent> entryEvts) {
+ public WatchEvent(List<EntryEvent> entryEvts, long revision) {
assert entryEvts != null && !entryEvts.isEmpty();
- this.single = entryEvts.size() == 1;
this.entryEvts = entryEvts;
+ this.revision = revision;
}
/**
@@ -49,8 +49,8 @@ public class WatchEvent {
*
* @param entryEvt Entry event.
*/
- public WatchEvent(@NotNull EntryEvent entryEvt) {
- this(List.of(entryEvt));
+ public WatchEvent(EntryEvent entryEvt) {
+ this(List.of(entryEvt), entryEvt.newEntry().revision());
}
/**
@@ -59,7 +59,7 @@ public class WatchEvent {
* @return {@code True} if watch event contains only one entry event.
*/
public boolean single() {
- return single;
+ return entryEvts.size() == 1;
}
/**
@@ -79,4 +79,18 @@ public class WatchEvent {
public EntryEvent entryEvent() {
return entryEvts.get(0);
}
+
+ /**
+ * Returns the revision of the modified entries.
+ *
+ * @return Event revision.
+ */
+ public long revision() {
+ return revision;
+ }
+
+ @Override
+ public String toString() {
+ return S.toString(this);
+ }
}
diff --git a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchListener.java b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchListener.java
index c8a03c75f7..a8a499b4c6 100644
--- a/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchListener.java
+++ b/modules/metastorage-api/src/main/java/org/apache/ignite/internal/metastorage/WatchListener.java
@@ -17,8 +17,6 @@
package org.apache.ignite.internal.metastorage;
-import org.jetbrains.annotations.NotNull;
-
/**
* The listener which receives and handles watch updates.
*/
@@ -26,16 +24,14 @@ public interface WatchListener {
/**
* The method will be called on each meta storage update.
*
- * @param evt A single event or a batch. The batch always contains updates for specific revision.
- * @return {@code True} if listener must continue event handling. If returns {@code false} then the listener and corresponding watch
- * will be unregistered.
+ * @param event A single event or a batch. The batch always contains updates for specific revision.
*/
- boolean onUpdate(@NotNull WatchEvent evt);
+ void onUpdate(WatchEvent event);
/**
* The method will be called in case of an error occurred. The listener and corresponding watch will be unregistered.
*
* @param e Exception.
*/
- void onError(@NotNull Throwable e);
+ void onError(Throwable e);
}
diff --git a/modules/metastorage/build.gradle b/modules/metastorage/build.gradle
index 3895855176..28ebdd81a1 100644
--- a/modules/metastorage/build.gradle
+++ b/modules/metastorage/build.gradle
@@ -39,6 +39,7 @@ dependencies {
testImplementation testFixtures(project(':ignite-core'))
testImplementation libs.mockito.junit
+ testImplementation libs.hamcrest.core
integrationTestImplementation libs.jetbrains.annotations
integrationTestImplementation project(":ignite-cluster-management")
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java
index 70ca2264fe..cc0c1accdb 100644
--- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageManagerImplTest.java
@@ -23,6 +23,7 @@ import static org.apache.ignite.internal.testframework.matchers.CompletableFutur
import static org.apache.ignite.utils.ClusterServiceTestUtils.clusterService;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.contains;
+import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -81,17 +82,23 @@ public class ItMetaStorageManagerImplTest {
raftManager = new Loza(clusterService, raftConfiguration, workDir.resolve("loza"), new HybridClockImpl());
+ VaultManager vaultManager = mock(VaultManager.class);
+
+ when(vaultManager.get(any())).thenReturn(completedFuture(null));
+
ClusterManagementGroupManager cmgManager = mock(ClusterManagementGroupManager.class);
when(cmgManager.metaStorageNodes())
.thenReturn(completedFuture(Set.of(clusterService.localConfiguration().getName())));
+ var storage = new RocksDbKeyValueStorage(clusterService.localConfiguration().getName(), workDir.resolve("metastorage"));
+
metaStorageManager = new MetaStorageManagerImpl(
- mock(VaultManager.class),
+ vaultManager,
clusterService,
cmgManager,
raftManager,
- new RocksDbKeyValueStorage(workDir.resolve("metastorage"))
+ storage
);
clusterService.start();
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServicePersistenceTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServicePersistenceTest.java
index c8f0ad554f..eaf3faecd0 100644
--- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServicePersistenceTest.java
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServicePersistenceTest.java
@@ -21,17 +21,22 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.function.BooleanSupplier;
import org.apache.ignite.internal.metastorage.Entry;
import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
+import org.apache.ignite.internal.raft.server.RaftServer;
import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
import org.apache.ignite.internal.raft.service.ItAbstractListenerSnapshotTest;
import org.apache.ignite.internal.raft.service.RaftGroupListener;
import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.util.IgniteUtils;
import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.network.ClusterService;
import org.junit.jupiter.api.AfterEach;
@@ -49,20 +54,20 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
private MetaStorageServiceImpl metaStorage;
- private KeyValueStorage storage;
+ private final Map<String, RocksDbKeyValueStorage> storageByName = new HashMap<>();
/** After each. */
@AfterEach
- void tearDown() {
- if (storage != null) {
- storage.close();
- }
+ void tearDown() throws Exception {
+ IgniteUtils.closeAll(storageByName.values().stream().map(storage -> storage::close));
}
/** {@inheritDoc} */
@Override
- public void beforeFollowerStop(RaftGroupService service) throws Exception {
- metaStorage = new MetaStorageServiceImpl(service, null, null);
+ public void beforeFollowerStop(RaftGroupService service, RaftServer server) throws Exception {
+ ClusterNode followerNode = getNode(server);
+
+ metaStorage = new MetaStorageServiceImpl(service, followerNode);
// Put some data in the metastorage
metaStorage.put(FIRST_KEY, FIRST_VALUE).get();
@@ -73,7 +78,15 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
/** {@inheritDoc} */
@Override
- public void afterFollowerStop(RaftGroupService service) throws Exception {
+ public void afterFollowerStop(RaftGroupService service, RaftServer server) throws Exception {
+ ClusterNode followerNode = getNode(server);
+
+ KeyValueStorage storage = storageByName.remove(followerNode.name());
+
+ if (storage != null) {
+ storage.close();
+ }
+
// Remove the first key from the metastorage
metaStorage.remove(FIRST_KEY).get();
@@ -96,7 +109,9 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
/** {@inheritDoc} */
@Override
public BooleanSupplier snapshotCheckClosure(JraftServerImpl restarted, boolean interactedAfterSnapshot) {
- KeyValueStorage storage = getListener(restarted, raftGroupId()).getStorage();
+ ClusterNode node = getNode(restarted);
+
+ KeyValueStorage storage = storageByName.get(node.name());
byte[] lastKey = interactedAfterSnapshot ? SECOND_KEY.bytes() : FIRST_KEY.bytes();
byte[] lastValue = interactedAfterSnapshot ? SECOND_VALUE : FIRST_VALUE;
@@ -111,16 +126,22 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
/** {@inheritDoc} */
@Override
- public Path getListenerPersistencePath(MetaStorageListener listener) {
- return ((RocksDbKeyValueStorage) listener.getStorage()).getDbPath();
+ public Path getListenerPersistencePath(MetaStorageListener listener, RaftServer server) {
+ return storageByName.get(getNode(server).name()).getDbPath();
}
/** {@inheritDoc} */
@Override
public RaftGroupListener createListener(ClusterService service, Path listenerPersistencePath) {
- storage = new RocksDbKeyValueStorage(listenerPersistencePath);
+ String nodeName = service.localConfiguration().getName();
+
+ KeyValueStorage storage = storageByName.computeIfAbsent(nodeName, name -> {
+ var s = new RocksDbKeyValueStorage(name, listenerPersistencePath);
- storage.start();
+ s.start();
+
+ return s;
+ });
return new MetaStorageListener(storage);
}
@@ -145,4 +166,8 @@ public class ItMetaStorageServicePersistenceTest extends ItAbstractListenerSnaps
assertEquals(expected, entry);
}
+
+ private static ClusterNode getNode(RaftServer server) {
+ return server.clusterService().topologyService().localMember();
+ }
}
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServiceTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServiceTest.java
index ef21661168..78c8fda92a 100644
--- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServiceTest.java
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageServiceTest.java
@@ -17,8 +17,8 @@
package org.apache.ignite.internal.metastorage.impl;
-import static java.util.concurrent.CompletableFuture.allOf;
import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toUnmodifiableSet;
import static org.apache.ignite.internal.metastorage.dsl.CompoundCondition.and;
import static org.apache.ignite.internal.metastorage.dsl.CompoundCondition.or;
import static org.apache.ignite.internal.metastorage.dsl.Conditions.revision;
@@ -31,13 +31,11 @@ import static org.apache.ignite.internal.metastorage.impl.ItMetaStorageServiceTe
import static org.apache.ignite.internal.metastorage.impl.ItMetaStorageServiceTest.ServerUpdateMatcher.upd;
import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
import static org.apache.ignite.utils.ClusterServiceTestUtils.findLocalAddresses;
-import static org.apache.ignite.utils.ClusterServiceTestUtils.waitForTopology;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyList;
import static org.mockito.ArgumentMatchers.eq;
@@ -60,20 +58,13 @@ import java.util.NoSuchElementException;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
import org.apache.ignite.internal.hlc.HybridClockImpl;
-import org.apache.ignite.internal.logger.IgniteLogger;
-import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.dsl.Condition;
import org.apache.ignite.internal.metastorage.dsl.Conditions;
import org.apache.ignite.internal.metastorage.dsl.If;
@@ -92,6 +83,7 @@ import org.apache.ignite.internal.metastorage.server.OrCondition;
import org.apache.ignite.internal.metastorage.server.RevisionCondition;
import org.apache.ignite.internal.metastorage.server.ValueCondition;
import org.apache.ignite.internal.metastorage.server.ValueCondition.Type;
+import org.apache.ignite.internal.metastorage.server.raft.MetaStorageLearnerListener;
import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
import org.apache.ignite.internal.metastorage.server.raft.MetastorageGroupId;
import org.apache.ignite.internal.raft.Loza;
@@ -101,21 +93,21 @@ import org.apache.ignite.internal.raft.RaftGroupEventsListener;
import org.apache.ignite.internal.raft.RaftManager;
import org.apache.ignite.internal.raft.RaftNodeId;
import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
+import org.apache.ignite.internal.raft.service.RaftGroupListener;
import org.apache.ignite.internal.raft.service.RaftGroupService;
import org.apache.ignite.internal.testframework.WorkDirectory;
import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.internal.util.IgniteUtils;
import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.network.ClusterService;
import org.apache.ignite.network.NetworkAddress;
import org.apache.ignite.network.StaticNodeFinder;
import org.apache.ignite.utils.ClusterServiceTestUtils;
import org.hamcrest.Description;
import org.hamcrest.TypeSafeMatcher;
-import org.jetbrains.annotations.NotNull;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
@@ -123,35 +115,23 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.ArgumentCaptor;
-import org.mockito.Mock;
-import org.mockito.junit.jupiter.MockitoExtension;
/**
* Meta storage client tests.
*/
@ExtendWith(WorkDirectoryExtension.class)
@ExtendWith(ConfigurationExtension.class)
-@ExtendWith(MockitoExtension.class)
public class ItMetaStorageServiceTest {
- /** The logger. */
- private static final IgniteLogger LOG = Loggers.forClass(ItMetaStorageServiceTest.class);
-
/** Base network port. */
private static final int NODE_PORT_BASE = 20_000;
- /** Nodes. */
- private static final int NODES = 2;
-
- /**
- * Expected server result entry.
- */
- private static final EntryImpl EXPECTED_RESULT_ENTRY =
- new EntryImpl(
- new byte[]{1},
- new byte[]{2},
- 10,
- 2
- );
+ /** Expected result entry. */
+ private static final Entry EXPECTED_RESULT_ENTRY = new EntryImpl(
+ new byte[]{1},
+ new byte[]{2},
+ 10,
+ 2
+ );
/**
* Expected result map.
@@ -161,38 +141,10 @@ public class ItMetaStorageServiceTest {
/** Expected server result collection. */
private static final Collection<Entry> EXPECTED_SRV_RESULT_COLL;
- /** Node 0 id. */
- private static final String NODE_ID_0 = "node-id-0";
-
- /** Node 1 id. */
- private static final String NODE_ID_1 = "node-id-1";
-
- /** Cluster. */
- private final ArrayList<ClusterService> cluster = new ArrayList<>();
-
- /** Meta storage raft server. */
- private final List<RaftManager> raftManagers = new ArrayList<>();
-
- /** List of Raft services (for resource management purposes). */
- private final List<RaftGroupService> raftGroupServices = new ArrayList<>();
-
- /** Mock Metastorage storage. */
- @Mock
- private KeyValueStorage mockStorage;
-
- /** Metastorage service. */
- private MetaStorageService metaStorageSvc;
-
- @WorkDirectory
- private Path dataPath;
-
- @InjectConfiguration
- private RaftConfiguration raftConfiguration;
-
static {
EXPECTED_RESULT_MAP = new TreeMap<>();
- EntryImpl entry1 = new EntryImpl(
+ Entry entry1 = new EntryImpl(
new byte[]{1},
new byte[]{2},
10,
@@ -201,7 +153,7 @@ public class ItMetaStorageServiceTest {
EXPECTED_RESULT_MAP.put(new ByteArray(entry1.key()), entry1);
- EntryImpl entry2 = new EntryImpl(
+ Entry entry2 = new EntryImpl(
new byte[]{3},
new byte[]{4},
10,
@@ -213,29 +165,115 @@ public class ItMetaStorageServiceTest {
EXPECTED_SRV_RESULT_COLL = List.of(entry1, entry2);
}
- /**
- * Run {@code NODES} cluster nodes.
- */
+ private static class Node {
+ private final ClusterService clusterService;
+
+ private final RaftManager raftManager;
+
+ private final KeyValueStorage mockStorage;
+
+ private RaftGroupService metaStorageRaftService;
+
+ private MetaStorageService metaStorageService;
+
+ Node(ClusterService clusterService, RaftConfiguration raftConfiguration, Path dataPath) {
+ this.clusterService = clusterService;
+
+ this.raftManager = new Loza(
+ clusterService,
+ raftConfiguration,
+ dataPath.resolve(name()),
+ new HybridClockImpl()
+ );
+
+ this.mockStorage = mock(KeyValueStorage.class);
+ }
+
+ void start(PeersAndLearners configuration) {
+ clusterService.start();
+ raftManager.start();
+
+ CompletableFuture<RaftGroupService> raftService = startRaftService(configuration);
+
+ assertThat(raftService, willCompleteSuccessfully());
+
+ metaStorageRaftService = raftService.join();
+
+ ClusterNode node = clusterService.topologyService().localMember();
+
+ metaStorageService = new MetaStorageServiceImpl(metaStorageRaftService, node);
+ }
+
+ String name() {
+ return clusterService.localConfiguration().getName();
+ }
+
+ private CompletableFuture<RaftGroupService> startRaftService(PeersAndLearners configuration) {
+ String name = name();
+
+ boolean isLearner = configuration.peer(name) == null;
+
+ Peer peer = isLearner ? configuration.learner(name) : configuration.peer(name);
+
+ assert peer != null;
+
+ RaftGroupListener listener = isLearner ? new MetaStorageLearnerListener(mockStorage) : new MetaStorageListener(mockStorage);
+
+ var raftNodeId = new RaftNodeId(MetastorageGroupId.INSTANCE, peer);
+
+ try {
+ return raftManager.startRaftGroupNode(raftNodeId, configuration, listener, RaftGroupEventsListener.noopLsnr);
+ } catch (NodeStoppingException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ void stop() throws Exception {
+ Stream<AutoCloseable> raftStop = Stream.of(
+ metaStorageRaftService == null ? null : (AutoCloseable) metaStorageRaftService::shutdown,
+ () -> raftManager.stopRaftNodes(MetastorageGroupId.INSTANCE)
+ );
+
+ Stream<AutoCloseable> beforeNodeStop = Stream.of(raftManager, clusterService).map(c -> c::beforeNodeStop);
+
+ Stream<AutoCloseable> nodeStop = Stream.of(raftManager, clusterService).map(c -> c::stop);
+
+ IgniteUtils.closeAll(Stream.of(raftStop, beforeNodeStop, nodeStop).flatMap(Function.identity()));
+ }
+ }
+
+ private TestInfo testInfo;
+
+ @WorkDirectory
+ private Path workDir;
+
+ @InjectConfiguration
+ private RaftConfiguration raftConfiguration;
+
+ private final List<Node> nodes = new ArrayList<>();
+
@BeforeEach
- public void beforeTest(TestInfo testInfo) throws Exception {
- List<NetworkAddress> localAddresses = findLocalAddresses(NODE_PORT_BASE, NODE_PORT_BASE + NODES);
+ public void beforeTest(TestInfo testInfo) {
+ this.testInfo = testInfo;
+ }
+
+ private List<Node> startNodes(int amount) {
+ List<NetworkAddress> localAddresses = findLocalAddresses(NODE_PORT_BASE, NODE_PORT_BASE + amount);
var nodeFinder = new StaticNodeFinder(localAddresses);
localAddresses.stream()
.map(addr -> ClusterServiceTestUtils.clusterService(testInfo, addr.port(), nodeFinder))
- .forEach(clusterService -> {
- clusterService.start();
- cluster.add(clusterService);
- });
+ .forEach(clusterService -> nodes.add(new Node(clusterService, raftConfiguration, workDir)));
- for (ClusterService node : cluster) {
- assertTrue(waitForTopology(node, NODES, 1000));
- }
+ PeersAndLearners metaStorageConfiguration = PeersAndLearners.fromConsistentIds(
+ Set.of(nodes.get(0).name()),
+ nodes.stream().skip(1).map(Node::name).collect(toUnmodifiableSet())
+ );
- LOG.info("Cluster started.");
+ nodes.parallelStream().forEach(node -> node.start(metaStorageConfiguration));
- metaStorageSvc = prepareMetaStorage();
+ return nodes;
}
/**
@@ -245,16 +283,7 @@ public class ItMetaStorageServiceTest {
*/
@AfterEach
public void afterTest() throws Exception {
- Stream<AutoCloseable> stopRaftGroupServices = raftGroupServices.stream().map(service -> service::shutdown);
-
- Stream<AutoCloseable> stopRaftGroups = raftManagers.stream()
- .map(manager -> () -> manager.stopRaftNodes(MetastorageGroupId.INSTANCE));
-
- Stream<AutoCloseable> beforeNodeStop = Stream.concat(raftManagers.stream(), cluster.stream()).map(c -> c::beforeNodeStop);
-
- Stream<AutoCloseable> nodeStop = Stream.concat(raftManagers.stream(), cluster.stream()).map(c -> c::stop);
-
- IgniteUtils.closeAll(Stream.of(stopRaftGroupServices, stopRaftGroups, beforeNodeStop, nodeStop).flatMap(Function.identity()));
+ IgniteUtils.closeAll(nodes.stream().map(node -> node::stop));
}
/**
@@ -264,9 +293,11 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testGet() throws Exception {
- when(mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenReturn(EXPECTED_RESULT_ENTRY);
+ Node node = startNodes(1).get(0);
+
+ when(node.mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenReturn(EXPECTED_RESULT_ENTRY);
- assertEquals(EXPECTED_RESULT_ENTRY, metaStorageSvc.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
+ assertEquals(EXPECTED_RESULT_ENTRY, node.metaStorageService.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
}
/**
@@ -276,11 +307,13 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testGetWithUpperBoundRevision() throws Exception {
- when(mockStorage.get(EXPECTED_RESULT_ENTRY.key(), EXPECTED_RESULT_ENTRY.revision())).thenReturn(EXPECTED_RESULT_ENTRY);
+ Node node = startNodes(1).get(0);
+
+ when(node.mockStorage.get(EXPECTED_RESULT_ENTRY.key(), EXPECTED_RESULT_ENTRY.revision())).thenReturn(EXPECTED_RESULT_ENTRY);
assertEquals(
EXPECTED_RESULT_ENTRY,
- metaStorageSvc.get(new ByteArray(EXPECTED_RESULT_ENTRY.key()), EXPECTED_RESULT_ENTRY.revision()).get()
+ node.metaStorageService.get(new ByteArray(EXPECTED_RESULT_ENTRY.key()), EXPECTED_RESULT_ENTRY.revision()).get()
);
}
@@ -291,9 +324,11 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testGetAll() throws Exception {
- when(mockStorage.getAll(anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
+ Node node = startNodes(1).get(0);
+
+ when(node.mockStorage.getAll(anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
- assertEquals(EXPECTED_RESULT_MAP, metaStorageSvc.getAll(EXPECTED_RESULT_MAP.keySet()).get());
+ assertEquals(EXPECTED_RESULT_MAP, node.metaStorageService.getAll(EXPECTED_RESULT_MAP.keySet()).get());
}
/**
@@ -303,11 +338,13 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testGetAllWithUpperBoundRevision() throws Exception {
- when(mockStorage.getAll(anyList(), eq(10L))).thenReturn(EXPECTED_SRV_RESULT_COLL);
+ Node node = startNodes(1).get(0);
+
+ when(node.mockStorage.getAll(anyList(), eq(10L))).thenReturn(EXPECTED_SRV_RESULT_COLL);
assertEquals(
EXPECTED_RESULT_MAP,
- metaStorageSvc.getAll(EXPECTED_RESULT_MAP.keySet(), 10).get()
+ node.metaStorageService.getAll(EXPECTED_RESULT_MAP.keySet(), 10).get()
);
}
@@ -318,13 +355,15 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testPut() throws Exception {
+ Node node = startNodes(1).get(0);
+
ByteArray expKey = new ByteArray(new byte[]{1});
byte[] expVal = {2};
- doNothing().when(mockStorage).put(expKey.bytes(), expVal);
+ doNothing().when(node.mockStorage).put(expKey.bytes(), expVal);
- metaStorageSvc.put(expKey, expVal).get();
+ node.metaStorageService.put(expKey, expVal).get();
}
/**
@@ -334,13 +373,15 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testGetAndPut() throws Exception {
+ Node node = startNodes(1).get(0);
+
byte[] expVal = {2};
- when(mockStorage.getAndPut(EXPECTED_RESULT_ENTRY.key(), expVal)).thenReturn(EXPECTED_RESULT_ENTRY);
+ when(node.mockStorage.getAndPut(EXPECTED_RESULT_ENTRY.key(), expVal)).thenReturn(EXPECTED_RESULT_ENTRY);
assertEquals(
EXPECTED_RESULT_ENTRY,
- metaStorageSvc.getAndPut(new ByteArray(EXPECTED_RESULT_ENTRY.key()), expVal).get()
+ node.metaStorageService.getAndPut(new ByteArray(EXPECTED_RESULT_ENTRY.key()), expVal).get()
);
}
@@ -351,7 +392,9 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testPutAll() throws Exception {
- metaStorageSvc.putAll(
+ Node node = startNodes(1).get(0);
+
+ node.metaStorageService.putAll(
EXPECTED_RESULT_MAP.entrySet().stream()
.collect(Collectors.toMap(
Map.Entry::getKey,
@@ -362,7 +405,7 @@ public class ItMetaStorageServiceTest {
ArgumentCaptor<List<byte[]>> keysCaptor = ArgumentCaptor.forClass(List.class);
ArgumentCaptor<List<byte[]>> valuesCaptor = ArgumentCaptor.forClass(List.class);
- verify(mockStorage).putAll(keysCaptor.capture(), valuesCaptor.capture());
+ verify(node.mockStorage).putAll(keysCaptor.capture(), valuesCaptor.capture());
// Assert keys equality.
assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keysCaptor.getValue().size());
@@ -392,9 +435,11 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testGetAndPutAll() throws Exception {
- when(mockStorage.getAndPutAll(anyList(), anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
+ Node node = startNodes(1).get(0);
+
+ when(node.mockStorage.getAndPutAll(anyList(), anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
- Map<ByteArray, Entry> gotRes = metaStorageSvc.getAndPutAll(
+ Map<ByteArray, Entry> gotRes = node.metaStorageService.getAndPutAll(
EXPECTED_RESULT_MAP.entrySet().stream()
.collect(Collectors.toMap(
Map.Entry::getKey,
@@ -407,7 +452,7 @@ public class ItMetaStorageServiceTest {
ArgumentCaptor<List<byte[]>> keysCaptor = ArgumentCaptor.forClass(List.class);
ArgumentCaptor<List<byte[]>> valuesCaptor = ArgumentCaptor.forClass(List.class);
- verify(mockStorage).getAndPutAll(keysCaptor.capture(), valuesCaptor.capture());
+ verify(node.mockStorage).getAndPutAll(keysCaptor.capture(), valuesCaptor.capture());
// Assert keys equality.
assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keysCaptor.getValue().size());
@@ -437,11 +482,13 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRemove() throws Exception {
+ Node node = startNodes(1).get(0);
+
ByteArray expKey = new ByteArray(new byte[]{1});
- doNothing().when(mockStorage).remove(expKey.bytes());
+ doNothing().when(node.mockStorage).remove(expKey.bytes());
- metaStorageSvc.remove(expKey).get();
+ node.metaStorageService.remove(expKey).get();
}
/**
@@ -451,16 +498,18 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testGetAndRemove() throws Exception {
- EntryImpl expRes = new EntryImpl(
+ Node node = startNodes(1).get(0);
+
+ Entry expRes = new EntryImpl(
new byte[]{1},
new byte[]{3},
10,
2
);
- when(mockStorage.getAndRemove(expRes.key())).thenReturn(expRes);
+ when(node.mockStorage.getAndRemove(expRes.key())).thenReturn(expRes);
- assertEquals(expRes, metaStorageSvc.getAndRemove(new ByteArray(expRes.key())).get());
+ assertEquals(expRes, node.metaStorageService.getAndRemove(new ByteArray(expRes.key())).get());
}
/**
@@ -470,16 +519,16 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRemoveAll() throws Exception {
- doNothing().when(mockStorage).removeAll(anyList());
+ Node node = startNodes(1).get(0);
- metaStorageSvc.removeAll(EXPECTED_RESULT_MAP.keySet()).get();
+ node.metaStorageService.removeAll(EXPECTED_RESULT_MAP.keySet()).get();
List<byte[]> expKeys = EXPECTED_RESULT_MAP.keySet().stream()
.map(ByteArray::bytes).collect(toList());
ArgumentCaptor<List<byte[]>> keysCaptor = ArgumentCaptor.forClass(List.class);
- verify(mockStorage).removeAll(keysCaptor.capture());
+ verify(node.mockStorage).removeAll(keysCaptor.capture());
assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keysCaptor.getValue().size());
@@ -495,15 +544,17 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testGetAndRemoveAll() throws Exception {
- when(mockStorage.getAndRemoveAll(anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
+ Node node = startNodes(1).get(0);
- Map<ByteArray, Entry> gotRes = metaStorageSvc.getAndRemoveAll(EXPECTED_RESULT_MAP.keySet()).get();
+ when(node.mockStorage.getAndRemoveAll(anyList())).thenReturn(EXPECTED_SRV_RESULT_COLL);
+
+ Map<ByteArray, Entry> gotRes = node.metaStorageService.getAndRemoveAll(EXPECTED_RESULT_MAP.keySet()).get();
assertEquals(EXPECTED_RESULT_MAP, gotRes);
ArgumentCaptor<List<byte[]>> keysCaptor = ArgumentCaptor.forClass(List.class);
- verify(mockStorage).getAndRemoveAll(keysCaptor.capture());
+ verify(node.mockStorage).getAndRemoveAll(keysCaptor.capture());
// Assert keys equality.
assertEquals(EXPECTED_RESULT_MAP.keySet().size(), keysCaptor.getValue().size());
@@ -522,15 +573,17 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRangeWitKeyToAndUpperBound() {
+ Node node = startNodes(1).get(0);
+
ByteArray expKeyFrom = new ByteArray(new byte[]{1});
ByteArray expKeyTo = new ByteArray(new byte[]{3});
long expRevUpperBound = 10;
- when(mockStorage.range(expKeyFrom.bytes(), expKeyTo.bytes(), expRevUpperBound, false)).thenReturn(mock(Cursor.class));
+ when(node.mockStorage.range(expKeyFrom.bytes(), expKeyTo.bytes(), expRevUpperBound, false)).thenReturn(mock(Cursor.class));
- metaStorageSvc.range(expKeyFrom, expKeyTo, expRevUpperBound).close();
+ node.metaStorageService.range(expKeyFrom, expKeyTo, expRevUpperBound).close();
}
/**
@@ -539,13 +592,15 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRangeWitKeyTo() {
+ Node node = startNodes(1).get(0);
+
ByteArray expKeyFrom = new ByteArray(new byte[]{1});
ByteArray expKeyTo = new ByteArray(new byte[]{3});
- when(mockStorage.range(expKeyFrom.bytes(), expKeyTo.bytes(), false)).thenReturn(mock(Cursor.class));
+ when(node.mockStorage.range(expKeyFrom.bytes(), expKeyTo.bytes(), false)).thenReturn(mock(Cursor.class));
- metaStorageSvc.range(expKeyFrom, expKeyTo).close();
+ node.metaStorageService.range(expKeyFrom, expKeyTo).close();
}
/**
@@ -554,11 +609,13 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRangeWitNullAsKeyTo() {
+ Node node = startNodes(1).get(0);
+
ByteArray expKeyFrom = new ByteArray(new byte[]{1});
- when(mockStorage.range(expKeyFrom.bytes(), null, false)).thenReturn(mock(Cursor.class));
+ when(node.mockStorage.range(expKeyFrom.bytes(), null, false)).thenReturn(mock(Cursor.class));
- metaStorageSvc.range(expKeyFrom, null).close();
+ node.metaStorageService.range(expKeyFrom, null).close();
}
/**
@@ -566,9 +623,11 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRangeHasNext() {
+ Node node = startNodes(1).get(0);
+
ByteArray expKeyFrom = new ByteArray(new byte[]{1});
- when(mockStorage.range(expKeyFrom.bytes(), null, false)).thenAnswer(invocation -> {
+ when(node.mockStorage.range(expKeyFrom.bytes(), null, false)).thenAnswer(invocation -> {
var cursor = mock(Cursor.class);
when(cursor.hasNext()).thenReturn(true);
@@ -576,7 +635,7 @@ public class ItMetaStorageServiceTest {
return cursor;
});
- Cursor<Entry> cursor = metaStorageSvc.range(expKeyFrom, null);
+ Cursor<Entry> cursor = node.metaStorageService.range(expKeyFrom, null);
assertTrue(cursor.iterator().hasNext());
}
@@ -586,7 +645,9 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRangeNext() {
- when(mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
+ Node node = startNodes(1).get(0);
+
+ when(node.mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
var cursor = mock(Cursor.class);
when(cursor.hasNext()).thenReturn(true);
@@ -595,7 +656,7 @@ public class ItMetaStorageServiceTest {
return cursor;
});
- Cursor<Entry> cursor = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+ Cursor<Entry> cursor = node.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
assertEquals(EXPECTED_RESULT_ENTRY, cursor.iterator().next());
}
@@ -605,7 +666,9 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRangeNextNoSuchElementException() {
- when(mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
+ Node node = startNodes(1).get(0);
+
+ when(node.mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
var cursor = mock(Cursor.class);
when(cursor.hasNext()).thenReturn(true);
@@ -614,7 +677,7 @@ public class ItMetaStorageServiceTest {
return cursor;
});
- Cursor<Entry> cursor = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+ Cursor<Entry> cursor = node.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
assertThrows(NoSuchElementException.class, () -> cursor.iterator().next());
}
@@ -625,124 +688,25 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testRangeClose() {
+ Node node = startNodes(1).get(0);
+
ByteArray expKeyFrom = new ByteArray(new byte[]{1});
- Cursor cursorMock = mock(Cursor.class);
+ Cursor<Entry> cursorMock = mock(Cursor.class);
- when(mockStorage.range(expKeyFrom.bytes(), null, false)).thenReturn(cursorMock);
+ when(node.mockStorage.range(expKeyFrom.bytes(), null, false)).thenReturn(cursorMock);
- Cursor<Entry> cursor = metaStorageSvc.range(expKeyFrom, null);
+ Cursor<Entry> cursor = node.metaStorageService.range(expKeyFrom, null);
cursor.close();
verify(cursorMock, times(1)).close();
}
- @Test
- public void testWatchOnUpdate() throws Exception {
- WatchEvent expectedEvent =
- new WatchEvent(List.of(
- new EntryEvent(
- new EntryImpl(
- new byte[]{2},
- new byte[]{20},
- 1,
- 1
- ),
- new EntryImpl(
- new byte[]{2},
- new byte[]{21},
- 2,
- 4
- )
- ),
- new EntryEvent(
- new EntryImpl(
- new byte[]{3},
- new byte[]{20},
- 1,
- 2
- ),
- new EntryImpl(
- new byte[]{3},
- new byte[]{},
- 2,
- 5
- )
- ),
- new EntryEvent(
- new EntryImpl(
- new byte[]{4},
- new byte[]{20},
- 1,
- 3
- ),
- new EntryImpl(
- new byte[]{4},
- new byte[]{},
- 3,
- 6
- )
- )
- ));
-
- ByteArray keyFrom = new ByteArray(new byte[]{1});
-
- ByteArray keyTo = new ByteArray(new byte[]{10});
-
- long rev = 2;
-
- when(mockStorage.watch(keyFrom.bytes(), keyTo.bytes(), rev)).thenAnswer(invocation -> {
- var cursor = mock(Cursor.class);
-
- when(cursor.hasNext()).thenReturn(true);
- when(cursor.next()).thenReturn(expectedEvent);
-
- return cursor;
- });
-
- CountDownLatch latch = new CountDownLatch(1);
-
- IgniteUuid watchId = metaStorageSvc.watch(keyFrom, keyTo, rev, new WatchListener() {
- @Override
- public boolean onUpdate(@NotNull WatchEvent event) {
- Collection<EntryEvent> expectedEvents = expectedEvent.entryEvents();
- Collection<EntryEvent> actualEvents = event.entryEvents();
-
- assertEquals(expectedEvents.size(), actualEvents.size());
-
- Iterator<EntryEvent> expectedIterator = expectedEvents.iterator();
- Iterator<EntryEvent> actualIterator = actualEvents.iterator();
-
- while (expectedIterator.hasNext() && actualIterator.hasNext()) {
- EntryEvent expectedEntryEvent = expectedIterator.next();
- EntryEvent actualEntryEvent = actualIterator.next();
-
- assertArrayEquals(expectedEntryEvent.oldEntry().key(), actualEntryEvent.oldEntry().key());
- assertArrayEquals(expectedEntryEvent.oldEntry().value(), actualEntryEvent.oldEntry().value());
- assertArrayEquals(expectedEntryEvent.newEntry().key(), actualEntryEvent.newEntry().key());
- assertArrayEquals(expectedEntryEvent.newEntry().value(), actualEntryEvent.newEntry().value());
- }
-
- latch.countDown();
-
- return true;
- }
-
- @Override
- public void onError(@NotNull Throwable e) {
- // Within given test it's not expected to get here.
- fail();
- }
- }).get();
-
- latch.await();
-
- metaStorageSvc.stopWatch(watchId).get();
- }
-
@Test
public void testMultiInvoke() throws Exception {
+ Node node = startNodes(1).get(0);
+
ByteArray key1 = new ByteArray(new byte[]{1});
ByteArray key2 = new ByteArray(new byte[]{2});
ByteArray key3 = new ByteArray(new byte[]{3});
@@ -780,11 +744,11 @@ public class ItMetaStorageServiceTest {
var ifCaptor = ArgumentCaptor.forClass(org.apache.ignite.internal.metastorage.server.If.class);
- when(mockStorage.invoke(any())).thenReturn(new StatementResult(true));
+ when(node.mockStorage.invoke(any())).thenReturn(new StatementResult(true));
- assertTrue(metaStorageSvc.invoke(iif).get().getAsBoolean());
+ assertTrue(node.metaStorageService.invoke(iif).get().getAsBoolean());
- verify(mockStorage).invoke(ifCaptor.capture());
+ verify(node.mockStorage).invoke(ifCaptor.capture());
var resultIf = ifCaptor.getValue();
@@ -818,11 +782,13 @@ public class ItMetaStorageServiceTest {
@Test
public void testInvoke() throws Exception {
+ Node node = startNodes(1).get(0);
+
ByteArray expKey = new ByteArray(new byte[]{1});
byte[] expVal = {2};
- when(mockStorage.invoke(any(), any(), any())).thenReturn(true);
+ when(node.mockStorage.invoke(any(), any(), any())).thenReturn(true);
Condition condition = Conditions.notExists(expKey);
@@ -830,7 +796,7 @@ public class ItMetaStorageServiceTest {
Operation failure = Operations.noop();
- assertTrue(metaStorageSvc.invoke(condition, success, failure).get());
+ assertTrue(node.metaStorageService.invoke(condition, success, failure).get());
var conditionCaptor = ArgumentCaptor.forClass(AbstractSimpleCondition.class);
@@ -838,7 +804,7 @@ public class ItMetaStorageServiceTest {
ArgumentCaptor<Collection<Operation>> failureCaptor = ArgumentCaptor.forClass(Collection.class);
- verify(mockStorage).invoke(conditionCaptor.capture(), successCaptor.capture(), failureCaptor.capture());
+ verify(node.mockStorage).invoke(conditionCaptor.capture(), successCaptor.capture(), failureCaptor.capture());
assertArrayEquals(expKey.bytes(), conditionCaptor.getValue().key());
@@ -857,10 +823,11 @@ public class ItMetaStorageServiceTest {
@Disabled // TODO: IGNITE-14693 Add tests for exception handling logic.
@Test
public void testGetThatThrowsCompactedException() {
- when(mockStorage.get(EXPECTED_RESULT_ENTRY.key()))
- .thenThrow(new org.apache.ignite.internal.metastorage.server.CompactedException());
+ Node node = startNodes(1).get(0);
- assertThrows(CompactedException.class, () -> metaStorageSvc.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
+ when(node.mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenThrow(new CompactedException());
+
+ assertThrows(CompactedException.class, () -> node.metaStorageService.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
}
/**
@@ -869,9 +836,11 @@ public class ItMetaStorageServiceTest {
@Disabled // TODO: IGNITE-14693 Add tests for exception handling logic.
@Test
public void testGetThatThrowsOperationTimeoutException() {
- when(mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenThrow(new OperationTimeoutException());
+ Node node = startNodes(1).get(0);
+
+ when(node.mockStorage.get(EXPECTED_RESULT_ENTRY.key())).thenThrow(new OperationTimeoutException());
- assertThrows(OperationTimeoutException.class, () -> metaStorageSvc.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
+ assertThrows(OperationTimeoutException.class, () -> node.metaStorageService.get(new ByteArray(EXPECTED_RESULT_ENTRY.key())).get());
}
/**
@@ -881,7 +850,12 @@ public class ItMetaStorageServiceTest {
*/
@Test
public void testCursorsCleanup() throws Exception {
- when(mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
+ startNodes(2);
+
+ Node leader = nodes.get(0);
+ Node learner = nodes.get(1);
+
+ when(leader.mockStorage.range(EXPECTED_RESULT_ENTRY.key(), null, false)).thenAnswer(invocation -> {
var cursor = mock(Cursor.class);
when(cursor.hasNext()).thenReturn(true);
@@ -890,31 +864,19 @@ public class ItMetaStorageServiceTest {
return cursor;
});
- String localName = cluster.get(0).topologyService().localMember().name();
-
- PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(Set.of(localName));
-
- RaftGroupService metaStorageRaftSvc2 = raftManagers.get(1)
- .startRaftGroupService(MetastorageGroupId.INSTANCE, configuration)
- .get(3, TimeUnit.SECONDS);
-
- raftGroupServices.add(metaStorageRaftSvc2);
-
- MetaStorageService metaStorageSvc2 = new MetaStorageServiceImpl(metaStorageRaftSvc2, NODE_ID_1, NODE_ID_1);
-
- Cursor<Entry> cursorNode0 = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+ Cursor<Entry> cursorNode0 = leader.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
assertTrue(cursorNode0.hasNext());
- Cursor<Entry> cursor2Node0 = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+ Cursor<Entry> cursor2Node0 = leader.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
assertTrue(cursor2Node0.hasNext());
- Cursor<Entry> cursorNode1 = metaStorageSvc2.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
+ Cursor<Entry> cursorNode1 = learner.metaStorageService.range(new ByteArray(EXPECTED_RESULT_ENTRY.key()), null);
assertTrue(cursorNode1.hasNext());
- metaStorageSvc.closeCursors(NODE_ID_0).get();
+ leader.metaStorageService.closeCursors(leader.clusterService.topologyService().localMember().id()).get();
assertThrows(NoSuchElementException.class, () -> cursorNode0.iterator().next());
@@ -976,7 +938,7 @@ public class ItMetaStorageServiceTest {
}
/**
- * Matcher for {@link org.apache.ignite.internal.metastorage.server.Condition}.
+ * Matcher for {@link Condition}.
*/
protected static class ServerConditionMatcher extends TypeSafeMatcher<org.apache.ignite.internal.metastorage.server.Condition> {
@@ -1029,54 +991,4 @@ public class ItMetaStorageServiceTest {
}
}
-
- /**
- * Prepares meta storage by instantiating corresponding raft server with {@link MetaStorageListener} and {@link
- * MetaStorageServiceImpl}.
- *
- * @return {@link MetaStorageService} instance.
- */
- private MetaStorageService prepareMetaStorage() throws Exception {
- String localName = cluster.get(0).topologyService().localMember().name();
-
- PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(Set.of(localName));
-
- CompletableFuture<RaftGroupService> firstService = startRaftService(cluster.get(0), configuration);
- CompletableFuture<RaftGroupService> secondService = startRaftService(cluster.get(1), configuration);
-
- assertThat(allOf(firstService, secondService), willCompleteSuccessfully());
-
- raftGroupServices.add(firstService.join());
- raftGroupServices.add(secondService.join());
-
- RaftGroupService metaStorageRaftGrpSvc = secondService.join();
-
- return new MetaStorageServiceImpl(metaStorageRaftGrpSvc, NODE_ID_0, NODE_ID_0);
- }
-
- private CompletableFuture<RaftGroupService> startRaftService(
- ClusterService node, PeersAndLearners configuration
- ) throws NodeStoppingException {
- var raftManager = new Loza(
- node,
- raftConfiguration,
- dataPath.resolve("raftManager" + raftManagers.size()),
- new HybridClockImpl());
-
- raftManager.start();
-
- raftManagers.add(raftManager);
-
- Peer serverPeer = configuration.peer(node.topologyService().localMember().name());
-
- if (serverPeer == null) {
- return raftManager.startRaftGroupService(MetastorageGroupId.INSTANCE, configuration);
- } else {
- var nodeId = new RaftNodeId(MetastorageGroupId.INSTANCE, serverPeer);
-
- return raftManager.startRaftGroupNode(
- nodeId, configuration, new MetaStorageListener(mockStorage), RaftGroupEventsListener.noopLsnr
- );
- }
- }
}
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageWatchTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageWatchTest.java
new file mode 100644
index 0000000000..2c8772adda
--- /dev/null
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/impl/ItMetaStorageWatchTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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.metastorage.impl;
+
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
+import static org.apache.ignite.utils.ClusterServiceTestUtils.findLocalAddresses;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+import org.apache.ignite.internal.metastorage.dsl.Conditions;
+import org.apache.ignite.internal.metastorage.dsl.Operations;
+import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
+import org.apache.ignite.internal.raft.Loza;
+import org.apache.ignite.internal.raft.RaftManager;
+import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.vault.VaultManager;
+import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.StaticNodeFinder;
+import org.apache.ignite.utils.ClusterServiceTestUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Tests for Meta Storage Watches.
+ */
+@ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
+public class ItMetaStorageWatchTest {
+ private static class Node {
+ private final ClusterService clusterService;
+
+ private final RaftManager raftManager;
+
+ private final MetaStorageManager metaStorageManager;
+
+ private final CompletableFuture<Set<String>> metaStorageNodesFuture = new CompletableFuture<>();
+
+ Node(ClusterService clusterService, RaftConfiguration raftConfiguration, Path dataPath) {
+ this.clusterService = clusterService;
+
+ Path basePath = dataPath.resolve(name());
+
+ this.raftManager = new Loza(
+ clusterService,
+ raftConfiguration,
+ basePath.resolve("raft"),
+ new HybridClockImpl()
+ );
+
+ var vaultManager = mock(VaultManager.class);
+
+ when(vaultManager.get(any())).thenReturn(CompletableFuture.completedFuture(null));
+ when(vaultManager.put(any(), any())).thenReturn(CompletableFuture.completedFuture(null));
+
+ var cmgManager = mock(ClusterManagementGroupManager.class);
+
+ when(cmgManager.metaStorageNodes()).thenReturn(metaStorageNodesFuture);
+
+ this.metaStorageManager = new MetaStorageManagerImpl(
+ vaultManager,
+ clusterService,
+ cmgManager,
+ raftManager,
+ new RocksDbKeyValueStorage(name(), basePath.resolve("storage"))
+ );
+ }
+
+ void start(Set<String> metaStorageNodes) {
+ clusterService.start();
+ raftManager.start();
+ metaStorageManager.start();
+
+ metaStorageNodesFuture.complete(metaStorageNodes);
+ }
+
+ String name() {
+ return clusterService.localConfiguration().getName();
+ }
+
+ void stop() throws Exception {
+ Stream<AutoCloseable> beforeNodeStop = Stream.of(metaStorageManager, raftManager, clusterService).map(c -> c::beforeNodeStop);
+
+ Stream<AutoCloseable> nodeStop = Stream.of(metaStorageManager, raftManager, clusterService).map(c -> c::stop);
+
+ IgniteUtils.closeAll(Stream.concat(beforeNodeStop, nodeStop));
+ }
+ }
+
+ private TestInfo testInfo;
+
+ @WorkDirectory
+ private Path workDir;
+
+ @InjectConfiguration
+ private RaftConfiguration raftConfiguration;
+
+ private final List<Node> nodes = new ArrayList<>();
+
+ @BeforeEach
+ public void beforeTest(TestInfo testInfo) {
+ this.testInfo = testInfo;
+ }
+
+ @AfterEach
+ void tearDown() throws Exception {
+ IgniteUtils.closeAll(nodes.stream().map(node -> node::stop));
+ }
+
+ private void startNodes(int amount) {
+ List<NetworkAddress> localAddresses = findLocalAddresses(10_000, 10_000 + nodes.size() + amount);
+
+ var nodeFinder = new StaticNodeFinder(localAddresses);
+
+ localAddresses.stream()
+ .map(addr -> ClusterServiceTestUtils.clusterService(testInfo, addr.port(), nodeFinder))
+ .forEach(clusterService -> nodes.add(new Node(clusterService, raftConfiguration, workDir)));
+
+ nodes.parallelStream().forEach(node -> node.start(Set.of(nodes.get(0).name())));
+ }
+
+ @Test
+ void testExactWatch() throws Exception {
+ testWatches((node, latch) -> node.metaStorageManager.registerExactWatch(new ByteArray("foo"), new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ assertThat(event.entryEvent().newEntry().key(), is("foo".getBytes(StandardCharsets.UTF_8)));
+ assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(StandardCharsets.UTF_8)));
+
+ latch.countDown();
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ fail();
+ }
+ }));
+ }
+
+ @Test
+ void testPrefixWatch() throws Exception {
+ testWatches((node, latch) -> node.metaStorageManager.registerPrefixWatch(new ByteArray("fo"), new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ assertThat(event.entryEvent().newEntry().key(), is("foo".getBytes(StandardCharsets.UTF_8)));
+ assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(StandardCharsets.UTF_8)));
+
+ latch.countDown();
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ fail();
+ }
+ }));
+ }
+
+ @Test
+ void testRangeWatch() throws Exception {
+ testWatches((node, latch) -> {
+ var startRange = new ByteArray("fo" + ('o' - 1));
+ var endRange = new ByteArray("foz");
+
+ node.metaStorageManager.registerRangeWatch(startRange, endRange, new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ assertThat(event.entryEvent().newEntry().key(), is("foo".getBytes(StandardCharsets.UTF_8)));
+ assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(StandardCharsets.UTF_8)));
+
+ latch.countDown();
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ fail();
+ }
+ });
+ });
+ }
+
+ private void testWatches(BiConsumer<Node, CountDownLatch> registerWatchAction) throws Exception {
+ int numNodes = 3;
+
+ startNodes(numNodes);
+
+ var latch = new CountDownLatch(numNodes);
+
+ for (Node node : nodes) {
+ registerWatchAction.accept(node, latch);
+
+ node.metaStorageManager.deployWatches();
+ }
+
+ var key = new ByteArray("foo");
+
+ CompletableFuture<Boolean> invokeFuture = nodes.get(0).metaStorageManager.invoke(
+ Conditions.notExists(key),
+ Operations.put(key, "bar".getBytes(StandardCharsets.UTF_8)),
+ Operations.noop()
+ );
+
+ assertThat(invokeFuture, willBe(true));
+
+ assertTrue(latch.await(10, TimeUnit.SECONDS));
+ }
+
+ /**
+ * Tests that metastorage missed metastorage events are replayed after deploying watches.
+ */
+ @Test
+ void testReplayUpdates() throws InterruptedException {
+ int numNodes = 3;
+
+ startNodes(numNodes);
+
+ var exactLatch = new CountDownLatch(numNodes);
+ var prefixLatch = new CountDownLatch(numNodes);
+
+ for (Node node : nodes) {
+ node.metaStorageManager.registerExactWatch(new ByteArray("foo"), new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ assertThat(event.entryEvent().newEntry().key(), is("foo".getBytes(StandardCharsets.UTF_8)));
+ assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(StandardCharsets.UTF_8)));
+
+ exactLatch.countDown();
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ fail();
+ }
+ });
+
+ node.metaStorageManager.registerPrefixWatch(new ByteArray("ba"), new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ List<String> keys = event.entryEvents().stream()
+ .map(e -> new String(e.newEntry().key(), StandardCharsets.UTF_8))
+ .collect(Collectors.toList());
+
+ List<String> values = event.entryEvents().stream()
+ .map(e -> new String(e.newEntry().value(), StandardCharsets.UTF_8))
+ .collect(Collectors.toList());
+
+ assertThat(keys, containsInAnyOrder("bar", "baz"));
+ assertThat(values, containsInAnyOrder("one", "two"));
+
+ prefixLatch.countDown();
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ fail();
+ }
+ });
+ }
+
+ CompletableFuture<Boolean> invokeFuture = nodes.get(0).metaStorageManager.invoke(
+ Conditions.notExists(new ByteArray("foo")),
+ Operations.put(new ByteArray("foo"), "bar".getBytes(StandardCharsets.UTF_8)),
+ Operations.noop()
+ );
+
+ assertThat(invokeFuture, willBe(true));
+
+ invokeFuture = nodes.get(0).metaStorageManager.invoke(
+ Conditions.exists(new ByteArray("foo")),
+ List.of(
+ Operations.put(new ByteArray("bar"), "one".getBytes(StandardCharsets.UTF_8)),
+ Operations.put(new ByteArray("baz"), "two".getBytes(StandardCharsets.UTF_8))
+ ),
+ List.of()
+ );
+
+ assertThat(invokeFuture, willBe(true));
+
+ nodes.forEach(node -> {
+ try {
+ node.metaStorageManager.deployWatches();
+ } catch (NodeStoppingException e) {
+ throw new RuntimeException(e);
+ }
+ });
+
+ assertTrue(exactLatch.await(10, TimeUnit.SECONDS));
+ assertTrue(prefixLatch.await(10, TimeUnit.SECONDS));
+ }
+}
diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java
index af270fa3e7..3c26c3d8b3 100644
--- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java
+++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/server/raft/ItMetaStorageRaftGroupTest.java
@@ -26,14 +26,12 @@ import static org.apache.ignite.utils.ClusterServiceTestUtils.waitForTopology;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotSame;
import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.Mockito.when;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
-import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
@@ -44,7 +42,6 @@ import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.metastorage.Entry;
import org.apache.ignite.internal.metastorage.impl.EntryImpl;
-import org.apache.ignite.internal.metastorage.impl.ItMetaStorageServiceTest;
import org.apache.ignite.internal.metastorage.impl.MetaStorageService;
import org.apache.ignite.internal.metastorage.impl.MetaStorageServiceImpl;
import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
@@ -84,7 +81,7 @@ import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class ItMetaStorageRaftGroupTest {
/** The logger. */
- private static final IgniteLogger LOG = Loggers.forClass(ItMetaStorageServiceTest.class);
+ private static final IgniteLogger LOG = Loggers.forClass(ItMetaStorageRaftGroupTest.class);
/** Base network port. */
private static final int NODE_PORT_BASE = 20_000;
@@ -215,8 +212,7 @@ public class ItMetaStorageRaftGroupTest {
final AtomicInteger replicatorStoppedCounter = new AtomicInteger(0);
when(mockStorage.range(EXPECTED_RESULT_ENTRY1.key(), new byte[]{4}, false)).thenAnswer(invocation -> {
- List<Entry> entries = new ArrayList<>(
- List.of(EXPECTED_RESULT_ENTRY1, EXPECTED_RESULT_ENTRY2));
+ List<Entry> entries = List.of(EXPECTED_RESULT_ENTRY1, EXPECTED_RESULT_ENTRY2);
return Cursor.fromBareIterator(entries.iterator());
});
@@ -228,55 +224,58 @@ public class ItMetaStorageRaftGroupTest {
String oldLeaderId = raftServersRaftGroups.get(0).value.leader().consistentId();
- Optional<RaftServer> oldLeaderServer = raftServers.stream()
- .filter(s -> localMemberName(s.clusterService()).equals(oldLeaderId)).findFirst();
+ RaftServer oldLeaderServer = raftServers.stream()
+ .filter(s -> localMemberName(s.clusterService()).equals(oldLeaderId))
+ .findFirst()
+ .orElseThrow();
//Server that will be alive after we stop leader.
- Optional<RaftServer> liveServer = raftServers.stream()
- .filter(s -> !localMemberName(s.clusterService()).equals(oldLeaderId)).findFirst();
-
- if (oldLeaderServer.isEmpty() || liveServer.isEmpty()) {
- fail();
- }
+ RaftServer liveServer = raftServers.stream()
+ .filter(s -> !localMemberName(s.clusterService()).equals(oldLeaderId))
+ .findFirst()
+ .orElseThrow();
RaftGroupService raftGroupServiceOfLiveServer = raftServersRaftGroups.stream()
- .filter(p -> p.key.equals(liveServer.get()))
- .findFirst().get().value;
+ .filter(p -> p.key.equals(liveServer))
+ .findFirst()
+ .orElseThrow()
+ .value;
MetaStorageService metaStorageSvc = new MetaStorageServiceImpl(
- raftGroupServiceOfLiveServer, "some_node", "some_node");
+ raftGroupServiceOfLiveServer, liveServer.clusterService().topologyService().localMember());
- Cursor<Entry> cursor = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY1.key()), new ByteArray(new byte[]{4}));
+ try (Cursor<Entry> cursor = metaStorageSvc.range(new ByteArray(EXPECTED_RESULT_ENTRY1.key()), new ByteArray(new byte[]{4}))) {
- assertTrue(waitForCondition(
- () -> replicatorStartedCounter.get() == 2, 5_000), replicatorStartedCounter.get() + "");
+ assertTrue(waitForCondition(
+ () -> replicatorStartedCounter.get() == 2, 5_000), replicatorStartedCounter.get() + "");
- assertTrue(cursor.hasNext());
+ assertTrue(cursor.hasNext());
- assertEquals(EXPECTED_RESULT_ENTRY1, (cursor.iterator().next()));
+ assertEquals(EXPECTED_RESULT_ENTRY1, (cursor.iterator().next()));
- // Ensure that leader has not been changed.
- // In a stable topology unexpected leader election shouldn't happen.
- assertTrue(waitForCondition(
- () -> replicatorStartedCounter.get() == 2, 5_000), replicatorStartedCounter.get() + "");
+ // Ensure that leader has not been changed.
+ // In a stable topology unexpected leader election shouldn't happen.
+ assertTrue(waitForCondition(
+ () -> replicatorStartedCounter.get() == 2, 5_000), replicatorStartedCounter.get() + "");
- //stop leader
- oldLeaderServer.get().stopRaftNodes(MetastorageGroupId.INSTANCE);
- oldLeaderServer.get().stop();
- cluster.stream().filter(c -> localMemberName(c).equals(oldLeaderId)).findFirst().get().stop();
+ //stop leader
+ oldLeaderServer.stopRaftNodes(MetastorageGroupId.INSTANCE);
+ oldLeaderServer.stop();
+ cluster.stream().filter(c -> localMemberName(c).equals(oldLeaderId)).findFirst().orElseThrow().stop();
- raftGroupServiceOfLiveServer.refreshLeader().get();
+ raftGroupServiceOfLiveServer.refreshLeader().get();
- assertNotSame(oldLeaderId, raftGroupServiceOfLiveServer.leader().consistentId());
+ assertNotSame(oldLeaderId, raftGroupServiceOfLiveServer.leader().consistentId());
- // ensure that leader has been changed only once
- assertTrue(waitForCondition(
- () -> replicatorStartedCounter.get() == 4, 5_000), replicatorStartedCounter.get() + "");
- assertTrue(waitForCondition(
- () -> replicatorStoppedCounter.get() == 2, 5_000), replicatorStoppedCounter.get() + "");
+ // ensure that leader has been changed only once
+ assertTrue(waitForCondition(
+ () -> replicatorStartedCounter.get() == 4, 5_000), replicatorStartedCounter.get() + "");
+ assertTrue(waitForCondition(
+ () -> replicatorStoppedCounter.get() == 2, 5_000), replicatorStoppedCounter.get() + "");
- assertTrue(cursor.hasNext());
- assertEquals(EXPECTED_RESULT_ENTRY2, (cursor.iterator().next()));
+ assertTrue(cursor.hasNext());
+ assertEquals(EXPECTED_RESULT_ENTRY2, (cursor.iterator().next()));
+ }
}
private List<Pair<RaftServer, RaftGroupService>> prepareJraftMetaStorages(AtomicInteger replicatorStartedCounter,
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/MetastorageCommandsMessageGroup.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/MetastorageCommandsMessageGroup.java
index 0166e56618..40f44346c1 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/MetastorageCommandsMessageGroup.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/MetastorageCommandsMessageGroup.java
@@ -100,12 +100,6 @@ public interface MetastorageCommandsMessageGroup {
/** Message type for {@link PrefixCommand}. */
short PREFIX = 61;
- /** Message type for {@link WatchExactKeysCommand}. */
- short WATCH_EXACT_KEYS = 70;
-
- /** Message type for {@link WatchRangeKeysCommand}. */
- short WATCH_RANGE_KEYS = 71;
-
/** Message type for {@link CursorHasNextCommand}. */
short CURSOR_HAS_NEXT = 80;
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchExactKeysCommand.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchExactKeysCommand.java
deleted file mode 100644
index 1d94c68f19..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchExactKeysCommand.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.metastorage.command;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import org.apache.ignite.internal.raft.WriteCommand;
-import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.network.annotations.Transferable;
-
-/**
- * Watch command for MetaStorageCommandListener that subscribes on meta storage updates matching the parameters.
- */
-@Transferable(MetastorageCommandsMessageGroup.WATCH_EXACT_KEYS)
-public interface WatchExactKeysCommand extends WriteCommand {
- /**
- * Returns the keys list. Couldn't be {@code null}.
- */
- List<byte[]> keys();
-
- /**
- * Returns start revision inclusive. {@code 0} - all revisions.
- */
- long revision();
-
- /**
- * Returns id of the node that requests range.
- */
- String requesterNodeId();
-
- /**
- * Returns id of cursor that is associated with the current command.
- */
- IgniteUuid cursorId();
-
- /**
- * Static constructor.
- *
- * @param commandsFactory Commands factory.
- * @param keys The keys collection. Couldn't be {@code null}.
- * @param revision Start revision inclusive. {@code 0} - all revisions.
- * @param requesterNodeId Id of the node that requests watch.
- * @param cursorId Id of cursor that is associated with the current command.
- */
- static WatchExactKeysCommand watchExactKeysCommand(
- MetaStorageCommandsFactory commandsFactory,
- Set<ByteArray> keys,
- long revision,
- String requesterNodeId,
- IgniteUuid cursorId
- ) {
- List<byte[]> list = new ArrayList<>(keys.size());
-
- for (ByteArray key : keys) {
- list.add(key.bytes());
- }
-
- return commandsFactory.watchExactKeysCommand()
- .keys(list)
- .revision(revision)
- .requesterNodeId(requesterNodeId)
- .cursorId(cursorId)
- .build();
- }
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchRangeKeysCommand.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchRangeKeysCommand.java
deleted file mode 100644
index 32cc85ace8..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/command/WatchRangeKeysCommand.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.metastorage.command;
-
-import org.apache.ignite.internal.raft.WriteCommand;
-import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.network.annotations.Transferable;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Watch command for MetaStorageCommandListener that subscribes on meta storage updates matching the parameters.
- */
-@Transferable(MetastorageCommandsMessageGroup.WATCH_RANGE_KEYS)
-public interface WatchRangeKeysCommand extends WriteCommand {
- /**
- * Returns start key of range (inclusive). Couldn't be {@code null}.
- */
- byte @Nullable [] keyFrom();
-
- /**
- * Returns end key of range (exclusive). Could be {@code null}.
- */
- byte @Nullable [] keyTo();
-
- /**
- * Returns start revision inclusive. {@code 0} - all revisions.
- */
- long revision();
-
- /**
- * Returns id of the node that requests range.
- */
- String requesterNodeId();
-
- /**
- * Returns id of cursor that is associated with the current command.
- */
- IgniteUuid cursorId();
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
index 4b41865fa8..a6ae8c8278 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
@@ -17,14 +17,16 @@
package org.apache.ignite.internal.metastorage.impl;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.stream.Collectors.toSet;
import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
import static org.apache.ignite.internal.util.ByteUtils.longToBytes;
import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_CLOSING_ERR;
import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_EXECUTION_ERR;
-import static org.apache.ignite.lang.ErrorGroups.MetaStorage.DEPLOYING_WATCH_ERR;
import java.util.Collection;
import java.util.Iterator;
+import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
@@ -32,6 +34,8 @@ import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.metastorage.Entry;
import org.apache.ignite.internal.metastorage.MetaStorageManager;
import org.apache.ignite.internal.metastorage.WatchListener;
@@ -43,25 +47,21 @@ import org.apache.ignite.internal.metastorage.exceptions.CompactedException;
import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
import org.apache.ignite.internal.metastorage.exceptions.OperationTimeoutException;
import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.raft.MetaStorageLearnerListener;
import org.apache.ignite.internal.metastorage.server.raft.MetaStorageListener;
import org.apache.ignite.internal.metastorage.server.raft.MetastorageGroupId;
-import org.apache.ignite.internal.metastorage.watch.AggregatedWatch;
-import org.apache.ignite.internal.metastorage.watch.KeyCriterion;
-import org.apache.ignite.internal.metastorage.watch.WatchAggregator;
import org.apache.ignite.internal.raft.Peer;
import org.apache.ignite.internal.raft.PeersAndLearners;
import org.apache.ignite.internal.raft.RaftGroupEventsListener;
import org.apache.ignite.internal.raft.RaftManager;
import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.Status;
import org.apache.ignite.internal.raft.service.RaftGroupService;
import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.internal.util.IgniteSpinBusyLock;
import org.apache.ignite.internal.util.IgniteUtils;
import org.apache.ignite.internal.vault.VaultManager;
import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.lang.NodeStoppingException;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.network.ClusterService;
@@ -80,11 +80,12 @@ import org.jetbrains.annotations.Nullable;
* </ul>
*/
public class MetaStorageManagerImpl implements MetaStorageManager {
+ private static final IgniteLogger LOG = Loggers.forClass(MetaStorageManagerImpl.class);
+
/**
- * Special key for the vault where the applied revision for {@link MetaStorageManagerImpl#storeEntries} operation is stored.
- * This mechanism is needed for committing processed watches to {@link VaultManager}.
+ * Special key for the vault where the applied revision for {@link MetaStorageManagerImpl#saveRevision} operation is stored.
*/
- public static final ByteArray APPLIED_REV = ByteArray.fromString("applied_revision");
+ private static final ByteArray APPLIED_REV = ByteArray.fromString("applied_revision");
private final ClusterService clusterService;
@@ -97,22 +98,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
private final ClusterManagementGroupManager cmgMgr;
/** Meta storage service. */
- private volatile CompletableFuture<MetaStorageService> metaStorageSvcFut;
-
- /**
- * Aggregator of multiple watches to deploy them as one batch.
- *
- * @see WatchAggregator
- */
- private final WatchAggregator watchAggregator = new WatchAggregator();
-
- /**
- * Future which will be completed with {@link IgniteUuid}, when aggregated watch will be successfully deployed. Can be resolved to
- * {@code null} if no watch deployed at the moment.
- *
- * <p>Multi-threaded access is guarded by {@code this}.
- */
- private CompletableFuture<IgniteUuid> deployFut = new CompletableFuture<>();
+ private volatile CompletableFuture<MetaStorageServiceImpl> metaStorageSvcFut;
/** Actual storage for the Metastorage. */
private final KeyValueStorage storage;
@@ -120,22 +106,13 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
/** Busy lock to stop synchronously. */
private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
- /**
- * If true - all new watches will be deployed immediately.
- *
- * <p>If false - all new watches will be aggregated to one batch for further deploy by {@link MetaStorageManager#deployWatches()}.
- *
- * <p>Multi-threaded access is guarded by {@code this}.
- */
- private boolean areWatchesDeployed = false;
+ /** Prevents double stopping of the component. */
+ private final AtomicBoolean isStopped = new AtomicBoolean();
/**
- * Flag that indicates that the component has been initialized.
+ * Applied revision of the Meta Storage, that is, the most recent revision that has been processed on this node.
*/
- private volatile boolean isInitialized = false;
-
- /** Prevents double stopping of the component. */
- private final AtomicBoolean isStopped = new AtomicBoolean();
+ private volatile long appliedRevision;
/**
* The constructor.
@@ -159,32 +136,58 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
this.storage = storage;
}
- private CompletableFuture<MetaStorageService> initializeMetaStorage(Set<String> metaStorageNodes) {
+ private CompletableFuture<MetaStorageServiceImpl> initializeMetaStorage(Set<String> metaStorageNodes) {
ClusterNode thisNode = clusterService.topologyService().localMember();
- PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(metaStorageNodes);
-
- Peer localPeer = configuration.peer(thisNode.name());
+ String thisNodeName = thisNode.name();
CompletableFuture<RaftGroupService> raftServiceFuture;
try {
- if (localPeer == null) {
- raftServiceFuture = raftMgr.startRaftGroupService(MetastorageGroupId.INSTANCE, configuration);
- } else {
- clusterService.topologyService().addEventHandler(new TopologyEventHandler() {
- @Override
- public void onDisappeared(ClusterNode member) {
- metaStorageSvcFut.thenAccept(svc -> svc.closeCursors(member.id()));
- }
- });
+ // We need to configure the replication protocol differently whether this node is a synchronous or asynchronous replica.
+ if (metaStorageNodes.contains(thisNodeName)) {
+ PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(metaStorageNodes);
+
+ Peer localPeer = configuration.peer(thisNodeName);
- storage.start();
+ assert localPeer != null;
raftServiceFuture = raftMgr.startRaftGroupNode(
new RaftNodeId(MetastorageGroupId.INSTANCE, localPeer),
configuration,
new MetaStorageListener(storage),
+ new RaftGroupEventsListener() {
+ @Override
+ public void onLeaderElected(long term) {
+ // TODO: add listener to remove learners when they leave Logical Topology
+ // see https://issues.apache.org/jira/browse/IGNITE-18554
+
+ registerTopologyEventListener();
+
+ // Update the configuration immediately in case we missed some updates.
+ addLearners(clusterService.topologyService().allMembers());
+ }
+
+ @Override
+ public void onNewPeersConfigurationApplied(PeersAndLearners configuration) {
+ }
+
+ @Override
+ public void onReconfigurationError(Status status, PeersAndLearners configuration, long term) {
+ }
+ }
+ );
+ } else {
+ PeersAndLearners configuration = PeersAndLearners.fromConsistentIds(metaStorageNodes, Set.of(thisNodeName));
+
+ Peer localPeer = configuration.learner(thisNodeName);
+
+ assert localPeer != null;
+
+ raftServiceFuture = raftMgr.startRaftGroupNode(
+ new RaftNodeId(MetastorageGroupId.INSTANCE, localPeer),
+ configuration,
+ new MetaStorageLearnerListener(storage),
RaftGroupEventsListener.noopLsnr
);
}
@@ -192,12 +195,97 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
return CompletableFuture.failedFuture(e);
}
- return raftServiceFuture.thenApply(service -> new MetaStorageServiceImpl(service, thisNode.id(), thisNode.name()));
+ return raftServiceFuture.thenApply(raftService -> new MetaStorageServiceImpl(raftService, thisNode));
+ }
+
+ private void registerTopologyEventListener() {
+ clusterService.topologyService().addEventHandler(new TopologyEventHandler() {
+ @Override
+ public void onAppeared(ClusterNode member) {
+ addLearners(List.of(member));
+ }
+
+ @Override
+ public void onDisappeared(ClusterNode member) {
+ metaStorageSvcFut.thenAccept(service -> isCurrentNodeLeader(service.raftGroupService())
+ .thenAccept(isLeader -> {
+ if (isLeader) {
+ service.closeCursors(member.id());
+ }
+ }));
+ }
+ });
+ }
+
+ private void addLearners(Collection<ClusterNode> nodes) {
+ if (!busyLock.enterBusy()) {
+ LOG.info("Skipping Meta Storage configuration update because the node is stopping");
+
+ return;
+ }
+
+ try {
+ metaStorageSvcFut
+ .thenApply(MetaStorageServiceImpl::raftGroupService)
+ .thenCompose(raftService -> isCurrentNodeLeader(raftService)
+ .thenCompose(isLeader -> isLeader ? addLearners(raftService, nodes) : completedFuture(null)))
+ .whenComplete((v, e) -> {
+ if (e != null) {
+ LOG.error("Unable to change peers on topology update", e);
+ }
+ });
+ } finally {
+ busyLock.leaveBusy();
+ }
+ }
+
+ private CompletableFuture<Void> addLearners(RaftGroupService raftService, Collection<ClusterNode> nodes) {
+ if (!busyLock.enterBusy()) {
+ LOG.info("Skipping Meta Storage configuration update because the node is stopping");
+
+ return completedFuture(null);
+ }
+
+ try {
+ Set<String> peers = raftService.peers().stream()
+ .map(Peer::consistentId)
+ .collect(toSet());
+
+ Set<String> learners = nodes.stream()
+ .map(ClusterNode::name)
+ .filter(name -> !peers.contains(name))
+ .collect(toSet());
+
+ if (learners.isEmpty()) {
+ return completedFuture(null);
+ }
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("New Meta Storage learners detected: " + learners);
+ }
+
+ PeersAndLearners newConfiguration = PeersAndLearners.fromConsistentIds(peers, learners);
+
+ return raftService.addLearners(newConfiguration.learners());
+ } finally {
+ busyLock.leaveBusy();
+ }
+ }
+
+ private CompletableFuture<Boolean> isCurrentNodeLeader(RaftGroupService raftService) {
+ String name = clusterService.topologyService().localMember().name();
+
+ return raftService.refreshLeader()
+ .thenApply(v -> raftService.leader().consistentId().equals(name));
}
/** {@inheritDoc} */
@Override
public void start() {
+ this.appliedRevision = readAppliedRevision().join();
+
+ storage.start();
+
this.metaStorageSvcFut = cmgMgr.metaStorageNodes()
.thenCompose(metaStorageNodes -> {
if (!busyLock.enterBusy()) {
@@ -205,8 +293,6 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
}
try {
- isInitialized = true;
-
return initializeMetaStorage(metaStorageNodes);
} finally {
busyLock.leaveBusy();
@@ -223,135 +309,59 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
busyLock.block();
- if (!isInitialized) {
- // Stop command was called before the init command was received
- return;
- }
-
- synchronized (this) {
- IgniteUtils.closeAll(
- this::stopDeployedWatches,
- () -> {
- if (raftMgr.stopRaftNodes(MetastorageGroupId.INSTANCE)) {
- storage.close();
- }
- }
- );
- }
- }
+ metaStorageSvcFut.cancel(true);
- private void stopDeployedWatches() throws Exception {
- if (!areWatchesDeployed) {
- return;
- }
-
- deployFut
- .thenCompose(watchId -> watchId == null
- ? CompletableFuture.completedFuture(null)
- : metaStorageSvcFut.thenAccept(service -> service.stopWatch(watchId))
- )
- .get();
+ IgniteUtils.closeAll(
+ () -> raftMgr.stopRaftNodes(MetastorageGroupId.INSTANCE),
+ storage::close
+ );
}
@Override
- public synchronized void deployWatches() throws NodeStoppingException {
- if (!busyLock.enterBusy()) {
- throw new NodeStoppingException();
- }
-
- try {
- CompletableFuture<IgniteUuid> deployFut = this.deployFut;
-
- updateAggregatedWatch().whenComplete((id, ex) -> {
- if (ex == null) {
- deployFut.complete(id);
- } else {
- deployFut.completeExceptionally(ex);
- }
- });
-
- areWatchesDeployed = true;
- } finally {
- busyLock.leaveBusy();
- }
+ public long appliedRevision() {
+ return appliedRevision;
}
@Override
- public synchronized CompletableFuture<Long> registerExactWatch(ByteArray key, WatchListener lsnr) {
- if (!busyLock.enterBusy()) {
- return CompletableFuture.failedFuture(new NodeStoppingException());
- }
-
- try {
- long watchId = watchAggregator.add(key, lsnr);
-
- return updateWatches().thenApply(uuid -> watchId);
- } finally {
- busyLock.leaveBusy();
- }
+ public void registerPrefixWatch(ByteArray key, WatchListener lsnr) {
+ storage.watchPrefix(key.bytes(), appliedRevision + 1, lsnr);
}
- /**
- * Register watch listener by collection of keys.
- *
- * @param keys Collection listen.
- * @param lsnr Listener which will be notified for each update.
- * @return Subscription identifier. Could be used in {@link #unregisterWatch} method in order to cancel subscription
- */
- public synchronized CompletableFuture<Long> registerWatch(Collection<ByteArray> keys, WatchListener lsnr) {
- if (!busyLock.enterBusy()) {
- return CompletableFuture.failedFuture(new NodeStoppingException());
- }
-
- try {
- long watchId = watchAggregator.add(keys, lsnr);
-
- return updateWatches().thenApply(uuid -> watchId);
- } finally {
- busyLock.leaveBusy();
- }
+ @Override
+ public void registerExactWatch(ByteArray key, WatchListener listener) {
+ storage.watchExact(key.bytes(), appliedRevision + 1, listener);
}
@Override
- public synchronized CompletableFuture<Long> registerRangeWatch(ByteArray keyFrom, ByteArray keyTo, WatchListener lsnr) {
- if (!busyLock.enterBusy()) {
- return CompletableFuture.failedFuture(new NodeStoppingException());
- }
-
- try {
- long watchId = watchAggregator.add(keyFrom, keyTo, lsnr);
-
- return updateWatches().thenApply(uuid -> watchId);
- } finally {
- busyLock.leaveBusy();
- }
+ public void registerRangeWatch(ByteArray keyFrom, @Nullable ByteArray keyTo, WatchListener listener) {
+ storage.watchRange(keyFrom.bytes(), keyTo == null ? null : keyTo.bytes(), appliedRevision + 1, listener);
}
@Override
- public synchronized CompletableFuture<Long> registerPrefixWatch(ByteArray key, WatchListener lsnr) {
- if (!busyLock.enterBusy()) {
- return CompletableFuture.failedFuture(new NodeStoppingException());
- }
-
- try {
- long watchId = watchAggregator.addPrefix(key, lsnr);
-
- return updateWatches().thenApply(uuid -> watchId);
- } finally {
- busyLock.leaveBusy();
- }
+ public void unregisterWatch(WatchListener lsnr) {
+ storage.removeWatch(lsnr);
}
@Override
- public synchronized CompletableFuture<Void> unregisterWatch(long id) {
+ public void deployWatches() throws NodeStoppingException {
if (!busyLock.enterBusy()) {
- return CompletableFuture.failedFuture(new NodeStoppingException());
+ throw new NodeStoppingException();
}
try {
- watchAggregator.cancel(id);
+ storage.startWatches(revision -> {
+ if (!busyLock.enterBusy()) {
+ LOG.info("Skipping applying MetaStorage revision because the node is stopping");
- return updateWatches().thenApply(uuid -> null);
+ return;
+ }
+
+ try {
+ saveRevision(revision);
+ } finally {
+ busyLock.leaveBusy();
+ }
+ });
} finally {
busyLock.leaveBusy();
}
@@ -594,8 +604,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
*
* @see MetaStorageService#range(ByteArray, ByteArray, long)
*/
- public @NotNull Cursor<Entry> range(@NotNull ByteArray keyFrom, @Nullable ByteArray keyTo, long revUpperBound)
- throws NodeStoppingException {
+ public Cursor<Entry> range(ByteArray keyFrom, @Nullable ByteArray keyTo, long revUpperBound) throws NodeStoppingException {
if (!busyLock.enterBusy()) {
throw new NodeStoppingException();
}
@@ -617,11 +626,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
*
* @see MetaStorageService#range(ByteArray, ByteArray, boolean)
*/
- public @NotNull Cursor<Entry> range(
- @NotNull ByteArray keyFrom,
- @Nullable ByteArray keyTo,
- boolean includeTombstones
- ) throws NodeStoppingException {
+ public Cursor<Entry> range(ByteArray keyFrom, @Nullable ByteArray keyTo, boolean includeTombstones) throws NodeStoppingException {
if (!busyLock.enterBusy()) {
throw new NodeStoppingException();
}
@@ -652,42 +657,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
}
try {
- CompletableFuture<Cursor<Entry>> cursorFuture = metaStorageSvcFut.thenCombine(
- appliedRevision(),
- (svc, appliedRevision) -> svc.range(keyFrom, keyTo, appliedRevision)
- );
-
- return new CursorWrapper<>(cursorFuture);
- } finally {
- busyLock.leaveBusy();
- }
- }
-
- /**
- * Retrieves entries for the given key prefix in lexicographic order. Entries will be filtered out by the current applied revision as an
- * upper bound. Applied revision is a revision of the last successful vault update.
- *
- * <p>Prefix query is a synonym of the range query {@code (prefixKey, nextKey(prefixKey))}.
- *
- * @param keyPrefix Prefix of the key to retrieve the entries. Couldn't be {@code null}.
- * @return Cursor built upon entries corresponding to the given range and applied revision.
- * @throws OperationTimeoutException If the operation is timed out.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
- * @see ByteArray
- * @see Entry
- */
- public Cursor<Entry> prefixWithAppliedRevision(ByteArray keyPrefix) throws NodeStoppingException {
- if (!busyLock.enterBusy()) {
- throw new NodeStoppingException();
- }
-
- try {
- CompletableFuture<Cursor<Entry>> cursorFuture = metaStorageSvcFut.thenCombine(
- appliedRevision(),
- (svc, appliedRevision) -> svc.prefix(keyPrefix, appliedRevision)
- );
-
- return new CursorWrapper<>(cursorFuture);
+ return new CursorWrapper<>(metaStorageSvcFut.thenApply(svc -> svc.range(keyFrom, keyTo, appliedRevision)));
} finally {
busyLock.leaveBusy();
}
@@ -729,69 +699,20 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
}
/**
- * Returns applied revision for {@link VaultManager#putAll} operation.
+ * Returns applied revision from the local storage.
*/
- private CompletableFuture<Long> appliedRevision() {
+ private CompletableFuture<Long> readAppliedRevision() {
return vaultMgr.get(APPLIED_REV)
.thenApply(appliedRevision -> appliedRevision == null ? 0L : bytesToLong(appliedRevision.value()));
}
/**
- * Stop current batch of consolidated watches and register new one from current {@link WatchAggregator}.
- *
- * <p>This method MUST always be called under a {@code synchronized} block.
- *
- * @return Ignite UUID of new consolidated watch.
- */
- private CompletableFuture<IgniteUuid> updateWatches() {
- if (!areWatchesDeployed) {
- return deployFut;
- }
-
- deployFut = deployFut
- .thenCompose(id -> id == null
- ? CompletableFuture.completedFuture(null)
- : metaStorageSvcFut.thenCompose(svc -> svc.stopWatch(id))
- )
- .thenCompose(r -> updateAggregatedWatch());
-
- return deployFut;
- }
-
- private CompletableFuture<IgniteUuid> updateAggregatedWatch() {
- return appliedRevision()
- .thenCompose(appliedRevision ->
- watchAggregator.watch(appliedRevision + 1, this::storeEntries)
- .map(this::dispatchAppropriateMetaStorageWatch)
- .orElseGet(() -> CompletableFuture.completedFuture(null))
- );
- }
-
- /**
- * Store entries with appropriate associated revision.
- *
- * @param entries to store.
- * @param revision associated revision.
+ * Save processed Meta Storage revision.
*/
- private void storeEntries(Collection<IgniteBiTuple<ByteArray, byte[]>> entries, long revision) {
- appliedRevision()
- .thenCompose(appliedRevision -> {
- if (revision <= appliedRevision) {
- throw new MetaStorageException(DEPLOYING_WATCH_ERR, String.format(
- "Current revision (%d) must be greater than the revision in the Vault (%d)",
- revision, appliedRevision
- ));
- }
+ private void saveRevision(long revision) {
+ vaultMgr.put(APPLIED_REV, longToBytes(revision)).join();
- Map<ByteArray, byte[]> batch = IgniteUtils.newHashMap(entries.size() + 1);
-
- batch.put(APPLIED_REV, longToBytes(revision));
-
- entries.forEach(e -> batch.put(e.getKey(), e.getValue()));
-
- return vaultMgr.putAll(batch);
- })
- .join();
+ appliedRevision = revision;
}
// TODO: IGNITE-14691 Temporally solution that should be removed after implementing reactive watches.
@@ -801,9 +722,6 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
/** Inner cursor future. */
private final CompletableFuture<Cursor<T>> innerCursorFut;
- /** Inner iterator future. */
- private final CompletableFuture<Iterator<T>> innerIterFut;
-
/**
* Constructor.
*
@@ -811,27 +729,12 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
*/
CursorWrapper(CompletableFuture<Cursor<T>> innerCursorFut) {
this.innerCursorFut = innerCursorFut;
- this.innerIterFut = innerCursorFut.thenApply(Iterable::iterator);
}
/** {@inheritDoc} */
@Override
public void close() {
- try {
- innerCursorFut.thenAccept(cursor -> {
- try {
- cursor.close();
- } catch (RuntimeException e) {
- throw new MetaStorageException(CURSOR_CLOSING_ERR, e);
- }
- }).get();
- } catch (ExecutionException e) {
- throw new IgniteInternalException("Exception while closing a cursor", e);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
-
- throw new IgniteInternalException("Interrupted while closing a cursor", e);
- }
+ get(innerCursorFut.thenAccept(Cursor::close), CURSOR_CLOSING_ERR);
}
/** {@inheritDoc} */
@@ -842,11 +745,7 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
}
try {
- try {
- return innerIterFut.thenApply(Iterator::hasNext).get();
- } catch (InterruptedException | ExecutionException e) {
- throw new MetaStorageException(CURSOR_EXECUTION_ERR, e);
- }
+ return get(innerCursorFut.thenApply(Iterator::hasNext), CURSOR_EXECUTION_ERR);
} finally {
busyLock.leaveBusy();
}
@@ -860,48 +759,22 @@ public class MetaStorageManagerImpl implements MetaStorageManager {
}
try {
- try {
- return innerIterFut.thenApply(Iterator::next).get();
- } catch (InterruptedException | ExecutionException e) {
- throw new MetaStorageException(CURSOR_EXECUTION_ERR, e);
- }
+ return get(innerCursorFut.thenApply(Iterator::next), CURSOR_EXECUTION_ERR);
} finally {
busyLock.leaveBusy();
}
}
- }
- /**
- * Dispatches appropriate meta storage watch method according to inferred watch criterion.
- *
- * @param aggregatedWatch Aggregated watch.
- * @return Future, which will be completed after new watch registration finished.
- */
- private CompletableFuture<IgniteUuid> dispatchAppropriateMetaStorageWatch(AggregatedWatch aggregatedWatch) {
- if (aggregatedWatch.keyCriterion() instanceof KeyCriterion.CollectionCriterion) {
- var criterion = (KeyCriterion.CollectionCriterion) aggregatedWatch.keyCriterion();
-
- return metaStorageSvcFut.thenCompose(metaStorageSvc -> metaStorageSvc.watch(
- criterion.keys(),
- aggregatedWatch.revision(),
- aggregatedWatch.listener()));
- } else if (aggregatedWatch.keyCriterion() instanceof KeyCriterion.ExactCriterion) {
- var criterion = (KeyCriterion.ExactCriterion) aggregatedWatch.keyCriterion();
-
- return metaStorageSvcFut.thenCompose(metaStorageSvc -> metaStorageSvc.watch(
- criterion.key(),
- aggregatedWatch.revision(),
- aggregatedWatch.listener()));
- } else if (aggregatedWatch.keyCriterion() instanceof KeyCriterion.RangeCriterion) {
- var criterion = (KeyCriterion.RangeCriterion) aggregatedWatch.keyCriterion();
-
- return metaStorageSvcFut.thenCompose(metaStorageSvc -> metaStorageSvc.watch(
- criterion.from(),
- criterion.to(),
- aggregatedWatch.revision(),
- aggregatedWatch.listener()));
- } else {
- throw new UnsupportedOperationException("Unsupported type of criterion");
+ private <R> R get(CompletableFuture<R> future, int errorCode) {
+ try {
+ return future.get();
+ } catch (ExecutionException e) {
+ throw new MetaStorageException(errorCode, e);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+
+ throw new MetaStorageException(errorCode, e);
+ }
}
}
}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageService.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageService.java
index 9d4f133066..b774971013 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageService.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageService.java
@@ -22,7 +22,6 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.dsl.Condition;
import org.apache.ignite.internal.metastorage.dsl.If;
import org.apache.ignite.internal.metastorage.dsl.Operation;
@@ -31,7 +30,6 @@ import org.apache.ignite.internal.metastorage.exceptions.CompactedException;
import org.apache.ignite.internal.metastorage.exceptions.OperationTimeoutException;
import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteUuid;
import org.jetbrains.annotations.Nullable;
/**
@@ -52,12 +50,12 @@ public interface MetaStorageService {
/**
* Retrieves an entry for the given key and the revision upper bound.
*
- * @param key The key. Couldn't be {@code null}.
+ * @param key The key. Couldn't be {@code null}.
* @param revUpperBound The upper bound for entry revisions. Must be positive.
* @return An entry for the given key and maximum revision limited by {@code revUpperBound}. Couldn't be {@code null}.
* @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on
- * getting future result.
+ * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on getting
+ * future result.
* @see ByteArray
* @see Entry
*/
@@ -77,12 +75,12 @@ public interface MetaStorageService {
/**
* Retrieves entries for given keys and the revision upper bound.
*
- * @param keys The set of keys. Couldn't be {@code null} or empty. Set elements couldn't be {@code null}.
+ * @param keys The set of keys. Couldn't be {@code null} or empty. Set elements couldn't be {@code null}.
* @param revUpperBound The upper bound for entry revisions. Must be positive.
* @return A map of entries for given keys and maximum revision limited by {@code revUpperBound}. Couldn't be {@code null}.
* @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on
- * getting future result.
+ * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on getting
+ * future result.
* @see ByteArray
* @see Entry
*/
@@ -91,7 +89,7 @@ public interface MetaStorageService {
/**
* Inserts or updates an entry with the given key and the given value.
*
- * @param key The key. Couldn't be {@code null}.
+ * @param key The key. Couldn't be {@code null}.
* @param value The value. Couldn't be {@code null}.
* @return Completed future.
* @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
@@ -103,7 +101,7 @@ public interface MetaStorageService {
/**
* Inserts or updates an entry with the given key and the given value and retrieves a previous entry for the given key.
*
- * @param key The key. Couldn't be {@code null}.
+ * @param key The key. Couldn't be {@code null}.
* @param value The value. Couldn't be {@code null}.
* @return A previous entry for the given key. Couldn't be {@code null}.
* @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
@@ -185,8 +183,8 @@ public interface MetaStorageService {
* <p>Conditional update could be treated as <i>if(condition)-then(success)-else(failure)</i> expression.</p>
*
* @param condition The condition.
- * @param success The update which will be applied in case of condition evaluation yields {@code true}.
- * @param failure The update which will be applied in case of condition evaluation yields {@code false}.
+ * @param success The update which will be applied in case of condition evaluation yields {@code true}.
+ * @param failure The update which will be applied in case of condition evaluation yields {@code false}.
* @return Future result {@code true} if {@code success} update was applied, otherwise {@code false}.
* @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
* @see ByteArray
@@ -202,8 +200,8 @@ public interface MetaStorageService {
* <p>Conditional update could be treated as <i>if(condition)-then(success)-else(failure)</i> expression.</p>
*
* @param condition The condition.
- * @param success The updates which will be applied in case of condition evaluation yields {@code true}.
- * @param failure The updates which will be applied in case of condition evaluation yields {@code false}.
+ * @param success The updates which will be applied in case of condition evaluation yields {@code true}.
+ * @param failure The updates which will be applied in case of condition evaluation yields {@code false}.
* @return Future result {@code true} if {@code success} update was applied, otherwise {@code false}.
* @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
* @see ByteArray
@@ -214,12 +212,11 @@ public interface MetaStorageService {
CompletableFuture<Boolean> invoke(Condition condition, Collection<Operation> success, Collection<Operation> failure);
/**
- * Invoke, which supports nested conditional statements.
- * For detailed docs about construction of new if statement, look at {@link If} javadocs.
+ * Invoke, which supports nested conditional statements. For detailed docs about construction of new if statement, look at {@link If}
+ * javadocs.
*
* @param iif {@link If} statement to invoke
* @return execution result
- *
* @see If
* @see StatementResult
*/
@@ -229,12 +226,12 @@ public interface MetaStorageService {
* Retrieves entries for the given key range in lexicographic order. Entries will be filtered out by upper bound of given revision
* number.
*
- * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
- * @param keyTo End key of range (exclusive). Could be {@code null}.
+ * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
+ * @param keyTo End key of range (exclusive). Could be {@code null}.
* @param revUpperBound The upper bound for entry revision. {@code -1} means latest revision.
* @return Cursor built upon entries corresponding to the given range and revision.
* @throws OperationTimeoutException If the operation is timed out.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
+ * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
* @see ByteArray
* @see Entry
*/
@@ -244,13 +241,13 @@ public interface MetaStorageService {
* Retrieves entries for the given key range in lexicographic order. Entries will be filtered out by upper bound of given revision
* number.
*
- * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
- * @param keyTo End key of range (exclusive). Could be {@code null}.
- * @param revUpperBound The upper bound for entry revision. {@code -1} means latest revision.
+ * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
+ * @param keyTo End key of range (exclusive). Could be {@code null}.
+ * @param revUpperBound The upper bound for entry revision. {@code -1} means latest revision.
* @param includeTombstones Whether to include tombstone entries.
* @return Cursor built upon entries corresponding to the given range and revision.
* @throws OperationTimeoutException If the operation is timed out.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
+ * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
* @see ByteArray
* @see Entry
*/
@@ -261,25 +258,25 @@ public interface MetaStorageService {
* {@code revUpperBound == -1}.
*
* @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
- * @param keyTo End key of range (exclusive). Could be {@code null}.
+ * @param keyTo End key of range (exclusive). Could be {@code null}.
* @return Cursor built upon entries corresponding to the given range and revision.
* @throws OperationTimeoutException If the operation is timed out.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
+ * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
* @see ByteArray
* @see Entry
*/
Cursor<Entry> range(ByteArray keyFrom, @Nullable ByteArray keyTo);
/**
- * Retrieves entries for the given key range in lexicographic order. Short cut for
- * {@link #range(ByteArray, ByteArray, long, boolean)} where {@code revUpperBound == -1}.
+ * Retrieves entries for the given key range in lexicographic order. Short cut for {@link #range(ByteArray, ByteArray, long, boolean)}
+ * where {@code revUpperBound == -1}.
*
- * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
- * @param keyTo End key of range (exclusive). Could be {@code null}.
+ * @param keyFrom Start key of range (inclusive). Couldn't be {@code null}.
+ * @param keyTo End key of range (exclusive). Could be {@code null}.
* @param includeTombstones Whether to include tombstone entries.
* @return Cursor built upon entries corresponding to the given range and revision.
* @throws OperationTimeoutException If the operation is timed out.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
+ * @throws CompactedException If the desired revisions are removed from the storage due to a compaction.
* @see ByteArray
* @see Entry
*/
@@ -296,64 +293,6 @@ public interface MetaStorageService {
*/
Cursor<Entry> prefix(ByteArray prefix, long revUpperBound);
- /**
- * Subscribes on meta storage updates matching the parameters.
- *
- * @param keyFrom Start key of range (inclusive). Could be {@code null}.
- * @param keyTo End key of range (exclusive). Could be {@code null}.
- * @param revision Start revision inclusive. {@code 0} - all revisions, {@code -1} - latest revision (accordingly to current meta
- * storage state).
- * @param lsnr Listener which will be notified for each update.
- * @return Subscription identifier. Could be used in {@link #stopWatch} method in order to cancel subscription.
- * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on
- * getting future result.
- * @see ByteArray
- * @see Entry
- */
- CompletableFuture<IgniteUuid> watch(@Nullable ByteArray keyFrom, @Nullable ByteArray keyTo, long revision, WatchListener lsnr);
-
- /**
- * Subscribes on meta storage updates for the given key.
- *
- * @param key The target key. Couldn't be {@code null}.
- * @param revision Start revision inclusive. {@code 0} - all revisions, {@code -1} - latest revision (accordingly to current meta
- * storage state).
- * @param lsnr Listener which will be notified for each update.
- * @return Subscription identifier. Could be used in {@link #stopWatch} method in order to cancel subscription.
- * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on
- * getting future result.
- * @see ByteArray
- * @see Entry
- */
- CompletableFuture<IgniteUuid> watch(ByteArray key, long revision, WatchListener lsnr);
-
- /**
- * Subscribes on meta storage updates for given keys.
- *
- * @param keys Set of target keys. Couldn't be {@code null} or empty.
- * @param revision Start revision inclusive. {@code 0} - all revision, {@code -1} - latest revision (accordingly to current meta storage
- * state).
- * @param lsnr Listener which will be notified for each update.
- * @return Subscription identifier. Could be used in {@link #stopWatch} method in order to cancel subscription.
- * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
- * @throws CompactedException If the desired revisions are removed from the storage due to a compaction. Will be thrown on
- * getting future result.
- * @see ByteArray
- * @see Entry
- */
- CompletableFuture<IgniteUuid> watch(Set<ByteArray> keys, long revision, WatchListener lsnr);
-
- /**
- * Cancels subscription for the given identifier.
- *
- * @param id Subscription identifier.
- * @return Completed future in case of operation success. Couldn't be {@code null}.
- * @throws OperationTimeoutException If the operation is timed out. Will be thrown on getting future result.
- */
- CompletableFuture<Void> stopWatch(IgniteUuid id);
-
/**
* Compacts meta storage (removes all tombstone entries and old entries except of entries with latest revision).
*
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageServiceImpl.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageServiceImpl.java
index 52e8611247..a1bbe42ca1 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageServiceImpl.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageServiceImpl.java
@@ -23,26 +23,16 @@ import static org.apache.ignite.internal.metastorage.command.GetAndPutAllCommand
import static org.apache.ignite.internal.metastorage.command.GetAndRemoveAllCommand.getAndRemoveAllCommand;
import static org.apache.ignite.internal.metastorage.command.PutAllCommand.putAllCommand;
import static org.apache.ignite.internal.metastorage.command.RemoveAllCommand.removeAllCommand;
-import static org.apache.ignite.internal.metastorage.command.WatchExactKeysCommand.watchExactKeysCommand;
-import static org.apache.ignite.lang.ErrorGroups.MetaStorage.WATCH_STOPPING_ERR;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.RejectedExecutionException;
-import org.apache.ignite.internal.logger.IgniteLogger;
-import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.command.GetAllCommand;
import org.apache.ignite.internal.metastorage.command.GetAndPutAllCommand;
import org.apache.ignite.internal.metastorage.command.GetAndPutCommand;
@@ -79,23 +69,17 @@ import org.apache.ignite.internal.metastorage.dsl.SimpleCondition.ValueCondition
import org.apache.ignite.internal.metastorage.dsl.Statement;
import org.apache.ignite.internal.metastorage.dsl.StatementResult;
import org.apache.ignite.internal.metastorage.dsl.Update;
-import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
import org.apache.ignite.internal.raft.service.RaftGroupService;
import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.lang.IgniteUuidGenerator;
-import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
import org.jetbrains.annotations.Nullable;
/**
* {@link MetaStorageService} implementation.
*/
public class MetaStorageServiceImpl implements MetaStorageService {
- /** The logger. */
- private static final IgniteLogger LOG = Loggers.forClass(MetaStorageServiceImpl.class);
-
/** IgniteUuid generator. */
private static final IgniteUuidGenerator uuidGenerator = new IgniteUuidGenerator(UUID.randomUUID(), 0);
@@ -105,28 +89,22 @@ public class MetaStorageServiceImpl implements MetaStorageService {
/** Meta storage raft group service. */
private final RaftGroupService metaStorageRaftGrpSvc;
- // TODO: IGNITE-14691 Temporally solution that should be removed after implementing reactive watches.
- /** Watch processor, that uses pulling logic in order to retrieve watch notifications from server. */
- private final WatchProcessor watchProcessor;
-
- /** Local node id. */
- private final String localNodeId;
-
- /** Local node name. */
- private final String localNodeName;
+ /** Local node. */
+ private final ClusterNode localNode;
/**
* Constructor.
*
* @param metaStorageRaftGrpSvc Meta storage raft group service.
- * @param localNodeId Local node id.
- * @param localNodeName Local node name.
+ * @param localNode Local node.
*/
- public MetaStorageServiceImpl(RaftGroupService metaStorageRaftGrpSvc, String localNodeId, String localNodeName) {
+ public MetaStorageServiceImpl(RaftGroupService metaStorageRaftGrpSvc, ClusterNode localNode) {
this.metaStorageRaftGrpSvc = metaStorageRaftGrpSvc;
- this.watchProcessor = new WatchProcessor();
- this.localNodeId = localNodeId;
- this.localNodeName = localNodeName;
+ this.localNode = localNode;
+ }
+
+ RaftGroupService raftGroupService() {
+ return metaStorageRaftGrpSvc;
}
/** {@inheritDoc} */
@@ -282,7 +260,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
commandsFactory.rangeCommand()
.keyFrom(keyFrom.bytes())
.keyTo(keyTo == null ? null : keyTo.bytes())
- .requesterNodeId(localNodeId)
+ .requesterNodeId(localNode.id())
.cursorId(uuidGenerator.randomUuid())
.revUpperBound(revUpperBound)
.includeTombstones(includeTombstones)
@@ -314,7 +292,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
commandsFactory.prefixCommand()
.prefix(prefix.bytes())
.revUpperBound(revUpperBound)
- .requesterNodeId(localNodeId)
+ .requesterNodeId(localNode.id())
.cursorId(uuidGenerator.randomUuid())
.includeTombstones(false)
.batchSize(PrefixCommand.DEFAULT_BATCH_SIZE)
@@ -324,74 +302,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
);
}
- /** {@inheritDoc} */
- @Override
- public CompletableFuture<IgniteUuid> watch(
- @Nullable ByteArray keyFrom,
- @Nullable ByteArray keyTo,
- long revision,
- WatchListener lsnr
- ) {
- CompletableFuture<IgniteUuid> watchRes = metaStorageRaftGrpSvc.run(commandsFactory.watchRangeKeysCommand()
- .keyFrom(keyFrom == null ? null : keyFrom.bytes())
- .keyTo(keyTo == null ? null : keyTo.bytes())
- .revision(revision)
- .requesterNodeId(localNodeId)
- .cursorId(uuidGenerator.randomUuid())
- .build()
- );
-
- watchRes.thenAccept(
- watchId -> watchProcessor.addWatch(
- watchId,
- new CursorImpl<>(commandsFactory, metaStorageRaftGrpSvc, watchRes, MetaStorageServiceImpl::watchResponse),
- lsnr
- )
- );
-
- return watchRes;
- }
-
- /** {@inheritDoc} */
- @Override
- public CompletableFuture<IgniteUuid> watch(
- ByteArray key,
- long revision,
- WatchListener lsnr
- ) {
- return watch(key, null, revision, lsnr);
- }
-
- /** {@inheritDoc} */
- @Override
- public CompletableFuture<IgniteUuid> watch(
- Set<ByteArray> keys,
- long revision,
- WatchListener lsnr
- ) {
- CompletableFuture<IgniteUuid> watchRes =
- metaStorageRaftGrpSvc.run(watchExactKeysCommand(commandsFactory, keys, revision, localNodeId, uuidGenerator.randomUuid()));
-
- watchRes.thenAccept(
- watchId -> watchProcessor.addWatch(
- watchId,
- new CursorImpl<>(commandsFactory, metaStorageRaftGrpSvc, watchRes, MetaStorageServiceImpl::watchResponse),
- lsnr
- )
- );
-
- return watchRes;
- }
-
- /** {@inheritDoc} */
- @Override
- public CompletableFuture<Void> stopWatch(IgniteUuid id) {
- return CompletableFuture.runAsync(() -> watchProcessor.stopWatch(id));
- }
-
// TODO: IGNITE-14734 Implement.
-
- /** {@inheritDoc} */
@Override
public CompletableFuture<Void> compact() {
throw new UnsupportedOperationException();
@@ -489,9 +400,7 @@ public class MetaStorageServiceImpl implements MetaStorageService {
.conditionType(cond.compoundConditionType().ordinal())
.build();
} else {
- assert false : "Unknown condition type: " + condition.getClass().getSimpleName();
-
- return null;
+ throw new IllegalArgumentException("Unknown condition type: " + condition.getClass().getSimpleName());
}
}
@@ -520,153 +429,4 @@ public class MetaStorageServiceImpl implements MetaStorageService {
return new EntryImpl(resp.key(), resp.value(), resp.revision(), resp.updateCounter());
}
-
- private static WatchEvent watchResponse(Object obj) {
- MultipleEntryResponse resp = (MultipleEntryResponse) obj;
-
- List<EntryEvent> evts = new ArrayList<>(resp.entries().size() / 2);
-
- Entry o = null;
- Entry n;
-
- for (int i = 0; i < resp.entries().size(); i++) {
- SingleEntryResponse s = resp.entries().get(i);
-
- EntryImpl e = new EntryImpl(s.key(), s.value(), s.revision(), s.updateCounter());
-
- if (i % 2 == 0) {
- o = e;
- } else {
- n = e;
-
- evts.add(new EntryEvent(o, n));
- }
- }
-
- return new WatchEvent(evts);
- }
-
- // TODO: IGNITE-14691 Temporally solution that should be removed after implementing reactive watches.
-
- /** Watch processor, that manages {@link Watcher} threads. */
- private final class WatchProcessor {
- /** Active Watcher threads that process notification pulling logic. */
- private final Map<IgniteUuid, Watcher> watchers = new ConcurrentHashMap<>();
-
- /**
- * Starts exclusive thread per watch that implement watch pulling logic and calls {@link WatchListener#onUpdate(WatchEvent)}} or
- * {@link WatchListener#onError(Throwable)}.
- *
- * @param watchId Watch id.
- * @param cursor Watch Cursor.
- * @param lsnr The listener which receives and handles watch updates.
- */
- private void addWatch(IgniteUuid watchId, CursorImpl<WatchEvent> cursor, WatchListener lsnr) {
- Watcher watcher = new Watcher(cursor, lsnr);
-
- watchers.put(watchId, watcher);
-
- watcher.start();
- }
-
- /**
- * Closes server cursor and interrupts watch pulling thread.
- *
- * @param watchId Watch id.
- */
- private void stopWatch(IgniteUuid watchId) {
- watchers.computeIfPresent(
- watchId,
- (k, v) -> {
- CompletableFuture.runAsync(() -> {
- v.stop = true;
-
- v.interrupt();
- }).thenRun(() -> {
- try {
- Thread.sleep(100);
-
- v.cursor.close();
- } catch (InterruptedException e) {
- throw new MetaStorageException(WATCH_STOPPING_ERR, e);
- } catch (Exception e) {
- if (e instanceof IgniteInternalException && e.getCause().getCause() instanceof RejectedExecutionException) {
- LOG.debug("Cursor close command was rejected because raft executor has been already stopped");
- return;
- }
-
- // TODO: IGNITE-14693 Implement Meta storage exception handling logic.
- LOG.warn("Unexpected exception", e);
- }
- });
- return null;
- }
- );
- }
-
- /** Watcher thread, uses pulling logic in order to retrieve watch notifications from server. */
- private final class Watcher extends Thread {
- private volatile boolean stop = false;
-
- /** Watch event cursor. */
- private Cursor<WatchEvent> cursor;
-
- /** The listener which receives and handles watch updates. */
- private WatchListener lsnr;
-
- /**
- * Constructor.
- *
- * @param cursor Watch event cursor.
- * @param lsnr The listener which receives and handles watch updates.
- */
- Watcher(Cursor<WatchEvent> cursor, WatchListener lsnr) {
- setName("ms-watcher-" + localNodeName);
- this.cursor = cursor;
- this.lsnr = lsnr;
- }
-
- /**
- * Pulls watch events from server side with the help of cursor.iterator.hasNext()/next() in the while(true) loop. Collects watch
- * events with same revision and fires either onUpdate or onError().
- */
- @Override
- public void run() {
- Iterator<WatchEvent> watchEvtsIter = cursor.iterator();
-
- while (!stop) {
- try {
- if (watchEvtsIter.hasNext()) {
- WatchEvent watchEvt = null;
-
- try {
- watchEvt = watchEvtsIter.next();
- } catch (Throwable e) {
- lsnr.onError(e);
-
- throw e;
- }
-
- assert watchEvt != null;
-
- lsnr.onUpdate(watchEvt);
- } else {
- Thread.sleep(10);
- }
- } catch (Throwable e) {
- if (e instanceof NodeStoppingException || e.getCause() instanceof NodeStoppingException) {
- break;
- } else if ((e instanceof InterruptedException || e.getCause() instanceof InterruptedException) && stop) {
- LOG.debug("Watcher has been stopped during node's stop");
-
- break;
- } else {
- // TODO: IGNITE-14693 Implement Meta storage exception handling logic.
- LOG.warn("Unexpected exception", e);
- }
- }
- }
- }
- }
- }
}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
index 33adcb47c1..38254c6dd1 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/KeyValueStorage.java
@@ -21,11 +21,13 @@ import java.nio.file.Path;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.function.LongConsumer;
import org.apache.ignite.internal.close.ManuallyCloseable;
import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.dsl.Operation;
import org.apache.ignite.internal.metastorage.dsl.StatementResult;
+import org.apache.ignite.internal.metastorage.exceptions.CompactedException;
import org.apache.ignite.internal.util.Cursor;
import org.jetbrains.annotations.Nullable;
@@ -220,32 +222,54 @@ public interface KeyValueStorage extends ManuallyCloseable {
* @param keyFrom Start key of range (inclusive).
* @param keyTo Last key of range (exclusive).
* @param rev Start revision number.
- * @return Cursor by update events.
*/
- Cursor<WatchEvent> watch(byte[] keyFrom, byte @Nullable [] keyTo, long rev);
+ void watchRange(byte[] keyFrom, byte @Nullable [] keyTo, long rev, WatchListener listener);
/**
- * Creates subscription on updates of entries corresponding to the given keys range (where upper bound is unlimited) and starting from
- * the given revision number.
+ * Registers a watch listener by a key prefix.
*
- * @param key Start key of range (inclusive).
- * @param rev Start revision number.
- * @return Cursor by update events.
+ * @param prefix Prefix to listen to.
+ * @param rev Starting Meta Storage revision.
+ * @param listener Listener which will be notified for each update.
*/
- Cursor<WatchEvent> watch(byte[] key, long rev);
+ void watchPrefix(byte[] prefix, long rev, WatchListener listener);
/**
- * Creates subscription on updates of entries corresponding to the given keys collection and starting from the given revision number.
+ * Registers a watch listener for the provided key.
*
- * @param keys Collection of keys
- * @param rev Start revision number.
- * @return Cursor by update events.
+ * @param key Meta Storage key.
+ * @param rev Starting Meta Storage revision.
+ * @param listener Listener which will be notified for each update.
*/
- Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev);
+ void watchExact(byte[] key, long rev, WatchListener listener);
+
+ /**
+ * Registers a watch listener for the provided keys.
+ *
+ * @param keys Meta Storage keys.
+ * @param rev Starting Meta Storage revision.
+ * @param listener Listener which will be notified for each update.
+ */
+ void watchExact(Collection<byte[]> keys, long rev, WatchListener listener);
+
+ /**
+ * Starts all registered watches.
+ *
+ * <p>Before calling this method, watches will not receive any updates.
+ *
+ * @param revisionCallback Callback that will be invoked after all watches of a particular revision are processed, with the revision
+ * as its argument.
+ */
+ void startWatches(LongConsumer revisionCallback);
+
+ /**
+ * Unregisters a watch listener.
+ */
+ void removeWatch(WatchListener listener);
/**
* Compacts storage (removes tombstones).
- * TODO: IGNITE-16444 Сorrect compaction for Metastorage.
+ * TODO: IGNITE-16444 Correct compaction for Metastorage.
*/
void compact();
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java
new file mode 100644
index 0000000000..489de7c898
--- /dev/null
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/Watch.java
@@ -0,0 +1,89 @@
+/*
+ * 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.metastorage.server;
+
+import java.util.function.Predicate;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
+
+/**
+ * Subscription on updates of Meta Storage entries corresponding to a subset of keys, starting from a given revision number.
+ */
+public class Watch {
+ /** Current revision. */
+ private volatile long targetRevision;
+
+ /** Key predicate. */
+ private final Predicate<byte[]> predicate;
+
+ /** Event listener. */
+ private final WatchListener listener;
+
+ /**
+ * Constructor.
+ *
+ * @param startRevision Starting revision.
+ * @param listener Event listener.
+ * @param predicate Key predicate.
+ */
+ public Watch(long startRevision, WatchListener listener, Predicate<byte[]> predicate) {
+ this.predicate = predicate;
+ this.listener = listener;
+ this.targetRevision = startRevision;
+ }
+
+ /**
+ * Returns {@code true} if a given key and its revision should be forwarded to the event listener.
+ *
+ * @param key Meta Storage key.
+ * @param revision Revision corresponding to the given {@code key}.
+ */
+ public boolean matches(byte[] key, long revision) {
+ return revision >= targetRevision && predicate.test(key);
+ }
+
+ /**
+ * Notifies the event listener about a Meta Storage event.
+ */
+ public void onUpdate(WatchEvent event) {
+ listener.onUpdate(event);
+
+ targetRevision = event.revision() + 1;
+ }
+
+ /**
+ * Callback that gets called if an error has occurred during the event processing.
+ */
+ public void onError(Throwable e) {
+ listener.onError(e);
+ }
+
+ /**
+ * Returns the event listener.
+ */
+ public WatchListener listener() {
+ return listener;
+ }
+
+ /**
+ * Returns the current Meta Storage revision this Watch is listening to.
+ */
+ public long targetRevision() {
+ return targetRevision;
+ }
+}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
index 6b321a170d..77ad1cffac 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/RocksDbKeyValueStorage.java
@@ -24,6 +24,8 @@ import static org.apache.ignite.internal.metastorage.server.persistence.RocksSto
import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.getAsLongs;
import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.keyToRocksKey;
import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.longToBytes;
+import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.revisionFromRocksKey;
+import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.rocksKeyToBytes;
import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.valueToBytes;
import static org.apache.ignite.internal.metastorage.server.persistence.StorageColumnFamilyType.DATA;
import static org.apache.ignite.internal.metastorage.server.persistence.StorageColumnFamilyType.INDEX;
@@ -34,6 +36,7 @@ import static org.apache.ignite.lang.ErrorGroups.MetaStorage.COMPACTION_ERR;
import static org.apache.ignite.lang.ErrorGroups.MetaStorage.OP_EXECUTION_ERR;
import static org.apache.ignite.lang.ErrorGroups.MetaStorage.RESTORING_STORAGE_ERR;
import static org.apache.ignite.lang.ErrorGroups.MetaStorage.STARTING_STORAGE_ERR;
+import static org.apache.ignite.lang.ErrorGroups.MetaStorage.WATCH_EXECUTION_ERR;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
@@ -44,14 +47,25 @@ import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.TreeSet;
+import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.LongConsumer;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.EntryEvent;
import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.dsl.Operation;
import org.apache.ignite.internal.metastorage.dsl.StatementResult;
import org.apache.ignite.internal.metastorage.dsl.Update;
@@ -62,10 +76,12 @@ import org.apache.ignite.internal.metastorage.server.If;
import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
import org.apache.ignite.internal.metastorage.server.Statement;
import org.apache.ignite.internal.metastorage.server.Value;
+import org.apache.ignite.internal.metastorage.server.Watch;
import org.apache.ignite.internal.rocksdb.ColumnFamily;
import org.apache.ignite.internal.rocksdb.RocksBiPredicate;
import org.apache.ignite.internal.rocksdb.RocksUtils;
import org.apache.ignite.internal.rocksdb.snapshot.RocksSnapshotManager;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.internal.util.IgniteUtils;
import org.apache.ignite.lang.IgniteBiTuple;
@@ -81,6 +97,7 @@ import org.rocksdb.ReadOptions;
import org.rocksdb.RocksDB;
import org.rocksdb.RocksDBException;
import org.rocksdb.RocksIterator;
+import org.rocksdb.Slice;
import org.rocksdb.WriteBatch;
import org.rocksdb.WriteOptions;
@@ -96,6 +113,8 @@ import org.rocksdb.WriteOptions;
* entry and the value is a {@code byte[]} that represents a {@code long[]} where every item is a revision of the storage.
*/
public class RocksDbKeyValueStorage implements KeyValueStorage {
+ private static final IgniteLogger LOG = Loggers.forClass(RocksDbKeyValueStorage.class);
+
/** A revision to store with system entries. */
private static final long SYSTEM_REVISION_MARKER_VALUE = 0;
@@ -127,7 +146,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
/** Thread-pool for snapshot operations execution. */
- private final ExecutorService snapshotExecutor = Executors.newFixedThreadPool(2);
+ private final ExecutorService snapshotExecutor;
/** Path to the rocksdb database. */
private final Path dbPath;
@@ -153,13 +172,55 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
/** Update counter. Will be incremented for each update of any particular entry. */
private volatile long updCntr;
+ /** Executor for processing watch events. */
+ private final ExecutorService watchExecutor;
+
+ /**
+ * Future representing the task of polling the {@code eventQueue}. Needed for cancelling the task on storage stop.
+ */
+ @Nullable
+ private volatile Future<?> watchExecutorFuture;
+
+ /** List of watches that listen to storage events. */
+ private final List<Watch> watches = new CopyOnWriteArrayList<>();
+
+ /** Status of the watch recovery process. */
+ private enum RecoveryStatus {
+ INITIAL,
+ PENDING,
+ IN_PROGRESS,
+ DONE
+ }
+
+ /**
+ * Current status of the watch recovery process. Watch recovery is needed for replaying missed updated when {@link #startWatches}
+ * is called.
+ */
+ private final AtomicReference<RecoveryStatus> recoveryStatus = new AtomicReference<>(RecoveryStatus.INITIAL);
+
+ /** Queue of update events, consumed by the {@link #watchExecutor}. */
+ private final BlockingQueue<List<Entry>> eventQueue = new LinkedBlockingQueue<>();
+
+ /**
+ * Current list of updated entries.
+ *
+ * <p>Since this list gets read and updated only on writes (under a write lock), no extra synchronisation is needed.
+ */
+ private final List<Entry> updatedEntries = new ArrayList<>();
+
+ /** Callback that gets notified after a {@link WatchEvent} has been processed by all registered watches. */
+ private volatile LongConsumer revisionCallback;
+
/**
* Constructor.
*
* @param dbPath RocksDB path.
*/
- public RocksDbKeyValueStorage(Path dbPath) {
+ public RocksDbKeyValueStorage(String nodeName, Path dbPath) {
this.dbPath = dbPath;
+
+ this.snapshotExecutor = Executors.newFixedThreadPool(2, NamedThreadFactory.create(nodeName, "metastorage-snapshot-executor", LOG));
+ this.watchExecutor = Executors.newSingleThreadExecutor(NamedThreadFactory.create(nodeName, "metastorage-watch-executor", LOG));
}
/** {@inheritDoc} */
@@ -228,6 +289,14 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
/** {@inheritDoc} */
@Override
public void close() {
+ Future<?> f = watchExecutorFuture;
+
+ if (f != null) {
+ f.cancel(true);
+ }
+
+ IgniteUtils.shutdownAndAwaitTermination(watchExecutor, 10, TimeUnit.SECONDS);
+
IgniteUtils.shutdownAndAwaitTermination(snapshotExecutor, 10, TimeUnit.SECONDS);
RocksUtils.closeAll(db, options);
@@ -242,6 +311,8 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
/** {@inheritDoc} */
@Override
public void restoreSnapshot(Path path) {
+ long currentRevision;
+
rwLock.writeLock().lock();
try {
@@ -252,7 +323,9 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
snapshotManager.restoreSnapshot(path);
- rev = bytesToLong(data.get(REVISION_KEY));
+ currentRevision = bytesToLong(data.get(REVISION_KEY));
+
+ rev = currentRevision;
updCntr = bytesToLong(data.get(UPDATE_COUNTER_KEY));
} catch (Exception e) {
@@ -260,6 +333,11 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
} finally {
rwLock.writeLock().unlock();
}
+
+ // Replay updates if startWatches() has already been called.
+ if (recoveryStatus.compareAndSet(RecoveryStatus.PENDING, RecoveryStatus.IN_PROGRESS)) {
+ replayUpdates(currentRevision);
+ }
}
/** {@inheritDoc} */
@@ -333,6 +411,14 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
rev = newRev;
updCntr = newCntr;
}
+
+ queueWatchEvent();
+ }
+
+ private static Entry entry(byte[] key, long revision, Value value) {
+ return value.tombstone()
+ ? EntryImpl.tombstone(key, revision, value.updateCounter())
+ : new EntryImpl(key, value.bytes(), revision, value.updateCounter());
}
/** {@inheritDoc} */
@@ -347,7 +433,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
long[] revs = getRevisions(key);
- final long lastRev = revs.length == 0 ? 0 : lastRevision(revs);
+ long lastRev = revs.length == 0 ? 0 : lastRevision(revs);
addDataToBatch(batch, key, value, curRev, cntr);
@@ -710,25 +796,34 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
}
@Override
- public Cursor<WatchEvent> watch(byte[] keyFrom, byte @Nullable [] keyTo, long rev) {
+ public void watchRange(byte[] keyFrom, byte @Nullable [] keyTo, long rev, WatchListener listener) {
assert keyFrom != null : "keyFrom couldn't be null.";
assert rev > 0 : "rev must be positive.";
- return new WatchCursor(this, rev, k ->
- CMP.compare(keyFrom, k) <= 0 && (keyTo == null || CMP.compare(k, keyTo) < 0)
- );
+ Predicate<byte[]> rangePredicate = keyTo == null
+ ? k -> CMP.compare(keyFrom, k) <= 0
+ : k -> CMP.compare(keyFrom, k) <= 0 && CMP.compare(keyTo, k) > 0;
+
+ watches.add(new Watch(rev, listener, rangePredicate));
+ }
+
+ @Override
+ public void watchPrefix(byte[] prefix, long rev, WatchListener listener) {
+ watchRange(prefix, incrementArray(prefix), rev, listener);
}
@Override
- public Cursor<WatchEvent> watch(byte[] key, long rev) {
+ public void watchExact(byte[] key, long rev, WatchListener listener) {
assert key != null : "key couldn't be null.";
assert rev > 0 : "rev must be positive.";
- return new WatchCursor(this, rev, k -> CMP.compare(k, key) == 0);
+ Predicate<byte[]> exactPredicate = k -> CMP.compare(k, key) == 0;
+
+ watches.add(new Watch(rev, listener, exactPredicate));
}
@Override
- public Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev) {
+ public void watchExact(Collection<byte[]> keys, long rev, WatchListener listener) {
assert keys != null && !keys.isEmpty() : "keys couldn't be null or empty: " + keys;
assert rev > 0 : "rev must be positive.";
@@ -736,7 +831,46 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
keySet.addAll(keys);
- return new WatchCursor(this, rev, keySet::contains);
+ Predicate<byte[]> inPredicate = keySet::contains;
+
+ watches.add(new Watch(rev, listener, inPredicate));
+ }
+
+ @Override
+ public void startWatches(LongConsumer revisionCallback) {
+ long currentRevision;
+
+ rwLock.readLock().lock();
+
+ try {
+ assert this.revisionCallback == null;
+
+ this.revisionCallback = revisionCallback;
+
+ currentRevision = rev;
+
+ // We update the recovery status under the read lock in order to avoid races between starting watches and applying a snapshot
+ // or concurrent writes. Replay of events can be done outside of the read lock relying on RocksDB snapshot isolation.
+ if (currentRevision == 0) {
+ // Revision can be 0 if there's no data in the storage. We set the status to PENDING expecting that it will be further
+ // updated either by applying a snapshot or by the first write to the storage.
+ recoveryStatus.set(RecoveryStatus.PENDING);
+ } else {
+ // If revision is not 0, we need to replay updates that match the existing data.
+ recoveryStatus.set(RecoveryStatus.IN_PROGRESS);
+ }
+ } finally {
+ rwLock.readLock().unlock();
+ }
+
+ if (currentRevision != 0) {
+ replayUpdates(currentRevision);
+ }
+ }
+
+ @Override
+ public void removeWatch(WatchListener listener) {
+ watches.removeIf(watch -> watch.listener() == listener);
}
@Override
@@ -815,7 +949,6 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
* @param rev Target revision.
* @return Collection of entries.
*/
- @NotNull
private Collection<Entry> doGetAll(Collection<byte[]> keys, long rev) {
assert keys != null : "keys list can't be null.";
assert !keys.isEmpty() : "keys list can't be empty.";
@@ -843,8 +976,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
* @param revUpperBound Target upper bound of revision.
* @return Value.
*/
- @NotNull
- Entry doGet(byte[] key, long revUpperBound) {
+ private Entry doGet(byte[] key, long revUpperBound) {
assert revUpperBound >= LATEST_REV : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
long[] revs;
@@ -878,7 +1010,7 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
* Get a list of the revisions of the entry corresponding to the key.
*
* @param key Key.
- * @return List of the revisions.
+ * @return Array of revisions.
* @throws RocksDBException If failed to perform {@link RocksDB#get(ColumnFamilyHandle, byte[])}.
*/
private long[] getRevisions(byte[] key) throws RocksDBException {
@@ -918,7 +1050,6 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
* @param revision Target revision.
* @return Entry.
*/
- @NotNull
Entry doGetValue(byte[] key, long revision) {
if (revision == 0) {
return EntryImpl.empty(key);
@@ -961,6 +1092,8 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
byte[] rocksValue = valueToBytes(value, cntr);
data.put(batch, rocksKey, rocksValue);
+
+ updatedEntries.add(entry(key, curRev, new Value(value, cntr)));
}
/**
@@ -1039,16 +1172,6 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
}
}
- /**
- * Creates a new iterator over the {@link StorageColumnFamilyType#DATA} column family.
- *
- * @param options Read options.
- * @return Iterator.
- */
- public RocksIterator newDataIterator(ReadOptions options) {
- return data.newIterator(options);
- }
-
/**
* Gets last revision from the list.
*
@@ -1060,17 +1183,166 @@ public class RocksDbKeyValueStorage implements KeyValueStorage {
}
/**
- * Returns database lock.
+ * Adds modified entries to the watch event queue.
*/
- ReadWriteLock lock() {
- return rwLock;
+ private void queueWatchEvent() {
+ if (updatedEntries.isEmpty()) {
+ return;
+ }
+
+ switch (recoveryStatus.get()) {
+ case INITIAL:
+ // Watches haven't been enabled yet, no need to queue any events, they will be replayed upon recovery.
+ updatedEntries.clear();
+
+ break;
+
+ case PENDING:
+ // Watches have been enabled, but no event replay happened because there was no data in the
+ // storage. Since we are adding some data here, we need to start the watch processor thread.
+ recoveryStatus.set(RecoveryStatus.IN_PROGRESS);
+
+ startWatchExecutor();
+
+ publishWatchEvent();
+
+ break;
+
+ default:
+ publishWatchEvent();
+
+ break;
+ }
+ }
+
+ private void publishWatchEvent() {
+ List<Entry> updatedEntriesCopy = List.copyOf(updatedEntries);
+
+ updatedEntries.clear();
+
+ try {
+ eventQueue.put(updatedEntriesCopy);
+ } catch (InterruptedException e) {
+ throw new MetaStorageException(WATCH_EXECUTION_ERR, "Interrupted when publishing a watch event", e);
+ }
}
/**
- * Returns database.
+ * Notifies registered watches about an update event.
*/
- RocksDB db() {
- return db;
+ private void notifyWatches(List<Entry> updatedEntries) {
+ // Revision must be the same for all entries.
+ long newRevision = updatedEntries.get(0).revision();
+
+ for (Watch watch : watches) {
+ var entryEvents = new ArrayList<EntryEvent>();
+
+ for (Entry newEntry : updatedEntries) {
+ byte[] newKey = newEntry.key();
+
+ assert newEntry.revision() == newRevision;
+
+ if (watch.matches(newKey, newRevision)) {
+ Entry oldEntry = get(newKey, newRevision - 1);
+
+ entryEvents.add(new EntryEvent(oldEntry, newEntry));
+ }
+ }
+
+ if (!entryEvents.isEmpty()) {
+ var event = new WatchEvent(entryEvents, newRevision);
+
+ try {
+ watch.onUpdate(event);
+ } catch (Exception e) {
+ watch.onError(e);
+
+ LOG.error("Error occurred when processing a watch event {}, watch will be disabled", e, event);
+
+ watches.remove(watch);
+ }
+ }
+ }
+
+ // Watch processing for the new revision is done, notify the revision callback.
+ revisionCallback.accept(newRevision);
+ }
+
+ private void replayUpdates(long upperRevision) {
+ long minWatchRevision = watches.stream()
+ .mapToLong(Watch::targetRevision)
+ .min()
+ .orElse(-1);
+
+ if (minWatchRevision == -1 || minWatchRevision > upperRevision) {
+ return;
+ }
+
+ var updatedEntries = new ArrayList<Entry>();
+
+ try (
+ var upperBound = new Slice(longToBytes(upperRevision + 1));
+ var options = new ReadOptions().setIterateUpperBound(upperBound);
+ RocksIterator it = data.newIterator(options)
+ ) {
+ it.seek(longToBytes(minWatchRevision));
+
+ long lastSeenRevision = minWatchRevision;
+
+ for (; it.isValid(); it.next()) {
+ byte[] rocksKey = it.key();
+ byte[] rocksValue = it.value();
+
+ long revision = revisionFromRocksKey(rocksKey);
+
+ if (revision != lastSeenRevision) {
+ if (!updatedEntries.isEmpty()) {
+ var updatedEntriesCopy = List.copyOf(updatedEntries);
+
+ watchExecutor.execute(() -> notifyWatches(updatedEntriesCopy));
+
+ updatedEntries.clear();
+ }
+
+ lastSeenRevision = revision;
+ }
+
+ updatedEntries.add(entry(rocksKeyToBytes(rocksKey), revision, bytesToValue(rocksValue)));
+ }
+
+ RocksUtils.checkIterator(it);
+
+ // Notify about the events left after finishing the cycle above.
+ if (!updatedEntries.isEmpty()) {
+ watchExecutor.execute(() -> notifyWatches(updatedEntries));
+ }
+ }
+
+ // Events replay is finished, now we can start processing more recent events from the event queue.
+ startWatchExecutor();
+ }
+
+ private void startWatchExecutor() {
+ if (recoveryStatus.compareAndSet(RecoveryStatus.IN_PROGRESS, RecoveryStatus.DONE)) {
+ watchExecutorFuture = watchExecutor.submit(() -> {
+ while (!Thread.currentThread().isInterrupted()) {
+ try {
+ notifyWatches(eventQueue.take());
+ } catch (InterruptedException e) {
+ LOG.info("Watch Executor interrupted, watches stopped");
+
+ return;
+ }
+ }
+ });
+ }
+ }
+
+ /**
+ * Returns database lock.
+ */
+ ReadWriteLock lock() {
+ return rwLock;
}
@TestOnly
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java
deleted file mode 100644
index c63f628715..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/persistence/WatchCursor.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * 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.metastorage.server.persistence;
-
-import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.bytesToValue;
-import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.longToBytes;
-import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.revisionFromRocksKey;
-import static org.apache.ignite.internal.metastorage.server.persistence.RocksStorageUtils.rocksKeyToBytes;
-import static org.apache.ignite.lang.ErrorGroups.MetaStorage.WATCH_EXECUTION_ERR;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.function.Predicate;
-import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
-import org.apache.ignite.internal.metastorage.impl.EntryImpl;
-import org.apache.ignite.internal.metastorage.server.Value;
-import org.apache.ignite.internal.rocksdb.RocksUtils;
-import org.apache.ignite.internal.util.Cursor;
-import org.rocksdb.ReadOptions;
-import org.rocksdb.RocksDBException;
-import org.rocksdb.RocksIterator;
-
-/**
- * Subscription on updates of entries corresponding to the given keys range (where the upper bound is unlimited) and starting from the given
- * revision number.
- */
-class WatchCursor implements Cursor<WatchEvent> {
- /** Storage. */
- private final RocksDbKeyValueStorage storage;
-
- /** Key predicate. */
- private final Predicate<byte[]> predicate;
-
- /** Options for {@link #nativeIterator}. */
- private final ReadOptions options = new ReadOptions();
-
- /** RocksDB iterator. */
- private final RocksIterator nativeIterator;
-
- /** Current revision. */
- private long currentRevision;
-
- /** Current value of the inner iterator's hasNext that is being reset to {@code false} after next is called. */
- private boolean currentHasNext = false;
-
- /**
- * Constructor.
- *
- * @param storage Storage.
- * @param rev Starting revision.
- * @param predicate Key predicate.
- */
- WatchCursor(RocksDbKeyValueStorage storage, long rev, Predicate<byte[]> predicate) {
- this.storage = storage;
- this.predicate = predicate;
-
- this.currentRevision = rev;
-
- this.nativeIterator = storage.newDataIterator(options);
- this.nativeIterator.seek(longToBytes(rev));
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean hasNext() {
- storage.lock().readLock().lock();
-
- try {
- if (currentHasNext) {
- return true;
- }
-
- if (!nativeIterator.isValid()) {
- nativeIterator.refresh();
-
- nativeIterator.seek(longToBytes(currentRevision));
- }
-
- // Check all keys to see if any one of them match the predicate.
- currentHasNext = RocksUtils.find(nativeIterator, (rocksKey, value) -> {
- byte[] key = rocksKeyToBytes(rocksKey);
-
- if (predicate.test(key)) {
- // We may have jumped to the next revision if there were no matching keys in previous.
- currentRevision = revisionFromRocksKey(rocksKey);
-
- return true;
- }
-
- return false;
- });
-
- return currentHasNext;
- } catch (RocksDBException e) {
- throw new MetaStorageException(WATCH_EXECUTION_ERR, e);
- } finally {
- storage.lock().readLock().unlock();
- }
- }
-
- /** {@inheritDoc} */
- @Override
- public WatchEvent next() {
- storage.lock().readLock().lock();
-
- try {
- if (!hasNext()) {
- throw new NoSuchElementException();
- }
-
- List<EntryEvent> evts = new ArrayList<>();
-
- long lastSeenRevision = currentRevision;
-
- // Iterate over the keys of the current revision and get all matching entries.
- for (; nativeIterator.isValid(); nativeIterator.next()) {
- byte[] rocksKey = nativeIterator.key();
- byte[] rocksValue = nativeIterator.value();
-
- long revision = revisionFromRocksKey(rocksKey);
-
- lastSeenRevision = revision;
-
- if (revision > currentRevision) {
- // There are no more keys for the current revision
- break;
- }
-
- byte[] key = rocksKeyToBytes(rocksKey);
-
- if (predicate.test(key)) {
- Value val = bytesToValue(rocksValue);
-
- Entry newEntry;
-
- if (val.tombstone()) {
- newEntry = EntryImpl.tombstone(key, revision, val.updateCounter());
- } else {
- newEntry = new EntryImpl(key, val.bytes(), revision, val.updateCounter());
- }
-
- Entry oldEntry = storage.doGet(key, revision - 1);
-
- evts.add(new EntryEvent(oldEntry, newEntry));
- }
- }
-
- currentHasNext = false;
-
- // Go to the next revision
- currentRevision = lastSeenRevision > currentRevision ? lastSeenRevision : currentRevision + 1;
-
- RocksUtils.checkIterator(nativeIterator);
-
- return new WatchEvent(evts);
- } finally {
- storage.lock().readLock().unlock();
- }
- }
-
- /** {@inheritDoc} */
- @Override
- public void close() {
- RocksUtils.closeAll(options, nativeIterator);
- }
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageLearnerListener.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageLearnerListener.java
new file mode 100644
index 0000000000..7902e66781
--- /dev/null
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageLearnerListener.java
@@ -0,0 +1,73 @@
+/*
+ * 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.metastorage.server.raft;
+
+import java.nio.file.Path;
+import java.util.Iterator;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.raft.ReadCommand;
+import org.apache.ignite.internal.raft.WriteCommand;
+import org.apache.ignite.internal.raft.service.CommandClosure;
+import org.apache.ignite.internal.raft.service.RaftGroupListener;
+
+/**
+ * Meta Storage Raft group listener for learner nodes. These nodes ignore read and cursor-related commands.
+ */
+public class MetaStorageLearnerListener implements RaftGroupListener {
+ private final KeyValueStorage storage;
+
+ private final MetaStorageWriteHandler writeHandler;
+
+ public MetaStorageLearnerListener(KeyValueStorage storage) {
+ this.storage = storage;
+ this.writeHandler = new MetaStorageWriteHandler(storage);
+ }
+
+ @Override
+ public void onRead(Iterator<CommandClosure<ReadCommand>> iterator) {
+ throw new UnsupportedOperationException("Reads should not be sent to learners");
+ }
+
+ @Override
+ public void onWrite(Iterator<CommandClosure<WriteCommand>> iter) {
+ while (iter.hasNext()) {
+ CommandClosure<WriteCommand> clo = iter.next();
+
+ if (!writeHandler.handleWriteCommand(clo)) {
+ // Ignore all commands that are not handled by the writeHandler.
+ clo.result(null);
+ }
+ }
+ }
+
+ @Override
+ public void onSnapshotSave(Path path, Consumer<Throwable> doneClo) {
+ storage.snapshot(path).whenComplete((unused, throwable) -> doneClo.accept(throwable));
+ }
+
+ @Override
+ public boolean onSnapshotLoad(Path path) {
+ storage.restoreSnapshot(path);
+ return true;
+ }
+
+ @Override
+ public void onShutdown() {
+ }
+}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
index b894fde792..ded27cc751 100644
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageListener.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.metastorage.server.raft;
import static java.util.Objects.requireNonNull;
-import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CLOSING_STORAGE_ERR;
import static org.apache.ignite.lang.ErrorGroups.MetaStorage.CURSOR_CLOSING_ERR;
import java.nio.file.Path;
@@ -31,54 +30,18 @@ import java.util.NoSuchElementException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
import org.apache.ignite.internal.metastorage.command.GetAllCommand;
-import org.apache.ignite.internal.metastorage.command.GetAndPutAllCommand;
-import org.apache.ignite.internal.metastorage.command.GetAndPutCommand;
-import org.apache.ignite.internal.metastorage.command.GetAndRemoveAllCommand;
-import org.apache.ignite.internal.metastorage.command.GetAndRemoveCommand;
import org.apache.ignite.internal.metastorage.command.GetCommand;
-import org.apache.ignite.internal.metastorage.command.InvokeCommand;
-import org.apache.ignite.internal.metastorage.command.MetaStorageCommandsFactory;
-import org.apache.ignite.internal.metastorage.command.MultiInvokeCommand;
import org.apache.ignite.internal.metastorage.command.MultipleEntryResponse;
import org.apache.ignite.internal.metastorage.command.PrefixCommand;
-import org.apache.ignite.internal.metastorage.command.PutAllCommand;
-import org.apache.ignite.internal.metastorage.command.PutCommand;
import org.apache.ignite.internal.metastorage.command.RangeCommand;
-import org.apache.ignite.internal.metastorage.command.RemoveAllCommand;
-import org.apache.ignite.internal.metastorage.command.RemoveCommand;
import org.apache.ignite.internal.metastorage.command.SingleEntryResponse;
-import org.apache.ignite.internal.metastorage.command.WatchExactKeysCommand;
-import org.apache.ignite.internal.metastorage.command.WatchRangeKeysCommand;
import org.apache.ignite.internal.metastorage.command.cursor.CursorCloseCommand;
import org.apache.ignite.internal.metastorage.command.cursor.CursorHasNextCommand;
import org.apache.ignite.internal.metastorage.command.cursor.CursorNextCommand;
import org.apache.ignite.internal.metastorage.command.cursor.CursorsCloseCommand;
-import org.apache.ignite.internal.metastorage.command.info.CompoundConditionInfo;
-import org.apache.ignite.internal.metastorage.command.info.ConditionInfo;
-import org.apache.ignite.internal.metastorage.command.info.IfInfo;
-import org.apache.ignite.internal.metastorage.command.info.OperationInfo;
-import org.apache.ignite.internal.metastorage.command.info.SimpleConditionInfo;
-import org.apache.ignite.internal.metastorage.command.info.StatementInfo;
-import org.apache.ignite.internal.metastorage.command.info.UpdateInfo;
-import org.apache.ignite.internal.metastorage.dsl.CompoundConditionType;
-import org.apache.ignite.internal.metastorage.dsl.ConditionType;
-import org.apache.ignite.internal.metastorage.dsl.Operation;
-import org.apache.ignite.internal.metastorage.dsl.StatementResult;
-import org.apache.ignite.internal.metastorage.dsl.Update;
import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
-import org.apache.ignite.internal.metastorage.server.AndCondition;
-import org.apache.ignite.internal.metastorage.server.Condition;
-import org.apache.ignite.internal.metastorage.server.ExistenceCondition;
-import org.apache.ignite.internal.metastorage.server.If;
import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
-import org.apache.ignite.internal.metastorage.server.OrCondition;
-import org.apache.ignite.internal.metastorage.server.RevisionCondition;
-import org.apache.ignite.internal.metastorage.server.Statement;
-import org.apache.ignite.internal.metastorage.server.TombstoneCondition;
-import org.apache.ignite.internal.metastorage.server.ValueCondition;
import org.apache.ignite.internal.raft.ReadCommand;
import org.apache.ignite.internal.raft.WriteCommand;
import org.apache.ignite.internal.raft.service.CommandClosure;
@@ -86,14 +49,13 @@ import org.apache.ignite.internal.raft.service.RaftGroupListener;
import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.lang.IgniteUuid;
import org.jetbrains.annotations.Nullable;
-import org.jetbrains.annotations.TestOnly;
/**
* Meta storage listener.
* TODO: IGNITE-14693 Implement Meta storage exception handling logic.
*/
public class MetaStorageListener implements RaftGroupListener {
- private final MetaStorageCommandsFactory commandsFactory = new MetaStorageCommandsFactory();
+ private final MetaStorageWriteHandler writeHandler;
/** Storage. */
private final KeyValueStorage storage;
@@ -108,6 +70,7 @@ public class MetaStorageListener implements RaftGroupListener {
*/
public MetaStorageListener(KeyValueStorage storage) {
this.storage = storage;
+ this.writeHandler = new MetaStorageWriteHandler(storage);
this.cursors = new ConcurrentHashMap<>();
}
@@ -130,9 +93,7 @@ public class MetaStorageListener implements RaftGroupListener {
e = storage.get(getCmd.key());
}
- SingleEntryResponse resp = new SingleEntryResponse(
- e.key(), e.value(), e.revision(), e.updateCounter()
- );
+ SingleEntryResponse resp = new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter());
clo.result(resp);
} else if (command instanceof GetAllCommand) {
@@ -171,85 +132,13 @@ public class MetaStorageListener implements RaftGroupListener {
while (iter.hasNext()) {
CommandClosure<WriteCommand> clo = iter.next();
- WriteCommand command = clo.command();
-
- if (command instanceof PutCommand) {
- PutCommand putCmd = (PutCommand) command;
-
- storage.put(putCmd.key(), putCmd.value());
-
- clo.result(null);
- } else if (command instanceof GetAndPutCommand) {
- GetAndPutCommand getAndPutCmd = (GetAndPutCommand) command;
-
- Entry e = storage.getAndPut(getAndPutCmd.key(), getAndPutCmd.value());
-
- clo.result(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
- } else if (command instanceof PutAllCommand) {
- PutAllCommand putAllCmd = (PutAllCommand) command;
-
- storage.putAll(putAllCmd.keys(), putAllCmd.values());
-
- clo.result(null);
- } else if (command instanceof GetAndPutAllCommand) {
- GetAndPutAllCommand getAndPutAllCmd = (GetAndPutAllCommand) command;
-
- Collection<Entry> entries = storage.getAndPutAll(getAndPutAllCmd.keys(), getAndPutAllCmd.values());
-
- List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
-
- for (Entry e : entries) {
- resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
- }
-
- clo.result(new MultipleEntryResponse(resp));
- } else if (command instanceof RemoveCommand) {
- RemoveCommand rmvCmd = (RemoveCommand) command;
-
- storage.remove(rmvCmd.key());
-
- clo.result(null);
- } else if (command instanceof GetAndRemoveCommand) {
- GetAndRemoveCommand getAndRmvCmd = (GetAndRemoveCommand) command;
-
- Entry e = storage.getAndRemove(getAndRmvCmd.key());
-
- clo.result(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
- } else if (command instanceof RemoveAllCommand) {
- RemoveAllCommand rmvAllCmd = (RemoveAllCommand) command;
-
- storage.removeAll(rmvAllCmd.keys());
-
- clo.result(null);
- } else if (command instanceof GetAndRemoveAllCommand) {
- GetAndRemoveAllCommand getAndRmvAllCmd = (GetAndRemoveAllCommand) command;
-
- Collection<Entry> entries = storage.getAndRemoveAll(getAndRmvAllCmd.keys());
-
- List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
-
- for (Entry e : entries) {
- resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
- }
-
- clo.result(new MultipleEntryResponse(resp));
- } else if (command instanceof InvokeCommand) {
- InvokeCommand cmd = (InvokeCommand) command;
-
- boolean res = storage.invoke(
- toCondition(cmd.condition()),
- toOperations(cmd.success()),
- toOperations(cmd.failure())
- );
-
- clo.result(res);
- } else if (command instanceof MultiInvokeCommand) {
- MultiInvokeCommand cmd = (MultiInvokeCommand) command;
+ if (writeHandler.handleWriteCommand(clo)) {
+ continue;
+ }
- StatementResult res = storage.invoke(toIf(cmd.iif()));
+ WriteCommand command = clo.command();
- clo.result(commandsFactory.statementResultInfo().result(res.bytes()).build());
- } else if (command instanceof RangeCommand) {
+ if (command instanceof RangeCommand) {
RangeCommand rangeCmd = (RangeCommand) command;
IgniteUuid cursorId = rangeCmd.cursorId();
@@ -258,7 +147,7 @@ public class MetaStorageListener implements RaftGroupListener {
? storage.range(rangeCmd.keyFrom(), rangeCmd.keyTo(), rangeCmd.revUpperBound(), rangeCmd.includeTombstones())
: storage.range(rangeCmd.keyFrom(), rangeCmd.keyTo(), rangeCmd.includeTombstones());
- var cursorMeta = new CursorMeta(cursor, CursorType.RANGE, rangeCmd.requesterNodeId(), rangeCmd.batchSize());
+ var cursorMeta = new CursorMeta(cursor, rangeCmd.requesterNodeId(), rangeCmd.batchSize());
cursors.put(cursorId, cursorMeta);
@@ -272,7 +161,7 @@ public class MetaStorageListener implements RaftGroupListener {
? storage.prefix(prefixCmd.prefix(), prefixCmd.includeTombstones())
: storage.prefix(prefixCmd.prefix(), prefixCmd.revUpperBound(), prefixCmd.includeTombstones());
- var cursorMeta = new CursorMeta(cursor, CursorType.RANGE, prefixCmd.requesterNodeId(), prefixCmd.batchSize());
+ var cursorMeta = new CursorMeta(cursor, prefixCmd.requesterNodeId(), prefixCmd.batchSize());
cursors.put(cursorId, cursorMeta);
@@ -289,39 +178,19 @@ public class MetaStorageListener implements RaftGroupListener {
}
try {
- if (cursorDesc.type() == CursorType.RANGE) {
- int batchSize = requireNonNull(cursorDesc.batchSize());
-
- List<SingleEntryResponse> resp = new ArrayList<>(batchSize);
-
- for (int i = 0; i < batchSize; i++) {
- if (cursorDesc.cursor().hasNext()) {
- Entry e = (Entry) cursorDesc.cursor().next();
-
- resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
- } else {
- break;
- }
- }
-
- clo.result(new MultipleEntryResponse(resp));
- } else if (cursorDesc.type() == CursorType.WATCH) {
- WatchEvent evt = (WatchEvent) cursorDesc.cursor().next();
+ int batchSize = requireNonNull(cursorDesc.batchSize());
- List<SingleEntryResponse> resp = new ArrayList<>(evt.entryEvents().size() * 2);
+ var resp = new ArrayList<SingleEntryResponse>(batchSize);
- for (EntryEvent e : evt.entryEvents()) {
- Entry o = e.oldEntry();
+ Cursor<Entry> cursor = cursorDesc.cursor();
- Entry n = e.newEntry();
+ for (int i = 0; i < batchSize && cursor.hasNext(); i++) {
+ Entry e = cursor.next();
- resp.add(new SingleEntryResponse(o.key(), o.value(), o.revision(), o.updateCounter()));
-
- resp.add(new SingleEntryResponse(n.key(), n.value(), n.revision(), n.updateCounter()));
- }
-
- clo.result(new MultipleEntryResponse(resp));
+ resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
}
+
+ clo.result(new MultipleEntryResponse(resp));
} catch (NoSuchElementException e) {
clo.result(e);
}
@@ -343,43 +212,6 @@ public class MetaStorageListener implements RaftGroupListener {
}
clo.result(null);
- } else if (command instanceof WatchRangeKeysCommand) {
- WatchRangeKeysCommand watchCmd = (WatchRangeKeysCommand) command;
-
- IgniteUuid cursorId = watchCmd.cursorId();
-
- Cursor<WatchEvent> cursor =
- storage.watch(watchCmd.keyFrom(), watchCmd.keyTo(), watchCmd.revision());
-
- cursors.put(
- cursorId,
- new CursorMeta(
- cursor,
- CursorType.WATCH,
- watchCmd.requesterNodeId(),
- null
- )
- );
-
- clo.result(cursorId);
- } else if (command instanceof WatchExactKeysCommand) {
- WatchExactKeysCommand watchCmd = (WatchExactKeysCommand) command;
-
- IgniteUuid cursorId = watchCmd.cursorId();
-
- Cursor<WatchEvent> cursor = storage.watch(watchCmd.keys(), watchCmd.revision());
-
- cursors.put(
- cursorId,
- new CursorMeta(
- cursor,
- CursorType.WATCH,
- watchCmd.requesterNodeId(),
- null
- )
- );
-
- clo.result(cursorId);
} else if (command instanceof CursorsCloseCommand) {
CursorsCloseCommand cursorsCloseCmd = (CursorsCloseCommand) command;
@@ -425,112 +257,14 @@ public class MetaStorageListener implements RaftGroupListener {
/** {@inheritDoc} */
@Override
public void onShutdown() {
- try {
- storage.close();
- } catch (RuntimeException e) {
- throw new MetaStorageException(CLOSING_STORAGE_ERR, "Failed to close storage: " + e.getMessage(), e);
- }
- }
-
- /**
- * Returns {@link KeyValueStorage} that is backing this listener.
- */
- @TestOnly
- public KeyValueStorage getStorage() {
- return storage;
- }
-
- private static If toIf(IfInfo iif) {
- return new If(toCondition(iif.cond()), toConditionBranch(iif.andThen()), toConditionBranch(iif.orElse()));
- }
-
- private static Update toUpdate(UpdateInfo updateInfo) {
- return new Update(toOperations(new ArrayList<>(updateInfo.operations())), new StatementResult(updateInfo.result().result()));
- }
-
- private static Statement toConditionBranch(StatementInfo statementInfo) {
- if (statementInfo.isTerminal()) {
- return new Statement(toUpdate(statementInfo.update()));
- } else {
- return new Statement(toIf(statementInfo.iif()));
- }
- }
-
- private static Condition toCondition(ConditionInfo info) {
- if (info instanceof SimpleConditionInfo) {
- SimpleConditionInfo inf = (SimpleConditionInfo) info;
- byte[] key = inf.key();
-
- ConditionType type = inf.type();
-
- if (type == ConditionType.KEY_EXISTS) {
- return new ExistenceCondition(ExistenceCondition.Type.EXISTS, key);
- } else if (type == ConditionType.KEY_NOT_EXISTS) {
- return new ExistenceCondition(ExistenceCondition.Type.NOT_EXISTS, key);
- } else if (type == ConditionType.TOMBSTONE) {
- return new TombstoneCondition(key);
- } else if (type == ConditionType.VAL_EQUAL) {
- return new ValueCondition(ValueCondition.Type.EQUAL, key, inf.value());
- } else if (type == ConditionType.VAL_NOT_EQUAL) {
- return new ValueCondition(ValueCondition.Type.NOT_EQUAL, key, inf.value());
- } else if (type == ConditionType.VAL_GREATER) {
- return new ValueCondition(ValueCondition.Type.GREATER, key, inf.value());
- } else if (type == ConditionType.VAL_GREATER_OR_EQUAL) {
- return new ValueCondition(ValueCondition.Type.GREATER_OR_EQUAL, key, inf.value());
- } else if (type == ConditionType.VAL_LESS) {
- return new ValueCondition(ValueCondition.Type.LESS, key, inf.value());
- } else if (type == ConditionType.VAL_LESS_OR_EQUAL) {
- return new ValueCondition(ValueCondition.Type.LESS_OR_EQUAL, key, inf.value());
- } else if (type == ConditionType.REV_EQUAL) {
- return new RevisionCondition(RevisionCondition.Type.EQUAL, key, inf.revision());
- } else if (type == ConditionType.REV_NOT_EQUAL) {
- return new RevisionCondition(RevisionCondition.Type.NOT_EQUAL, key, inf.revision());
- } else if (type == ConditionType.REV_GREATER) {
- return new RevisionCondition(RevisionCondition.Type.GREATER, key, inf.revision());
- } else if (type == ConditionType.REV_GREATER_OR_EQUAL) {
- return new RevisionCondition(RevisionCondition.Type.GREATER_OR_EQUAL, key, inf.revision());
- } else if (type == ConditionType.REV_LESS) {
- return new RevisionCondition(RevisionCondition.Type.LESS, key, inf.revision());
- } else if (type == ConditionType.REV_LESS_OR_EQUAL) {
- return new RevisionCondition(RevisionCondition.Type.LESS_OR_EQUAL, key, inf.revision());
- } else {
- throw new IllegalArgumentException("Unknown condition type: " + type);
- }
- } else if (info instanceof CompoundConditionInfo) {
- CompoundConditionInfo inf = (CompoundConditionInfo) info;
-
- if (inf.type() == CompoundConditionType.AND) {
- return new AndCondition(toCondition(inf.leftConditionInfo()), toCondition(inf.rightConditionInfo()));
-
- } else if (inf.type() == CompoundConditionType.OR) {
- return new OrCondition(toCondition(inf.leftConditionInfo()), toCondition(inf.rightConditionInfo()));
- } else {
- throw new IllegalArgumentException("Unknown compound condition " + inf.type());
- }
- } else {
- throw new IllegalArgumentException("Unknown condition info type " + info);
- }
- }
-
- private static List<Operation> toOperations(List<OperationInfo> infos) {
- List<Operation> ops = new ArrayList<>(infos.size());
-
- for (OperationInfo info : infos) {
- ops.add(new Operation(info.type(), info.key(), info.value()));
- }
-
- return ops;
}
/**
* Cursor meta information: origin node id and type.
*/
- private class CursorMeta {
+ private static class CursorMeta {
/** Cursor. */
- private final Cursor<?> cursor;
-
- /** Cursor type. */
- private final CursorType type;
+ private final Cursor<Entry> cursor;
/** Id of the node that creates cursor. */
private final String requesterNodeId;
@@ -542,17 +276,11 @@ public class MetaStorageListener implements RaftGroupListener {
* The constructor.
*
* @param cursor Cursor.
- * @param type Cursor type.
* @param requesterNodeId Id of the node that creates cursor.
* @param batchSize Batch size.
*/
- CursorMeta(Cursor<?> cursor,
- CursorType type,
- String requesterNodeId,
- @Nullable Integer batchSize
- ) {
+ CursorMeta(Cursor<Entry> cursor, String requesterNodeId, @Nullable Integer batchSize) {
this.cursor = cursor;
- this.type = type;
this.requesterNodeId = requesterNodeId;
this.batchSize = batchSize;
}
@@ -560,36 +288,22 @@ public class MetaStorageListener implements RaftGroupListener {
/**
* Returns cursor.
*/
- public Cursor<?> cursor() {
+ Cursor<Entry> cursor() {
return cursor;
}
- /**
- * Returns cursor type.
- */
- public CursorType type() {
- return type;
- }
-
/**
* Returns id of the node that creates cursor.
*/
- public String requesterNodeId() {
+ String requesterNodeId() {
return requesterNodeId;
}
/**
* Returns maximum size of the batch that is sent in single response message.
*/
- public @Nullable Integer batchSize() {
+ @Nullable Integer batchSize() {
return batchSize;
}
}
-
- /** Cursor type. */
- private enum CursorType {
- RANGE,
-
- WATCH;
- }
}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageWriteHandler.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageWriteHandler.java
new file mode 100644
index 0000000000..fbcebdbdc3
--- /dev/null
+++ b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/raft/MetaStorageWriteHandler.java
@@ -0,0 +1,245 @@
+/*
+ * 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.metastorage.server.raft;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.command.GetAndPutAllCommand;
+import org.apache.ignite.internal.metastorage.command.GetAndPutCommand;
+import org.apache.ignite.internal.metastorage.command.GetAndRemoveAllCommand;
+import org.apache.ignite.internal.metastorage.command.GetAndRemoveCommand;
+import org.apache.ignite.internal.metastorage.command.InvokeCommand;
+import org.apache.ignite.internal.metastorage.command.MetaStorageCommandsFactory;
+import org.apache.ignite.internal.metastorage.command.MultiInvokeCommand;
+import org.apache.ignite.internal.metastorage.command.MultipleEntryResponse;
+import org.apache.ignite.internal.metastorage.command.PutAllCommand;
+import org.apache.ignite.internal.metastorage.command.PutCommand;
+import org.apache.ignite.internal.metastorage.command.RemoveAllCommand;
+import org.apache.ignite.internal.metastorage.command.RemoveCommand;
+import org.apache.ignite.internal.metastorage.command.SingleEntryResponse;
+import org.apache.ignite.internal.metastorage.command.info.CompoundConditionInfo;
+import org.apache.ignite.internal.metastorage.command.info.ConditionInfo;
+import org.apache.ignite.internal.metastorage.command.info.IfInfo;
+import org.apache.ignite.internal.metastorage.command.info.OperationInfo;
+import org.apache.ignite.internal.metastorage.command.info.SimpleConditionInfo;
+import org.apache.ignite.internal.metastorage.command.info.StatementInfo;
+import org.apache.ignite.internal.metastorage.command.info.UpdateInfo;
+import org.apache.ignite.internal.metastorage.dsl.CompoundConditionType;
+import org.apache.ignite.internal.metastorage.dsl.ConditionType;
+import org.apache.ignite.internal.metastorage.dsl.Operation;
+import org.apache.ignite.internal.metastorage.dsl.StatementResult;
+import org.apache.ignite.internal.metastorage.dsl.Update;
+import org.apache.ignite.internal.metastorage.server.AndCondition;
+import org.apache.ignite.internal.metastorage.server.Condition;
+import org.apache.ignite.internal.metastorage.server.ExistenceCondition;
+import org.apache.ignite.internal.metastorage.server.If;
+import org.apache.ignite.internal.metastorage.server.KeyValueStorage;
+import org.apache.ignite.internal.metastorage.server.OrCondition;
+import org.apache.ignite.internal.metastorage.server.RevisionCondition;
+import org.apache.ignite.internal.metastorage.server.Statement;
+import org.apache.ignite.internal.metastorage.server.TombstoneCondition;
+import org.apache.ignite.internal.metastorage.server.ValueCondition;
+import org.apache.ignite.internal.raft.WriteCommand;
+import org.apache.ignite.internal.raft.service.CommandClosure;
+
+/**
+ * Class containing some common logic for Meta Storage Raft group listeners.
+ */
+class MetaStorageWriteHandler {
+ private final MetaStorageCommandsFactory commandsFactory = new MetaStorageCommandsFactory();
+
+ private final KeyValueStorage storage;
+
+ MetaStorageWriteHandler(KeyValueStorage storage) {
+ this.storage = storage;
+ }
+
+ /**
+ * Tries to process a {@link WriteCommand}, returning {@code true} if the command has been successfully processed or {@code false}
+ * if the command requires external processing.
+ */
+ boolean handleWriteCommand(CommandClosure<WriteCommand> clo) {
+ WriteCommand command = clo.command();
+
+ if (command instanceof PutCommand) {
+ PutCommand putCmd = (PutCommand) command;
+
+ storage.put(putCmd.key(), putCmd.value());
+
+ clo.result(null);
+ } else if (command instanceof GetAndPutCommand) {
+ GetAndPutCommand getAndPutCmd = (GetAndPutCommand) command;
+
+ Entry e = storage.getAndPut(getAndPutCmd.key(), getAndPutCmd.value());
+
+ clo.result(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+ } else if (command instanceof PutAllCommand) {
+ PutAllCommand putAllCmd = (PutAllCommand) command;
+
+ storage.putAll(putAllCmd.keys(), putAllCmd.values());
+
+ clo.result(null);
+ } else if (command instanceof GetAndPutAllCommand) {
+ GetAndPutAllCommand getAndPutAllCmd = (GetAndPutAllCommand) command;
+
+ Collection<Entry> entries = storage.getAndPutAll(getAndPutAllCmd.keys(), getAndPutAllCmd.values());
+
+ List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
+
+ for (Entry e : entries) {
+ resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+ }
+
+ clo.result(new MultipleEntryResponse(resp));
+ } else if (command instanceof RemoveCommand) {
+ RemoveCommand rmvCmd = (RemoveCommand) command;
+
+ storage.remove(rmvCmd.key());
+
+ clo.result(null);
+ } else if (command instanceof GetAndRemoveCommand) {
+ GetAndRemoveCommand getAndRmvCmd = (GetAndRemoveCommand) command;
+
+ Entry e = storage.getAndRemove(getAndRmvCmd.key());
+
+ clo.result(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+ } else if (command instanceof RemoveAllCommand) {
+ RemoveAllCommand rmvAllCmd = (RemoveAllCommand) command;
+
+ storage.removeAll(rmvAllCmd.keys());
+
+ clo.result(null);
+ } else if (command instanceof GetAndRemoveAllCommand) {
+ GetAndRemoveAllCommand getAndRmvAllCmd = (GetAndRemoveAllCommand) command;
+
+ Collection<Entry> entries = storage.getAndRemoveAll(getAndRmvAllCmd.keys());
+
+ List<SingleEntryResponse> resp = new ArrayList<>(entries.size());
+
+ for (Entry e : entries) {
+ resp.add(new SingleEntryResponse(e.key(), e.value(), e.revision(), e.updateCounter()));
+ }
+
+ clo.result(new MultipleEntryResponse(resp));
+ } else if (command instanceof InvokeCommand) {
+ InvokeCommand cmd = (InvokeCommand) command;
+
+ boolean res = storage.invoke(
+ toCondition(cmd.condition()),
+ toOperations(cmd.success()),
+ toOperations(cmd.failure())
+ );
+
+ clo.result(res);
+ } else if (command instanceof MultiInvokeCommand) {
+ MultiInvokeCommand cmd = (MultiInvokeCommand) command;
+
+ StatementResult res = storage.invoke(toIf(cmd.iif()));
+
+ clo.result(commandsFactory.statementResultInfo().result(res.bytes()).build());
+ } else {
+ return false;
+ }
+
+ return true;
+ }
+
+ private static If toIf(IfInfo iif) {
+ return new If(toCondition(iif.cond()), toConditionBranch(iif.andThen()), toConditionBranch(iif.orElse()));
+ }
+
+ private static Update toUpdate(UpdateInfo updateInfo) {
+ return new Update(toOperations(new ArrayList<>(updateInfo.operations())), new StatementResult(updateInfo.result().result()));
+ }
+
+ private static Statement toConditionBranch(StatementInfo statementInfo) {
+ if (statementInfo.isTerminal()) {
+ return new Statement(toUpdate(statementInfo.update()));
+ } else {
+ return new Statement(toIf(statementInfo.iif()));
+ }
+ }
+
+ private static Condition toCondition(ConditionInfo info) {
+ if (info instanceof SimpleConditionInfo) {
+ SimpleConditionInfo inf = (SimpleConditionInfo) info;
+ byte[] key = inf.key();
+
+ ConditionType type = inf.type();
+
+ if (type == ConditionType.KEY_EXISTS) {
+ return new ExistenceCondition(ExistenceCondition.Type.EXISTS, key);
+ } else if (type == ConditionType.KEY_NOT_EXISTS) {
+ return new ExistenceCondition(ExistenceCondition.Type.NOT_EXISTS, key);
+ } else if (type == ConditionType.TOMBSTONE) {
+ return new TombstoneCondition(key);
+ } else if (type == ConditionType.VAL_EQUAL) {
+ return new ValueCondition(ValueCondition.Type.EQUAL, key, inf.value());
+ } else if (type == ConditionType.VAL_NOT_EQUAL) {
+ return new ValueCondition(ValueCondition.Type.NOT_EQUAL, key, inf.value());
+ } else if (type == ConditionType.VAL_GREATER) {
+ return new ValueCondition(ValueCondition.Type.GREATER, key, inf.value());
+ } else if (type == ConditionType.VAL_GREATER_OR_EQUAL) {
+ return new ValueCondition(ValueCondition.Type.GREATER_OR_EQUAL, key, inf.value());
+ } else if (type == ConditionType.VAL_LESS) {
+ return new ValueCondition(ValueCondition.Type.LESS, key, inf.value());
+ } else if (type == ConditionType.VAL_LESS_OR_EQUAL) {
+ return new ValueCondition(ValueCondition.Type.LESS_OR_EQUAL, key, inf.value());
+ } else if (type == ConditionType.REV_EQUAL) {
+ return new RevisionCondition(RevisionCondition.Type.EQUAL, key, inf.revision());
+ } else if (type == ConditionType.REV_NOT_EQUAL) {
+ return new RevisionCondition(RevisionCondition.Type.NOT_EQUAL, key, inf.revision());
+ } else if (type == ConditionType.REV_GREATER) {
+ return new RevisionCondition(RevisionCondition.Type.GREATER, key, inf.revision());
+ } else if (type == ConditionType.REV_GREATER_OR_EQUAL) {
+ return new RevisionCondition(RevisionCondition.Type.GREATER_OR_EQUAL, key, inf.revision());
+ } else if (type == ConditionType.REV_LESS) {
+ return new RevisionCondition(RevisionCondition.Type.LESS, key, inf.revision());
+ } else if (type == ConditionType.REV_LESS_OR_EQUAL) {
+ return new RevisionCondition(RevisionCondition.Type.LESS_OR_EQUAL, key, inf.revision());
+ } else {
+ throw new IllegalArgumentException("Unknown condition type: " + type);
+ }
+ } else if (info instanceof CompoundConditionInfo) {
+ CompoundConditionInfo inf = (CompoundConditionInfo) info;
+
+ if (inf.type() == CompoundConditionType.AND) {
+ return new AndCondition(toCondition(inf.leftConditionInfo()), toCondition(inf.rightConditionInfo()));
+
+ } else if (inf.type() == CompoundConditionType.OR) {
+ return new OrCondition(toCondition(inf.leftConditionInfo()), toCondition(inf.rightConditionInfo()));
+ } else {
+ throw new IllegalArgumentException("Unknown compound condition " + inf.type());
+ }
+ } else {
+ throw new IllegalArgumentException("Unknown condition info type " + info);
+ }
+ }
+
+ private static List<Operation> toOperations(List<OperationInfo> infos) {
+ List<Operation> ops = new ArrayList<>(infos.size());
+
+ for (OperationInfo info : infos) {
+ ops.add(new Operation(info.type(), info.key(), info.value()));
+ }
+
+ return ops;
+ }
+}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java
deleted file mode 100644
index d00ccb10f0..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/AggregatedWatch.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.metastorage.watch;
-
-import org.apache.ignite.internal.metastorage.WatchListener;
-
-/**
- * Watch implementation with associated revision. Instance of this watch produced by {@link WatchAggregator}.
- */
-public class AggregatedWatch {
- /** Watch key criterion. */
- private final KeyCriterion keyCriterion;
-
- /** Aggregated watch listener. */
- private final WatchListener lsnr;
-
- /** Watch revision. */
- private final long revision;
-
- /**
- * Creates the instance of aggregated watch.
- *
- * @param keyCriterion Aggregated key criterion.
- * @param revision Aggregated revision.
- * @param lsnr Aggregated listener.
- */
- public AggregatedWatch(KeyCriterion keyCriterion, long revision, WatchListener lsnr) {
- this.keyCriterion = keyCriterion;
- this.revision = revision;
- this.lsnr = lsnr;
- }
-
- /**
- * Returns key criterion.
- */
- public KeyCriterion keyCriterion() {
- return keyCriterion;
- }
-
- /**
- * Returns watch listener.
- */
- public WatchListener listener() {
- return lsnr;
- }
-
- /**
- * Returns revision.
- */
- public long revision() {
- return revision;
- }
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java
deleted file mode 100644
index df58d102f0..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/KeyCriterion.java
+++ /dev/null
@@ -1,369 +0,0 @@
-/*
- * 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.metastorage.watch;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Objects;
-import java.util.Set;
-import org.apache.ignite.lang.ByteArray;
-
-/**
- * Filter for listen key's changes on metastore.
- */
-public abstract class KeyCriterion {
- /**
- * Checks if this key criterion contains the key.
- *
- * @return {@code true} if criterion contains the key, {@code false} otherwise.
- */
- public abstract boolean contains(ByteArray key);
-
- /**
- * Union current key criterion with another one.
- *
- * @param keyCriterion Criterion to calculate the union with.
- * @param swapTry Set to {@code true} if current criterion can't calculate the union and trying to calculate it from the opposite
- * side.
- * @return Result key criterion.
- */
- protected abstract KeyCriterion union(KeyCriterion keyCriterion, boolean swapTry);
-
- /**
- * Union two key criteria and produce the new one.
- *
- * <p>Rules for the union of different types of criteria:
- * <pre>
- * exact + exact = collection|exact
- * collection + exact = collection
- * collection + collection = collection
- * range + exact = range
- * range + collection = range
- * range + range = range
- * </pre>
- *
- * @param keyCriterion Criterion to calculate the union with.
- * @return Result of criteria union.
- */
- public KeyCriterion union(KeyCriterion keyCriterion) {
- return union(keyCriterion, false);
- }
-
- /**
- * Creates a common exception that indicates the given key criterions cannot be combined.
- *
- * @param keyCriterion1 Criterion.
- * @param keyCriterion2 Criterion.
- * @return Common exception that indicates the given key criterions cannot be combined.
- */
- private static RuntimeException unsupportedUnionException(KeyCriterion keyCriterion1, KeyCriterion keyCriterion2) {
- return new UnsupportedOperationException("Can't calculate the union between " + keyCriterion1.getClass()
- + "and " + keyCriterion2.getClass() + " key criteria.");
- }
-
- /**
- * Criterion which contains the range of keys.
- */
- public static class RangeCriterion extends KeyCriterion {
- /** Start of the range. */
- private final ByteArray from;
-
- /** End of the range (exclusive). */
- private final ByteArray to;
-
- /**
- * Creates the instance of range criterion.
- *
- * @param from Start of the range.
- * @param to End of the range (exclusive).
- */
- public RangeCriterion(ByteArray from, ByteArray to) {
- this.from = from;
- this.to = to;
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean contains(ByteArray key) {
- return key.compareTo(from) >= 0 && key.compareTo(to) < 0;
- }
-
- /**
- * Calculates range representation for prefix criterion as {@code (prefixKey, nextKey(prefixKey)) }.
- *
- * @param prefixKey Prefix criterion.
- * @return Calculated range
- */
- public static RangeCriterion fromPrefixKey(ByteArray prefixKey) {
- return new RangeCriterion(prefixKey, nextKey(prefixKey));
- }
-
- /** {@inheritDoc} */
- @Override
- protected KeyCriterion union(KeyCriterion keyCriterion, boolean swapTry) {
- ByteArray from;
- ByteArray to;
-
- if (keyCriterion instanceof ExactCriterion) {
- from = ((ExactCriterion) keyCriterion).key;
- to = nextKey(from);
- } else if (keyCriterion instanceof CollectionCriterion) {
- from = Collections.min(((CollectionCriterion) keyCriterion).keys);
- to = nextKey(Collections.max(((CollectionCriterion) keyCriterion).keys));
- } else if (keyCriterion instanceof RangeCriterion) {
- from = ((RangeCriterion) keyCriterion).from;
- to = ((RangeCriterion) keyCriterion).to;
- } else if (!swapTry) {
- return keyCriterion.union(this, true);
- } else {
- throw KeyCriterion.unsupportedUnionException(this, keyCriterion);
- }
-
- return new RangeCriterion(
- minFromNullables(this.from, from),
- maxFromNullables(this.to, to)
- );
-
- }
-
- /**
- * Calculates the maximum of two keys in the scope of keys' range. According to the logic of range keys - null is an equivalent to
- * +Inf in the range end position.
- *
- * @param key1 The first key to compare.
- * @param key2 The second key to compare.
- * @return Maximum key.
- */
- private static ByteArray maxFromNullables(ByteArray key1, ByteArray key2) {
- if (key1 != null && key2 != null) {
- return (key1.compareTo(key2) >= 0) ? key1 : key2;
- } else {
- return null;
- }
- }
-
- /**
- * Calculates the minimum of two keys in the scope of keys' range. According to the logic of range keys - null is an equivalent to
- * -Inf in the range start position.
- *
- * @param key1 The first key to compare.
- * @param key2 The second key to compare.
- * @return Minimum key.
- */
- private static ByteArray minFromNullables(ByteArray key1, ByteArray key2) {
- if (key1 != null && key2 != null) {
- return (key1.compareTo(key2) < 0) ? key1 : key2;
- } else {
- return null;
- }
- }
-
- /**
- * Calculates the next key for received key.
- *
- * @param key Input key.
- * @return Next key.
- */
- private static ByteArray nextKey(ByteArray key) {
- var bytes = Arrays.copyOf(key.bytes(), key.bytes().length);
-
- if (bytes[bytes.length - 1] != Byte.MAX_VALUE) {
- bytes[bytes.length - 1]++;
- } else {
- bytes = Arrays.copyOf(bytes, bytes.length + 1);
- }
-
- return new ByteArray(bytes);
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- RangeCriterion criterion = (RangeCriterion) o;
- return Objects.equals(from, criterion.from) && Objects.equals(to, criterion.to);
- }
-
- /** {@inheritDoc} */
- @Override
- public int hashCode() {
- return Objects.hash(from, to);
- }
-
- /**
- * Returns start of the range.
- */
- public ByteArray from() {
- return from;
- }
-
- /**
- * Returns end of the range (exclusive).
- */
- public ByteArray to() {
- return to;
- }
- }
-
- /**
- * Criterion which consists collection of keys.
- */
- public static class CollectionCriterion extends KeyCriterion {
- /** Collection of keys. */
- private final Set<ByteArray> keys;
-
- /**
- * Creates the instance of collection criterion.
- *
- * @param keys Collection of keys.
- */
- public CollectionCriterion(Collection<ByteArray> keys) {
- this.keys = new HashSet<>(keys);
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean contains(ByteArray key) {
- return keys.contains(key);
- }
-
- /** {@inheritDoc} */
- @Override
- protected KeyCriterion union(KeyCriterion keyCriterion, boolean swapTry) {
- var newKeys = new HashSet<>(keys);
-
- if (keyCriterion instanceof ExactCriterion) {
- newKeys.add(((ExactCriterion) keyCriterion).key);
-
- return new CollectionCriterion(newKeys);
- } else if (keyCriterion instanceof CollectionCriterion) {
- newKeys.addAll(((CollectionCriterion) keyCriterion).keys);
-
- return new CollectionCriterion(newKeys);
- } else if (!swapTry) {
- return keyCriterion.union(keyCriterion, true);
- } else {
- throw KeyCriterion.unsupportedUnionException(this, keyCriterion);
- }
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- CollectionCriterion criterion = (CollectionCriterion) o;
- return Objects.equals(keys, criterion.keys);
- }
-
- /** {@inheritDoc} */
- @Override
- public int hashCode() {
- return Objects.hash(keys);
- }
-
- /**
- * Returns collection of keys.
- */
- public Set<ByteArray> keys() {
- return keys;
- }
- }
-
- /**
- * Simple criterion which contains exactly one key.
- */
- public static class ExactCriterion extends KeyCriterion {
- /** The key of criterion. */
- private final ByteArray key;
-
- /**
- * Creates the instance of exact criterion.
- *
- * @param key Instance of the reference key.
- */
- public ExactCriterion(ByteArray key) {
- this.key = key;
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean contains(ByteArray key) {
- return this.key.equals(key);
- }
-
- /** {@inheritDoc} */
- @Override
- protected KeyCriterion union(KeyCriterion keyCriterion, boolean swapTry) {
- if (keyCriterion instanceof ExactCriterion) {
- if (equals(keyCriterion)) {
- return this;
- } else {
- return new CollectionCriterion(Arrays.asList(key, ((ExactCriterion) keyCriterion).key));
- }
- } else if (!swapTry) {
- return keyCriterion.union(this, true);
- } else {
- throw KeyCriterion.unsupportedUnionException(this, keyCriterion);
- }
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
-
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- ExactCriterion criterion = (ExactCriterion) o;
- return Objects.equals(key, criterion.key);
- }
-
- /** {@inheritDoc} */
- @Override
- public int hashCode() {
- return Objects.hash(key);
- }
-
- /**
- * Returns the key of criterion.
- */
- public ByteArray key() {
- return key;
- }
- }
-}
diff --git a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java
deleted file mode 100644
index 4cbd89c6c7..0000000000
--- a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/watch/WatchAggregator.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/*
- * 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.metastorage.watch;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.BiConsumer;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.WatchListener;
-import org.apache.ignite.lang.ByteArray;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.jetbrains.annotations.NotNull;
-
-/**
- * Needed to aggregate multiple watches to one aggregated watch. This approach needed to provide the following additional guarantees to
- * watching mechanism: - watch events will be processed sequentially - watch events will be resolved in the order of watch registration
- */
-public class WatchAggregator {
- /**
- * Watches' map must be synchronized because of changes from WatchListener in separate thread.
- */
- private final Map<Long, Watch> watches = Collections.synchronizedMap(new LinkedHashMap<>());
-
- /** Simple auto increment id for internal watches. */
- private final AtomicLong idCntr = new AtomicLong(0);
-
- /**
- * Adds new watch with simple exact criterion.
- *
- * @param key Key for watching.
- * @param lsnr Listener which will be executed on watch event.
- * @return id of registered watch. Can be used for remove watch from later.
- */
- public long add(ByteArray key, WatchListener lsnr) {
- var watch = new Watch(new KeyCriterion.ExactCriterion(key), lsnr);
- var id = idCntr.incrementAndGet();
- watches.put(id, watch);
- return id;
- }
-
- /**
- * Adds new watch with filter by collection of keys.
- *
- * @param keys Collection of keys to listen.
- * @param lsnr Listener which will be executed on watch event.
- * @return id of registered watch. Can be used for remove watch from later.
- */
- public long add(Collection<ByteArray> keys, WatchListener lsnr) {
- var watch = new Watch(new KeyCriterion.CollectionCriterion(keys), lsnr);
- var id = idCntr.incrementAndGet();
- watches.put(id, watch);
- return id;
- }
-
- /**
- * Adds new watch with filter by collection of keys.
- *
- * @param from Start key of range to listen.
- * @param to End key of range (exclusively)..
- * @param lsnr Listener which will be executed on watch event.
- * @return id of registered watch. Can be used for remove watch from later.
- */
- public long add(ByteArray from, ByteArray to, WatchListener lsnr) {
- var watch = new Watch(new KeyCriterion.RangeCriterion(from, to), lsnr);
- var id = idCntr.incrementAndGet();
- watches.put(id, watch);
- return id;
- }
-
- /**
- * Adds new watch with filter by key prefix.
- *
- * @param key Prefix for key.
- * @param lsnr Listener which will be executed on watch event.
- * @return id of registered watch. Can be used for remove watch from later.
- */
- public long addPrefix(ByteArray key, WatchListener lsnr) {
- var watch = new Watch(KeyCriterion.RangeCriterion.fromPrefixKey(key), lsnr);
- var id = idCntr.incrementAndGet();
- watches.put(id, watch);
- return id;
- }
-
- /**
- * Cancel watch by id.
- *
- * @param id of watch to cancel.
- */
- public void cancel(long id) {
- watches.remove(id);
- }
-
- /**
- * Cancel multiple watches by ids.
- *
- * @param ids of watches to cancel.
- */
- public void cancelAll(Collection<Long> ids) {
- watches.keySet().removeAll(ids);
- }
-
- /**
- * Produce watch with aggregated key criterion and general watch listener dispatcher.
- *
- * @param revision start revision to listen event.
- * @param saveRevisionAct action to commit keys-revision pair to persistent store for processed keys.
- * @return result aggregated watch.
- */
- public Optional<AggregatedWatch> watch(
- long revision,
- BiConsumer<Collection<IgniteBiTuple<ByteArray, byte[]>>, Long> saveRevisionAct
- ) {
- synchronized (watches) {
- if (watches.isEmpty()) {
- return Optional.empty();
- } else {
- return Optional.of(new AggregatedWatch(inferGeneralCriteria(), revision, watchListener(saveRevisionAct)));
- }
- }
- }
-
- /**
- * Returns general criterion, which overlays all aggregated criteria.
- *
- * @return aggregated criterion.
- */
- private KeyCriterion inferGeneralCriteria() {
- return
- watches.values().stream()
- .map(Watch::keyCriterion)
- .reduce(KeyCriterion::union)
- .get();
- }
-
- /**
- * Produces the watch listener, which will dispatch events to appropriate watches.
- *
- * @param storeRevision action to commit keys-revision pair to persistent store for processed keys.
- * @return watch listener, which will dispatch events to appropriate watches.
- */
- private WatchListener watchListener(BiConsumer<Collection<IgniteBiTuple<ByteArray, byte[]>>, Long> storeRevision) {
- // Copy watches to separate collection, because all changes on the WatchAggregator watches
- // shouldn't be propagated to listener watches immediately.
- // WatchAggregator will be redeployed with new watches if needed instead.
- final LinkedHashMap<Long, Watch> cpWatches = new LinkedHashMap<>(watches);
-
- return new WatchListener() {
-
- @Override
- public boolean onUpdate(@NotNull WatchEvent evt) {
- //TODO: IGNITE-15858 Fix stopWatch may solve the issue.
- synchronized (watches) {
- processWatchEvents(evt);
- }
-
- return true;
- }
-
- /**
- * Process watch events synchronously.
- *
- * @param evt Watch event.
- */
- private void processWatchEvents(@NotNull WatchEvent evt) {
- var watchIt = cpWatches.entrySet().iterator();
- Collection<Long> toCancel = new ArrayList<>();
-
- while (watchIt.hasNext()) {
- Map.Entry<Long, Watch> entry = watchIt.next();
- Watch watch = entry.getValue();
- var filteredEvts = new ArrayList<EntryEvent>();
-
- for (EntryEvent entryEvt : evt.entryEvents()) {
- if (watch.keyCriterion().contains(new ByteArray(entryEvt.oldEntry().key()))) {
- filteredEvts.add(entryEvt);
- }
- }
-
- if (!filteredEvts.isEmpty()) {
- if (!watch.listener().onUpdate(new WatchEvent(filteredEvts))) {
- watchIt.remove();
-
- toCancel.add(entry.getKey());
- }
- }
- }
-
- // Cancel finished watches from the global watch map
- // to prevent finished watches from redeploy.
- if (!toCancel.isEmpty()) {
- cancelAll(toCancel);
- }
-
- var revision = 0L;
- var entries = new ArrayList<IgniteBiTuple<ByteArray, byte[]>>();
- for (EntryEvent entryEvt : evt.entryEvents()) {
- revision = entryEvt.newEntry().revision();
-
- entries.add(new IgniteBiTuple<>(new ByteArray(entryEvt.newEntry().key()), entryEvt.newEntry().value()));
- }
-
- storeRevision.accept(entries, revision);
- }
-
- @Override
- public void onError(@NotNull Throwable e) {
- watches.values().forEach(w -> w.listener().onError(e));
- }
- };
- }
-
- /**
- * (key criterion, watch listener) container.
- */
- private static class Watch {
- /** Key criterion. */
- private final KeyCriterion keyCriterion;
-
- /** Watch listener. */
- private final WatchListener lsnr;
-
- /** Creates the watch. */
- private Watch(KeyCriterion keyCriterion, WatchListener lsnr) {
- this.keyCriterion = keyCriterion;
- this.lsnr = lsnr;
- }
-
- /**
- * Returns key criterion.
- */
- public KeyCriterion keyCriterion() {
- return keyCriterion;
- }
-
- /**
- * Returns watch listener.
- */
- public WatchListener listener() {
- return lsnr;
- }
- }
-}
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageRangeCursorTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageRangeCursorTest.java
index d948732013..462ffc6269 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageRangeCursorTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageRangeCursorTest.java
@@ -49,6 +49,8 @@ import org.apache.ignite.internal.raft.service.CommandClosure;
import org.apache.ignite.internal.raft.service.RaftGroupService;
import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.lang.ByteArray;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.NetworkAddress;
import org.jetbrains.annotations.Nullable;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.params.ParameterizedTest;
@@ -62,13 +64,13 @@ import org.mockito.junit.jupiter.MockitoExtension;
*/
@ExtendWith(MockitoExtension.class)
public class MetaStorageRangeCursorTest {
- MetaStorageListener listener;
+ private MetaStorageListener listener;
@Mock
- RaftGroupService raftGroupService;
+ private RaftGroupService raftGroupService;
@Mock
- KeyValueStorage storage;
+ private KeyValueStorage storage;
@ParameterizedTest
@MethodSource("getParameters")
@@ -82,7 +84,9 @@ public class MetaStorageRangeCursorTest {
when(raftGroupService.run(any())).thenAnswer(invocation -> runCommand(invocation.getArgument(0)));
- MetaStorageService metaStorageService = new MetaStorageServiceImpl(raftGroupService, "test", "test");
+ var localNode = new ClusterNode("test", "test", new NetworkAddress("localhost", 10000));
+
+ MetaStorageService metaStorageService = new MetaStorageServiceImpl(raftGroupService, localNode);
checkCursor(metaStorageService.range(intToBytes(0), intToBytes(keyTo)), limit);
checkCursor(metaStorageService.range(intToBytes(0), intToBytes(keyTo), 0), limit);
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/AbstractKeyValueStorageTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/AbstractKeyValueStorageTest.java
index e4736c8a5c..b918e31da7 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/AbstractKeyValueStorageTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/AbstractKeyValueStorageTest.java
@@ -19,25 +19,30 @@ package org.apache.ignite.internal.metastorage.server;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.function.Function.identity;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import java.nio.ByteBuffer;
import java.util.Collection;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
import java.util.stream.Collectors;
import org.apache.ignite.internal.metastorage.Entry;
import org.apache.ignite.internal.metastorage.EntryEvent;
import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.dsl.Operation;
import org.apache.ignite.internal.metastorage.dsl.OperationType;
import org.apache.ignite.internal.metastorage.dsl.StatementResult;
@@ -45,6 +50,7 @@ import org.apache.ignite.internal.metastorage.dsl.Update;
import org.apache.ignite.internal.metastorage.server.ValueCondition.Type;
import org.apache.ignite.internal.util.Cursor;
import org.apache.ignite.lang.ByteArray;
+import org.jetbrains.annotations.Nullable;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -53,7 +59,7 @@ import org.junit.jupiter.api.Test;
* Tests for key-value storage implementations.
*/
public abstract class AbstractKeyValueStorageTest {
- private KeyValueStorage storage;
+ protected KeyValueStorage storage;
/**
* Before each.
@@ -1949,88 +1955,84 @@ public abstract class AbstractKeyValueStorageTest {
assertEquals(3, storage.updateCounter());
// Range for latest revision without max bound.
- Cursor<Entry> cur = storage.range(key1, null, false);
+ try (Cursor<Entry> cur = storage.range(key1, null, false)) {
+ assertTrue(cur.hasNext());
- Iterator<Entry> it = cur.iterator();
+ Entry e1 = cur.next();
- assertTrue(it.hasNext());
+ assertFalse(e1.empty());
+ assertFalse(e1.tombstone());
+ assertArrayEquals(key1, e1.key());
+ assertArrayEquals(val1, e1.value());
+ assertEquals(1, e1.revision());
+ assertEquals(1, e1.updateCounter());
- Entry e1 = it.next();
+ assertTrue(cur.hasNext());
- assertFalse(e1.empty());
- assertFalse(e1.tombstone());
- assertArrayEquals(key1, e1.key());
- assertArrayEquals(val1, e1.value());
- assertEquals(1, e1.revision());
- assertEquals(1, e1.updateCounter());
+ Entry e2 = cur.next();
- assertTrue(it.hasNext());
+ assertFalse(e2.empty());
+ assertFalse(e2.tombstone());
+ assertArrayEquals(key2, e2.key());
+ assertArrayEquals(val2, e2.value());
+ assertEquals(1, e2.revision());
+ assertEquals(2, e2.updateCounter());
- Entry e2 = it.next();
+ // Deliberately don't call cur.hasNext()
- assertFalse(e2.empty());
- assertFalse(e2.tombstone());
- assertArrayEquals(key2, e2.key());
- assertArrayEquals(val2, e2.value());
- assertEquals(1, e2.revision());
- assertEquals(2, e2.updateCounter());
-
- // Deliberately don't call it.hasNext()
-
- Entry e3 = it.next();
+ Entry e3 = cur.next();
- assertFalse(e3.empty());
- assertFalse(e3.tombstone());
- assertArrayEquals(key3, e3.key());
- assertArrayEquals(val3, e3.value());
- assertEquals(1, e3.revision());
- assertEquals(3, e3.updateCounter());
+ assertFalse(e3.empty());
+ assertFalse(e3.tombstone());
+ assertArrayEquals(key3, e3.key());
+ assertArrayEquals(val3, e3.value());
+ assertEquals(1, e3.revision());
+ assertEquals(3, e3.updateCounter());
- assertFalse(it.hasNext());
+ assertFalse(cur.hasNext());
- try {
- it.next();
+ try {
+ cur.next();
- fail();
- } catch (NoSuchElementException e) {
- // No-op.
+ fail();
+ } catch (NoSuchElementException e) {
+ // No-op.
+ }
}
// Range for latest revision with max bound.
- cur = storage.range(key1, key3, false);
-
- it = cur.iterator();
+ try (Cursor<Entry> cur = storage.range(key1, key3, false)) {
+ assertTrue(cur.hasNext());
- assertTrue(it.hasNext());
+ Entry e1 = cur.next();
- e1 = it.next();
+ assertFalse(e1.empty());
+ assertFalse(e1.tombstone());
+ assertArrayEquals(key1, e1.key());
+ assertArrayEquals(val1, e1.value());
+ assertEquals(1, e1.revision());
+ assertEquals(1, e1.updateCounter());
- assertFalse(e1.empty());
- assertFalse(e1.tombstone());
- assertArrayEquals(key1, e1.key());
- assertArrayEquals(val1, e1.value());
- assertEquals(1, e1.revision());
- assertEquals(1, e1.updateCounter());
+ assertTrue(cur.hasNext());
- assertTrue(it.hasNext());
+ Entry e2 = cur.next();
- e2 = it.next();
+ assertFalse(e2.empty());
+ assertFalse(e2.tombstone());
+ assertArrayEquals(key2, e2.key());
+ assertArrayEquals(val2, e2.value());
+ assertEquals(1, e2.revision());
+ assertEquals(2, e2.updateCounter());
- assertFalse(e2.empty());
- assertFalse(e2.tombstone());
- assertArrayEquals(key2, e2.key());
- assertArrayEquals(val2, e2.value());
- assertEquals(1, e2.revision());
- assertEquals(2, e2.updateCounter());
-
- assertFalse(it.hasNext());
+ assertFalse(cur.hasNext());
- try {
- it.next();
+ try {
+ cur.next();
- fail();
- } catch (NoSuchElementException e) {
- // No-op.
+ fail();
+ } catch (NoSuchElementException e) {
+ // No-op.
+ }
}
}
@@ -2079,7 +2081,7 @@ public abstract class AbstractKeyValueStorageTest {
}
@Test
- public void watchCursorLexicographicTest() throws Exception {
+ public void watchLexicographicTest() {
assertEquals(0, storage.revision());
assertEquals(0, storage.updateCounter());
@@ -2088,37 +2090,30 @@ public abstract class AbstractKeyValueStorageTest {
int count = 1000; // Exceeds 1 byte
- for (int i = 0; i < count; i++) {
- storage.put(key, val);
- }
-
- assertEquals(count, storage.revision());
- assertEquals(count, storage.updateCounter());
-
- int i = 1;
- int countSeen = 0;
-
- try (Cursor<WatchEvent> cur = storage.watch(key, 1)) {
- for (WatchEvent event : cur) {
- assertTrue(event.single());
+ CompletableFuture<Void> awaitFuture = watchExact(key, 1, count, (event, state) -> {
+ assertTrue(event.single());
- Entry entry = event.entryEvent().newEntry();
+ Entry entry = event.entryEvent().newEntry();
- byte[] entryKey = entry.key();
+ byte[] entryKey = entry.key();
- assertEquals(i++, entry.revision());
+ assertEquals((long) state, entry.revision());
- assertArrayEquals(key, entryKey);
+ assertArrayEquals(key, entryKey);
+ });
- countSeen++;
- }
+ for (int i = 0; i < count; i++) {
+ storage.put(key, val);
}
- assertEquals(count, countSeen);
+ assertEquals(count, storage.revision());
+ assertEquals(count, storage.updateCounter());
+
+ assertThat(awaitFuture, willCompleteSuccessfully());
}
@Test
- public void watchCursorForRange() throws Exception {
+ public void testWatchRange() {
byte[] key1 = key(1);
byte[] val11 = keyValue(1, 11);
@@ -2133,119 +2128,99 @@ public abstract class AbstractKeyValueStorageTest {
assertEquals(0, storage.updateCounter());
// Watch for all updates starting from revision 2.
- Cursor<WatchEvent> cur = storage.watch(key1, null, 2);
+ CompletableFuture<Void> awaitFuture = watchRange(key1, null, 2, 2, (event, state) -> {
+ if (state == 1) {
+ assertFalse(event.single());
- Iterator<WatchEvent> it = cur.iterator();
+ Map<ByteArray, EntryEvent> map = event.entryEvents().stream()
+ .collect(Collectors.toMap(evt -> new ByteArray(evt.newEntry().key()), identity()));
- assertFalse(it.hasNext());
- assertThrows(NoSuchElementException.class, it::next);
+ assertEquals(2, map.size());
- storage.putAll(List.of(key1, key2), List.of(val11, val21));
+ // First update under revision.
+ EntryEvent e2 = map.get(new ByteArray(key2));
- assertEquals(1, storage.revision());
- assertEquals(2, storage.updateCounter());
+ assertNotNull(e2);
- // Revision is less than 2.
- assertFalse(it.hasNext());
- assertThrows(NoSuchElementException.class, it::next);
+ Entry oldEntry2 = e2.oldEntry();
- storage.putAll(List.of(key2, key3), List.of(val22, val31));
+ assertFalse(oldEntry2.empty());
+ assertFalse(oldEntry2.tombstone());
+ assertEquals(1, oldEntry2.revision());
+ assertEquals(2, oldEntry2.updateCounter());
+ assertArrayEquals(key2, oldEntry2.key());
+ assertArrayEquals(val21, oldEntry2.value());
- assertEquals(2, storage.revision());
- assertEquals(4, storage.updateCounter());
+ Entry newEntry2 = e2.newEntry();
- // Revision is 2.
- assertTrue(it.hasNext());
+ assertFalse(newEntry2.empty());
+ assertFalse(newEntry2.tombstone());
+ assertEquals(2, newEntry2.revision());
+ assertEquals(3, newEntry2.updateCounter());
+ assertArrayEquals(key2, newEntry2.key());
+ assertArrayEquals(val22, newEntry2.value());
- WatchEvent watchEvent = it.next();
+ // Second update under revision.
+ EntryEvent e3 = map.get(new ByteArray(key3));
- assertFalse(watchEvent.single());
+ assertNotNull(e3);
- Map<ByteArray, EntryEvent> map = watchEvent.entryEvents().stream()
- .collect(Collectors.toMap(evt -> new ByteArray(evt.newEntry().key()), identity()));
+ Entry oldEntry3 = e3.oldEntry();
- assertEquals(2, map.size());
+ assertTrue(oldEntry3.empty());
+ assertFalse(oldEntry3.tombstone());
+ assertArrayEquals(key3, oldEntry3.key());
- // First update under revision.
- EntryEvent e2 = map.get(new ByteArray(key2));
-
- assertNotNull(e2);
+ Entry newEntry3 = e3.newEntry();
- Entry oldEntry2 = e2.oldEntry();
-
- assertFalse(oldEntry2.empty());
- assertFalse(oldEntry2.tombstone());
- assertEquals(1, oldEntry2.revision());
- assertEquals(2, oldEntry2.updateCounter());
- assertArrayEquals(key2, oldEntry2.key());
- assertArrayEquals(val21, oldEntry2.value());
+ assertFalse(newEntry3.empty());
+ assertFalse(newEntry3.tombstone());
+ assertEquals(2, newEntry3.revision());
+ assertEquals(4, newEntry3.updateCounter());
+ assertArrayEquals(key3, newEntry3.key());
+ assertArrayEquals(val31, newEntry3.value());
+ } else if (state == 2) {
+ assertTrue(event.single());
- Entry newEntry2 = e2.newEntry();
+ EntryEvent e1 = event.entryEvent();
- assertFalse(newEntry2.empty());
- assertFalse(newEntry2.tombstone());
- assertEquals(2, newEntry2.revision());
- assertEquals(3, newEntry2.updateCounter());
- assertArrayEquals(key2, newEntry2.key());
- assertArrayEquals(val22, newEntry2.value());
+ Entry oldEntry1 = e1.oldEntry();
- // Second update under revision.
- EntryEvent e3 = map.get(new ByteArray(key3));
+ assertFalse(oldEntry1.empty());
+ assertFalse(oldEntry1.tombstone());
+ assertEquals(1, oldEntry1.revision());
+ assertEquals(1, oldEntry1.updateCounter());
+ assertArrayEquals(key1, oldEntry1.key());
+ assertArrayEquals(val11, oldEntry1.value());
- assertNotNull(e3);
+ Entry newEntry1 = e1.newEntry();
- Entry oldEntry3 = e3.oldEntry();
+ assertFalse(newEntry1.empty());
+ assertTrue(newEntry1.tombstone());
+ assertEquals(3, newEntry1.revision());
+ assertEquals(5, newEntry1.updateCounter());
+ assertArrayEquals(key1, newEntry1.key());
+ assertNull(newEntry1.value());
+ }
+ });
- assertTrue(oldEntry3.empty());
- assertFalse(oldEntry3.tombstone());
- assertArrayEquals(key3, oldEntry3.key());
+ storage.putAll(List.of(key1, key2), List.of(val11, val21));
- Entry newEntry3 = e3.newEntry();
+ assertEquals(1, storage.revision());
+ assertEquals(2, storage.updateCounter());
- assertFalse(newEntry3.empty());
- assertFalse(newEntry3.tombstone());
- assertEquals(2, newEntry3.revision());
- assertEquals(4, newEntry3.updateCounter());
- assertArrayEquals(key3, newEntry3.key());
- assertArrayEquals(val31, newEntry3.value());
+ storage.putAll(List.of(key2, key3), List.of(val22, val31));
- assertFalse(it.hasNext());
+ assertEquals(2, storage.revision());
+ assertEquals(4, storage.updateCounter());
storage.remove(key1);
- assertTrue(it.hasNext());
-
- watchEvent = it.next();
-
- assertTrue(watchEvent.single());
-
- EntryEvent e1 = watchEvent.entryEvent();
-
- Entry oldEntry1 = e1.oldEntry();
-
- assertFalse(oldEntry1.empty());
- assertFalse(oldEntry1.tombstone());
- assertEquals(1, oldEntry1.revision());
- assertEquals(1, oldEntry1.updateCounter());
- assertArrayEquals(key1, oldEntry1.key());
- assertArrayEquals(val11, oldEntry1.value());
-
- Entry newEntry1 = e1.newEntry();
-
- assertFalse(newEntry1.empty());
- assertTrue(newEntry1.tombstone());
- assertEquals(3, newEntry1.revision());
- assertEquals(5, newEntry1.updateCounter());
- assertArrayEquals(key1, newEntry1.key());
- assertNull(newEntry1.value());
-
- assertFalse(it.hasNext());
-
- cur.close();
+ assertThat(awaitFuture, willCompleteSuccessfully());
}
@Test
- public void watchCursorForKey() {
+ public void testWatchExact() {
byte[] key1 = key(1);
byte[] val11 = keyValue(1, 11);
byte[] val12 = keyValue(1, 12);
@@ -2257,79 +2232,63 @@ public abstract class AbstractKeyValueStorageTest {
assertEquals(0, storage.revision());
assertEquals(0, storage.updateCounter());
- Cursor<WatchEvent> cur = storage.watch(key1, 1);
+ CompletableFuture<Void> awaitFuture = watchExact(key1, 1, 2, (event, state) -> {
+ if (state == 1) {
+ assertTrue(event.single());
- Iterator<WatchEvent> it = cur.iterator();
+ EntryEvent e1 = event.entryEvent();
- assertFalse(it.hasNext());
- assertThrows(NoSuchElementException.class, it::next);
+ Entry oldEntry1 = e1.oldEntry();
- storage.putAll(List.of(key1, key2), List.of(val11, val21));
+ assertTrue(oldEntry1.empty());
+ assertFalse(oldEntry1.tombstone());
- assertEquals(1, storage.revision());
- assertEquals(2, storage.updateCounter());
+ Entry newEntry1 = e1.newEntry();
- assertTrue(it.hasNext());
-
- WatchEvent watchEvent = it.next();
+ assertFalse(newEntry1.empty());
+ assertFalse(newEntry1.tombstone());
+ assertEquals(1, newEntry1.revision());
+ assertEquals(1, newEntry1.updateCounter());
+ assertArrayEquals(key1, newEntry1.key());
+ assertArrayEquals(val11, newEntry1.value());
+ } else if (state == 2) {
+ assertTrue(event.single());
- assertTrue(watchEvent.single());
+ EntryEvent e1 = event.entryEvent();
- EntryEvent e1 = watchEvent.entryEvent();
+ Entry oldEntry1 = e1.oldEntry();
- Entry oldEntry1 = e1.oldEntry();
+ assertFalse(oldEntry1.empty());
+ assertFalse(oldEntry1.tombstone());
+ assertEquals(1, oldEntry1.revision());
+ assertEquals(1, oldEntry1.updateCounter());
+ assertArrayEquals(key1, oldEntry1.key());
+ assertArrayEquals(val11, oldEntry1.value());
- assertTrue(oldEntry1.empty());
- assertFalse(oldEntry1.tombstone());
+ Entry newEntry1 = e1.newEntry();
- Entry newEntry1 = e1.newEntry();
+ assertFalse(newEntry1.empty());
+ assertFalse(newEntry1.tombstone());
+ assertEquals(3, newEntry1.revision());
+ assertEquals(4, newEntry1.updateCounter());
+ assertArrayEquals(key1, newEntry1.key());
+ assertArrayEquals(val12, newEntry1.value());
+ }
+ });
- assertFalse(newEntry1.empty());
- assertFalse(newEntry1.tombstone());
- assertEquals(1, newEntry1.revision());
- assertEquals(1, newEntry1.updateCounter());
- assertArrayEquals(key1, newEntry1.key());
- assertArrayEquals(val11, newEntry1.value());
+ storage.putAll(List.of(key1, key2), List.of(val11, val21));
- assertFalse(it.hasNext());
+ assertEquals(1, storage.revision());
+ assertEquals(2, storage.updateCounter());
storage.put(key2, val22);
-
- assertFalse(it.hasNext());
-
storage.put(key1, val12);
- assertTrue(it.hasNext());
-
- watchEvent = it.next();
-
- assertTrue(watchEvent.single());
-
- e1 = watchEvent.entryEvent();
-
- oldEntry1 = e1.oldEntry();
-
- assertFalse(oldEntry1.empty());
- assertFalse(oldEntry1.tombstone());
- assertEquals(1, oldEntry1.revision());
- assertEquals(1, oldEntry1.updateCounter());
- assertArrayEquals(key1, newEntry1.key());
- assertArrayEquals(val11, newEntry1.value());
-
- newEntry1 = e1.newEntry();
-
- assertFalse(newEntry1.empty());
- assertFalse(newEntry1.tombstone());
- assertEquals(3, newEntry1.revision());
- assertEquals(4, newEntry1.updateCounter());
- assertArrayEquals(key1, newEntry1.key());
- assertArrayEquals(val12, newEntry1.value());
-
- assertFalse(it.hasNext());
+ assertThat(awaitFuture, willCompleteSuccessfully());
}
@Test
- public void watchCursorForKeySkipNonMatchingEntries() throws Exception {
+ public void watchExactkipNonMatchingEntries() {
byte[] key1 = key(1);
byte[] val1v1 = keyValue(1, 11);
byte[] val1v2 = keyValue(1, 12);
@@ -2340,32 +2299,10 @@ public abstract class AbstractKeyValueStorageTest {
assertEquals(0, storage.revision());
assertEquals(0, storage.updateCounter());
- try (Cursor<WatchEvent> cur = storage.watch(key2, 1)) {
- assertFalse(cur.hasNext());
- assertThrows(NoSuchElementException.class, cur::next);
-
- storage.put(key1, val1v1);
-
- assertFalse(cur.hasNext());
- assertThrows(NoSuchElementException.class, cur::next);
-
- storage.put(key1, val1v2);
-
- assertFalse(cur.hasNext());
- assertThrows(NoSuchElementException.class, cur::next);
-
- storage.put(key2, val2);
-
- assertEquals(3, storage.revision());
- assertEquals(3, storage.updateCounter());
-
- assertTrue(cur.hasNext());
-
- WatchEvent watchEvent = cur.next();
+ CompletableFuture<Void> awaitFuture = watchExact(key2, 1, 1, (event, state) -> {
+ assertTrue(event.single());
- assertTrue(watchEvent.single());
-
- EntryEvent e1 = watchEvent.entryEvent();
+ EntryEvent e1 = event.entryEvent();
Entry oldEntry1 = e1.oldEntry();
@@ -2380,11 +2317,20 @@ public abstract class AbstractKeyValueStorageTest {
assertEquals(3, newEntry1.updateCounter());
assertArrayEquals(key2, newEntry1.key());
assertArrayEquals(val2, newEntry1.value());
- }
+ });
+
+ storage.put(key1, val1v1);
+ storage.put(key1, val1v2);
+ storage.put(key2, val2);
+
+ assertEquals(3, storage.revision());
+ assertEquals(3, storage.updateCounter());
+
+ assertThat(awaitFuture, willCompleteSuccessfully());
}
@Test
- public void watchCursorForKeys() {
+ public void watchExactForKeys() {
byte[] key1 = key(1);
byte[] val11 = keyValue(1, 11);
@@ -2399,39 +2345,24 @@ public abstract class AbstractKeyValueStorageTest {
assertEquals(0, storage.revision());
assertEquals(0, storage.updateCounter());
- Cursor<WatchEvent> cur = storage.watch(List.of(key1, key2), 1);
-
- Iterator<WatchEvent> it = cur.iterator();
-
- assertFalse(it.hasNext());
- assertThrows(NoSuchElementException.class, it::next);
+ CompletableFuture<Void> awaitFuture = watchExact(List.of(key1, key2), 1, 2, (event, state) -> {
+ if (state == 1) {
+ assertFalse(event.single());
+ } else if (state == 2) {
+ assertTrue(event.single());
+ }
+ });
storage.putAll(List.of(key1, key2, key3), List.of(val11, val21, val31));
assertEquals(1, storage.revision());
assertEquals(3, storage.updateCounter());
- assertTrue(it.hasNext());
-
- WatchEvent watchEvent = it.next();
-
- assertFalse(watchEvent.single());
-
- assertFalse(it.hasNext());
-
storage.put(key2, val22);
- assertTrue(it.hasNext());
-
- watchEvent = it.next();
-
- assertTrue(watchEvent.single());
-
- assertFalse(it.hasNext());
-
storage.put(key3, val32);
- assertFalse(it.hasNext());
+ assertThat(awaitFuture, willCompleteSuccessfully());
}
private static void fill(KeyValueStorage storage, int keySuffix, int num) {
@@ -2447,4 +2378,56 @@ public abstract class AbstractKeyValueStorageTest {
private static byte[] keyValue(int k, int v) {
return ("key" + k + '_' + "val" + v).getBytes(UTF_8);
}
+
+ private CompletableFuture<Void> watchExact(
+ byte[] key, long revision, int expectedNumCalls, BiConsumer<WatchEvent, Integer> testCondition
+ ) {
+ return watch(listener -> storage.watchExact(key, revision, listener), testCondition, expectedNumCalls);
+ }
+
+ private CompletableFuture<Void> watchExact(
+ Collection<byte[]> keys, long revision, int expectedNumCalls, BiConsumer<WatchEvent, Integer> testCondition
+ ) {
+ return watch(listener -> storage.watchExact(keys, revision, listener), testCondition, expectedNumCalls);
+ }
+
+ private CompletableFuture<Void> watchRange(
+ byte[] keyFrom, byte @Nullable [] keyTo, long revision, int expectedNumCalls, BiConsumer<WatchEvent, Integer> testCondition
+ ) {
+ return watch(listener -> storage.watchRange(keyFrom, keyTo, revision, listener), testCondition, expectedNumCalls);
+ }
+
+ private CompletableFuture<Void> watch(
+ Consumer<WatchListener> watchMethod, BiConsumer<WatchEvent, Integer> testCondition, int expectedNumCalls
+ ) {
+ var state = new AtomicInteger();
+
+ var resultFuture = new CompletableFuture<Void>();
+
+ watchMethod.accept(new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ try {
+ var curState = state.incrementAndGet();
+
+ testCondition.accept(event, curState);
+
+ if (curState == expectedNumCalls) {
+ resultFuture.complete(null);
+ }
+ } catch (Exception e) {
+ resultFuture.completeExceptionally(e);
+ }
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ resultFuture.completeExceptionally(e);
+ }
+ });
+
+ storage.startWatches(revision -> {});
+
+ return resultFuture;
+ }
}
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/RocksDbKeyValueStorageTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/RocksDbKeyValueStorageTest.java
index 5c87a65902..530aca3cf8 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/RocksDbKeyValueStorageTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/RocksDbKeyValueStorageTest.java
@@ -17,10 +17,22 @@
package org.apache.ignite.internal.metastorage.server;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
import java.nio.file.Path;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
import org.apache.ignite.internal.testframework.WorkDirectory;
import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
+import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
/**
@@ -34,6 +46,62 @@ public class RocksDbKeyValueStorageTest extends AbstractKeyValueStorageTest {
/** {@inheritDoc} */
@Override
KeyValueStorage storage() {
- return new RocksDbKeyValueStorage(workDir);
+ return new RocksDbKeyValueStorage("test", workDir.resolve("storage"));
+ }
+
+ @Test
+ void testWatchReplayOnSnapshotLoad() throws InterruptedException {
+ storage.put("foo".getBytes(UTF_8), "bar".getBytes(UTF_8));
+ storage.put("baz".getBytes(UTF_8), "quux".getBytes(UTF_8));
+
+ long revisionBeforeSnapshot = storage.revision();
+
+ Path snapshotPath = workDir.resolve("snapshot");
+
+ assertThat(storage.snapshot(snapshotPath), willCompleteSuccessfully());
+
+ storage.close();
+
+ storage = storage();
+
+ storage.start();
+
+ var latch = new CountDownLatch(2);
+
+ storage.watchExact("foo".getBytes(UTF_8), 1, new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ assertThat(event.entryEvent().newEntry().value(), is("bar".getBytes(UTF_8)));
+
+ latch.countDown();
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ fail();
+ }
+ });
+
+ storage.watchExact("baz".getBytes(UTF_8), 1, new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ assertThat(event.entryEvent().newEntry().value(), is("quux".getBytes(UTF_8)));
+
+ latch.countDown();
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ fail();
+ }
+ });
+
+ storage.startWatches(revision -> {});
+
+ storage.restoreSnapshot(snapshotPath);
+
+ assertThat(storage.revision(), is(revisionBeforeSnapshot));
+
+ assertTrue(latch.await(10, TimeUnit.SECONDS));
}
}
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
index 5fa0ad41d7..3050c8ff00 100644
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
+++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorageTest.java
@@ -24,6 +24,6 @@ class SimpleInMemoryKeyValueStorageTest extends AbstractKeyValueStorageTest {
/** {@inheritDoc} */
@Override
KeyValueStorage storage() {
- return new SimpleInMemoryKeyValueStorage();
+ return new SimpleInMemoryKeyValueStorage("test");
}
}
diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/watch/WatchAggregatorTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/watch/WatchAggregatorTest.java
deleted file mode 100644
index 4d23ca00d2..0000000000
--- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/watch/WatchAggregatorTest.java
+++ /dev/null
@@ -1,312 +0,0 @@
-/*
- * 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.metastorage.watch;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import org.apache.ignite.internal.metastorage.Entry;
-import org.apache.ignite.internal.metastorage.EntryEvent;
-import org.apache.ignite.internal.metastorage.WatchEvent;
-import org.apache.ignite.internal.metastorage.WatchListener;
-import org.apache.ignite.lang.ByteArray;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-import org.junit.jupiter.api.Test;
-import org.mockito.ArgumentCaptor;
-
-/**
- * Test class for {@link WatchAggregator}.
- */
-public class WatchAggregatorTest {
- @Test
- public void testEventsRouting() {
- var watchAggregator = new WatchAggregator();
- var lsnr1 = mock(WatchListener.class);
- var lsnr2 = mock(WatchListener.class);
-
- watchAggregator.add(new ByteArray("1"), lsnr1);
- watchAggregator.add(new ByteArray("2"), lsnr2);
-
- var entryEvt1 = new EntryEvent(
- entry("1", "value1", 1, 1),
- entry("1", "value1n", 1, 1)
- );
-
- var entryEvt2 = new EntryEvent(
- entry("2", "value2", 1, 1),
- entry("2", "value2n", 1, 1)
- );
-
- watchAggregator.watch(1, (v1, v2) -> {
- }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
- var watchEvt1Res = ArgumentCaptor.forClass(WatchEvent.class);
- verify(lsnr1).onUpdate(watchEvt1Res.capture());
- assertEquals(List.of(entryEvt1), watchEvt1Res.getValue().entryEvents());
-
- var watchEvt2Res = ArgumentCaptor.forClass(WatchEvent.class);
- verify(lsnr2).onUpdate(watchEvt2Res.capture());
- assertEquals(List.of(entryEvt2), watchEvt2Res.getValue().entryEvents());
- }
-
- @Test
- public void testCancel() {
- var watchAggregator = new WatchAggregator();
- var lsnr1 = mock(WatchListener.class);
- when(lsnr1.onUpdate(any())).thenReturn(true);
- var lsnr2 = mock(WatchListener.class);
- when(lsnr2.onUpdate(any())).thenReturn(true);
- final var id1 = watchAggregator.add(new ByteArray("1"), lsnr1);
- var id2 = watchAggregator.add(new ByteArray("2"), lsnr2);
-
- var entryEvt1 = new EntryEvent(
- entry("1", "value1", 1, 1),
- entry("1", "value1n", 1, 1)
- );
-
- var entryEvt2 = new EntryEvent(
- entry("2", "value2", 1, 1),
- entry("2", "value2n", 1, 1)
- );
-
- watchAggregator.watch(1, (v1, v2) -> {
- }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
- verify(lsnr1, times(1)).onUpdate(any());
- verify(lsnr2, times(1)).onUpdate(any());
-
- watchAggregator.cancel(id1);
- watchAggregator.watch(1, (v1, v2) -> {
- }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
- verify(lsnr1, times(1)).onUpdate(any());
- verify(lsnr2, times(2)).onUpdate(any());
- }
-
- @Test
- public void testCancelByFalseFromListener() {
- var watchAggregator = new WatchAggregator();
- var lsnr1 = mock(WatchListener.class);
- when(lsnr1.onUpdate(any())).thenReturn(false);
- var lsnr2 = mock(WatchListener.class);
- when(lsnr2.onUpdate(any())).thenReturn(true);
- var id1 = watchAggregator.add(new ByteArray("1"), lsnr1);
- var id2 = watchAggregator.add(new ByteArray("2"), lsnr2);
-
- var entryEvt1 = new EntryEvent(
- entry("1", "value1", 1, 1),
- entry("1", "value1n", 1, 1)
- );
-
- var entryEvt2 = new EntryEvent(
- entry("2", "value2", 1, 1),
- entry("2", "value2n", 1, 1)
- );
-
- watchAggregator.watch(1, (v1, v2) -> {
- }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
- verify(lsnr1, times(1)).onUpdate(any());
- verify(lsnr2, times(1)).onUpdate(any());
-
- watchAggregator.watch(1, (v1, v2) -> {
- }).get().listener().onUpdate(new WatchEvent(List.of(entryEvt1, entryEvt2)));
-
- verify(lsnr1, times(1)).onUpdate(any());
- verify(lsnr2, times(2)).onUpdate(any());
-
- }
-
- @Test
- public void testOneCriterionInference() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(new ByteArray("key"), null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- assertEquals(new KeyCriterion.ExactCriterion(new ByteArray("key")), keyCriterion);
- }
-
- @Test
- public void testTwoExactCriteriaUnion() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(new ByteArray("key1"), null);
- watchAggregator.add(new ByteArray("key2"), null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- var expKeyCriterion = new KeyCriterion.CollectionCriterion(
- new HashSet<>(Arrays.asList(new ByteArray("key1"), new ByteArray("key2")))
- );
- assertEquals(expKeyCriterion, keyCriterion);
- }
-
- @Test
- public void testTwoEqualExactCriteriaUnion() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(new ByteArray("key1"), null);
- watchAggregator.add(new ByteArray("key1"), null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- assertEquals(keyCriterion, keyCriterion);
- }
-
- @Test
- public void testThatKeyCriteriaUnionAssociative() {
- var data = Arrays.asList(
- new KeyCriterion.RangeCriterion(new ByteArray("0"), new ByteArray("5")),
- new KeyCriterion.CollectionCriterion(Arrays.asList(new ByteArray("1"), new ByteArray("2"))),
- new KeyCriterion.ExactCriterion(new ByteArray("3")));
-
- for (int i = 0; i < data.size() - 1; i++) {
- for (int j = i + 1; j < data.size(); j++) {
- assertEquals(data.get(i).union(data.get(j)), data.get(j).union(data.get(i)));
- }
- }
- }
-
- @Test
- public void testTwoEqualCollectionCriteriaUnion() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(Arrays.asList(new ByteArray("key1"), new ByteArray("key2")), null);
- watchAggregator.add(Arrays.asList(new ByteArray("key1"), new ByteArray("key2")), null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- var expKeyCriterion = new KeyCriterion.CollectionCriterion(
- new HashSet<>(Arrays.asList(new ByteArray("key1"), new ByteArray("key2")))
- );
- assertEquals(expKeyCriterion, keyCriterion);
- }
-
- @Test
- public void testExactInTheMiddleAndRangeCriteriaOnTheEdgesUnion() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(new ByteArray("key1"), null);
- watchAggregator.add(new ByteArray("key0"), new ByteArray("key2"), null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- var expKeyCriterion = new KeyCriterion.RangeCriterion(
- new ByteArray("key0"), new ByteArray("key2"));
- assertEquals(expKeyCriterion, keyCriterion);
- }
-
- @Test
- public void testHighExactAndLowerRangeCriteriaUnion() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(new ByteArray("key3"), null);
- watchAggregator.add(new ByteArray("key0"), new ByteArray("key2"), null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- var expKeyCriterion = new KeyCriterion.RangeCriterion(
- new ByteArray("key0"), new ByteArray("key4"));
- assertEquals(expKeyCriterion, keyCriterion);
- }
-
- @Test
- public void testNullKeyAsStartOfRangeCriterion() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(new ByteArray("key0"), null);
- watchAggregator.add(null, new ByteArray("key2"), null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- var expKeyCriterion = new KeyCriterion.RangeCriterion(
- null, new ByteArray("key2"));
- assertEquals(expKeyCriterion, keyCriterion);
- }
-
- @Test
- public void testNullKeyAsEndOfRangeCriterion() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(new ByteArray("key3"), null);
- watchAggregator.add(new ByteArray("key1"), null, null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- var expKeyCriterion = new KeyCriterion.RangeCriterion(
- new ByteArray("key1"), null);
- assertEquals(expKeyCriterion, keyCriterion);
- }
-
- @Test
- public void testAllTypesOfCriteriaUnion() {
- var watchAggregator = new WatchAggregator();
-
- watchAggregator.add(new ByteArray("key0"), null);
- watchAggregator.add(new ByteArray("key1"), new ByteArray("key2"), null);
- watchAggregator.add(Arrays.asList(new ByteArray("key0"), new ByteArray("key3")), null);
-
- var keyCriterion = watchAggregator.watch(0, null).get().keyCriterion();
- var expKeyCriterion = new KeyCriterion.RangeCriterion(
- new ByteArray("key0"), new ByteArray("key4"));
- assertEquals(expKeyCriterion, keyCriterion);
- }
-
- private Entry entry(String key, String val, long revision, long updateCntr) {
- return new Entry() {
- /** {@inheritDoc} */
- @Override
- public @NotNull byte[] key() {
- return key.getBytes(StandardCharsets.UTF_8);
- }
-
- /** {@inheritDoc} */
- @Override
- public @Nullable byte[] value() {
- return val.getBytes(StandardCharsets.UTF_8);
- }
-
- /** {@inheritDoc} */
- @Override
- public long revision() {
- return revision;
- }
-
- /** {@inheritDoc} */
- @Override
- public long updateCounter() {
- return updateCntr;
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean empty() {
- return false;
- }
-
- /** {@inheritDoc} */
- @Override
- public boolean tombstone() {
- return false;
- }
- };
- }
-}
diff --git a/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java b/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
index 855137d330..15cf0c2a0b 100644
--- a/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
+++ b/modules/metastorage/src/testFixtures/java/org/apache/ignite/internal/metastorage/server/SimpleInMemoryKeyValueStorage.java
@@ -26,8 +26,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
@@ -35,22 +33,33 @@ import java.util.NoSuchElementException;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.function.LongConsumer;
import java.util.function.Predicate;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.metastorage.Entry;
import org.apache.ignite.internal.metastorage.EntryEvent;
import org.apache.ignite.internal.metastorage.WatchEvent;
+import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.dsl.Operation;
import org.apache.ignite.internal.metastorage.dsl.StatementResult;
import org.apache.ignite.internal.metastorage.exceptions.MetaStorageException;
import org.apache.ignite.internal.metastorage.impl.EntryImpl;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
import org.apache.ignite.internal.util.Cursor;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.internal.util.IgniteUtils;
import org.jetbrains.annotations.Nullable;
/**
- * Simple in-memory key/value storage.
+ * Simple in-memory key/value storage for tests.
*/
public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
+ private static final IgniteLogger LOG = Loggers.forClass(SimpleInMemoryKeyValueStorage.class);
+
/** Lexicographical comparator. */
private static final Comparator<byte[]> CMP = Arrays::compareUnsigned;
@@ -63,7 +72,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
private NavigableMap<byte[], List<Long>> keysIdx = new TreeMap<>(CMP);
/** Revisions index. Value contains all entries which were modified under particular revision. */
- private Map<Long, NavigableMap<byte[], Value>> revsIdx = new HashMap<>();
+ private NavigableMap<Long, NavigableMap<byte[], Value>> revsIdx = new TreeMap<>();
/** Revision. Will be incremented for each single-entry or multi-entry update operation. */
private long rev;
@@ -74,6 +83,20 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
/** All operations are queued on this lock. */
private final Object mux = new Object();
+ private final List<Watch> watches = new CopyOnWriteArrayList<>();
+
+ private boolean areWatchesEnabled = false;
+
+ private LongConsumer revisionCallback;
+
+ private final ExecutorService watchExecutor;
+
+ private final List<Entry> updatedEntries = new ArrayList<>();
+
+ public SimpleInMemoryKeyValueStorage(String nodeName) {
+ this.watchExecutor = Executors.newSingleThreadExecutor(NamedThreadFactory.create(nodeName, "watch-executor", LOG));
+ }
+
/** {@inheritDoc} */
@Override
public void start() {
@@ -104,12 +127,17 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
doPut(key, value, curRev);
- rev = curRev;
+ updateRevision(curRev);
}
}
+ private void updateRevision(long newRevision) {
+ rev = newRevision;
+
+ notifyWatches(newRevision);
+ }
+
/** {@inheritDoc} */
- @NotNull
@Override
public Entry getAndPut(byte[] key, byte[] bytes) {
synchronized (mux) {
@@ -117,7 +145,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
long lastRev = doPut(key, bytes, curRev);
- rev = curRev;
+ updateRevision(curRev);
// Return previous value.
return doGetValue(key, lastRev);
@@ -135,7 +163,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
/** {@inheritDoc} */
- @NotNull
@Override
public Collection<Entry> getAndPutAll(List<byte[]> keys, List<byte[]> values) {
Collection<Entry> res;
@@ -152,7 +179,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
/** {@inheritDoc} */
- @NotNull
@Override
public Entry get(byte[] key) {
synchronized (mux) {
@@ -161,7 +187,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
/** {@inheritDoc} */
- @NotNull
@Override
public Entry get(byte[] key, long revUpperBound) {
synchronized (mux) {
@@ -170,14 +195,12 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
/** {@inheritDoc} */
- @NotNull
@Override
public Collection<Entry> getAll(List<byte[]> keys) {
return doGetAll(keys, LATEST_REV);
}
/** {@inheritDoc} */
- @NotNull
@Override
public Collection<Entry> getAll(List<byte[]> keys, long revUpperBound) {
return doGetAll(keys, revUpperBound);
@@ -190,13 +213,12 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
long curRev = rev + 1;
if (doRemove(key, curRev)) {
- rev = curRev;
+ updateRevision(curRev);
}
}
}
/** {@inheritDoc} */
- @NotNull
@Override
public Entry getAndRemove(byte[] key) {
synchronized (mux) {
@@ -237,7 +259,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
/** {@inheritDoc} */
- @NotNull
@Override
public Collection<Entry> getAndRemoveAll(List<byte[]> keys) {
Collection<Entry> res = new ArrayList<>(keys.size());
@@ -306,7 +327,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
if (modified) {
- rev = curRev;
+ updateRevision(curRev);
}
return branch;
@@ -350,7 +371,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
if (modified) {
- rev = curRev;
+ updateRevision(curRev);
}
return branch.update().result();
@@ -394,25 +415,34 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
@Override
- public Cursor<WatchEvent> watch(byte[] keyFrom, byte @Nullable [] keyTo, long rev) {
+ public void watchRange(byte[] keyFrom, byte @Nullable [] keyTo, long rev, WatchListener listener) {
assert keyFrom != null : "keyFrom couldn't be null.";
assert rev > 0 : "rev must be positive.";
- return new WatchCursor(rev, k ->
- CMP.compare(keyFrom, k) <= 0 && (keyTo == null || CMP.compare(k, keyTo) < 0)
- );
+ Predicate<byte[]> rangePredicate = keyTo == null
+ ? k -> CMP.compare(keyFrom, k) <= 0
+ : k -> CMP.compare(keyFrom, k) <= 0 && CMP.compare(keyTo, k) > 0;
+
+ watches.add(new Watch(rev, listener, rangePredicate));
}
@Override
- public Cursor<WatchEvent> watch(byte[] key, long rev) {
+ public void watchPrefix(byte[] prefix, long rev, WatchListener listener) {
+ watchRange(prefix, incrementArray(prefix), rev, listener);
+ }
+
+ @Override
+ public void watchExact(byte[] key, long rev, WatchListener listener) {
assert key != null : "key couldn't be null.";
assert rev > 0 : "rev must be positive.";
- return new WatchCursor(rev, k -> CMP.compare(k, key) == 0);
+ Predicate<byte[]> exactPredicate = k -> CMP.compare(k, key) == 0;
+
+ watches.add(new Watch(rev, listener, exactPredicate));
}
@Override
- public Cursor<WatchEvent> watch(Collection<byte[]> keys, long rev) {
+ public void watchExact(Collection<byte[]> keys, long rev, WatchListener listener) {
assert keys != null && !keys.isEmpty() : "keys couldn't be null or empty: " + keys;
assert rev > 0 : "rev must be positive.";
@@ -420,7 +450,95 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
keySet.addAll(keys);
- return new WatchCursor(rev, keySet::contains);
+ Predicate<byte[]> inPredicate = keySet::contains;
+
+ watches.add(new Watch(rev, listener, inPredicate));
+ }
+
+ @Override
+ public void startWatches(LongConsumer revisionCallback) {
+ synchronized (mux) {
+ areWatchesEnabled = true;
+
+ assert this.revisionCallback == null;
+
+ this.revisionCallback = revisionCallback;
+
+ replayUpdates();
+ }
+ }
+
+ private void replayUpdates() {
+ long minWatchRevision = watches.stream()
+ .mapToLong(Watch::targetRevision)
+ .min()
+ .orElse(-1);
+
+ if (minWatchRevision == -1) {
+ return;
+ }
+
+ revsIdx.tailMap(minWatchRevision)
+ .forEach((revision, entries) -> {
+ entries.forEach((key, value) -> {
+ var entry = new EntryImpl(key, value.bytes(), revision, value.updateCounter());
+
+ updatedEntries.add(entry);
+ });
+
+ notifyWatches(revision);
+ });
+ }
+
+ private void notifyWatches(long revision) {
+ if (!areWatchesEnabled || updatedEntries.isEmpty()) {
+ return;
+ }
+
+ // Make a local copy of the non-volatile field while we are still under the lock.
+ LongConsumer revisionCallback = this.revisionCallback;
+
+ var updatedEntriesCopy = List.copyOf(updatedEntries);
+
+ updatedEntries.clear();
+
+ watchExecutor.execute(() -> {
+ for (Watch watch : watches) {
+ var entryEvents = new ArrayList<EntryEvent>();
+
+ for (Entry newEntry : updatedEntriesCopy) {
+ byte[] newKey = newEntry.key();
+
+ if (watch.matches(newKey, revision)) {
+ Entry oldEntry = get(newKey, revision - 1);
+
+ entryEvents.add(new EntryEvent(oldEntry, newEntry));
+ }
+ }
+
+ if (!entryEvents.isEmpty()) {
+ var event = new WatchEvent(entryEvents, revision);
+
+ try {
+ watch.onUpdate(event);
+ } catch (Exception e) {
+ watch.onError(e);
+
+ LOG.error("Error occurred when processing a watch event {}, watch will be disabled", e, event);
+
+ watches.remove(watch);
+ }
+ }
+ }
+
+ // Watch processing for the new revision is done, notify the revision callback.
+ revisionCallback.accept(revision);
+ });
+ }
+
+ @Override
+ public void removeWatch(WatchListener listener) {
+ watches.removeIf(watch -> watch.listener() == listener);
}
@Override
@@ -428,7 +546,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
synchronized (mux) {
NavigableMap<byte[], List<Long>> compactedKeysIdx = new TreeMap<>(CMP);
- Map<Long, NavigableMap<byte[], Value>> compactedRevsIdx = new HashMap<>();
+ NavigableMap<Long, NavigableMap<byte[], Value>> compactedRevsIdx = new TreeMap<>();
keysIdx.forEach((key, revs) -> compactForKey(key, revs, compactedKeysIdx, compactedRevsIdx));
@@ -440,10 +558,9 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
@Override
public void close() {
- // No-op.
+ IgniteUtils.shutdownAndAwaitTermination(watchExecutor, 10, TimeUnit.SECONDS);
}
- @NotNull
@Override
public CompletableFuture<Void> snapshot(Path snapshotPath) {
throw new UnsupportedOperationException();
@@ -490,7 +607,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
}
- @NotNull
private Collection<Entry> doGetAll(List<byte[]> keys, long rev) {
assert keys != null : "keys list can't be null.";
assert !keys.isEmpty() : "keys list can't be empty.";
@@ -507,7 +623,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
return res;
}
- @NotNull
private Entry doGet(byte[] key, long revUpperBound) {
assert revUpperBound >= LATEST_REV : "Invalid arguments: [revUpperBound=" + revUpperBound + ']';
@@ -537,7 +652,7 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
* Returns maximum revision which must be less or equal to {@code upperBoundRev}. If there is no such revision then {@code -1} will be
* returned.
*
- * @param revs Revisions list.
+ * @param revs Revisions list.
* @param upperBoundRev Revision upper bound.
* @return Appropriate revision or {@code -1} if there is no such revision.
*/
@@ -555,7 +670,6 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
return -1;
}
- @NotNull
private Entry doGetValue(byte[] key, long lastRev) {
if (lastRev == 0) {
return EntryImpl.empty(key);
@@ -602,6 +716,10 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
}
);
+ var updatedEntry = new EntryImpl(key, val.tombstone() ? null : bytes, curRev, curUpdCntr);
+
+ updatedEntries.add(updatedEntry);
+
return lastRev;
}
@@ -626,10 +744,12 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
entries.put(key, val);
+ updatedEntries.add(new EntryImpl(key, bytes, curRev, curUpdCntr));
+
revsIdx.put(curRev, entries);
}
- rev = curRev;
+ updateRevision(curRev);
return curRev;
}
@@ -653,12 +773,11 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
private class RangeCursor implements Cursor<Entry> {
private final byte[] keyFrom;
- private final byte[] keyTo;
+ private final byte @Nullable [] keyTo;
private final long rev;
- private final Iterator<Entry> it;
-
+ @Nullable
private Entry nextRetEntry;
private byte[] lastRetKey;
@@ -667,148 +786,93 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
private boolean finished;
- RangeCursor(byte[] keyFrom, byte[] keyTo, long rev, boolean includeTombstones) {
+ RangeCursor(byte[] keyFrom, byte @Nullable [] keyTo, long rev, boolean includeTombstones) {
this.keyFrom = keyFrom;
this.keyTo = keyTo;
this.rev = rev;
this.includeTombstones = includeTombstones;
- this.it = createIterator();
}
/** {@inheritDoc} */
@Override
public boolean hasNext() {
- return it.hasNext();
- }
-
- /** {@inheritDoc} */
- @Override
- public Entry next() {
- return it.next();
- }
-
- /** {@inheritDoc} */
- @Override
- public void close() {
- // No-op.
- }
-
- @NotNull
- Iterator<Entry> createIterator() {
- return new Iterator<>() {
- /** {@inheritDoc} */
- @Override
- public boolean hasNext() {
- synchronized (mux) {
- while (true) {
- if (finished) {
- return false;
- }
-
- if (nextRetEntry != null) {
- return true;
- }
-
- byte[] key = lastRetKey;
-
- while (!finished || nextRetEntry == null) {
- Map.Entry<byte[], List<Long>> e =
- key == null ? keysIdx.ceilingEntry(keyFrom) : keysIdx.higherEntry(key);
-
- if (e == null) {
- finished = true;
+ synchronized (mux) {
+ while (true) {
+ if (finished) {
+ return false;
+ }
- break;
- }
+ if (nextRetEntry != null) {
+ return true;
+ }
- key = e.getKey();
+ byte[] key = lastRetKey;
- if (keyTo != null && CMP.compare(key, keyTo) >= 0) {
- finished = true;
+ while (!finished || nextRetEntry == null) {
+ Map.Entry<byte[], List<Long>> e =
+ key == null ? keysIdx.ceilingEntry(keyFrom) : keysIdx.higherEntry(key);
- break;
- }
+ if (e == null) {
+ finished = true;
- List<Long> revs = e.getValue();
+ break;
+ }
- assert revs != null && !revs.isEmpty() :
- "Revisions should not be empty or null: [revs=" + revs + ']';
+ key = e.getKey();
- long lastRev = maxRevision(revs, rev);
+ if (keyTo != null && CMP.compare(key, keyTo) >= 0) {
+ finished = true;
- if (lastRev == -1) {
- continue;
- }
+ break;
+ }
- Entry entry = doGetValue(key, lastRev);
+ List<Long> revs = e.getValue();
- if (!entry.tombstone() || includeTombstones) {
- assert !entry.empty() : "Iterator should not return empty entry.";
+ assert revs != null && !revs.isEmpty() :
+ "Revisions should not be empty or null: [revs=" + revs + ']';
- nextRetEntry = entry;
+ long lastRev = maxRevision(revs, rev);
- break;
- }
- }
+ if (lastRev == -1) {
+ continue;
}
- }
- }
-
- /** {@inheritDoc} */
- @Override
- public Entry next() {
- synchronized (mux) {
- while (true) {
- if (finished) {
- throw new NoSuchElementException();
- }
- if (nextRetEntry != null) {
- Entry e = nextRetEntry;
+ Entry entry = doGetValue(key, lastRev);
- nextRetEntry = null;
+ if (!entry.tombstone() || includeTombstones) {
+ assert !entry.empty() : "Iterator should not return empty entry.";
- lastRetKey = e.key();
+ nextRetEntry = entry;
- return e;
- } else {
- hasNext();
- }
+ break;
}
}
}
- };
+ }
}
- }
-
- /**
- * Extension of {@link Cursor}.
- */
- private class WatchCursor implements Cursor<WatchEvent> {
- private final Predicate<byte[]> predicate;
- private final Iterator<WatchEvent> it;
-
- private long lastRetRev;
+ /** {@inheritDoc} */
+ @Override
+ public Entry next() {
+ synchronized (mux) {
+ while (true) {
+ if (finished) {
+ throw new NoSuchElementException();
+ }
- private long nextRetRev = -1;
+ if (nextRetEntry != null) {
+ Entry e = nextRetEntry;
- WatchCursor(long rev, Predicate<byte[]> predicate) {
- this.predicate = predicate;
- this.lastRetRev = rev - 1;
- this.it = createIterator();
- }
+ nextRetEntry = null;
- /** {@inheritDoc} */
- @Override
- public boolean hasNext() {
- return it.hasNext();
- }
+ lastRetKey = e.key();
- /** {@inheritDoc} */
- @Override
- public WatchEvent next() {
- return it.next();
+ return e;
+ } else {
+ hasNext();
+ }
+ }
+ }
}
/** {@inheritDoc} */
@@ -816,90 +880,5 @@ public class SimpleInMemoryKeyValueStorage implements KeyValueStorage {
public void close() {
// No-op.
}
-
- @NotNull
- Iterator<WatchEvent> createIterator() {
- return new Iterator<>() {
- /** {@inheritDoc} */
- @Override
- public boolean hasNext() {
- synchronized (mux) {
- if (nextRetRev != -1) {
- return true;
- }
-
- while (true) {
- long curRev = lastRetRev + 1;
-
- NavigableMap<byte[], Value> entries = revsIdx.get(curRev);
-
- if (entries == null) {
- return false;
- }
-
- for (byte[] key : entries.keySet()) {
- if (predicate.test(key)) {
- nextRetRev = curRev;
-
- return true;
- }
- }
-
- lastRetRev++;
- }
- }
- }
-
- /** {@inheritDoc} */
- @Override
- public WatchEvent next() {
- synchronized (mux) {
- while (true) {
- if (nextRetRev != -1) {
- NavigableMap<byte[], Value> entries = revsIdx.get(nextRetRev);
-
- if (entries == null) {
- throw new NoSuchElementException();
- }
-
- List<EntryEvent> evts = new ArrayList<>(entries.size());
-
- for (Map.Entry<byte[], Value> e : entries.entrySet()) {
- byte[] key = e.getKey();
-
- Value val = e.getValue();
-
- if (predicate.test(key)) {
- Entry newEntry;
-
- if (val.tombstone()) {
- newEntry = EntryImpl.tombstone(key, nextRetRev, val.updateCounter());
- } else {
- newEntry = new EntryImpl(key, val.bytes(), nextRetRev, val.updateCounter());
- }
-
- Entry oldEntry = doGet(key, nextRetRev - 1);
-
- evts.add(new EntryEvent(oldEntry, newEntry));
- }
- }
-
- if (evts.isEmpty()) {
- continue;
- }
-
- lastRetRev = nextRetRev;
-
- nextRetRev = -1;
-
- return new WatchEvent(evts);
- } else if (!hasNext()) {
- throw new NoSuchElementException();
- }
- }
- }
- }
- };
- }
}
}
diff --git a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/service/RaftGroupService.java b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/service/RaftGroupService.java
index 731896acde..e482d5c42f 100644
--- a/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/service/RaftGroupService.java
+++ b/modules/raft-api/src/main/java/org/apache/ignite/internal/raft/service/RaftGroupService.java
@@ -179,10 +179,10 @@ public interface RaftGroupService {
*
* <p>This operation is executed on a group leader.
*
- * @param learners List of learners.
+ * @param learners Collection of learners.
* @return A future.
*/
- CompletableFuture<Void> addLearners(List<Peer> learners);
+ CompletableFuture<Void> addLearners(Collection<Peer> learners);
/**
* Removes learners.
diff --git a/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
index 3640f6e593..5f41d131c1 100644
--- a/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
+++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/RaftGroupServiceImpl.java
@@ -363,7 +363,7 @@ public class RaftGroupServiceImpl implements RaftGroupService {
}
@Override
- public CompletableFuture<Void> addLearners(List<Peer> learners) {
+ public CompletableFuture<Void> addLearners(Collection<Peer> learners) {
Peer leader = this.leader;
if (leader == null) {
diff --git a/modules/raft/src/testFixtures/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java b/modules/raft/src/testFixtures/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java
index 360f317d6f..0e448509e7 100644
--- a/modules/raft/src/testFixtures/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java
+++ b/modules/raft/src/testFixtures/java/org/apache/ignite/internal/raft/service/ItAbstractListenerSnapshotTest.java
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.raft.Loza;
import org.apache.ignite.internal.raft.PeersAndLearners;
import org.apache.ignite.internal.raft.RaftGroupServiceImpl;
import org.apache.ignite.internal.raft.RaftNodeId;
+import org.apache.ignite.internal.raft.server.RaftServer;
import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
import org.apache.ignite.internal.replicator.ReplicationGroupId;
import org.apache.ignite.internal.testframework.WorkDirectory;
@@ -193,7 +194,16 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
// Set up a raft group service
RaftGroupService service = prepareRaftGroup(testInfo);
- beforeFollowerStop(service);
+ CompletableFuture<Void> refreshLeaderFuture = service.refreshLeader()
+ .thenCompose(v -> {
+ if (service.leader() == null) {
+ return service.refreshLeader();
+ } else {
+ return CompletableFuture.completedFuture(null);
+ }
+ });
+
+ assertThat(refreshLeaderFuture, willCompleteSuccessfully());
// Select any node that is not the leader of the group
JraftServerImpl toStop = servers.stream()
@@ -201,13 +211,15 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
.findAny()
.orElseThrow();
+ beforeFollowerStop(service, toStop);
+
var nodeId = new RaftNodeId(raftGroupId(), toStop.localPeers(raftGroupId()).get(0));
// Get the path to that node's raft directory
Path serverDataPath = toStop.getServerDataPath(nodeId);
// Get the path to that node's RocksDB key-value storage
- Path dbPath = getListenerPersistencePath(getListener(toStop, raftGroupId()));
+ Path dbPath = getListenerPersistencePath(getListener(toStop, raftGroupId()), toStop);
int stopIdx = servers.indexOf(toStop);
@@ -222,7 +234,7 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
// Create a snapshot of the raft group
service.snapshot(service.leader()).get();
- afterFollowerStop(service);
+ afterFollowerStop(service, toStop);
// Create another raft snapshot
service.snapshot(service.leader()).get();
@@ -256,17 +268,19 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
* Interacts with the raft group before a follower is stopped.
*
* @param service Raft group service.
+ * @param server Raft server that is going to be stopped.
* @throws Exception If failed.
*/
- public abstract void beforeFollowerStop(RaftGroupService service) throws Exception;
+ public abstract void beforeFollowerStop(RaftGroupService service, RaftServer server) throws Exception;
/**
* Interacts with the raft group after a follower is stopped.
*
* @param service Raft group service.
+ * @param server Raft server that has been stopped.
* @throws Exception If failed.
*/
- public abstract void afterFollowerStop(RaftGroupService service) throws Exception;
+ public abstract void afterFollowerStop(RaftGroupService service, RaftServer server) throws Exception;
/**
* Interacts with a raft group after the leader has captured a snapshot.
@@ -290,9 +304,10 @@ public abstract class ItAbstractListenerSnapshotTest<T extends RaftGroupListener
* Returns path to the group's persistence.
*
* @param listener Raft group listener.
+ * @param server Raft server, where the listener has been registered.
* @return Path to the group's persistence.
*/
- public abstract Path getListenerPersistencePath(T listener);
+ public abstract Path getListenerPersistencePath(T listener, RaftServer server);
/**
* Creates raft group listener.
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java
index c82a5bd2c8..5197674999 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/ItDistributedConfigurationPropertiesTest.java
@@ -56,11 +56,11 @@ import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
import org.apache.ignite.internal.util.IgniteUtils;
import org.apache.ignite.internal.vault.VaultManager;
import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
+import org.apache.ignite.lang.NodeStoppingException;
import org.apache.ignite.network.ClusterService;
import org.apache.ignite.network.NetworkAddress;
import org.apache.ignite.network.StaticNodeFinder;
import org.apache.ignite.utils.ClusterServiceTestUtils;
-import org.jetbrains.annotations.NotNull;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -138,14 +138,14 @@ public class ItDistributedConfigurationPropertiesTest {
clusterService,
cmgManager,
raftManager,
- new SimpleInMemoryKeyValueStorage()
+ new SimpleInMemoryKeyValueStorage(name())
);
// create a custom storage implementation that is able to "lose" some storage updates
var distributedCfgStorage = new DistributedConfigurationStorage(metaStorageManager, vaultManager) {
/** {@inheritDoc} */
@Override
- public synchronized void registerConfigurationListener(@NotNull ConfigurationStorageListener listener) {
+ public synchronized void registerConfigurationListener(ConfigurationStorageListener listener) {
super.registerConfigurationListener(changedEntries -> {
if (receivesUpdates) {
return listener.onEntriesChanged(changedEntries);
@@ -168,14 +168,18 @@ public class ItDistributedConfigurationPropertiesTest {
/**
* Starts the created components.
*/
- void start() throws Exception {
+ void start() {
vaultManager.start();
Stream.of(clusterService, raftManager, cmgManager, metaStorageManager)
.forEach(IgniteComponent::start);
// deploy watches to propagate data from the metastore into the vault
- metaStorageManager.deployWatches();
+ try {
+ metaStorageManager.deployWatches();
+ } catch (NodeStoppingException e) {
+ throw new RuntimeException(e);
+ }
distributedCfgManager.start();
}
@@ -205,7 +209,7 @@ public class ItDistributedConfigurationPropertiesTest {
}
String name() {
- return clusterService.topologyService().localMember().name();
+ return clusterService.localConfiguration().getName();
}
}
@@ -240,8 +244,7 @@ public class ItDistributedConfigurationPropertiesTest {
raftConfiguration
);
- firstNode.start();
- secondNode.start();
+ Stream.of(firstNode, secondNode).parallel().forEach(Node::start);
firstNode.cmgManager.initCluster(List.of(firstNode.name()), List.of(), "cluster");
}
@@ -285,7 +288,7 @@ public class ItDistributedConfigurationPropertiesTest {
assertThat(firstValue.value(), is("bar"));
assertThat(directProxy(secondValue).value(), is("bar"));
- assertTrue(waitForCondition(() -> "bar".equals(secondValue.value()), 100));
+ assertTrue(waitForCondition(() -> "bar".equals(secondValue.value()), 1000));
// disable storage updates on the second node. This way the new values will never be propagated into the
// configuration storage
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
index 3ef167c751..7b31d379aa 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItDistributedConfigurationStorageTest.java
@@ -23,12 +23,12 @@ import static org.apache.ignite.internal.testframework.matchers.CompletableFutur
import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
+import static org.junit.jupiter.api.Assertions.assertTrue;
import java.io.Serializable;
import java.nio.file.Path;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.stream.Stream;
import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
@@ -112,7 +112,7 @@ public class ItDistributedConfigurationStorageTest {
clusterService,
cmgManager,
raftManager,
- new SimpleInMemoryKeyValueStorage()
+ new SimpleInMemoryKeyValueStorage(name())
);
cfgStorage = new DistributedConfigurationStorage(metaStorageManager, vaultManager);
@@ -150,7 +150,7 @@ public class ItDistributedConfigurationStorageTest {
}
String name() {
- return clusterService.topologyService().localMember().name();
+ return clusterService.localConfiguration().getName();
}
}
@@ -174,7 +174,7 @@ public class ItDistributedConfigurationStorageTest {
assertThat(node.cfgStorage.write(data, 0), willBe(equalTo(true)));
assertThat(node.cfgStorage.writeConfigurationRevision(0, 1), willCompleteSuccessfully());
- waitForCondition(() -> Objects.nonNull(node.vaultManager.get(MetaStorageManagerImpl.APPLIED_REV).join()), 3000);
+ assertTrue(waitForCondition(() -> node.metaStorageManager.appliedRevision() != 0, 3000));
} finally {
node.stop();
}
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
index 9cfdb9437e..fed69aedd6 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
@@ -509,7 +509,7 @@ public class ItRebalanceDistributedTest {
clusterService,
cmgManager,
raftManager,
- new SimpleInMemoryKeyValueStorage()
+ new SimpleInMemoryKeyValueStorage(clusterService.localConfiguration().getName())
);
cfgStorage = new DistributedConfigurationStorage(metaStorageManager, vaultManager);
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItDataSchemaSyncTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItDataSchemaSyncTest.java
index 2e9709d179..9ba9d228b5 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItDataSchemaSyncTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItDataSchemaSyncTest.java
@@ -164,7 +164,7 @@ public class ItDataSchemaSyncTest extends IgniteAbstractTest {
IgnitionManager.stop(nodeToStop);
- listenerInhibitor.stopWithoutResend();
+ listenerInhibitor.stopInhibit();
CompletableFuture<Ignite> ignite1Fut = nodesBootstrapCfg.entrySet().stream()
.filter(k -> k.getKey().equals(nodeToStop))
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
index cb52ced0c7..1831f1a14f 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.runner.app;
import static org.apache.ignite.internal.recovery.ConfigurationCatchUpListener.CONFIGURATION_CATCH_UP_DIFFERENCE_PROPERTY;
import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
import static org.apache.ignite.utils.ClusterServiceTestUtils.defaultSerializationRegistry;
import static org.hamcrest.MatcherAssert.assertThat;
@@ -38,6 +39,7 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.ListIterator;
+import java.util.Objects;
import java.util.ServiceLoader;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
@@ -48,6 +50,7 @@ import java.util.function.Function;
import java.util.function.IntFunction;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
+import java.util.stream.Stream;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgnitionManager;
import org.apache.ignite.internal.app.IgniteImpl;
@@ -263,7 +266,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
clusterSvc,
cmgManager,
raftMgr,
- new RocksDbKeyValueStorage(dir.resolve("metastorage"))
+ new RocksDbKeyValueStorage(name, dir.resolve("metastorage"))
);
var cfgStorage = new DistributedConfigurationStorage(metaStorageMgr, vault);
@@ -575,7 +578,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
return (IgniteImpl) future.join();
})
- .collect(Collectors.toUnmodifiableList());
+ .collect(Collectors.toList());
}
/**
@@ -688,10 +691,10 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
* Restarts the node which stores some data.
*/
@Test
- public void nodeWithDataTest() {
- Ignite ignite = startNode(0);
+ public void nodeWithDataTest() throws InterruptedException {
+ IgniteImpl ignite = startNode(0);
- createTableWithData(ignite, TABLE_NAME, 1);
+ createTableWithData(List.of(ignite), TABLE_NAME, 1);
stopNode(0);
@@ -704,7 +707,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
* Starts two nodes and checks that the data are storing through restarts. Nodes restart in the same order when they started at first.
*/
@Test
- public void testTwoNodesRestartDirect() {
+ public void testTwoNodesRestartDirect() throws InterruptedException {
twoNodesRestart(true);
}
@@ -712,7 +715,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
* Starts two nodes and checks that the data are storing through restarts. Nodes restart in reverse order when they started at first.
*/
@Test
- public void testTwoNodesRestartReverse() {
+ public void testTwoNodesRestartReverse() throws InterruptedException {
twoNodesRestart(false);
}
@@ -721,17 +724,17 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
*
* @param directOrder When the parameter is true, nodes restart in direct order, otherwise they restart in reverse order.
*/
- private void twoNodesRestart(boolean directOrder) {
- Ignite ignite = startNode(0);
-
- startNode(1);
+ private void twoNodesRestart(boolean directOrder) throws InterruptedException {
+ List<IgniteImpl> nodes = startNodes(2);
- createTableWithData(ignite, TABLE_NAME, 2);
- createTableWithData(ignite, TABLE_NAME_2, 2);
+ createTableWithData(nodes, TABLE_NAME, 2);
+ createTableWithData(nodes, TABLE_NAME_2, 2);
stopNode(0);
stopNode(1);
+ Ignite ignite;
+
if (directOrder) {
startNode(0);
ignite = startNode(1);
@@ -797,12 +800,12 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
* checks that the table created before node stop, is not available when majority if lost.
*/
@Test
- public void testOneNodeRestartWithGap() {
- Ignite ignite = startNode(0);
+ public void testOneNodeRestartWithGap() throws InterruptedException {
+ IgniteImpl ignite = startNode(0);
List<IgniteComponent> components = startPartialNode(1, null);
- createTableWithData(ignite, TABLE_NAME, 2);
+ createTableWithData(List.of(ignite), TABLE_NAME, 2);
stopPartialNode(components);
@@ -828,12 +831,12 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
* Checks that the table created in cluster of 2 nodes, is recovered on a node after restart of this node.
*/
@Test
- public void testRecoveryOnOneNode() {
- Ignite ignite = startNode(0);
+ public void testRecoveryOnOneNode() throws InterruptedException {
+ IgniteImpl ignite = startNode(0);
List<IgniteComponent> components = startPartialNode(1, null);
- createTableWithData(ignite, TABLE_NAME, 2, 1);
+ createTableWithData(List.of(ignite), TABLE_NAME, 2, 1);
stopPartialNode(components);
@@ -850,13 +853,11 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
* Checks that a cluster is able to restart when some changes were made in configuration.
*/
@Test
- public void testRestartDiffConfig() {
+ public void testRestartDiffConfig() throws InterruptedException {
List<IgniteImpl> ignites = startNodes(2);
- Ignite ignite0 = ignites.get(0);
-
- createTableWithData(ignite0, TABLE_NAME, 2);
- createTableWithData(ignite0, TABLE_NAME_2, 2);
+ createTableWithData(ignites, TABLE_NAME, 2);
+ createTableWithData(ignites, TABLE_NAME_2, 2);
stopNode(0);
stopNode(1);
@@ -880,17 +881,19 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
*/
@Test
@WithSystemProperty(key = CONFIGURATION_CATCH_UP_DIFFERENCE_PROPERTY, value = "0")
- public void testCfgGapWithoutData() {
+ public void testCfgGapWithoutData() throws InterruptedException {
List<IgniteImpl> nodes = startNodes(3);
- createTableWithData(nodes.get(0), TABLE_NAME, nodes.size());
+ createTableWithData(nodes, TABLE_NAME, nodes.size());
log.info("Stopping the node.");
stopNode(nodes.size() - 1);
- createTableWithData(nodes.get(0), TABLE_NAME_2, nodes.size());
- createTableWithData(nodes.get(0), TABLE_NAME_2 + "0", nodes.size());
+ nodes.set(nodes.size() - 1, null);
+
+ createTableWithData(nodes, TABLE_NAME_2, nodes.size());
+ createTableWithData(nodes, TABLE_NAME_2 + "0", nodes.size());
log.info("Starting the node.");
@@ -909,7 +912,7 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
*/
@Test
@WithSystemProperty(key = CONFIGURATION_CATCH_UP_DIFFERENCE_PROPERTY, value = "0")
- public void testMetastorageStop() {
+ public void testMetastorageStop() throws InterruptedException {
int cfgGap = 4;
List<IgniteImpl> nodes = startNodes(3);
@@ -918,8 +921,10 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
stopNode(nodes.size() - 1);
+ nodes.set(nodes.size() - 1, null);
+
for (int i = 0; i < cfgGap; i++) {
- createTableWithData(nodes.get(0), "t" + i, nodes.size(), 1);
+ createTableWithData(nodes, "t" + i, nodes.size(), 1);
}
log.info("Starting the node.");
@@ -956,18 +961,20 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
*/
@Test
@Disabled("https://issues.apache.org/jira/browse/IGNITE-17770")
- public void testCfgGap() {
+ public void testCfgGap() throws InterruptedException {
List<IgniteImpl> nodes = startNodes(4);
- createTableWithData(nodes.get(0), "t1", nodes.size());
+ createTableWithData(nodes, "t1", nodes.size());
log.info("Stopping the node.");
stopNode(nodes.size() - 1);
+ nodes.set(nodes.size() - 1, null);
+
checkTableWithData(nodes.get(0), "t1");
- createTableWithData(nodes.get(0), "t2", nodes.size());
+ createTableWithData(nodes, "t2", nodes.size());
log.info("Starting the node.");
@@ -1001,27 +1008,30 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
/**
* Creates a table and load data to it.
*
- * @param ignite Ignite.
+ * @param nodes Ignite nodes.
* @param name Table name.
* @param replicas Replica factor.
*/
- private static void createTableWithData(Ignite ignite, String name, int replicas) {
- createTableWithData(ignite, name, replicas, 10);
+ private void createTableWithData(List<IgniteImpl> nodes, String name, int replicas) throws InterruptedException {
+ createTableWithData(nodes, name, replicas, 10);
}
/**
* Creates a table and load data to it.
*
- * @param ignite Ignite.
+ * @param nodes Ignite nodes.
* @param name Table name.
* @param replicas Replica factor.
* @param partitions Partitions count.
*/
- private static void createTableWithData(Ignite ignite, String name, int replicas, int partitions) {
- try (Session session = ignite.sql().createSession()) {
+ private void createTableWithData(List<IgniteImpl> nodes, String name, int replicas, int partitions)
+ throws InterruptedException {
+ try (Session session = nodes.get(0).sql().createSession()) {
session.execute(null, "CREATE TABLE " + name
+ "(id INT PRIMARY KEY, name VARCHAR) WITH replicas=" + replicas + ", partitions=" + partitions);
+ waitForIndex(nodes, name + "_PK");
+
for (int i = 0; i < 100; i++) {
session.execute(null, "INSERT INTO " + name + "(id, name) VALUES (?, ?)",
i, VALUE_PRODUCER.apply(i));
@@ -1029,6 +1039,37 @@ public class ItIgniteNodeRestartTest extends IgniteAbstractTest {
}
}
+ private void waitForIndex(Collection<IgniteImpl> nodes, String indexName) throws InterruptedException {
+ // FIXME: Wait for the index to be created on all nodes,
+ // this is a workaround for https://issues.apache.org/jira/browse/IGNITE-18203 to avoid missed updates to the PK index.
+
+ Stream<TablesConfiguration> partialTablesConfiguration = Stream.empty();
+
+ if (partialNode != null) {
+ partialTablesConfiguration = partialNode.stream()
+ .filter(ConfigurationManager.class::isInstance)
+ .map(c -> ((ConfigurationManager) c).configurationRegistry().getConfiguration(TablesConfiguration.KEY))
+ .filter(Objects::nonNull)
+ .findAny()
+ .map(Stream::of)
+ .orElseThrow();
+ }
+
+ Stream<TablesConfiguration> nodesTablesConfigurations = nodes.stream()
+ .filter(Objects::nonNull)
+ .map(node -> node.clusterConfiguration().getConfiguration(TablesConfiguration.KEY));
+
+ List<TablesConfiguration> tablesConfigurations = Stream.concat(nodesTablesConfigurations, partialTablesConfiguration)
+ .collect(Collectors.toList());
+
+ assertTrue(waitForCondition(
+ () -> tablesConfigurations.stream()
+ .map(cfg -> cfg.indexes().get(indexName.toUpperCase()))
+ .allMatch(Objects::nonNull),
+ 10_000
+ ));
+ }
+
/**
* Creates a table.
*
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
index 13ebf99dad..ba9eb80043 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
@@ -158,7 +158,6 @@ public class ItTablesApiTest extends IgniteAbstractTest {
* @throws Exception If failed.
*/
@Test
- @Disabled("https://issues.apache.org/jira/browse/IGNITE-18429")
public void testTableAlreadyCreatedFromLaggedNode() throws Exception {
clusterNodes.forEach(ign -> assertNull(ign.tables().table(TABLE_NAME)));
@@ -395,7 +394,6 @@ public class ItTablesApiTest extends IgniteAbstractTest {
* @throws Exception If failed.
*/
@Test
- @Disabled("https://issues.apache.org/jira/browse/IGNITE-18429")
public void testCreateDropTable() throws Exception {
clusterNodes.forEach(ign -> assertNull(ign.tables().table(TABLE_NAME)));
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
index 113f4e91b7..e9d0766518 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexSpoolTest.java
@@ -18,14 +18,22 @@
package org.apache.ignite.internal.sql.engine;
import static org.apache.ignite.internal.sql.engine.util.Commons.IN_BUFFER_SIZE;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.List;
+import java.util.Objects;
import java.util.stream.Stream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.app.IgniteImpl;
import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.schema.configuration.index.TableIndexConfiguration;
import org.apache.ignite.table.Table;
+import org.jetbrains.annotations.Nullable;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
@@ -71,7 +79,7 @@ public class ItIndexSpoolTest extends AbstractBasicIntegrationTest {
*/
@ParameterizedTest(name = "tableSize={0}, partitions={1}")
@MethodSource("rowsWithPartitionsArgs")
- public void test(int rows, int partitions) {
+ public void test(int rows, int partitions) throws InterruptedException {
prepareDataSet(rows, partitions);
var res = sql("SELECT /*+ DISABLE_RULE('NestedLoopJoinConverter', 'MergeJoinConverter') */"
@@ -84,7 +92,7 @@ public class ItIndexSpoolTest extends AbstractBasicIntegrationTest {
res.forEach(r -> assertThat(r.get(0), is(r.get(1))));
}
- private void prepareDataSet(int rowsCount, int parts) {
+ private void prepareDataSet(int rowsCount, int parts) throws InterruptedException {
Object[][] dataRows = new Object[rowsCount][];
for (int i = 0; i < rowsCount; i++) {
@@ -94,10 +102,28 @@ public class ItIndexSpoolTest extends AbstractBasicIntegrationTest {
for (String name : List.of("TEST0", "TEST1")) {
sql(String.format("CREATE TABLE " + name + "(id INT PRIMARY KEY, jid INT, val VARCHAR) WITH replicas=2,partitions=%d", parts));
+ waitForIndex(name + "_PK");
+
// TODO: https://issues.apache.org/jira/browse/IGNITE-17304 uncomment this
// sql("CREATE INDEX " + name + "_jid_idx ON " + name + "(jid)");
insertData(name, List.of("ID", "JID", "VAL"), dataRows);
}
}
+
+ private static void waitForIndex(String indexName) throws InterruptedException {
+ // FIXME: Wait for the index to be created on all nodes,
+ // this is a workaround for https://issues.apache.org/jira/browse/IGNITE-18203 to avoid missed updates to the PK index.
+ assertTrue(waitForCondition(
+ () -> CLUSTER_NODES.stream().map(node -> getIndexConfiguration(node, indexName)).allMatch(Objects::nonNull),
+ 10_000)
+ );
+ }
+
+ private static @Nullable TableIndexConfiguration getIndexConfiguration(Ignite node, String indexName) {
+ return ((IgniteImpl) node).clusterConfiguration()
+ .getConfiguration(TablesConfiguration.KEY)
+ .indexes()
+ .get(indexName.toUpperCase());
+ }
}
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
index 8b2b3e777e..b2498853ba 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
@@ -23,9 +23,17 @@ import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsIn
import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsSubPlan;
import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsTableScan;
import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsUnion;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
import static org.hamcrest.Matchers.not;
+import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.List;
+import java.util.Objects;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.schema.configuration.index.TableIndexConfiguration;
+import org.jetbrains.annotations.Nullable;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
@@ -46,12 +54,16 @@ public class ItSecondaryIndexTest extends AbstractBasicIntegrationTest {
* Before all.
*/
@BeforeAll
- static void initTestData() {
+ static void initTestData() throws InterruptedException {
sql("CREATE TABLE developer (id INT PRIMARY KEY, name VARCHAR, depid INT, city VARCHAR, age INT)");
sql("CREATE INDEX " + DEPID_IDX + " ON developer (depid)");
sql("CREATE INDEX " + NAME_CITY_IDX + " ON developer (name DESC, city DESC)");
sql("CREATE INDEX " + NAME_DEPID_CITY_IDX + " ON developer (name DESC, depid DESC, city DESC)");
+ waitForIndex(DEPID_IDX);
+ waitForIndex(NAME_CITY_IDX);
+ waitForIndex(NAME_DEPID_CITY_IDX);
+
insertData("DEVELOPER", List.of("ID", "NAME", "DEPID", "CITY", "AGE"), new Object[][]{
{1, "Mozart", 3, "Vienna", 33},
{2, "Beethoven", 2, "Vienna", 44},
@@ -81,6 +93,8 @@ public class ItSecondaryIndexTest extends AbstractBasicIntegrationTest {
sql("CREATE TABLE unwrap_pk(f1 VARCHAR, f2 BIGINT, f3 BIGINT, f4 BIGINT, primary key(f2, f1))");
sql("CREATE INDEX " + PK_SORTED_IDX + " ON unwrap_pk(f2, f1)");
+ waitForIndex(PK_SORTED_IDX);
+
insertData("UNWRAP_PK", List.of("F1", "F2", "F3", "F4"), new Object[][]{
{"Petr", 1L, 2L, 3L},
{"Ivan", 2L, 2L, 4L},
@@ -94,6 +108,8 @@ public class ItSecondaryIndexTest extends AbstractBasicIntegrationTest {
sql("CREATE TABLE t1 (id INT PRIMARY KEY, val INT)");
sql("CREATE INDEX t1_idx on t1(val DESC)");
+ waitForIndex("t1_idx");
+
insertData("T1", List.of("ID", "VAL"), new Object[][]{
{1, null},
{2, null},
@@ -105,6 +121,22 @@ public class ItSecondaryIndexTest extends AbstractBasicIntegrationTest {
});
}
+ private static void waitForIndex(String indexName) throws InterruptedException {
+ // FIXME: Wait for the sorted index to be created on all nodes,
+ // this is a workaround for https://issues.apache.org/jira/browse/IGNITE-18203 to avoid missed updates to the sorted index.
+ assertTrue(waitForCondition(
+ () -> CLUSTER_NODES.stream().map(node -> getSortedIndexConfig(node, indexName)).allMatch(Objects::nonNull),
+ 10_000)
+ );
+ }
+
+ private static @Nullable TableIndexConfiguration getSortedIndexConfig(Ignite node, String indexName) {
+ return ((IgniteImpl) node).clusterConfiguration()
+ .getConfiguration(TablesConfiguration.KEY)
+ .indexes()
+ .get(indexName.toUpperCase());
+ }
+
@Test
public void testEqualsFilterWithUnwrpKey() {
assertQuery("SELECT F1 FROM UNWRAP_PK WHERE F2=2")
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
index a9ed2853a0..e8e59e8acb 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.table;
import static org.apache.ignite.internal.index.SortedIndex.INCLUDE_LEFT;
import static org.apache.ignite.internal.index.SortedIndex.INCLUDE_RIGHT;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -30,6 +31,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.ArrayList;
import java.util.List;
+import java.util.Objects;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Flow.Publisher;
@@ -39,6 +41,7 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
import org.apache.ignite.internal.app.IgniteImpl;
import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
import org.apache.ignite.internal.schema.BinaryRow;
@@ -50,6 +53,7 @@ import org.apache.ignite.internal.schema.Column;
import org.apache.ignite.internal.schema.NativeTypes;
import org.apache.ignite.internal.schema.SchemaDescriptor;
import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.schema.configuration.index.TableIndexConfiguration;
import org.apache.ignite.internal.schema.row.Row;
import org.apache.ignite.internal.schema.row.RowAssembler;
import org.apache.ignite.internal.sql.engine.AbstractBasicIntegrationTest;
@@ -59,6 +63,7 @@ import org.apache.ignite.lang.IgniteStringFormatter;
import org.apache.ignite.table.KeyValueView;
import org.apache.ignite.table.Tuple;
import org.apache.ignite.tx.IgniteTransactions;
+import org.jetbrains.annotations.Nullable;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
@@ -87,9 +92,16 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
);
@BeforeEach
- public void beforeTest() {
+ public void beforeTest() throws InterruptedException {
TableImpl table = getOrCreateTable();
+ // FIXME: Wait for the sorted index to be created on all nodes,
+ // this is a workaround for https://issues.apache.org/jira/browse/IGNITE-18203 to avoid missed updates to the sorted index.
+ assertTrue(waitForCondition(
+ () -> CLUSTER_NODES.stream().map(ItTableScanTest::getSortedIndexConfig).allMatch(Objects::nonNull),
+ 10_000)
+ );
+
loadData(table);
}
@@ -120,7 +132,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
subscription.request(2);
- IgniteTestUtils.waitForCondition(() -> scannedRows.size() == 2, 10_000);
+ waitForCondition(() -> scannedRows.size() == 2, 10_000);
assertEquals(2, scannedRows.size());
assertFalse(scanned.isDone());
@@ -162,7 +174,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
subscription.request(1);
- IgniteTestUtils.waitForCondition(() -> !scannedRows.isEmpty(), 10_000);
+ waitForCondition(() -> !scannedRows.isEmpty(), 10_000);
assertEquals(1, scannedRows.size());
assertFalse(scanned.isDone());
@@ -406,7 +418,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
subscription.request(1);
- IgniteTestUtils.waitForCondition(() -> !scannedRows.isEmpty(), 10_000);
+ waitForCondition(() -> !scannedRows.isEmpty(), 10_000);
assertEquals(1, scannedRows.size());
assertFalse(scanned.isDone());
@@ -417,7 +429,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
subscription.request(2);
- IgniteTestUtils.waitForCondition(() -> scannedRows.size() == 3, 10_000);
+ waitForCondition(() -> scannedRows.size() == 3, 10_000);
assertEquals(3, scannedRows.size());
assertFalse(scanned.isDone());
@@ -462,7 +474,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
subscription.request(3);
- IgniteTestUtils.waitForCondition(() -> scannedRows.size() == 3, 10_000);
+ waitForCondition(() -> scannedRows.size() == 3, 10_000);
assertEquals(3, scannedRows.size());
assertFalse(scanned.isDone());
@@ -597,7 +609,7 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
subscription.request(1_000); // Request so much entries here to close the publisher.
- IgniteTestUtils.waitForCondition(() -> scanned.isDone(), 10_000);
+ waitForCondition(() -> scanned.isDone(), 10_000);
return scannedRows;
}
@@ -638,8 +650,14 @@ public class ItTableScanTest extends AbstractBasicIntegrationTest {
* @return Index id.
*/
private static UUID getSortedIndexId() {
- return ((IgniteImpl) CLUSTER_NODES.get(0)).clusterConfiguration().getConfiguration(TablesConfiguration.KEY).indexes()
- .get(SORTED_IDX.toUpperCase()).id().value();
+ return getSortedIndexConfig(CLUSTER_NODES.get(0)).id().value();
+ }
+
+ private static @Nullable TableIndexConfiguration getSortedIndexConfig(Ignite node) {
+ return ((IgniteImpl) node).clusterConfiguration()
+ .getConfiguration(TablesConfiguration.KEY)
+ .indexes()
+ .get(SORTED_IDX.toUpperCase());
}
/**
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/test/WatchListenerInhibitor.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/test/WatchListenerInhibitor.java
index 3d06f9c8f9..ce220e5af8 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/test/WatchListenerInhibitor.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/test/WatchListenerInhibitor.java
@@ -17,144 +17,82 @@
package org.apache.ignite.internal.test;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.Mockito.doAnswer;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.getFieldValue;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Optional;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
import org.apache.ignite.Ignite;
import org.apache.ignite.internal.app.IgniteImpl;
import org.apache.ignite.internal.metastorage.WatchEvent;
import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
-import org.apache.ignite.internal.metastorage.watch.AggregatedWatch;
-import org.apache.ignite.internal.metastorage.watch.WatchAggregator;
-import org.apache.ignite.internal.testframework.IgniteTestUtils;
-import org.jetbrains.annotations.NotNull;
-import org.junit.platform.commons.util.ReflectionUtils;
-import org.mockito.Mockito;
+import org.apache.ignite.internal.metastorage.server.Watch;
+import org.apache.ignite.internal.metastorage.server.persistence.RocksDbKeyValueStorage;
/**
* Listener which wraps another one to inhibit events.
*/
-public class WatchListenerInhibitor implements WatchListener {
- /** Inhibited events. Guarded by {@code this}. */
- private final Collection<WatchEvent> inhibitEvents = new ArrayList<>();
+public class WatchListenerInhibitor {
+ /** "watches" field captured from the {@link RocksDbKeyValueStorage} instance. */
+ private final List<Watch> watches;
- /** Inhibit flag. Guarded by {@code this}. */
- private boolean inhibit = false;
-
- /** Wrapped listener. Guarded by {@code this}. */
- private WatchListener realListener;
+ /** Latch used to block the watch notification thread. */
+ private final CountDownLatch inhibitLatch = new CountDownLatch(1);
/**
* Creates the specific listener which can inhibit events for real metastorage listener.
*
* @param ignite Ignite.
* @return Listener inhibitor.
- * @throws Exception If something wrong when creating the listener inhibitor.
*/
- public static WatchListenerInhibitor metastorageEventsInhibitor(Ignite ignite)
- throws Exception {
+ public static WatchListenerInhibitor metastorageEventsInhibitor(Ignite ignite) {
//TODO: IGNITE-15723 After a component factory will be implemented, need to got rid of reflection here.
- MetaStorageManagerImpl metaMngr = (MetaStorageManagerImpl) ReflectionUtils.tryToReadFieldValue(
- IgniteImpl.class,
- "metaStorageMgr",
- (IgniteImpl) ignite
- ).get();
-
- assertNotNull(metaMngr);
-
- WatchAggregator aggregator = (WatchAggregator) ReflectionUtils.tryToReadFieldValue(
- MetaStorageManagerImpl.class,
- "watchAggregator",
- metaMngr
- ).get();
-
- assertNotNull(aggregator);
-
- WatchAggregator aggregatorSpy = Mockito.spy(aggregator);
-
- WatchListenerInhibitor inhibitor = new WatchListenerInhibitor();
-
- doAnswer(mock -> {
- Optional<AggregatedWatch> op = (Optional<AggregatedWatch>) mock.callRealMethod();
-
- assertTrue(op.isPresent());
+ var metaStorageManager = (MetaStorageManagerImpl) getFieldValue(ignite, IgniteImpl.class, "metaStorageMgr");
- inhibitor.setRealListener(op.get().listener());
+ var storage = (RocksDbKeyValueStorage) getFieldValue(metaStorageManager, MetaStorageManagerImpl.class, "storage");
- return Optional.of(new AggregatedWatch(op.get().keyCriterion(), op.get().revision(),
- inhibitor));
- }).when(aggregatorSpy).watch(anyLong(), any());
+ var watches = (List<Watch>) getFieldValue(storage, RocksDbKeyValueStorage.class, "watches");
- IgniteTestUtils.setFieldValue(metaMngr, "watchAggregator", aggregatorSpy);
-
- // Redeploy metastorage watch. The Watch inhibitor will be used after.
- metaMngr.unregisterWatch(-1);
-
- return inhibitor;
+ return new WatchListenerInhibitor(watches);
}
- /**
- * Default constructor.
- */
- private WatchListenerInhibitor() {
+ private WatchListenerInhibitor(List<Watch> watches) {
+ this.watches = watches;
}
/**
- * Sets a wrapped listener.
- *
- * @param realListener Listener to wrap.
+ * Starts inhibiting events.
*/
- private synchronized void setRealListener(WatchListener realListener) {
- this.realListener = realListener;
- }
-
- /** {@inheritDoc} */
- @Override public synchronized boolean onUpdate(@NotNull WatchEvent evt) {
- if (!inhibit) {
- return realListener.onUpdate(evt);
- }
-
- return inhibitEvents.add(evt);
- }
-
- /** {@inheritDoc} */
- @Override public synchronized void onError(@NotNull Throwable e) {
- realListener.onError(e);
- }
-
- /**
- * Starts inhibit events.
- */
- public synchronized void startInhibit() {
- inhibit = true;
+ public void startInhibit() {
+ // Inject a watch that matches all keys and revisions and blocks the watch notification thread until the latch is released.
+ var blockingWatch = new Watch(
+ 0,
+ new WatchListener() {
+ @Override
+ public void onUpdate(WatchEvent event) {
+ try {
+ inhibitLatch.await();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ }
+ },
+ key -> true
+ );
+
+ watches.add(0, blockingWatch);
}
/**
- * Stops inhibit events.
+ * Stops inhibiting events.
*/
- public synchronized void stopInhibit() {
- inhibit = false;
-
- for (WatchEvent evt : inhibitEvents) {
- realListener.onUpdate(evt);
- }
-
- inhibitEvents.clear();
- }
-
- /**
- * Stops silently, no events resend.
- */
- public synchronized void stopWithoutResend() {
- inhibit = false;
-
- inhibitEvents.clear();
+ public void stopInhibit() {
+ inhibitLatch.countDown();
}
}
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index e9d936e4ce..6d40dc0e95 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -342,7 +342,7 @@ public class IgniteImpl implements Ignite {
clusterSvc,
cmgMgr,
raftMgr,
- new RocksDbKeyValueStorage(workDir.resolve(METASTORAGE_DB_PATH))
+ new RocksDbKeyValueStorage(name, workDir.resolve(METASTORAGE_DB_PATH))
);
placementDriverMgr = new PlacementDriverManager(metaStorageMgr);
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
index 5c3b27e9d9..3964041b1f 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorage.java
@@ -27,6 +27,7 @@ import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
@@ -46,7 +47,6 @@ import org.apache.ignite.internal.metastorage.dsl.Conditions;
import org.apache.ignite.internal.metastorage.dsl.Operation;
import org.apache.ignite.internal.metastorage.dsl.Operations;
import org.apache.ignite.internal.metastorage.dsl.SimpleCondition;
-import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
import org.apache.ignite.internal.thread.NamedThreadFactory;
import org.apache.ignite.internal.util.ByteUtils;
import org.apache.ignite.internal.util.Cursor;
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.util.IgniteUtils;
import org.apache.ignite.internal.vault.VaultEntry;
import org.apache.ignite.internal.vault.VaultManager;
import org.apache.ignite.lang.ByteArray;
-import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
/**
@@ -108,12 +107,11 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
* revision of {@link #MASTER_KEY}.
*
* <p>This is true for all cases except for node restart. Key-specific revision values are lost on local vault copy after restart, so
- * stored {@link MetaStorageManagerImpl#APPLIED_REV} value is used instead. This fact has very important side effect: it's no longer
+ * stored {@link MetaStorageManager#appliedRevision} value is used instead. This fact has very important side effect: it's no longer
* possible to use {@link ConditionType#REV_EQUAL} on {@link #MASTER_KEY}
* in {@link DistributedConfigurationStorage#write(Map, long)}. {@link ConditionType#REV_LESS_OR_EQUAL} must be used instead.
*
* @see #MASTER_KEY
- * @see MetaStorageManagerImpl#APPLIED_REV
* @see #write(Map, long)
*/
private final AtomicLong changeId = new AtomicLong(0L);
@@ -196,8 +194,8 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
/** {@inheritDoc} */
@Override
public CompletableFuture<Data> readDataOnRecovery() throws StorageException {
- CompletableFuture<Data> future = vaultMgr.get(MetaStorageManagerImpl.APPLIED_REV)
- .thenCombine(vaultMgr.get(CONFIGURATION_REVISIONS_KEY), this::resolveRevision)
+ CompletableFuture<Data> future = vaultMgr.get(CONFIGURATION_REVISIONS_KEY)
+ .thenApply(configurationRevision -> resolveRevision(metaStorageMgr.appliedRevision(), configurationRevision))
.thenApplyAsync(this::readDataOnRecovery0, threadPool);
return registerFuture(future);
@@ -207,13 +205,11 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
* Resolves current configuration revision based on the saved in the Vault revision of the metastorage and also
* previous and current revisions of the configuration saved in the Vault.
*
- * @param appliedRevEntry Applied revision entry.
+ * @param metaStorageRevision Meta Storage revision.
* @param revisionsEntry Configuration revisions entry.
* @return Configuration revision.
*/
- private long resolveRevision(@Nullable VaultEntry appliedRevEntry, @Nullable VaultEntry revisionsEntry) {
- long appliedRevision = appliedRevEntry == null ? 0L : ByteUtils.bytesToLong(appliedRevEntry.value());
-
+ private static long resolveRevision(long metaStorageRevision, @Nullable VaultEntry revisionsEntry) {
if (revisionsEntry != null) {
byte[] value = revisionsEntry.value();
long prevMasterKeyRevision = ByteUtils.bytesToLong(value, 0);
@@ -221,7 +217,7 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
// If current master key revision is higher than applied revision, then node failed
// before applied revision changed, so we have to use previous master key revision
- return curMasterKeyRevision <= appliedRevision ? curMasterKeyRevision : prevMasterKeyRevision;
+ return curMasterKeyRevision <= metaStorageRevision ? curMasterKeyRevision : prevMasterKeyRevision;
} else {
// Configuration has not been updated yet, so it is safe to return 0 as the revision for the master key.
return 0L;
@@ -302,8 +298,10 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
// TODO: properly handle such cases https://issues.apache.org/jira/browse/IGNITE-14604
metaStorageMgr.registerPrefixWatch(DST_KEYS_START_RANGE, new WatchListener() {
@Override
- public boolean onUpdate(@NotNull WatchEvent events) {
- Map<String, Serializable> data = new HashMap<>();
+ public void onUpdate(WatchEvent events) {
+ var data = new HashMap<String, Serializable>();
+
+ var vaultData = new HashMap<ByteArray, byte[]>();
Entry masterKeyEntry = null;
@@ -319,6 +317,8 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
data.put(key, value);
}
+
+ vaultData.put(new ByteArray(e.key()), e.value());
}
// Contract of meta storage ensures that all updates of one revision will come in one batch.
@@ -332,13 +332,21 @@ public class DistributedConfigurationStorage implements ConfigurationStorage {
changeId.set(newChangeId);
- lsnr.onEntriesChanged(new Data(data, newChangeId)).join();
+ try {
+ vaultMgr.putAll(vaultData)
+ .thenCompose(v -> lsnr.onEntriesChanged(new Data(data, newChangeId)))
+ .get();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
- return true;
+ throw new StorageException("Interrupted when processing Meta Storage watch", e);
+ } catch (ExecutionException e) {
+ throw new StorageException("Error when processing Meta Storage watch", e);
+ }
}
@Override
- public void onError(@NotNull Throwable e) {
+ public void onError(Throwable e) {
// TODO: need to handle this case and there should some mechanism for registering new watch as far as
// TODO: onError unregisters failed watch https://issues.apache.org/jira/browse/IGNITE-14604
LOG.warn("Meta storage listener issue", e);
diff --git a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
index c7b17fb09a..4ef2cc87aa 100644
--- a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
+++ b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationCatchUpTest.java
@@ -21,6 +21,7 @@ import static org.apache.ignite.configuration.annotation.ConfigurationType.DISTR
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -42,8 +43,6 @@ import org.apache.ignite.internal.metastorage.WatchEvent;
import org.apache.ignite.internal.metastorage.WatchListener;
import org.apache.ignite.internal.metastorage.dsl.Operation;
import org.apache.ignite.internal.metastorage.impl.EntryImpl;
-import org.apache.ignite.internal.metastorage.impl.MetaStorageManagerImpl;
-import org.apache.ignite.internal.util.ByteUtils;
import org.apache.ignite.internal.vault.VaultManager;
import org.apache.ignite.internal.vault.inmemory.InMemoryVaultService;
import org.apache.ignite.lang.ByteArray;
@@ -124,7 +123,7 @@ public class DistributedConfigurationCatchUpTest {
vaultManager.put(MetaStorageMockWrapper.TEST_KEY, new byte[]{4, 1, 2, 3, 4}).get();
// This emulates a change in MetaStorage that is not related to the configuration.
- vaultManager.put(MetaStorageManagerImpl.APPLIED_REV, ByteUtils.longToBytes(2)).get();
+ when(wrapper.mock.appliedRevision()).thenReturn(1L);
storage = storage(wrapper);
@@ -196,11 +195,11 @@ public class DistributedConfigurationCatchUpTest {
.then(invocation -> triggerStorageListener());
// This captures the listener.
- when(mock.registerPrefixWatch(any(), any())).then(invocation -> {
+ doAnswer(invocation -> {
lsnr = invocation.getArgument(1);
- return CompletableFuture.completedFuture(null);
- });
+ return null;
+ }).when(mock).registerPrefixWatch(any(), any());
}
/**
diff --git a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorageTest.java b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorageTest.java
index 1b76c29845..a4580144ff 100644
--- a/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorageTest.java
+++ b/modules/runner/src/test/java/org/apache/ignite/internal/configuration/storage/DistributedConfigurationStorageTest.java
@@ -45,7 +45,7 @@ import org.junit.jupiter.api.BeforeEach;
public class DistributedConfigurationStorageTest extends ConfigurationStorageTest {
private final VaultManager vaultManager = new VaultManager(new InMemoryVaultService());
- private final KeyValueStorage metaStorage = new SimpleInMemoryKeyValueStorage();
+ private final KeyValueStorage metaStorage = new SimpleInMemoryKeyValueStorage("test");
private final MetaStorageManager metaStorageManager = mockMetaStorageManager();
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
index 55863ca962..7c52a4325a 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
@@ -38,6 +38,7 @@ import java.util.function.Function;
import org.apache.ignite.internal.hlc.HybridClock;
import org.apache.ignite.internal.hlc.HybridClockImpl;
import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.raft.server.RaftServer;
import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
import org.apache.ignite.internal.raft.service.ItAbstractListenerSnapshotTest;
import org.apache.ignite.internal.raft.service.RaftGroupListener;
@@ -118,7 +119,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
/** {@inheritDoc} */
@Override
- public void beforeFollowerStop(RaftGroupService service) throws Exception {
+ public void beforeFollowerStop(RaftGroupService service, RaftServer server) throws Exception {
// TODO: https://issues.apache.org/jira/browse/IGNITE-17817 Use Replica layer with new transaction protocol.
TxManagerImpl txManager = new TxManagerImpl(replicaService, new HeapLockManager(), new HybridClockImpl());
@@ -144,7 +145,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
/** {@inheritDoc} */
@Override
- public void afterFollowerStop(RaftGroupService service) throws Exception {
+ public void afterFollowerStop(RaftGroupService service, RaftServer server) throws Exception {
// TODO: https://issues.apache.org/jira/browse/IGNITE-17817 Use Replica layer with new transaction protocol.
TxManagerImpl txManager = new TxManagerImpl(replicaService, new HeapLockManager(), new HybridClockImpl());
@@ -252,7 +253,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
/** {@inheritDoc} */
@Override
- public Path getListenerPersistencePath(PartitionListener listener) {
+ public Path getListenerPersistencePath(PartitionListener listener, RaftServer server) {
return paths.get(listener);
}
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
index b6b438ef80..4a92a73f98 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
@@ -1765,7 +1765,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
private void registerRebalanceListeners() {
metaStorageMgr.registerPrefixWatch(ByteArray.fromString(PENDING_ASSIGNMENTS_PREFIX), new WatchListener() {
@Override
- public boolean onUpdate(@NotNull WatchEvent evt) {
+ public void onUpdate(WatchEvent evt) {
if (!busyLock.enterBusy()) {
throw new IgniteInternalException(new NodeStoppingException());
}
@@ -1776,7 +1776,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
Entry pendingAssignmentsWatchEvent = evt.entryEvent().newEntry();
if (pendingAssignmentsWatchEvent.value() == null) {
- return true;
+ return;
}
int partId = extractPartitionNumber(pendingAssignmentsWatchEvent.key());
@@ -1901,7 +1901,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
// Do not change peers of the raft group if this is a stale event.
// Note that we start raft node before for the sake of the consistency in a starting and stopping raft nodes.
if (pendingAssignmentsWatchEvent.revision() < pendingAssignmentsEntry.revision()) {
- return true;
+ return;
}
RaftGroupService partGrpSvc = internalTable.partitionRaftGroupService(partId);
@@ -1916,22 +1916,20 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
partGrpSvc.changePeersAsync(pendingConfiguration, leaderWithTerm.term()).join();
}
-
- return true;
} finally {
busyLock.leaveBusy();
}
}
@Override
- public void onError(@NotNull Throwable e) {
+ public void onError(Throwable e) {
LOG.warn("Unable to process pending assignments event", e);
}
});
metaStorageMgr.registerPrefixWatch(ByteArray.fromString(STABLE_ASSIGNMENTS_PREFIX), new WatchListener() {
@Override
- public boolean onUpdate(@NotNull WatchEvent evt) {
+ public void onUpdate(WatchEvent evt) {
if (!busyLock.enterBusy()) {
throw new IgniteInternalException(new NodeStoppingException());
}
@@ -1942,7 +1940,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
Entry stableAssignmentsWatchEvent = evt.entryEvent().newEntry();
if (stableAssignmentsWatchEvent.value() == null) {
- return true;
+ return;
}
int part = extractPartitionNumber(stableAssignmentsWatchEvent.key());
@@ -1973,22 +1971,20 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
// no-op
}
}
-
- return true;
} finally {
busyLock.leaveBusy();
}
}
@Override
- public void onError(@NotNull Throwable e) {
+ public void onError(Throwable e) {
LOG.warn("Unable to process stable assignments event", e);
}
});
metaStorageMgr.registerPrefixWatch(ByteArray.fromString(ASSIGNMENTS_SWITCH_REDUCE_PREFIX), new WatchListener() {
@Override
- public boolean onUpdate(@NotNull WatchEvent evt) {
+ public void onUpdate(WatchEvent evt) {
byte[] key = evt.entryEvent().newEntry().key();
int partitionNumber = extractPartitionNumber(key);
@@ -2008,12 +2004,10 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
replicaGrpId,
evt
);
-
- return true;
}
@Override
- public void onError(@NotNull Throwable e) {
+ public void onError(Throwable e) {
LOG.warn("Unable to process switch reduce event", e);
}
});