Upgrade to Atomix 3.0-rc5
* Upgrade Raft primitives to Atomix 3.0
* Replace cluster store and messaging implementations with Atomix cluster management/messaging
* Add test scripts for installing/starting Atomix cluster
* Replace core primitives with Atomix primitives.
Change-Id: I7623653c81292a34f21b01f5f38ca11b5ef15cad
diff --git a/core/store/dist/src/main/java/org/onosproject/store/cluster/impl/DistributedClusterStore.java b/core/store/dist/src/main/java/org/onosproject/store/cluster/impl/DistributedClusterStore.java
index e8a6f01..084c198 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/cluster/impl/DistributedClusterStore.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/cluster/impl/DistributedClusterStore.java
@@ -38,6 +38,7 @@
import org.onosproject.cluster.ControllerNode;
import org.onosproject.cluster.ControllerNode.State;
import org.onosproject.cluster.DefaultControllerNode;
+import org.onosproject.cluster.Node;
import org.onosproject.cluster.NodeId;
import org.onosproject.core.Version;
import org.onosproject.core.VersionService;
@@ -70,7 +71,7 @@
import static org.onosproject.cluster.ClusterEvent.Type.INSTANCE_READY;
import static org.slf4j.LoggerFactory.getLogger;
-@Component(immediate = true)
+@Component(enabled = false)
@Service
/**
* Distributed cluster nodes store that employs an accrual failure
@@ -222,6 +223,11 @@
}
@Override
+ public Set<Node> getStorageNodes() {
+ return ImmutableSet.of();
+ }
+
+ @Override
public ControllerNode getNode(NodeId nodeId) {
checkNotNull(nodeId, INSTANCE_ID_NULL);
return allNodes.get(nodeId);
diff --git a/core/store/dist/src/main/java/org/onosproject/store/cluster/messaging/impl/NettyMessagingManager.java b/core/store/dist/src/main/java/org/onosproject/store/cluster/messaging/impl/NettyMessagingManager.java
index 454f1bc..c98c72c 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/cluster/messaging/impl/NettyMessagingManager.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/cluster/messaging/impl/NettyMessagingManager.java
@@ -101,7 +101,7 @@
/**
* Netty based MessagingService.
*/
-@Component(immediate = true)
+@Component(enabled = false)
@Service
public class NettyMessagingManager implements MessagingService {
private static final long HISTORY_EXPIRE_MILLIS = Duration.ofMinutes(1).toMillis();
diff --git a/core/store/dist/src/main/java/org/onosproject/store/config/impl/DistributedNetworkConfigStore.java b/core/store/dist/src/main/java/org/onosproject/store/config/impl/DistributedNetworkConfigStore.java
index c8547fa..7c2081e 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/config/impl/DistributedNetworkConfigStore.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/config/impl/DistributedNetworkConfigStore.java
@@ -129,7 +129,7 @@
// Sweep through any pending configurations, validate them and then prune them.
private void processPendingConfigs(ConfigFactory configFactory) {
- ImmutableSet.copyOf(configs.keySet()).forEach(k -> {
+ configs.keySet().forEach(k -> {
if (Objects.equals(k.configKey, configFactory.configKey()) &&
isAssignableFrom(configFactory, k)) {
// Prune whether valid or not
@@ -175,7 +175,7 @@
// Sweep through any configurations for the config factory, set back to pending state.
private void processExistingConfigs(ConfigFactory configFactory) {
- ImmutableSet.copyOf(configs.keySet()).forEach(k -> {
+ configs.keySet().forEach(k -> {
if (Objects.equals(configFactory.configClass().getName(), k.configClass)) {
Versioned<JsonNode> remove = configs.remove(k);
if (remove != null) {
@@ -294,7 +294,7 @@
@Override
public <S> void clearConfig(S subject) {
- ImmutableSet.copyOf(configs.keySet()).forEach(k -> {
+ configs.keySet().forEach(k -> {
if (Objects.equals(subject, k.subject) && delegate != null) {
configs.remove(k);
}
@@ -303,7 +303,7 @@
@Override
public <S> void clearConfig() {
- ImmutableSet.copyOf(configs.keySet()).forEach(k -> {
+ configs.keySet().forEach(k -> {
if (delegate != null) {
configs.remove(k);
}
diff --git a/core/store/dist/src/test/java/org/onosproject/store/cluster/messaging/impl/NettyMessagingManagerTest.java b/core/store/dist/src/test/java/org/onosproject/store/cluster/messaging/impl/NettyMessagingManagerTest.java
index e48b225..dfb2f67 100644
--- a/core/store/dist/src/test/java/org/onosproject/store/cluster/messaging/impl/NettyMessagingManagerTest.java
+++ b/core/store/dist/src/test/java/org/onosproject/store/cluster/messaging/impl/NettyMessagingManagerTest.java
@@ -245,7 +245,7 @@
@Override
public ClusterMetadata getClusterMetadata() {
return new ClusterMetadata(new ProviderId(DUMMY_NAME, DUMMY_NAME),
- name, Sets.newHashSet(), Sets.newHashSet());
+ name, getLocalNode(), Sets.newHashSet());
}
@Override
diff --git a/core/store/primitives/BUCK b/core/store/primitives/BUCK
index 1860c07..5de553d 100644
--- a/core/store/primitives/BUCK
+++ b/core/store/primitives/BUCK
@@ -1,10 +1,12 @@
+include_defs('//core/store/primitives/primitives.bucklet')
+
COMPILE_DEPS = [
'//lib:CORE_DEPS',
'//lib:KRYO',
+ '//lib:ATOMIX',
'//core/common:onos-core-common',
'//incubator/api:onos-incubator-api',
'//core/store/serializers:onos-core-serializers',
- '//lib:atomix',
]
TEST_DEPS = [
@@ -22,4 +24,5 @@
deps = COMPILE_DEPS,
test_deps = TEST_DEPS,
visibility = ['PUBLIC'],
+ import_packages = INCLUDE_PACKAGES + ',' + ALL_PACKAGES,
)
diff --git a/core/store/primitives/pom.xml b/core/store/primitives/pom.xml
index 0d15ddc..68c9d1f 100644
--- a/core/store/primitives/pom.xml
+++ b/core/store/primitives/pom.xml
@@ -69,7 +69,13 @@
<dependency>
<groupId>io.atomix</groupId>
<artifactId>atomix</artifactId>
- <version>2.0.23</version>
+ <version>3.0.0-rc5</version>
+ </dependency>
+
+ <dependency>
+ <groupId>io.atomix</groupId>
+ <artifactId>atomix-raft</artifactId>
+ <version>3.0.0-rc5</version>
</dependency>
<dependency>
diff --git a/core/store/primitives/primitives.bucklet b/core/store/primitives/primitives.bucklet
new file mode 100644
index 0000000..c9c2fc7
--- /dev/null
+++ b/core/store/primitives/primitives.bucklet
@@ -0,0 +1,41 @@
+# Base package names for core Atomix primitives
+_primitive_package_names = (
+ 'barrier',
+ 'collection',
+ 'counter',
+ 'election',
+ 'idgenerator',
+ 'iterator',
+ 'list',
+ 'lock',
+ 'map',
+ 'multimap',
+ 'multiset',
+ 'queue',
+ 'semaphore',
+ 'set',
+ 'transaction',
+ 'tree',
+ 'value',
+ 'workqueue',
+)
+
+# A list of all core primitive packages for dynamic class loading
+PRIMITIVE_PACKAGES = ','.join(['io.atomix.primitive.session.impl'] + ['io.atomix.core.{primitive},io.atomix.core.{primitive}.impl'.format(primitive=primitive) for primitive in _primitive_package_names])
+
+# Base package names for core Atomix protocols
+_protocol_package_names = (
+ 'raft',
+ 'backup'
+)
+
+# A list of all core protocol packages for dynamic class loading
+PROTOCOL_PACKAGES = ','.join(['io.atomix.primitive.partition.impl'] + ['io.atomix.protocols.{protocol},io.atomix.protocols.{protocol}.partition'.format(protocol=protocol) for protocol in _protocol_package_names] + ['io.atomix.protocols.gossip'])
+
+# A list of all additional core packages used in dynamic class loading
+CORE_PACKAGES = ','.join(['io.atomix.core.profile', 'io.atomix.core.profile.impl'])
+
+# A list of all dynamically loadable Atomix packages
+INCLUDE_PACKAGES = ','.join([PRIMITIVE_PACKAGES, PROTOCOL_PACKAGES, CORE_PACKAGES])
+
+ALL_PACKAGES = '*'
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/AtomixClusterStore.java b/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/AtomixClusterStore.java
new file mode 100644
index 0000000..0afc462
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/AtomixClusterStore.java
@@ -0,0 +1,241 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.cluster.impl;
+
+import java.time.Instant;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Maps;
+import io.atomix.cluster.ClusterMembershipEvent;
+import io.atomix.cluster.ClusterMembershipEventListener;
+import io.atomix.cluster.ClusterMembershipService;
+import io.atomix.cluster.Member;
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Reference;
+import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.apache.felix.scr.annotations.Service;
+import org.onlab.packet.IpAddress;
+import org.onosproject.cluster.ClusterEvent;
+import org.onosproject.cluster.ClusterStore;
+import org.onosproject.cluster.ClusterStoreDelegate;
+import org.onosproject.cluster.ControllerNode;
+import org.onosproject.cluster.DefaultControllerNode;
+import org.onosproject.cluster.Node;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.core.Version;
+import org.onosproject.core.VersionService;
+import org.onosproject.store.AbstractStore;
+import org.onosproject.store.impl.AtomixManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Atomix cluster store.
+ */
+@Component(immediate = true)
+@Service
+public class AtomixClusterStore extends AbstractStore<ClusterEvent, ClusterStoreDelegate> implements ClusterStore {
+ private static final String INSTANCE_ID_NULL = "Instance ID cannot be null";
+
+ private static final String STATE_KEY = "state";
+ private static final String VERSION_KEY = "version";
+
+ private final Logger log = LoggerFactory.getLogger(getClass());
+
+ @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+ protected AtomixManager atomixManager;
+
+ @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+ protected VersionService versionService;
+
+ private ClusterMembershipService membershipService;
+ private ControllerNode localNode;
+ private final Map<NodeId, ControllerNode> nodes = Maps.newConcurrentMap();
+ private final Map<NodeId, ControllerNode.State> states = Maps.newConcurrentMap();
+ private final Map<NodeId, Version> versions = Maps.newConcurrentMap();
+ private final Map<NodeId, Instant> updates = Maps.newConcurrentMap();
+ private final ClusterMembershipEventListener membershipEventListener = this::changeMembership;
+
+ @Activate
+ public void activate() {
+ membershipService = atomixManager.getAtomix().getMembershipService();
+ membershipService.addListener(membershipEventListener);
+ membershipService.getMembers().forEach(member -> {
+ ControllerNode node = toControllerNode(member);
+ nodes.put(node.id(), node);
+ updateState(node, member);
+ updateVersion(node, member);
+ });
+ membershipService.getLocalMember().properties().put(STATE_KEY, ControllerNode.State.ACTIVE.name());
+ membershipService.getLocalMember().properties().put(VERSION_KEY, versionService.version().toString());
+ localNode = toControllerNode(membershipService.getLocalMember());
+ states.put(localNode.id(), ControllerNode.State.ACTIVE);
+ versions.put(localNode.id(), versionService.version());
+ log.info("Started");
+ }
+
+ @Deactivate
+ public void deactivate() {
+ membershipService.removeListener(membershipEventListener);
+ log.info("Stopped");
+ }
+
+ private void changeMembership(ClusterMembershipEvent event) {
+ ControllerNode node = nodes.get(NodeId.nodeId(event.subject().id().id()));
+ switch (event.type()) {
+ case MEMBER_ADDED:
+ case METADATA_CHANGED:
+ if (node == null) {
+ node = toControllerNode(event.subject());
+ nodes.put(node.id(), node);
+ notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_ADDED, node));
+ }
+ updateVersion(node, event.subject());
+ updateState(node, event.subject());
+ break;
+ case MEMBER_REMOVED:
+ if (node != null
+ && states.put(node.id(), ControllerNode.State.INACTIVE) != ControllerNode.State.INACTIVE) {
+ notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_DEACTIVATED, node));
+ notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_REMOVED, node));
+ }
+ break;
+ default:
+ break;
+ }
+ }
+
+ private void updateState(ControllerNode node, Member member) {
+ String state = member.properties().getProperty(STATE_KEY);
+ if (state == null || !state.equals(ControllerNode.State.READY.name())) {
+ if (states.put(node.id(), ControllerNode.State.ACTIVE) != ControllerNode.State.ACTIVE) {
+ log.info("Updated node {} state to {}", node.id(), ControllerNode.State.ACTIVE);
+ markUpdated(node.id());
+ notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_ACTIVATED, node));
+ }
+ } else {
+ if (states.put(node.id(), ControllerNode.State.READY) != ControllerNode.State.READY) {
+ log.info("Updated node {} state to {}", node.id(), ControllerNode.State.READY);
+ markUpdated(node.id());
+ notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_READY, node));
+ }
+ }
+ }
+
+ private void updateVersion(ControllerNode node, Member member) {
+ String versionString = member.properties().getProperty(VERSION_KEY);
+ if (versionString != null) {
+ Version version = Version.version(versionString);
+ if (!Objects.equals(versions.put(node.id(), version), version)) {
+ log.info("Updated node {} version to {}", node.id(), version);
+ }
+ }
+ }
+
+ private void markUpdated(NodeId nodeId) {
+ updates.put(nodeId, Instant.now());
+ }
+
+ private ControllerNode toControllerNode(Member member) {
+ return new DefaultControllerNode(
+ NodeId.nodeId(member.id().id()),
+ IpAddress.valueOf(member.address().address()),
+ member.address().port());
+ }
+
+ @Override
+ public ControllerNode getLocalNode() {
+ return toControllerNode(membershipService.getLocalMember());
+ }
+
+ @Override
+ public Set<Node> getStorageNodes() {
+ return membershipService.getMembers()
+ .stream()
+ .filter(member -> !Objects.equals(member.properties().getProperty("type"), "onos"))
+ .map(this::toControllerNode)
+ .collect(Collectors.toSet());
+ }
+
+ @Override
+ public Set<ControllerNode> getNodes() {
+ return membershipService.getMembers()
+ .stream()
+ .filter(member -> Objects.equals(member.properties().getProperty("type"), "onos"))
+ .map(this::toControllerNode)
+ .collect(Collectors.toSet());
+ }
+
+ @Override
+ public ControllerNode getNode(NodeId nodeId) {
+ Member member = membershipService.getMember(nodeId.id());
+ return member != null ? toControllerNode(member) : null;
+ }
+
+ @Override
+ public ControllerNode.State getState(NodeId nodeId) {
+ checkNotNull(nodeId, INSTANCE_ID_NULL);
+ return states.get(nodeId);
+ }
+
+ @Override
+ public Version getVersion(NodeId nodeId) {
+ checkNotNull(nodeId, INSTANCE_ID_NULL);
+ return versions.get(nodeId);
+ }
+
+ @Override
+ public Instant getLastUpdatedInstant(NodeId nodeId) {
+ checkNotNull(nodeId, INSTANCE_ID_NULL);
+ return updates.get(nodeId);
+ }
+
+ @Override
+ public void markFullyStarted(boolean started) {
+ ControllerNode.State state = started ? ControllerNode.State.READY : ControllerNode.State.ACTIVE;
+ states.put(localNode.id(), state);
+ membershipService.getLocalMember().properties().setProperty(STATE_KEY, state.name());
+ }
+
+ @Override
+ public ControllerNode addNode(NodeId nodeId, IpAddress ip, int tcpPort) {
+ checkNotNull(nodeId, INSTANCE_ID_NULL);
+ ControllerNode node = new DefaultControllerNode(nodeId, ip, tcpPort);
+ nodes.put(node.id(), node);
+ ControllerNode.State state = node.equals(localNode)
+ ? ControllerNode.State.ACTIVE : ControllerNode.State.INACTIVE;
+ membershipService.getMember(node.id().id()).properties().setProperty(STATE_KEY, state.name());
+ notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_ADDED, node));
+ return node;
+ }
+
+ @Override
+ public void removeNode(NodeId nodeId) {
+ checkNotNull(nodeId, INSTANCE_ID_NULL);
+ ControllerNode node = nodes.remove(nodeId);
+ if (node != null) {
+ states.remove(nodeId);
+ notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_REMOVED, node));
+ }
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java b/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/package-info.java
similarity index 76%
copy from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
copy to core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/package-info.java
index aa11664..23bd9c4 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/package-info.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2016-present Open Networking Foundation
+ * Copyright 2018-present Open Networking Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -15,6 +15,6 @@
*/
/**
- * State machine implementation classes for distributed primitives.
+ * Implementation classes for cluster stores.
*/
-package org.onosproject.store.primitives.resources.impl;
+package org.onosproject.store.cluster.impl;
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/AtomixMessagingManager.java b/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/AtomixMessagingManager.java
new file mode 100644
index 0000000..46b27bf
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/AtomixMessagingManager.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.cluster.messaging.impl;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+
+import io.atomix.utils.net.Address;
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Reference;
+import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.apache.felix.scr.annotations.Service;
+import org.onlab.packet.IpAddress;
+import org.onosproject.store.cluster.messaging.Endpoint;
+import org.onosproject.store.cluster.messaging.MessagingService;
+import org.onosproject.store.impl.AtomixManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Atomix messaging manager.
+ */
+@Component(immediate = true)
+@Service
+public class AtomixMessagingManager implements MessagingService {
+
+ private final Logger log = LoggerFactory.getLogger(getClass());
+
+ @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+ protected AtomixManager atomixManager;
+
+ private io.atomix.cluster.messaging.MessagingService messagingService;
+
+ @Activate
+ public void activate() {
+ messagingService = atomixManager.getAtomix().getMessagingService();
+ log.info("Started");
+ }
+
+ @Deactivate
+ public void deactivate() {
+ log.info("Stopped");
+ }
+
+ private Address toAddress(Endpoint ep) {
+ return new Address(ep.host().toString(), ep.port(), ep.host().toInetAddress());
+ }
+
+ private Endpoint toEndpoint(Address address) {
+ return new Endpoint(IpAddress.valueOf(address.address()), address.port());
+ }
+
+ @Override
+ public CompletableFuture<Void> sendAsync(Endpoint ep, String type, byte[] payload) {
+ return messagingService.sendAsync(toAddress(ep), type, payload);
+ }
+
+ @Override
+ public CompletableFuture<byte[]> sendAndReceive(Endpoint ep, String type, byte[] payload) {
+ return messagingService.sendAndReceive(toAddress(ep), type, payload);
+ }
+
+ @Override
+ public CompletableFuture<byte[]> sendAndReceive(Endpoint ep, String type, byte[] payload, Executor executor) {
+ return messagingService.sendAndReceive(toAddress(ep), type, payload, executor);
+ }
+
+ @Override
+ public void registerHandler(String type, BiConsumer<Endpoint, byte[]> handler, Executor executor) {
+ BiConsumer<Address, byte[]> consumer = (address, payload) -> handler.accept(toEndpoint(address), payload);
+ messagingService.registerHandler(type, consumer, executor);
+ }
+
+ @Override
+ public void registerHandler(String type, BiFunction<Endpoint, byte[], byte[]> handler, Executor executor) {
+ BiFunction<Address, byte[], byte[]> function = (address, payload) ->
+ handler.apply(toEndpoint(address), payload);
+ messagingService.registerHandler(type, function, executor);
+ }
+
+ @Override
+ public void registerHandler(String type, BiFunction<Endpoint, byte[], CompletableFuture<byte[]>> handler) {
+ BiFunction<Address, byte[], CompletableFuture<byte[]>> function = (address, payload) ->
+ handler.apply(toEndpoint(address), payload);
+ messagingService.registerHandler(type, function);
+ }
+
+ @Override
+ public void unregisterHandler(String type) {
+ messagingService.unregisterHandler(type);
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java b/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/package-info.java
similarity index 76%
rename from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
rename to core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/package-info.java
index aa11664..b88387a 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/package-info.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2016-present Open Networking Foundation
+ * Copyright 2018-present Open Networking Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -15,6 +15,6 @@
*/
/**
- * State machine implementation classes for distributed primitives.
+ * Implementation classes for cluster messaging.
*/
-package org.onosproject.store.primitives.resources.impl;
+package org.onosproject.store.cluster.messaging.impl;
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/impl/AtomixManager.java b/core/store/primitives/src/main/java/org/onosproject/store/impl/AtomixManager.java
new file mode 100644
index 0000000..43cde78
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/impl/AtomixManager.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.impl;
+
+import java.util.stream.Collectors;
+
+import io.atomix.cluster.discovery.BootstrapDiscoveryProvider;
+import io.atomix.core.Atomix;
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Reference;
+import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.apache.felix.scr.annotations.Service;
+import org.onosproject.cluster.ClusterMetadataService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Atomix manager.
+ */
+@Component(immediate = true)
+@Service(value = AtomixManager.class)
+public class AtomixManager {
+
+ private final Logger log = LoggerFactory.getLogger(getClass());
+
+ @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+ protected ClusterMetadataService metadataService;
+
+ private Atomix atomix;
+
+ /**
+ * Returns the Atomix instance.
+ *
+ * @return the Atomix instance
+ */
+ public Atomix getAtomix() {
+ return atomix;
+ }
+
+ @Activate
+ public void activate() {
+ atomix = createAtomix();
+ atomix.start().join();
+ log.info("Started");
+ }
+
+ @Deactivate
+ public void deactivate() {
+ atomix.stop().join();
+ log.info("Stopped");
+ }
+
+ private Atomix createAtomix() {
+ return Atomix.builder(getClass().getClassLoader())
+ .withClusterId(metadataService.getClusterMetadata().getName())
+ .withMemberId(metadataService.getLocalNode().id().id())
+ .withAddress(metadataService.getLocalNode().ip().toString(), metadataService.getLocalNode().tcpPort())
+ .withProperty("type", "onos")
+ .withMembershipProvider(BootstrapDiscoveryProvider.builder()
+ .withNodes(metadataService.getClusterMetadata().getStorageNodes().stream()
+ .map(node -> io.atomix.cluster.Node.builder()
+ .withId(node.id().id())
+ .withAddress(node.ip().toString(), node.tcpPort())
+ .build())
+ .collect(Collectors.toList()))
+ .build())
+ .build();
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java b/core/store/primitives/src/main/java/org/onosproject/store/impl/package-info.java
similarity index 76%
copy from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
copy to core/store/primitives/src/main/java/org/onosproject/store/impl/package-info.java
index aa11664..6f34a50 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/impl/package-info.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2016-present Open Networking Foundation
+ * Copyright 2018-present Open Networking Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -15,6 +15,6 @@
*/
/**
- * State machine implementation classes for distributed primitives.
+ * Implementation classes for various Distributed primitives.
*/
-package org.onosproject.store.primitives.resources.impl;
+package org.onosproject.store.impl;
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AntiEntropyAdvertisement.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AntiEntropyAdvertisement.java
index 373273c..0004fe1 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AntiEntropyAdvertisement.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AntiEntropyAdvertisement.java
@@ -15,13 +15,12 @@
*/
package org.onosproject.store.primitives.impl;
+import java.util.Map;
+
import com.google.common.base.MoreObjects;
import com.google.common.collect.ImmutableMap;
-
import org.onosproject.cluster.NodeId;
-import java.util.Map;
-
import static com.google.common.base.Preconditions.checkNotNull;
/**
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounter.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounter.java
new file mode 100644
index 0000000..ce5c248
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounter.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncAtomicCounter;
+
+/**
+ * Atomix atomic counter.
+ */
+public class AtomixAtomicCounter implements AsyncAtomicCounter {
+ private final io.atomix.core.counter.AsyncAtomicCounter atomixCounter;
+
+ public AtomixAtomicCounter(io.atomix.core.counter.AsyncAtomicCounter atomixCounter) {
+ this.atomixCounter = atomixCounter;
+ }
+
+ @Override
+ public String name() {
+ return atomixCounter.name();
+ }
+
+ @Override
+ public CompletableFuture<Long> incrementAndGet() {
+ return atomixCounter.incrementAndGet();
+ }
+
+ @Override
+ public CompletableFuture<Long> getAndIncrement() {
+ return atomixCounter.getAndIncrement();
+ }
+
+ @Override
+ public CompletableFuture<Long> getAndAdd(long delta) {
+ return atomixCounter.getAndAdd(delta);
+ }
+
+ @Override
+ public CompletableFuture<Long> addAndGet(long delta) {
+ return atomixCounter.addAndGet(delta);
+ }
+
+ @Override
+ public CompletableFuture<Long> get() {
+ return atomixCounter.get();
+ }
+
+ @Override
+ public CompletableFuture<Void> set(long value) {
+ return atomixCounter.set(value);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> compareAndSet(long expectedValue, long updateValue) {
+ return atomixCounter.compareAndSet(expectedValue, updateValue);
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterBuilder.java
new file mode 100644
index 0000000..b18c128
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterBuilder.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncAtomicCounter;
+import org.onosproject.store.service.AtomicCounterBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncAtomicCounter} builder.
+ */
+public class AtomixAtomicCounterBuilder extends AtomicCounterBuilder {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixAtomicCounterBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public AsyncAtomicCounter build() {
+ return new AtomixAtomicCounter(atomix.atomicCounterBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMap.java
new file mode 100644
index 0000000..fc0aca1
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMap.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncAtomicCounterMap;
+
+/**
+ * Atomix atomic counter map.
+ */
+public class AtomixAtomicCounterMap<K> implements AsyncAtomicCounterMap<K> {
+ private final io.atomix.core.map.AsyncAtomicCounterMap<K> atomixMap;
+
+ public AtomixAtomicCounterMap(io.atomix.core.map.AsyncAtomicCounterMap<K> atomixMap) {
+ this.atomixMap = atomixMap;
+ }
+
+ @Override
+ public String name() {
+ return atomixMap.name();
+ }
+
+ @Override
+ public CompletableFuture<Long> incrementAndGet(K key) {
+ return atomixMap.incrementAndGet(key);
+ }
+
+ @Override
+ public CompletableFuture<Long> decrementAndGet(K key) {
+ return atomixMap.decrementAndGet(key);
+ }
+
+ @Override
+ public CompletableFuture<Long> getAndIncrement(K key) {
+ return atomixMap.getAndIncrement(key);
+ }
+
+ @Override
+ public CompletableFuture<Long> getAndDecrement(K key) {
+ return atomixMap.getAndDecrement(key);
+ }
+
+ @Override
+ public CompletableFuture<Long> addAndGet(K key, long delta) {
+ return atomixMap.addAndGet(key, delta);
+ }
+
+ @Override
+ public CompletableFuture<Long> getAndAdd(K key, long delta) {
+ return atomixMap.getAndAdd(key, delta);
+ }
+
+ @Override
+ public CompletableFuture<Long> get(K key) {
+ return atomixMap.get(key);
+ }
+
+ @Override
+ public CompletableFuture<Long> put(K key, long newValue) {
+ return atomixMap.put(key, newValue);
+ }
+
+ @Override
+ public CompletableFuture<Long> putIfAbsent(K key, long newValue) {
+ return atomixMap.putIfAbsent(key, newValue);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> replace(K key, long expectedOldValue, long newValue) {
+ return atomixMap.replace(key, expectedOldValue, newValue);
+ }
+
+ @Override
+ public CompletableFuture<Long> remove(K key) {
+ return atomixMap.remove(key);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> remove(K key, long value) {
+ return atomixMap.remove(key, value);
+ }
+
+ @Override
+ public CompletableFuture<Integer> size() {
+ return atomixMap.size();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> isEmpty() {
+ return atomixMap.isEmpty();
+ }
+
+ @Override
+ public CompletableFuture<Void> clear() {
+ return atomixMap.clear();
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMapBuilder.java
new file mode 100644
index 0000000..03177bf
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMapBuilder.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncAtomicCounterMap;
+import org.onosproject.store.service.AtomicCounterMap;
+import org.onosproject.store.service.AtomicCounterMapBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncAtomicCounterMap} builder.
+ *
+ * @param <K> type for tree value
+ */
+public class AtomixAtomicCounterMapBuilder<K> extends AtomicCounterMapBuilder<K> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixAtomicCounterMapBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public AtomicCounterMap<K> build() {
+ return buildAsyncMap().asAtomicCounterMap();
+ }
+
+ @Override
+ public AsyncAtomicCounterMap<K> buildAsyncMap() {
+ return new AtomixAtomicCounterMap<K>(atomix.<K>atomicCounterMapBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGenerator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGenerator.java
new file mode 100644
index 0000000..2bafa0b
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGenerator.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncAtomicIdGenerator;
+
+/**
+ * Atomix atomic ID generator.
+ */
+public class AtomixAtomicIdGenerator implements AsyncAtomicIdGenerator {
+ private final io.atomix.core.idgenerator.AsyncAtomicIdGenerator atomixIdGenerator;
+
+ public AtomixAtomicIdGenerator(io.atomix.core.idgenerator.AsyncAtomicIdGenerator atomixIdGenerator) {
+ this.atomixIdGenerator = atomixIdGenerator;
+ }
+
+ @Override
+ public String name() {
+ return atomixIdGenerator.name();
+ }
+
+ @Override
+ public CompletableFuture<Long> nextId() {
+ return atomixIdGenerator.nextId();
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGeneratorBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGeneratorBuilder.java
new file mode 100644
index 0000000..2b7bd85
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGeneratorBuilder.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncAtomicIdGenerator;
+import org.onosproject.store.service.AtomicIdGeneratorBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncAtomicIdGenerator} builder.
+ */
+public class AtomixAtomicIdGeneratorBuilder extends AtomicIdGeneratorBuilder {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixAtomicIdGeneratorBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public AsyncAtomicIdGenerator build() {
+ return new AtomixAtomicIdGenerator(atomix.atomicIdGeneratorBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValue.java
new file mode 100644
index 0000000..977b206
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValue.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import com.google.common.collect.Maps;
+import org.onosproject.store.service.AsyncAtomicValue;
+import org.onosproject.store.service.AtomicValueEvent;
+import org.onosproject.store.service.AtomicValueEventListener;
+
+/**
+ * Atomix atomic value.
+ */
+public class AtomixAtomicValue<V> implements AsyncAtomicValue<V> {
+ private final io.atomix.core.value.AsyncAtomicValue<V> atomixValue;
+ private final Map<AtomicValueEventListener<V>, io.atomix.core.value.AtomicValueEventListener<V>> listenerMap =
+ Maps.newIdentityHashMap();
+
+ public AtomixAtomicValue(io.atomix.core.value.AsyncAtomicValue<V> atomixValue) {
+ this.atomixValue = atomixValue;
+ }
+
+ @Override
+ public String name() {
+ return atomixValue.name();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> compareAndSet(V expect, V update) {
+ return atomixValue.compareAndSet(expect, update);
+ }
+
+ @Override
+ public CompletableFuture<V> get() {
+ return atomixValue.get();
+ }
+
+ @Override
+ public CompletableFuture<V> getAndSet(V value) {
+ return atomixValue.getAndSet(value);
+ }
+
+ @Override
+ public CompletableFuture<Void> set(V value) {
+ return atomixValue.set(value);
+ }
+
+ @Override
+ public synchronized CompletableFuture<Void> addListener(AtomicValueEventListener<V> listener) {
+ io.atomix.core.value.AtomicValueEventListener<V> atomixListener = event ->
+ listener.event(new AtomicValueEvent<V>(
+ name(),
+ event.newValue(),
+ event.oldValue()));
+ listenerMap.put(listener, atomixListener);
+ return atomixValue.addListener(atomixListener);
+ }
+
+ @Override
+ public synchronized CompletableFuture<Void> removeListener(AtomicValueEventListener<V> listener) {
+ io.atomix.core.value.AtomicValueEventListener<V> atomixListener = listenerMap.remove(listener);
+ if (atomixListener != null) {
+ return atomixValue.removeListener(atomixListener);
+ }
+ return CompletableFuture.completedFuture(null);
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValueBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValueBuilder.java
new file mode 100644
index 0000000..6b2e5df
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValueBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncAtomicValue;
+import org.onosproject.store.service.AtomicValueBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncAtomicValue} builder.
+ *
+ * @param <V> type for set value
+ */
+public class AtomixAtomicValueBuilder<V> extends AtomicValueBuilder<V> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixAtomicValueBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public AsyncAtomicValue<V> build() {
+ return new AtomixAtomicValue<V>(atomix.<V>atomicValueBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMap.java
new file mode 100644
index 0000000..d1a601d
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMap.java
@@ -0,0 +1,259 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.Maps;
+import io.atomix.core.collection.impl.TranscodingAsyncDistributedCollection;
+import io.atomix.core.set.impl.TranscodingAsyncDistributedSet;
+import org.onosproject.store.primitives.MapUpdate;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.AsyncConsistentMap;
+import org.onosproject.store.service.AsyncIterator;
+import org.onosproject.store.service.ConsistentMapException;
+import org.onosproject.store.service.MapEvent;
+import org.onosproject.store.service.MapEventListener;
+import org.onosproject.store.service.TransactionLog;
+import org.onosproject.store.service.Version;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * Atomix consistent map.
+ */
+public class AtomixConsistentMap<K, V> implements AsyncConsistentMap<K, V> {
+ private final io.atomix.core.map.AsyncAtomicMap<K, V> atomixMap;
+ private final Map<MapEventListener<K, V>, io.atomix.core.map.AtomicMapEventListener<K, V>> listenerMap =
+ Maps.newIdentityHashMap();
+
+ public AtomixConsistentMap(io.atomix.core.map.AsyncAtomicMap<K, V> atomixMap) {
+ this.atomixMap = atomixMap;
+ }
+
+ @Override
+ public String name() {
+ return atomixMap.name();
+ }
+
+ @Override
+ public CompletableFuture<Integer> size() {
+ return atomixMap.size();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> containsKey(K key) {
+ return atomixMap.containsKey(key);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> containsValue(V value) {
+ return atomixMap.containsValue(value);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> get(K key) {
+ return atomixMap.get(key).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
+ return atomixMap.getOrDefault(key, defaultValue).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> computeIf(
+ K key, Predicate<? super V> condition, BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
+ return adapt(atomixMap.computeIf(key, condition, remappingFunction).thenApply(this::toVersioned));
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> put(K key, V value) {
+ return adapt(atomixMap.put(key, value).thenApply(this::toVersioned));
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
+ return adapt(atomixMap.putAndGet(key, value).thenApply(this::toVersioned));
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> remove(K key) {
+ return adapt(atomixMap.remove(key).thenApply(this::toVersioned));
+ }
+
+ @Override
+ public CompletableFuture<Void> clear() {
+ return atomixMap.clear();
+ }
+
+ @Override
+ public CompletableFuture<Set<K>> keySet() {
+ return CompletableFuture.completedFuture(atomixMap.keySet().sync());
+ }
+
+ @Override
+ public CompletableFuture<Collection<Versioned<V>>> values() {
+ return CompletableFuture.completedFuture(
+ new TranscodingAsyncDistributedCollection<Versioned<V>, io.atomix.utils.time.Versioned<V>>(
+ atomixMap.values(),
+ v -> new io.atomix.utils.time.Versioned<>(v.value(), v.version()),
+ v -> new Versioned<>(v.value(), v.version())).sync());
+ }
+
+ @Override
+ public CompletableFuture<Set<Map.Entry<K, Versioned<V>>>> entrySet() {
+ return CompletableFuture.completedFuture(
+ new TranscodingAsyncDistributedSet<Map.Entry<K, Versioned<V>>,
+ Map.Entry<K, io.atomix.utils.time.Versioned<V>>>(
+ atomixMap.entrySet(),
+ e -> Maps.immutableEntry(e.getKey(),
+ new io.atomix.utils.time.Versioned<>(e.getValue().value(), e.getValue().version())),
+ e -> Maps.immutableEntry(e.getKey(), new Versioned<>(e.getValue().value(), e.getValue().version())))
+ .sync());
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
+ return adapt(atomixMap.putIfAbsent(key, value).thenApply(this::toVersioned));
+ }
+
+ @Override
+ public CompletableFuture<Boolean> remove(K key, V value) {
+ return adapt(atomixMap.remove(key, value));
+ }
+
+ @Override
+ public CompletableFuture<Boolean> remove(K key, long version) {
+ return adapt(atomixMap.remove(key, version));
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> replace(K key, V value) {
+ return adapt(atomixMap.replace(key, value).thenApply(this::toVersioned));
+ }
+
+ @Override
+ public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
+ return adapt(atomixMap.replace(key, oldValue, newValue));
+ }
+
+ @Override
+ public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
+ return adapt(atomixMap.replace(key, oldVersion, newValue));
+ }
+
+ @Override
+ public CompletableFuture<AsyncIterator<Map.Entry<K, Versioned<V>>>> iterator() {
+ io.atomix.core.iterator.AsyncIterator<Map.Entry<K, io.atomix.utils.time.Versioned<V>>> atomixIterator
+ = atomixMap.entrySet().iterator();
+ return CompletableFuture.completedFuture(new AsyncIterator<Map.Entry<K, Versioned<V>>>() {
+ @Override
+ public CompletableFuture<Boolean> hasNext() {
+ return atomixIterator.hasNext();
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<K, Versioned<V>>> next() {
+ return atomixIterator.next()
+ .thenApply(entry -> Maps.immutableEntry(entry.getKey(), toVersioned(entry.getValue())));
+ }
+ });
+ }
+
+ @Override
+ public synchronized CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
+ io.atomix.core.map.AtomicMapEventListener<K, V> atomixListener = event ->
+ listener.event(new MapEvent<K, V>(
+ MapEvent.Type.valueOf(event.type().name()),
+ name(),
+ event.key(),
+ toVersioned(event.newValue()),
+ toVersioned(event.oldValue())));
+ listenerMap.put(listener, atomixListener);
+ return atomixMap.addListener(atomixListener, executor);
+ }
+
+ @Override
+ public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
+ io.atomix.core.map.AtomicMapEventListener<K, V> atomixListener = listenerMap.remove(listener);
+ if (atomixListener != null) {
+ return atomixMap.removeListener(atomixListener);
+ }
+ return CompletableFuture.completedFuture(null);
+ }
+
+ @Override
+ public CompletableFuture<Version> begin(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K, V>> transactionLog) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K, V>> transactionLog) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Void> commit(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Void> rollback(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ private <T> CompletableFuture<T> adapt(CompletableFuture<T> future) {
+ CompletableFuture<T> newFuture = new CompletableFuture<>();
+ future.whenComplete((result, error) -> {
+ if (error == null) {
+ newFuture.complete(result);
+ } else {
+ Throwable cause = Throwables.getRootCause(error);
+ if (cause instanceof io.atomix.primitive.PrimitiveException.ConcurrentModification) {
+ newFuture.completeExceptionally(
+ new ConsistentMapException.ConcurrentModification(cause.getMessage()));
+ } else if (cause instanceof io.atomix.primitive.PrimitiveException.Timeout) {
+ newFuture.completeExceptionally(new ConsistentMapException.Timeout(cause.getMessage()));
+ } else if (cause instanceof io.atomix.primitive.PrimitiveException.Interrupted) {
+ newFuture.completeExceptionally(new ConsistentMapException.Interrupted());
+ } else if (cause instanceof io.atomix.primitive.PrimitiveException.Unavailable) {
+ newFuture.completeExceptionally(new ConsistentMapException.Unavailable());
+ } else if (cause instanceof io.atomix.primitive.PrimitiveException) {
+ newFuture.completeExceptionally(new ConsistentMapException(cause.getMessage()));
+ }
+ }
+ });
+ return newFuture;
+ }
+
+ private Versioned<V> toVersioned(io.atomix.utils.time.Versioned<V> versioned) {
+ return versioned != null
+ ? new Versioned<>(versioned.value(), versioned.version(), versioned.creationTime())
+ : null;
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMapBuilder.java
new file mode 100644
index 0000000..0eadcde
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMapBuilder.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2016-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncConsistentMap;
+import org.onosproject.store.service.ConsistentMap;
+import org.onosproject.store.service.ConsistentMapBuilder;
+
+/**
+ * Default {@link AsyncConsistentMap} builder.
+ *
+ * @param <K> type for map key
+ * @param <V> type for map value
+ */
+public class AtomixConsistentMapBuilder<K, V> extends ConsistentMapBuilder<K, V> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixConsistentMapBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public ConsistentMap<K, V> build() {
+ return buildAsyncMap().asConsistentMap();
+ }
+
+ @Override
+ public AsyncConsistentMap<K, V> buildAsyncMap() {
+ return new AtomixConsistentMap<>(atomix.<K, V>atomicMapBuilder(name())
+ .withRegistrationRequired()
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withCacheEnabled(relaxedReadConsistency())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimap.java
new file mode 100644
index 0000000..6627a81
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimap.java
@@ -0,0 +1,191 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.Function;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multiset;
+import io.atomix.core.map.impl.TranscodingAsyncDistributedMap;
+import org.onosproject.store.service.AsyncConsistentMultimap;
+import org.onosproject.store.service.AsyncIterator;
+import org.onosproject.store.service.MultimapEvent;
+import org.onosproject.store.service.MultimapEventListener;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * Atomix consistent map.
+ */
+public class AtomixConsistentMultimap<K, V> implements AsyncConsistentMultimap<K, V> {
+ private final io.atomix.core.multimap.AsyncAtomicMultimap<K, V> atomixMultimap;
+ private final Map<MultimapEventListener<K, V>, io.atomix.core.multimap.AtomicMultimapEventListener<K, V>>
+ listenerMap = Maps.newIdentityHashMap();
+
+ public AtomixConsistentMultimap(io.atomix.core.multimap.AsyncAtomicMultimap<K, V> atomixMultimap) {
+ this.atomixMultimap = atomixMultimap;
+ }
+
+ @Override
+ public String name() {
+ return atomixMultimap.name();
+ }
+
+ @Override
+ public CompletableFuture<Integer> size() {
+ return atomixMultimap.size();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> containsKey(K key) {
+ return atomixMultimap.containsKey(key);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> containsValue(V value) {
+ return atomixMultimap.containsValue(value);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> isEmpty() {
+ return atomixMultimap.isEmpty();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> containsEntry(K key, V value) {
+ return atomixMultimap.containsEntry(key, value);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> put(K key, V value) {
+ return atomixMultimap.put(key, value);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<Collection<? extends V>>> putAndGet(K key, V value) {
+ return atomixMultimap.put(key, value).thenCompose(v -> atomixMultimap.get(key)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> remove(K key, V value) {
+ return atomixMultimap.remove(key, value);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<Collection<? extends V>>> removeAndGet(K key, V value) {
+ return atomixMultimap.remove(key, value).thenCompose(v -> atomixMultimap.get(key)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> removeAll(K key, Collection<? extends V> values) {
+ return atomixMultimap.removeAll(key, values);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<Collection<? extends V>>> removeAll(K key) {
+ return atomixMultimap.removeAll(key).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> putAll(K key, Collection<? extends V> values) {
+ return atomixMultimap.putAll(key, values);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public CompletableFuture<Versioned<Collection<? extends V>>> replaceValues(K key, Collection<V> values) {
+ return atomixMultimap.replaceValues(key, values).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Void> clear() {
+ return atomixMultimap.clear();
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public CompletableFuture<Versioned<Collection<? extends V>>> get(K key) {
+ return atomixMultimap.get(key).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Set<K>> keySet() {
+ return CompletableFuture.completedFuture(atomixMultimap.keySet().sync());
+ }
+
+ @Override
+ public CompletableFuture<Multiset<K>> keys() {
+ return CompletableFuture.completedFuture(atomixMultimap.keys().sync());
+ }
+
+ @Override
+ public CompletableFuture<Multiset<V>> values() {
+ return CompletableFuture.completedFuture(atomixMultimap.values().sync());
+ }
+
+ @Override
+ public CompletableFuture<Collection<Map.Entry<K, V>>> entries() {
+ return CompletableFuture.completedFuture(atomixMultimap.entries().sync());
+ }
+
+ @Override
+ public CompletableFuture<AsyncIterator<Map.Entry<K, V>>> iterator() {
+ return CompletableFuture.completedFuture(new AtomixIterator<>(atomixMultimap.entries().iterator()));
+ }
+
+ @Override
+ public CompletableFuture<Map<K, Collection<V>>> asMap() {
+ return CompletableFuture.completedFuture(
+ new TranscodingAsyncDistributedMap<K, Collection<V>, K, io.atomix.utils.time.Versioned<Collection<V>>>(
+ atomixMultimap.asMap(),
+ Function.identity(),
+ Function.identity(),
+ v -> new io.atomix.utils.time.Versioned<>(v, 0),
+ v -> v.value()).sync());
+ }
+
+ @Override
+ public synchronized CompletableFuture<Void> addListener(MultimapEventListener<K, V> listener, Executor executor) {
+ io.atomix.core.multimap.AtomicMultimapEventListener<K, V> atomixListener = event ->
+ listener.event(new MultimapEvent<K, V>(
+ name(),
+ event.key(),
+ event.newValue(),
+ event.oldValue()));
+ listenerMap.put(listener, atomixListener);
+ return atomixMultimap.addListener(atomixListener, executor);
+ }
+
+ @Override
+ public CompletableFuture<Void> removeListener(MultimapEventListener<K, V> listener) {
+ io.atomix.core.multimap.AtomicMultimapEventListener<K, V> atomixListener = listenerMap.remove(listener);
+ if (atomixListener != null) {
+ return atomixMultimap.removeListener(atomixListener);
+ }
+ return CompletableFuture.completedFuture(null);
+ }
+
+ private Versioned<Collection<? extends V>> toVersioned(
+ io.atomix.utils.time.Versioned<Collection<V>> versioned) {
+ return versioned != null
+ ? new Versioned<>(versioned.value(), versioned.version(), versioned.creationTime())
+ : null;
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimapBuilder.java
new file mode 100644
index 0000000..88f6bd2
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimapBuilder.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncConsistentMultimap;
+import org.onosproject.store.service.ConsistentMultimap;
+import org.onosproject.store.service.ConsistentMultimapBuilder;
+
+/**
+ * Default {@link AsyncConsistentMultimap} builder.
+ *
+ * @param <K> type for multimap key
+ * @param <V> type for multimap value
+ */
+public class AtomixConsistentMultimapBuilder<K, V> extends ConsistentMultimapBuilder<K, V> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixConsistentMultimapBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public AsyncConsistentMultimap<K, V> buildMultimap() {
+ return new AtomixConsistentMultimap<>(atomix.<K, V>atomicMultimapBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withCacheEnabled(relaxedReadConsistency())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+
+ @Override
+ public ConsistentMultimap<K, V> build() {
+ return buildMultimap().asMultimap();
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMap.java
new file mode 100644
index 0000000..a7bedbb
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMap.java
@@ -0,0 +1,345 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.Maps;
+import io.atomix.core.collection.impl.TranscodingAsyncDistributedCollection;
+import io.atomix.core.map.impl.DelegatingAsyncDistributedNavigableMap;
+import org.onosproject.store.primitives.MapUpdate;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.AsyncConsistentTreeMap;
+import org.onosproject.store.service.AsyncIterator;
+import org.onosproject.store.service.ConsistentMapException;
+import org.onosproject.store.service.MapEvent;
+import org.onosproject.store.service.MapEventListener;
+import org.onosproject.store.service.TransactionLog;
+import org.onosproject.store.service.Version;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * Atomix consistent tree map.
+ */
+public class AtomixConsistentTreeMap<V> implements AsyncConsistentTreeMap<V> {
+ private final io.atomix.core.map.AsyncAtomicNavigableMap<String, V> atomixTreeMap;
+ private final Map<MapEventListener<String, V>, io.atomix.core.map.AtomicMapEventListener<String, V>> listenerMap =
+ Maps.newIdentityHashMap();
+
+ public AtomixConsistentTreeMap(io.atomix.core.map.AsyncAtomicNavigableMap<String, V> atomixTreeMap) {
+ this.atomixTreeMap = atomixTreeMap;
+ }
+
+ @Override
+ public String name() {
+ return atomixTreeMap.name();
+ }
+
+ @Override
+ public CompletableFuture<Integer> size() {
+ return atomixTreeMap.size();
+ }
+
+ @Override
+ public CompletableFuture<String> firstKey() {
+ return atomixTreeMap.firstKey();
+ }
+
+ @Override
+ public CompletableFuture<String> lastKey() {
+ return atomixTreeMap.lastKey();
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> ceilingEntry(String key) {
+ return atomixTreeMap.ceilingEntry(key).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> floorEntry(String key) {
+ return atomixTreeMap.floorEntry(key).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> higherEntry(String key) {
+ return atomixTreeMap.higherEntry(key).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> lowerEntry(String key) {
+ return atomixTreeMap.lowerEntry(key).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> firstEntry() {
+ return atomixTreeMap.firstEntry().thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> lastEntry() {
+ return atomixTreeMap.lastEntry().thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> pollFirstEntry() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> pollLastEntry() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> containsKey(String key) {
+ return atomixTreeMap.containsKey(key);
+ }
+
+ @Override
+ public CompletableFuture<String> lowerKey(String key) {
+ return atomixTreeMap.lowerKey(key);
+ }
+
+ @Override
+ public CompletableFuture<String> floorKey(String key) {
+ return atomixTreeMap.floorKey(key);
+ }
+
+ @Override
+ public CompletableFuture<String> ceilingKey(String key) {
+ return atomixTreeMap.ceilingKey(key);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> get(String key) {
+ return atomixTreeMap.get(key).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<String> higherKey(String key) {
+ return atomixTreeMap.higherKey(key);
+ }
+
+ @Override
+ public CompletableFuture<NavigableSet<String>> navigableKeySet() {
+ return CompletableFuture.completedFuture(atomixTreeMap.navigableKeySet().sync());
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> getOrDefault(String key, V defaultValue) {
+ return atomixTreeMap.getOrDefault(key, defaultValue).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<NavigableMap<String, V>> subMap(
+ String upperKey, String lowerKey, boolean inclusiveUpper, boolean inclusiveLower) {
+ return CompletableFuture.completedFuture(
+ new DelegatingAsyncDistributedNavigableMap<>(
+ atomixTreeMap.subMap(lowerKey, inclusiveLower, upperKey, inclusiveUpper)).sync());
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> computeIf(
+ String key,
+ Predicate<? super V> condition, BiFunction<? super String, ? super V, ? extends V> remappingFunction) {
+ return adapt(atomixTreeMap.computeIf(key, condition, remappingFunction)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> put(String key, V value) {
+ return adapt(atomixTreeMap.put(key, value)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> putAndGet(String key, V value) {
+ return adapt(atomixTreeMap.putAndGet(key, value)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> remove(String key) {
+ return adapt(atomixTreeMap.remove(key)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Set<String>> keySet() {
+ return CompletableFuture.completedFuture(atomixTreeMap.keySet().sync());
+ }
+
+ @Override
+ public CompletableFuture<Set<Map.Entry<String, Versioned<V>>>> entrySet() {
+ return CompletableFuture.completedFuture(atomixTreeMap.entrySet().stream()
+ .map(this::toVersioned)
+ .collect(Collectors.toSet()));
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> putIfAbsent(String key, V value) {
+ return adapt(atomixTreeMap.putIfAbsent(key, value)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> remove(String key, V value) {
+ return adapt(atomixTreeMap.remove(key, value));
+ }
+
+ @Override
+ public CompletableFuture<Boolean> remove(String key, long version) {
+ return adapt(atomixTreeMap.remove(key, version));
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> replace(String key, V value) {
+ return adapt(atomixTreeMap.replace(key, value)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> replace(String key, V oldValue, V newValue) {
+ return adapt(atomixTreeMap.replace(key, oldValue, newValue));
+ }
+
+ @Override
+ public CompletableFuture<Boolean> replace(String key, long oldVersion, V newValue) {
+ return adapt(atomixTreeMap.replace(key, oldVersion, newValue));
+ }
+
+ @Override
+ public CompletableFuture<Boolean> containsValue(V value) {
+ return atomixTreeMap.containsValue(value);
+ }
+
+ @Override
+ public CompletableFuture<AsyncIterator<Map.Entry<String, Versioned<V>>>> iterator() {
+ io.atomix.core.iterator.AsyncIterator<Map.Entry<String, io.atomix.utils.time.Versioned<V>>> atomixIterator
+ = atomixTreeMap.entrySet().iterator();
+ return CompletableFuture.completedFuture(new AsyncIterator<Map.Entry<String, Versioned<V>>>() {
+ @Override
+ public CompletableFuture<Boolean> hasNext() {
+ return atomixIterator.hasNext();
+ }
+
+ @Override
+ public CompletableFuture<Map.Entry<String, Versioned<V>>> next() {
+ return atomixIterator.next()
+ .thenApply(entry -> Maps.immutableEntry(entry.getKey(), toVersioned(entry.getValue())));
+ }
+ });
+ }
+
+ @Override
+ public CompletableFuture<Void> clear() {
+ return atomixTreeMap.clear();
+ }
+
+ @Override
+ public CompletableFuture<Collection<Versioned<V>>> values() {
+ return CompletableFuture.completedFuture(
+ new TranscodingAsyncDistributedCollection<Versioned<V>, io.atomix.utils.time.Versioned<V>>(
+ atomixTreeMap.values(),
+ e -> new io.atomix.utils.time.Versioned<>(e.value(), e.version()),
+ e -> new Versioned<>(e.value(), e.version())).sync());
+ }
+
+ @Override
+ public synchronized CompletableFuture<Void> addListener(MapEventListener<String, V> listener, Executor executor) {
+ io.atomix.core.map.AtomicMapEventListener<String, V> atomixListener = event ->
+ listener.event(new MapEvent<String, V>(
+ MapEvent.Type.valueOf(event.type().name()),
+ name(),
+ event.key(),
+ toVersioned(event.newValue()),
+ toVersioned(event.oldValue())));
+ listenerMap.put(listener, atomixListener);
+ return atomixTreeMap.addListener(atomixListener, executor);
+ }
+
+ @Override
+ public CompletableFuture<Void> removeListener(MapEventListener<String, V> listener) {
+ io.atomix.core.map.AtomicMapEventListener<String, V> atomixListener = listenerMap.remove(listener);
+ if (atomixListener != null) {
+ return atomixTreeMap.removeListener(atomixListener);
+ }
+ return CompletableFuture.completedFuture(null);
+ }
+
+ @Override
+ public CompletableFuture<Version> begin(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<String, V>> transactionLog) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<String, V>> transactionLog) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Void> commit(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Void> rollback(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ private <T> CompletableFuture<T> adapt(CompletableFuture<T> future) {
+ CompletableFuture<T> newFuture = new CompletableFuture<>();
+ future.whenComplete((result, error) -> {
+ if (error == null) {
+ newFuture.complete(result);
+ } else {
+ Throwable cause = Throwables.getRootCause(error);
+ if (cause instanceof io.atomix.primitive.PrimitiveException.ConcurrentModification) {
+ newFuture.completeExceptionally(
+ new ConsistentMapException.ConcurrentModification(error.getMessage()));
+ } else if (cause instanceof io.atomix.primitive.PrimitiveException.Timeout) {
+ newFuture.completeExceptionally(new ConsistentMapException.Timeout(error.getMessage()));
+ } else if (cause instanceof io.atomix.primitive.PrimitiveException.Interrupted) {
+ newFuture.completeExceptionally(new ConsistentMapException.Interrupted());
+ } else if (cause instanceof io.atomix.primitive.PrimitiveException.Unavailable) {
+ newFuture.completeExceptionally(new ConsistentMapException.Unavailable());
+ } else if (cause instanceof io.atomix.primitive.PrimitiveException) {
+ newFuture.completeExceptionally(new ConsistentMapException(cause.getMessage()));
+ }
+ }
+ });
+ return newFuture;
+ }
+
+ private Versioned<V> toVersioned(io.atomix.utils.time.Versioned<V> versioned) {
+ return versioned != null
+ ? new Versioned<>(versioned.value(), versioned.version(), versioned.creationTime())
+ : null;
+ }
+
+ private Map.Entry<String, Versioned<V>> toVersioned(Map.Entry<String, io.atomix.utils.time.Versioned<V>> entry) {
+ return entry != null ? Maps.immutableEntry(entry.getKey(), toVersioned(entry.getValue())) : null;
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMapBuilder.java
new file mode 100644
index 0000000..14178fd
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMapBuilder.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncConsistentTreeMap;
+import org.onosproject.store.service.ConsistentTreeMap;
+import org.onosproject.store.service.ConsistentTreeMapBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncConsistentTreeMap} builder.
+ *
+ * @param <V> type for set value
+ */
+public class AtomixConsistentTreeMapBuilder<V> extends ConsistentTreeMapBuilder<V> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixConsistentTreeMapBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public ConsistentTreeMap<V> build() {
+ return buildTreeMap().asTreeMap();
+ }
+
+ @Override
+ public AsyncConsistentTreeMap<V> buildTreeMap() {
+ return new AtomixConsistentTreeMap<>(atomix.<String, V>atomicNavigableMapBuilder(name())
+ .withRegistrationRequired()
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withCacheEnabled(relaxedReadConsistency())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLock.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLock.java
new file mode 100644
index 0000000..ef2f502
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLock.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.time.Duration;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncDistributedLock;
+import org.onosproject.store.service.Version;
+
+/**
+ * Atomix distributed lock.
+ */
+public class AtomixDistributedLock implements AsyncDistributedLock {
+ private final io.atomix.core.lock.AsyncAtomicLock atomixLock;
+
+ public AtomixDistributedLock(io.atomix.core.lock.AsyncAtomicLock atomixLock) {
+ this.atomixLock = atomixLock;
+ }
+
+ @Override
+ public String name() {
+ return atomixLock.name();
+ }
+
+ @Override
+ public CompletableFuture<Version> lock() {
+ return atomixLock.lock().thenApply(this::toVersion);
+ }
+
+ @Override
+ public CompletableFuture<Optional<Version>> tryLock() {
+ return atomixLock.tryLock().thenApply(optional -> optional.map(this::toVersion));
+ }
+
+ @Override
+ public CompletableFuture<Optional<Version>> tryLock(Duration timeout) {
+ return atomixLock.tryLock(timeout).thenApply(optional -> optional.map(this::toVersion));
+ }
+
+ @Override
+ public CompletableFuture<Void> unlock() {
+ return atomixLock.unlock();
+ }
+
+ private Version toVersion(io.atomix.utils.time.Version version) {
+ return new Version(version.value());
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLockBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLockBuilder.java
new file mode 100644
index 0000000..e16a610
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLockBuilder.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncDistributedLock;
+import org.onosproject.store.service.DistributedLockBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncDistributedLock} builder.
+ */
+public class AtomixDistributedLockBuilder extends DistributedLockBuilder {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixDistributedLockBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public AsyncDistributedLock build() {
+ return new AtomixDistributedLock(atomix.atomicLockBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSet.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSet.java
new file mode 100644
index 0000000..e1aadb3
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSet.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Maps;
+import org.onosproject.store.service.AsyncDistributedSet;
+import org.onosproject.store.service.SetEvent;
+import org.onosproject.store.service.SetEventListener;
+
+/**
+ * Atomix distributed set.
+ */
+public class AtomixDistributedSet<E> implements AsyncDistributedSet<E> {
+ private final io.atomix.core.set.AsyncDistributedSet<E> atomixSet;
+ private final Map<SetEventListener<E>, io.atomix.core.collection.CollectionEventListener<E>> listenerMap =
+ Maps.newIdentityHashMap();
+
+ public AtomixDistributedSet(io.atomix.core.set.AsyncDistributedSet<E> atomixSet) {
+ this.atomixSet = atomixSet;
+ }
+
+ @Override
+ public String name() {
+ return atomixSet.name();
+ }
+
+ @Override
+ public CompletableFuture<Integer> size() {
+ return atomixSet.size();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> add(E element) {
+ return atomixSet.add(element);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> remove(E element) {
+ return atomixSet.remove(element);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> isEmpty() {
+ return atomixSet.isEmpty();
+ }
+
+ @Override
+ public CompletableFuture<Void> clear() {
+ return atomixSet.clear();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> contains(E element) {
+ return atomixSet.contains(element);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> addAll(Collection<? extends E> c) {
+ return atomixSet.addAll(c);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> containsAll(Collection<? extends E> c) {
+ return atomixSet.containsAll(c);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> retainAll(Collection<? extends E> c) {
+ return atomixSet.retainAll(c);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> removeAll(Collection<? extends E> c) {
+ return atomixSet.removeAll(c);
+ }
+
+ @Override
+ public CompletableFuture<? extends Set<E>> getAsImmutableSet() {
+ return CompletableFuture.completedFuture(atomixSet.stream().collect(Collectors.toSet()));
+ }
+
+ @Override
+ public synchronized CompletableFuture<Void> addListener(SetEventListener<E> listener) {
+ io.atomix.core.collection.CollectionEventListener<E> atomixListener = event ->
+ listener.event(new SetEvent<E>(
+ name(),
+ SetEvent.Type.valueOf(event.type().name()),
+ event.element()));
+ listenerMap.put(listener, atomixListener);
+ return atomixSet.addListener(atomixListener);
+ }
+
+ @Override
+ public CompletableFuture<Void> removeListener(SetEventListener<E> listener) {
+ io.atomix.core.collection.CollectionEventListener<E> atomixListener = listenerMap.remove(listener);
+ if (atomixListener != null) {
+ return atomixSet.removeListener(atomixListener);
+ }
+ return CompletableFuture.completedFuture(null);
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSetBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSetBuilder.java
new file mode 100644
index 0000000..73c6f9d
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSetBuilder.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncDistributedSet;
+import org.onosproject.store.service.DistributedSetBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncDistributedSet} builder.
+ *
+ * @param <E> type for set value
+ */
+public class AtomixDistributedSetBuilder<E> extends DistributedSetBuilder<E> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixDistributedSetBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public AsyncDistributedSet<E> build() {
+ return new AtomixDistributedSet<E>(atomix.<E>setBuilder(name())
+ .withRegistrationRequired()
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ // TODO: Enable caching for DistributedSet in Atomix
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopic.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopic.java
similarity index 78%
rename from core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopic.java
rename to core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopic.java
index 7581c37..01b7da8 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopic.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopic.java
@@ -20,30 +20,29 @@
import java.util.concurrent.Executor;
import java.util.function.Consumer;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AtomicValueEventListener;
+import com.google.common.collect.Maps;
+import io.atomix.core.value.AsyncAtomicValue;
+import io.atomix.core.value.AtomicValueEventListener;
import org.onosproject.store.service.DistributedPrimitive;
import org.onosproject.store.service.Topic;
-import com.google.common.collect.Maps;
-
/**
* Default implementation of {@link Topic}.
*
* @param <T> topic message type.
*/
-public class DefaultDistributedTopic<T> implements Topic<T> {
+public class AtomixDistributedTopic<T> implements Topic<T> {
- private final AsyncAtomicValue<T> atomicValue;
+ private final AsyncAtomicValue<T> atomixValue;
private final Map<Consumer<T>, AtomicValueEventListener<T>> callbacks = Maps.newIdentityHashMap();
- DefaultDistributedTopic(AsyncAtomicValue<T> atomicValue) {
- this.atomicValue = atomicValue;
+ AtomixDistributedTopic(AsyncAtomicValue<T> atomixValue) {
+ this.atomixValue = atomixValue;
}
@Override
public String name() {
- return atomicValue.name();
+ return atomixValue.name();
}
@Override
@@ -52,13 +51,8 @@
}
@Override
- public CompletableFuture<Void> destroy() {
- return atomicValue.destroy();
- }
-
- @Override
public CompletableFuture<Void> publish(T message) {
- return atomicValue.set(message);
+ return atomixValue.set(message);
}
@Override
@@ -66,7 +60,7 @@
AtomicValueEventListener<T> valueListener =
event -> executor.execute(() -> callback.accept(event.newValue()));
if (callbacks.putIfAbsent(callback, valueListener) == null) {
- return atomicValue.addListener(valueListener);
+ return atomixValue.addListener(valueListener);
}
return CompletableFuture.completedFuture(null);
}
@@ -75,8 +69,13 @@
public CompletableFuture<Void> unsubscribe(Consumer<T> callback) {
AtomicValueEventListener<T> valueListener = callbacks.remove(callback);
if (valueListener != null) {
- return atomicValue.removeListener(valueListener);
+ return atomixValue.removeListener(valueListener);
}
return CompletableFuture.completedFuture(null);
}
+
+ @Override
+ public CompletableFuture<Void> destroy() {
+ return atomixValue.close();
+ }
}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopicBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopicBuilder.java
new file mode 100644
index 0000000..34dda44
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopicBuilder.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.Topic;
+import org.onosproject.store.service.TopicBuilder;
+
+/**
+ * Default topic builder.
+ */
+public class AtomixDistributedTopicBuilder<T> extends TopicBuilder<T> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixDistributedTopicBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public Topic<T> build() {
+ return new AtomixDistributedTopic<>(atomix.<T>atomicValueBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTree.java
new file mode 100644
index 0000000..4bbc822
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTree.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Maps;
+import org.onosproject.store.primitives.NodeUpdate;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.AsyncDocumentTree;
+import org.onosproject.store.service.DocumentPath;
+import org.onosproject.store.service.DocumentTreeEvent;
+import org.onosproject.store.service.DocumentTreeListener;
+import org.onosproject.store.service.TransactionLog;
+import org.onosproject.store.service.Version;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * Atomix document tree.
+ */
+public class AtomixDocumentTree<V> implements AsyncDocumentTree<V> {
+ private final io.atomix.core.tree.AsyncAtomicDocumentTree<V> atomixTree;
+ private final Map<DocumentTreeListener<V>, io.atomix.core.tree.DocumentTreeEventListener<V>> listenerMap =
+ Maps.newIdentityHashMap();
+
+ public AtomixDocumentTree(io.atomix.core.tree.AsyncAtomicDocumentTree<V> atomixTree) {
+ this.atomixTree = atomixTree;
+ }
+
+ @Override
+ public String name() {
+ return atomixTree.name();
+ }
+
+ @Override
+ public DocumentPath root() {
+ return toOnosPath(atomixTree.root());
+ }
+
+ @Override
+ public CompletableFuture<Map<String, Versioned<V>>> getChildren(DocumentPath path) {
+ return atomixTree.getChildren(toAtomixPath(path))
+ .thenApply(map -> map.entrySet().stream()
+ .collect(Collectors.toMap(e -> e.getKey(),
+ e -> toVersioned(e.getValue()))));
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> get(DocumentPath path) {
+ return atomixTree.get(toAtomixPath(path)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
+ return atomixTree.set(toAtomixPath(path), value).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> create(DocumentPath path, V value) {
+ return atomixTree.create(toAtomixPath(path), value);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
+ return atomixTree.createRecursive(toAtomixPath(path), value);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
+ return atomixTree.replace(toAtomixPath(path), newValue, version);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
+ return atomixTree.replace(toAtomixPath(path), newValue, currentValue);
+ }
+
+ @Override
+ public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
+ return atomixTree.removeNode(toAtomixPath(path)).thenApply(this::toVersioned);
+ }
+
+ @Override
+ public synchronized CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
+ io.atomix.core.tree.DocumentTreeEventListener<V> atomixListener = event ->
+ listener.event(new DocumentTreeEvent<V>(
+ DocumentPath.from(event.path().pathElements()),
+ DocumentTreeEvent.Type.valueOf(event.type().name()),
+ event.newValue().map(this::toVersioned),
+ event.oldValue().map(this::toVersioned)));
+ listenerMap.put(listener, atomixListener);
+ return atomixTree.addListener(toAtomixPath(path), atomixListener);
+ }
+
+ @Override
+ public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
+ io.atomix.core.tree.DocumentTreeEventListener<V> atomixListener = listenerMap.remove(listener);
+ if (atomixListener != null) {
+ return atomixTree.removeListener(atomixListener);
+ }
+ return CompletableFuture.completedFuture(null);
+ }
+
+ @Override
+ public CompletableFuture<Version> begin(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<V>> transactionLog) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<V>> transactionLog) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Void> commit(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public CompletableFuture<Void> rollback(TransactionId transactionId) {
+ throw new UnsupportedOperationException();
+ }
+
+ private DocumentPath toOnosPath(io.atomix.core.tree.DocumentPath path) {
+ return DocumentPath.from(path.pathElements());
+ }
+
+ private io.atomix.core.tree.DocumentPath toAtomixPath(DocumentPath path) {
+ return io.atomix.core.tree.DocumentPath.from(path.pathElements());
+ }
+
+ private Versioned<V> toVersioned(io.atomix.utils.time.Versioned<V> versioned) {
+ return versioned != null
+ ? new Versioned<>(versioned.value(), versioned.version(), versioned.creationTime())
+ : null;
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTreeBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTreeBuilder.java
new file mode 100644
index 0000000..91dea85
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTreeBuilder.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncDocumentTree;
+import org.onosproject.store.service.DocumentTreeBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncDocumentTree} builder.
+ *
+ * @param <V> type for tree value
+ */
+public class AtomixDocumentTreeBuilder<V> extends DocumentTreeBuilder<V> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixDocumentTreeBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public AsyncDocumentTree<V> build() {
+ return buildDocumentTree();
+ }
+
+ @Override
+ public AsyncDocumentTree<V> buildDocumentTree() {
+ return new AtomixDocumentTree<>(atomix.<V>atomicDocumentTreeBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ // TODO: Enable caching for DocumentTree in Atomix
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixIterator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixIterator.java
new file mode 100644
index 0000000..c3288d3
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixIterator.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncIterator;
+
+/**
+ * Atomix asynchronous iterator.
+ */
+public class AtomixIterator<T> implements AsyncIterator<T> {
+ private final io.atomix.core.iterator.AsyncIterator<T> atomixIterator;
+
+ public AtomixIterator(io.atomix.core.iterator.AsyncIterator<T> atomixIterator) {
+ this.atomixIterator = atomixIterator;
+ }
+
+ @Override
+ public CompletableFuture<Boolean> hasNext() {
+ return atomixIterator.hasNext();
+ }
+
+ @Override
+ public CompletableFuture<T> next() {
+ return atomixIterator.next();
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElector.java
new file mode 100644
index 0000000..1098981
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElector.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Maps;
+import io.atomix.core.election.LeadershipEventListener;
+import org.onosproject.cluster.Leader;
+import org.onosproject.cluster.Leadership;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.event.Change;
+import org.onosproject.store.service.AsyncLeaderElector;
+
+/**
+ * Atomix leader elector.
+ */
+public class AtomixLeaderElector implements AsyncLeaderElector {
+ private final io.atomix.core.election.AsyncLeaderElector<NodeId> atomixElector;
+ private final NodeId localNodeId;
+ private final Map<Consumer<Change<Leadership>>, LeadershipEventListener<NodeId>> listenerMap =
+ Maps.newIdentityHashMap();
+
+ public AtomixLeaderElector(io.atomix.core.election.AsyncLeaderElector<NodeId> atomixElector, NodeId localNodeId) {
+ this.atomixElector = atomixElector;
+ this.localNodeId = localNodeId;
+ }
+
+ @Override
+ public String name() {
+ return atomixElector.name();
+ }
+
+ @Override
+ public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
+ return atomixElector.run(topic, nodeId).thenApply(leadership -> toLeadership(topic, leadership));
+ }
+
+ @Override
+ public CompletableFuture<Void> withdraw(String topic) {
+ return atomixElector.withdraw(topic, localNodeId);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
+ return atomixElector.anoint(topic, nodeId);
+ }
+
+ @Override
+ public CompletableFuture<Void> evict(NodeId nodeId) {
+ return atomixElector.evict(nodeId);
+ }
+
+ @Override
+ public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
+ return atomixElector.promote(topic, nodeId);
+ }
+
+ @Override
+ public CompletableFuture<Leadership> getLeadership(String topic) {
+ return atomixElector.getLeadership(topic).thenApply(leadership -> toLeadership(topic, leadership));
+ }
+
+ @Override
+ public CompletableFuture<Map<String, Leadership>> getLeaderships() {
+ return atomixElector.getLeaderships()
+ .thenApply(leaderships -> leaderships.entrySet().stream()
+ .collect(Collectors.toMap(e -> e.getKey(), e -> toLeadership(e.getKey(), e.getValue()))));
+ }
+
+ @Override
+ public CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> consumer) {
+ LeadershipEventListener<NodeId> atomixListener = event ->
+ consumer.accept(new Change<>(
+ toLeadership(event.topic(), event.oldLeadership()),
+ toLeadership(event.topic(), event.newLeadership())));
+ listenerMap.put(consumer, atomixListener);
+ return atomixElector.addListener(atomixListener);
+ }
+
+ @Override
+ public CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
+ LeadershipEventListener<NodeId> atomixListener = listenerMap.remove(consumer);
+ if (atomixListener != null) {
+ return atomixElector.removeListener(atomixListener);
+ }
+ return CompletableFuture.completedFuture(null);
+ }
+
+ private Leadership toLeadership(String topic, io.atomix.core.election.Leadership<NodeId> leadership) {
+ return leadership != null
+ ? new Leadership(topic, toLeader(leadership.leader()), leadership.candidates())
+ : null;
+ }
+
+ private Leader toLeader(io.atomix.core.election.Leader<NodeId> leader) {
+ return leader != null ? new Leader(leader.id(), leader.term(), leader.timestamp()) : null;
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElectorBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElectorBuilder.java
new file mode 100644
index 0000000..feb8127
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElectorBuilder.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.store.serializers.KryoNamespaces;
+import org.onosproject.store.service.AsyncLeaderElector;
+import org.onosproject.store.service.LeaderElectorBuilder;
+import org.onosproject.store.service.Serializer;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncLeaderElector} builder.
+ */
+public class AtomixLeaderElectorBuilder extends LeaderElectorBuilder {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+ private final NodeId localNodeId;
+
+ public AtomixLeaderElectorBuilder(Atomix atomix, String group, NodeId localNodeId) {
+ this.atomix = atomix;
+ this.group = group;
+ this.localNodeId = localNodeId;
+ }
+
+ @Override
+ public AsyncLeaderElector build() {
+ Serializer serializer = Serializer.using(KryoNamespaces.API);
+ return new AtomixLeaderElector(atomix.<NodeId>leaderElectorBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ // TODO: Enable caching for LeaderElector in Atomix
+ .withSerializer(new AtomixSerializerAdapter(serializer))
+ .build()
+ .async(), localNodeId);
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixSerializerAdapter.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixSerializerAdapter.java
similarity index 80%
rename from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixSerializerAdapter.java
rename to core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixSerializerAdapter.java
index abb1580..46ca0b8 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixSerializerAdapter.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixSerializerAdapter.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2017-present Open Networking Foundation
+ * Copyright 2018-present Open Networking Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -13,14 +13,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.onosproject.store.primitives.resources.impl;
+package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.Serializer;
/**
- * ONOS to Atomix serializer adapter.
+ * Atomix serializer.
*/
-public class AtomixSerializerAdapter implements io.atomix.serializer.Serializer {
+public class AtomixSerializerAdapter implements io.atomix.utils.serializer.Serializer {
private final Serializer serializer;
public AtomixSerializerAdapter(Serializer serializer) {
@@ -36,4 +36,4 @@
public <T> T decode(byte[] bytes) {
return serializer.decode(bytes);
}
-}
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContext.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContext.java
new file mode 100644
index 0000000..9260591
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContext.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import io.atomix.core.transaction.AsyncTransaction;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.CommitStatus;
+import org.onosproject.store.service.DistributedPrimitive;
+import org.onosproject.store.service.Serializer;
+import org.onosproject.store.service.StorageException;
+import org.onosproject.store.service.TransactionContext;
+import org.onosproject.store.service.TransactionalMap;
+
+/**
+ * Atomix transaction context.
+ */
+public class AtomixTransactionContext implements TransactionContext {
+ private static final int MAX_RETRIES = 5;
+
+ private final AsyncTransaction atomixTransaction;
+ private final String group;
+
+ public AtomixTransactionContext(AsyncTransaction atomixTransaction, String group) {
+ this.atomixTransaction = atomixTransaction;
+ this.group = group;
+ }
+
+ @Override
+ public String name() {
+ return atomixTransaction.name();
+ }
+
+ @Override
+ public TransactionId transactionId() {
+ return TransactionId.from(atomixTransaction.transactionId().id());
+ }
+
+ @Override
+ public boolean isOpen() {
+ return atomixTransaction.isOpen();
+ }
+
+ @Override
+ public void begin() {
+ try {
+ atomixTransaction.begin().get(DistributedPrimitive.DEFAULT_OPERATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ throw new StorageException.Interrupted();
+ } catch (TimeoutException e) {
+ throw new StorageException.Timeout();
+ } catch (ExecutionException e) {
+ throw new StorageException.Unavailable();
+ }
+ }
+
+ @Override
+ public CompletableFuture<CommitStatus> commit() {
+ return atomixTransaction.commit().thenApply(status -> CommitStatus.valueOf(status.name()));
+ }
+
+ @Override
+ public void abort() {
+ try {
+ atomixTransaction.abort().get(DistributedPrimitive.DEFAULT_OPERATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ throw new StorageException.Interrupted();
+ } catch (TimeoutException e) {
+ throw new StorageException.Timeout();
+ } catch (ExecutionException e) {
+ throw new StorageException.Unavailable();
+ }
+ }
+
+ @Override
+ public <K, V> TransactionalMap<K, V> getTransactionalMap(String mapName, Serializer serializer) {
+ return new AtomixTransactionalMap<>(atomixTransaction.<K, V>mapBuilder(mapName)
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withSerializer(new AtomixSerializerAdapter(serializer))
+ .build());
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContextBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContextBuilder.java
new file mode 100644
index 0000000..70ef628
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContextBuilder.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.core.transaction.Isolation;
+import org.onosproject.store.service.TransactionContext;
+import org.onosproject.store.service.TransactionContextBuilder;
+
+/**
+ * Atomix transaction context builder.
+ */
+public class AtomixTransactionContextBuilder extends TransactionContextBuilder {
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixTransactionContextBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public TransactionContext build() {
+ return new AtomixTransactionContext(atomix.transactionBuilder()
+ .withIsolation(Isolation.REPEATABLE_READS)
+ .build()
+ .async(), group);
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionalMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionalMap.java
new file mode 100644
index 0000000..8577a84
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionalMap.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import org.onosproject.store.service.TransactionalMap;
+
+/**
+ * Atomix transactional map.
+ */
+public class AtomixTransactionalMap<K, V> implements TransactionalMap<K, V> {
+ private final io.atomix.core.transaction.TransactionalMap<K, V> atomixMap;
+
+ public AtomixTransactionalMap(io.atomix.core.transaction.TransactionalMap<K, V> atomixMap) {
+ this.atomixMap = atomixMap;
+ }
+
+ @Override
+ public V get(K key) {
+ return atomixMap.get(key);
+ }
+
+ @Override
+ public boolean containsKey(K key) {
+ return atomixMap.containsKey(key);
+ }
+
+ @Override
+ public V put(K key, V value) {
+ return atomixMap.put(key, value);
+ }
+
+ @Override
+ public V remove(K key) {
+ return atomixMap.remove(key);
+ }
+
+ @Override
+ public V putIfAbsent(K key, V value) {
+ return atomixMap.putIfAbsent(key, value);
+ }
+
+ @Override
+ public boolean remove(K key, V value) {
+ return atomixMap.remove(key, value);
+ }
+
+ @Override
+ public boolean replace(K key, V oldValue, V newValue) {
+ return atomixMap.replace(key, oldValue, newValue);
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueue.java
new file mode 100644
index 0000000..d3b9907
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueue.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.onosproject.store.service.Task;
+import org.onosproject.store.service.WorkQueue;
+import org.onosproject.store.service.WorkQueueStats;
+
+/**
+ * Atomix work queue.
+ */
+public class AtomixWorkQueue<E> implements WorkQueue<E> {
+ private final io.atomix.core.workqueue.AsyncWorkQueue<E> atomixWorkQueue;
+
+ public AtomixWorkQueue(io.atomix.core.workqueue.AsyncWorkQueue<E> atomixWorkQueue) {
+ this.atomixWorkQueue = atomixWorkQueue;
+ }
+
+ @Override
+ public String name() {
+ return atomixWorkQueue.name();
+ }
+
+ @Override
+ public CompletableFuture<Void> addMultiple(Collection<E> items) {
+ return atomixWorkQueue.addMultiple(items);
+ }
+
+ @Override
+ public CompletableFuture<Collection<Task<E>>> take(int maxItems) {
+ return atomixWorkQueue.take(maxItems)
+ .thenApply(tasks -> tasks.stream()
+ .map(task -> new Task<>(task.taskId(), task.payload()))
+ .collect(Collectors.toList()));
+ }
+
+ @Override
+ public CompletableFuture<Void> complete(Collection<String> taskIds) {
+ return atomixWorkQueue.complete(taskIds);
+ }
+
+ @Override
+ public CompletableFuture<Void> registerTaskProcessor(
+ Consumer<E> taskProcessor, int parallelism, Executor executor) {
+ return atomixWorkQueue.registerTaskProcessor(taskProcessor, parallelism, executor);
+ }
+
+ @Override
+ public CompletableFuture<Void> stopProcessing() {
+ return atomixWorkQueue.stopProcessing();
+ }
+
+ @Override
+ public CompletableFuture<WorkQueueStats> stats() {
+ return atomixWorkQueue.stats()
+ .thenApply(stats -> WorkQueueStats.builder()
+ .withTotalCompleted(stats.totalCompleted())
+ .withTotalInProgress(stats.totalInProgress())
+ .withTotalPending(stats.totalPending())
+ .build());
+ }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueueBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueueBuilder.java
new file mode 100644
index 0000000..9c65106
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueueBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * Licensed 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.onosproject.store.primitives.impl;
+
+import io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.WorkQueue;
+import org.onosproject.store.service.WorkQueueBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.WorkQueue} builder.
+ *
+ * @param <E> type for set value
+ */
+public class AtomixWorkQueueBuilder<E> extends WorkQueueBuilder<E> {
+ private static final int MAX_RETRIES = 5;
+ private final Atomix atomix;
+ private final String group;
+
+ public AtomixWorkQueueBuilder(Atomix atomix, String group) {
+ this.atomix = atomix;
+ this.group = group;
+ }
+
+ @Override
+ public WorkQueue<E> build() {
+ return new AtomixWorkQueue<>(atomix.<E>workQueueBuilder(name())
+ .withProtocol(MultiRaftProtocol.builder(group)
+ .withRecoveryStrategy(Recovery.RECOVER)
+ .withMaxRetries(MAX_RETRIES)
+ .build())
+ .withReadOnly(readOnly())
+ .withSerializer(new AtomixSerializerAdapter(serializer()))
+ .build()
+ .async());
+ }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMap.java
deleted file mode 100644
index a25fb79..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMap.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import static org.slf4j.LoggerFactory.getLogger;
-
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-
-import com.google.common.util.concurrent.MoreExecutors;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Versioned;
-import org.slf4j.Logger;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-
-import static org.onosproject.store.service.DistributedPrimitive.Status.INACTIVE;
-import static org.onosproject.store.service.DistributedPrimitive.Status.SUSPENDED;
-
-/**
- * {@code AsyncConsistentMap} that caches entries on read.
- * <p>
- * The cache entries are automatically invalidated when updates are detected either locally or
- * remotely.
- * <p> This implementation only attempts to serve cached entries for {@link AsyncConsistentMap#get get}
- * {@link AsyncConsistentMap#getOrDefault(Object, Object) getOrDefault}, and
- * {@link AsyncConsistentMap#containsKey(Object) containsKey} calls. All other calls skip the cache
- * and directly go the backing map.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class CachingAsyncConsistentMap<K, V> extends DelegatingAsyncConsistentMap<K, V> {
- private static final int DEFAULT_CACHE_SIZE = 10000;
- private final Logger log = getLogger(getClass());
-
- private final Map<MapEventListener<K, V>, Executor> mapEventListeners = new ConcurrentHashMap<>();
- private final LoadingCache<K, CompletableFuture<Versioned<V>>> cache;
- private final AsyncConsistentMap<K, V> backingMap;
- private final MapEventListener<K, V> cacheUpdater;
- private final Consumer<Status> statusListener;
-
- /**
- * Default constructor.
- *
- * @param backingMap a distributed, strongly consistent map for backing
- */
- public CachingAsyncConsistentMap(AsyncConsistentMap<K, V> backingMap) {
- this(backingMap, DEFAULT_CACHE_SIZE);
- }
-
- /**
- * Constructor to configure cache size.
- *
- * @param backingMap a distributed, strongly consistent map for backing
- * @param cacheSize the maximum size of the cache
- */
- public CachingAsyncConsistentMap(AsyncConsistentMap<K, V> backingMap, int cacheSize) {
- super(backingMap);
- this.backingMap = backingMap;
- cache = CacheBuilder.newBuilder()
- .maximumSize(cacheSize)
- .build(CacheLoader.from(CachingAsyncConsistentMap.super::get));
- cacheUpdater = event -> {
- Versioned<V> newValue = event.newValue();
- if (newValue == null) {
- cache.invalidate(event.key());
- } else {
- cache.put(event.key(), CompletableFuture.completedFuture(newValue));
- }
- mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event)));
- };
- statusListener = status -> {
- log.debug("{} status changed to {}", this.name(), status);
- // If the status of the underlying map is SUSPENDED or INACTIVE
- // we can no longer guarantee that the cache will be in sync.
- if (status == SUSPENDED || status == INACTIVE) {
- cache.invalidateAll();
- }
- };
- super.addListener(cacheUpdater, MoreExecutors.directExecutor());
- super.addStatusChangeListener(statusListener);
- }
-
- @Override
- public CompletableFuture<Void> destroy() {
- super.removeStatusChangeListener(statusListener);
- return super.destroy().thenCompose(v -> removeListener(cacheUpdater));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(K key) {
- return cache.getUnchecked(key)
- .whenComplete((r, e) -> {
- if (e != null) {
- cache.invalidate(key);
- }
- });
- }
-
- @Override
- public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
- return cache.getUnchecked(key).thenCompose(r -> {
- if (r == null) {
- CompletableFuture<Versioned<V>> versioned = backingMap.getOrDefault(key, defaultValue);
- cache.put(key, versioned);
- return versioned;
- } else {
- return CompletableFuture.completedFuture(r);
- }
- }).whenComplete((r, e) -> {
- if (e != null) {
- cache.invalidate(key);
- }
- });
- }
-
- @Override
- public CompletableFuture<Versioned<V>> computeIf(K key,
- Predicate<? super V> condition,
- BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- return super.computeIf(key, condition, remappingFunction)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(K key, V value) {
- return super.put(key, value)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
- return super.putAndGet(key, value)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
- return super.putIfAbsent(key, value)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> remove(K key) {
- return super.remove(key)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- return cache.getUnchecked(key).thenApply(Objects::nonNull)
- .whenComplete((r, e) -> {
- if (e != null) {
- cache.invalidate(key);
- }
- });
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return super.clear()
- .whenComplete((r, e) -> cache.invalidateAll());
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return super.remove(key, value)
- .whenComplete((r, e) -> {
- if (r) {
- cache.invalidate(key);
- }
- });
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, long version) {
- return super.remove(key, version)
- .whenComplete((r, e) -> {
- if (r) {
- cache.invalidate(key);
- }
- });
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(K key, V value) {
- return super.replace(key, value)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
- return super.replace(key, oldValue, newValue)
- .whenComplete((r, e) -> {
- if (r) {
- cache.invalidate(key);
- }
- });
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
- return super.replace(key, oldVersion, newValue)
- .whenComplete((r, e) -> {
- if (r) {
- cache.invalidate(key);
- }
- });
- }
-
- @Override
- public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
- mapEventListeners.put(listener, executor);
- return CompletableFuture.completedFuture(null);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
- mapEventListeners.remove(listener);
- return CompletableFuture.completedFuture(null);
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimap.java
deleted file mode 100644
index 3d49f7f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimap.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.util.concurrent.MoreExecutors;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-import org.slf4j.Logger;
-
-import static org.onosproject.store.service.DistributedPrimitive.Status.INACTIVE;
-import static org.onosproject.store.service.DistributedPrimitive.Status.SUSPENDED;
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * Caching {@link AsyncConsistentMultimap} implementation.
- */
-public class CachingAsyncConsistentMultimap<K, V> extends DelegatingAsyncConsistentMultimap<K, V> {
- private static final int DEFAULT_CACHE_SIZE = 10000;
- private final Logger log = getLogger(getClass());
-
- private final Map<MultimapEventListener<K, V>, Executor> mapEventListeners = new ConcurrentHashMap<>();
- private final LoadingCache<K, CompletableFuture<Versioned<Collection<? extends V>>>> cache;
- private final MultimapEventListener<K, V> cacheUpdater;
- private final Consumer<Status> statusListener;
-
- /**
- * Default constructor.
- *
- * @param backingMap a distributed, strongly consistent map for backing
- */
- public CachingAsyncConsistentMultimap(AsyncConsistentMultimap<K, V> backingMap) {
- this(backingMap, DEFAULT_CACHE_SIZE);
- }
-
- /**
- * Constructor to configure cache size.
- *
- * @param backingMap a distributed, strongly consistent map for backing
- * @param cacheSize the maximum size of the cache
- */
- public CachingAsyncConsistentMultimap(AsyncConsistentMultimap<K, V> backingMap, int cacheSize) {
- super(backingMap);
- cache = CacheBuilder.newBuilder()
- .maximumSize(cacheSize)
- .build(CacheLoader.from(CachingAsyncConsistentMultimap.super::get));
- cacheUpdater = event -> {
- V oldValue = event.oldValue();
- V newValue = event.newValue();
- CompletableFuture<Versioned<Collection<? extends V>>> future = cache.getUnchecked(event.key());
- switch (event.type()) {
- case INSERT:
- if (future.isDone()) {
- Versioned<Collection<? extends V>> oldVersioned = future.join();
- Versioned<Collection<? extends V>> newVersioned = new Versioned<>(
- ImmutableSet.<V>builder().addAll(oldVersioned.value()).add(newValue).build(),
- oldVersioned.version(),
- oldVersioned.creationTime());
- cache.put(event.key(), CompletableFuture.completedFuture(newVersioned));
- } else {
- cache.put(event.key(), future.thenApply(versioned -> new Versioned<>(
- ImmutableSet.<V>builder().addAll(versioned.value()).add(newValue).build(),
- versioned.version(),
- versioned.creationTime())));
- }
- break;
- case REMOVE:
- if (future.isDone()) {
- Versioned<Collection<? extends V>> oldVersioned = future.join();
- cache.put(event.key(), CompletableFuture.completedFuture(new Versioned<>(oldVersioned.value()
- .stream()
- .filter(value -> !Objects.equals(value, oldValue))
- .collect(Collectors.toSet()), oldVersioned.version(), oldVersioned.creationTime())));
- } else {
- cache.put(event.key(), future.thenApply(versioned -> new Versioned<>(versioned.value()
- .stream()
- .filter(value -> !Objects.equals(value, oldValue))
- .collect(Collectors.toSet()), versioned.version(), versioned.creationTime())));
- }
- break;
- default:
- break;
- }
- mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event)));
- };
- statusListener = status -> {
- log.debug("{} status changed to {}", this.name(), status);
- // If the status of the underlying map is SUSPENDED or INACTIVE
- // we can no longer guarantee that the cache will be in sync.
- if (status == SUSPENDED || status == INACTIVE) {
- cache.invalidateAll();
- }
- };
- super.addListener(cacheUpdater, MoreExecutors.directExecutor());
- super.addStatusChangeListener(statusListener);
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- return get(key).thenApply(value -> value != null && !value.value().isEmpty());
- }
-
- @Override
- public CompletableFuture<Boolean> put(K key, V value) {
- return super.put(key, value)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return super.remove(key, value)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Boolean> removeAll(K key, Collection<? extends V> values) {
- return super.removeAll(key, values)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> removeAll(K key) {
- return super.removeAll(key)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Boolean> putAll(K key, Collection<? extends V> values) {
- return super.putAll(key, values)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> replaceValues(K key, Collection<V> values) {
- return super.replaceValues(key, values)
- .whenComplete((r, e) -> cache.invalidate(key));
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return super.clear()
- .whenComplete((r, e) -> cache.invalidateAll());
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> get(K key) {
- return cache.getUnchecked(key)
- .whenComplete((r, e) -> {
- if (e != null) {
- cache.invalidate(key);
- }
- });
- }
-
- @Override
- public CompletableFuture<Void> addListener(MultimapEventListener<K, V> listener, Executor executor) {
- mapEventListeners.put(listener, executor);
- return CompletableFuture.completedFuture(null);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MultimapEventListener<K, V> listener) {
- mapEventListeners.remove(listener);
- return CompletableFuture.completedFuture(null);
- }
-
- @Override
- public CompletableFuture<Void> destroy() {
- super.removeStatusChangeListener(statusListener);
- return super.destroy().thenCompose(v -> removeListener(cacheUpdater));
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTree.java
deleted file mode 100644
index eece91e..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTree.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.Maps;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.Versioned;
-import org.slf4j.Logger;
-
-import static org.onosproject.store.service.DistributedPrimitive.Status.INACTIVE;
-import static org.onosproject.store.service.DistributedPrimitive.Status.SUSPENDED;
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * Caching asynchronous document tree.
- */
-public class CachingAsyncDocumentTree<V> extends DelegatingAsyncDocumentTree<V> implements AsyncDocumentTree<V> {
- private static final int DEFAULT_CACHE_SIZE = 10000;
- private final Logger log = getLogger(getClass());
-
- private final Map<DocumentTreeListener<V>, DocumentTreeListener<V>> eventListeners = Maps.newConcurrentMap();
- private final LoadingCache<DocumentPath, CompletableFuture<Versioned<V>>> cache;
- private final DocumentTreeListener<V> cacheUpdater;
- private final Consumer<Status> statusListener;
-
- /**
- * Default constructor.
- *
- * @param backingTree a distributed, strongly consistent map for backing
- */
- public CachingAsyncDocumentTree(AsyncDocumentTree<V> backingTree) {
- this(backingTree, DEFAULT_CACHE_SIZE);
- }
-
- /**
- * Constructor to configure cache size.
- *
- * @param backingTree a distributed, strongly consistent map for backing
- * @param cacheSize the maximum size of the cache
- */
- public CachingAsyncDocumentTree(AsyncDocumentTree<V> backingTree, int cacheSize) {
- super(backingTree);
- cache = CacheBuilder.newBuilder()
- .maximumSize(cacheSize)
- .build(CacheLoader.from(CachingAsyncDocumentTree.super::get));
- cacheUpdater = event -> {
- if (!event.newValue().isPresent()) {
- cache.invalidate(event.path());
- } else {
- cache.put(event.path(), CompletableFuture.completedFuture(event.newValue().get()));
- }
- eventListeners.values().forEach(listener -> listener.event(event));
- };
- statusListener = status -> {
- log.debug("{} status changed to {}", this.name(), status);
- // If the status of the underlying map is SUSPENDED or INACTIVE
- // we can no longer guarantee that the cache will be in sync.
- if (status == SUSPENDED || status == INACTIVE) {
- cache.invalidateAll();
- }
- };
- super.addListener(root(), cacheUpdater);
- super.addStatusChangeListener(statusListener);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(DocumentPath path) {
- return cache.getUnchecked(path);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
- return super.set(path, value)
- .whenComplete((r, e) -> cache.invalidate(path));
- }
-
- @Override
- public CompletableFuture<Boolean> create(DocumentPath path, V value) {
- return super.create(path, value)
- .whenComplete((r, e) -> cache.invalidate(path));
- }
-
- @Override
- public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
- return super.createRecursive(path, value)
- .whenComplete((r, e) -> cache.invalidate(path));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
- return super.replace(path, newValue, version)
- .whenComplete((r, e) -> {
- if (r) {
- cache.invalidate(path);
- }
- });
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
- return super.replace(path, newValue, currentValue)
- .whenComplete((r, e) -> {
- if (r) {
- cache.invalidate(path);
- }
- });
- }
-
- @Override
- public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
- return super.removeNode(path)
- .whenComplete((r, e) -> cache.invalidate(path));
- }
-
- @Override
- public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
- eventListeners.put(listener, new InternalListener(path, listener));
- return CompletableFuture.completedFuture(null);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
- eventListeners.remove(listener);
- return CompletableFuture.completedFuture(null);
- }
-
- private class InternalListener implements DocumentTreeListener<V> {
- private final DocumentPath path;
- private final DocumentTreeListener<V> listener;
-
- public InternalListener(DocumentPath path, DocumentTreeListener<V> listener) {
- this.path = path;
- this.listener = listener;
- }
-
- @Override
- public void event(DocumentTreeEvent<V> event) {
- if (event.path().isDescendentOf(path)) {
- listener.event(event);
- }
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncLeaderElector.java
deleted file mode 100644
index 91dcca5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncLeaderElector.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.service.AsyncLeaderElector;
-
-/**
- * Caching async leader elector.
- */
-public class CachingAsyncLeaderElector extends DelegatingAsyncLeaderElector {
- private final LoadingCache<String, CompletableFuture<Leadership>> cache;
- private final Consumer<Change<Leadership>> cacheUpdater;
- private final Consumer<Status> statusListener;
-
- public CachingAsyncLeaderElector(AsyncLeaderElector delegateLeaderElector) {
- super(delegateLeaderElector);
- cache = CacheBuilder.newBuilder()
- .maximumSize(1000)
- .build(CacheLoader.from(super::getLeadership));
-
- cacheUpdater = change -> {
- Leadership leadership = change.newValue();
- cache.put(leadership.topic(), CompletableFuture.completedFuture(leadership));
- };
- statusListener = status -> {
- if (status == Status.SUSPENDED || status == Status.INACTIVE) {
- cache.invalidateAll();
- }
- };
- addChangeListener(cacheUpdater);
- addStatusChangeListener(statusListener);
- }
-
- @Override
- public CompletableFuture<Leadership> getLeadership(String topic) {
- return cache.getUnchecked(topic)
- .whenComplete((r, e) -> {
- if (e != null) {
- cache.invalidate(topic);
- }
- });
- }
-
- @Override
- public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
- return super.run(topic, nodeId).whenComplete((r, e) -> cache.invalidate(topic));
- }
-
- @Override
- public CompletableFuture<Void> withdraw(String topic) {
- return super.withdraw(topic).whenComplete((r, e) -> cache.invalidate(topic));
- }
-
- @Override
- public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
- return super.anoint(topic, nodeId).whenComplete((r, e) -> cache.invalidate(topic));
- }
-
- @Override
- public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
- return super.promote(topic, nodeId).whenComplete((r, e) -> cache.invalidate(topic));
- }
-
- @Override
- public CompletableFuture<Void> evict(NodeId nodeId) {
- return super.evict(nodeId).whenComplete((r, e) -> cache.invalidateAll());
- }
-
- @Override
- public CompletableFuture<Void> destroy() {
- removeStatusChangeListener(statusListener);
- return removeChangeListener(cacheUpdater);
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CompatibleValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CompatibleValue.java
deleted file mode 100644
index 73c4d12..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CompatibleValue.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.core.Version;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-/**
- * Compatibility wrapper for primitive values.
- */
-public class CompatibleValue<T> {
- private final T value;
- private final Version version;
-
- public CompatibleValue(T value, Version version) {
- this.value = value;
- this.version = version;
- }
-
- /**
- * Returns the wrapped value.
- *
- * @return the wrapped value
- */
- public T value() {
- return value;
- }
-
- /**
- * Returns the compatibilty version.
- *
- * @return the compatibility version
- */
- public Version version() {
- return version;
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("value", value)
- .add("version", version)
- .toString();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CoordinationManager.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CoordinationManager.java
index 1f6310f..aa9a78ef 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CoordinationManager.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CoordinationManager.java
@@ -15,9 +15,12 @@
*/
package org.onosproject.store.primitives.impl;
+import java.util.List;
import java.util.function.Supplier;
import java.util.stream.Collectors;
+import io.atomix.core.Atomix;
+import io.atomix.primitive.partition.PartitionGroup;
import org.apache.felix.scr.annotations.Activate;
import org.apache.felix.scr.annotations.Component;
import org.apache.felix.scr.annotations.Deactivate;
@@ -26,14 +29,10 @@
import org.apache.felix.scr.annotations.Service;
import org.onosproject.cluster.ClusterService;
import org.onosproject.cluster.ControllerNode;
-import org.onosproject.cluster.DefaultPartition;
import org.onosproject.cluster.NodeId;
-import org.onosproject.cluster.PartitionId;
import org.onosproject.persistence.PersistenceService;
import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicValue;
+import org.onosproject.store.impl.AtomixManager;
import org.onosproject.store.service.AsyncConsistentMultimap;
import org.onosproject.store.service.AsyncConsistentTreeMap;
import org.onosproject.store.service.AsyncDocumentTree;
@@ -54,17 +53,15 @@
import org.onosproject.store.service.TopicBuilder;
import org.onosproject.store.service.TransactionContextBuilder;
import org.onosproject.store.service.WorkQueue;
+import org.onosproject.store.service.WorkQueueBuilder;
import org.slf4j.Logger;
-import java.util.List;
-
import static org.onosproject.security.AppGuard.checkPermission;
import static org.onosproject.security.AppPermission.Type.STORAGE_WRITE;
import static org.slf4j.LoggerFactory.getLogger;
/**
- * Implementation of {@code CoordinationService} that uses a {@link StoragePartition} that spans all the nodes
- * in the cluster regardless of version.
+ * Implementation of {@code CoordinationService} that uses the Atomix management partition group.
*/
@Service
@Component(immediate = true)
@@ -81,23 +78,16 @@
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
protected PersistenceService persistenceService;
- private StoragePartition partition;
- private DistributedPrimitiveCreator primitiveCreator;
+ @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+ protected AtomixManager atomixManager;
+
+ private Atomix atomix;
+ private PartitionGroup group;
@Activate
public void activate() {
- partition = new StoragePartition(
- new DefaultPartition(
- PartitionId.from(0),
- null,
- clusterService.getNodes()
- .stream()
- .map(ControllerNode::id)
- .collect(Collectors.toSet())),
- clusterCommunicator,
- clusterService);
- partition.open().join();
- primitiveCreator = partition.client();
+ atomix = atomixManager.getAtomix();
+ group = atomix.getPartitionService().getSystemPartitionGroup();
log.info("Started");
}
@@ -112,134 +102,146 @@
final NodeId localNodeId = clusterService.getLocalNode().id();
Supplier<List<NodeId>> peersSupplier = () -> clusterService.getNodes().stream()
- .map(ControllerNode::id)
- .filter(nodeId -> !nodeId.equals(localNodeId))
- .filter(id -> clusterService.getState(id).isActive())
- .collect(Collectors.toList());
+ .map(ControllerNode::id)
+ .filter(nodeId -> !nodeId.equals(localNodeId))
+ .filter(id -> clusterService.getState(id).isActive())
+ .collect(Collectors.toList());
Supplier<List<NodeId>> bootstrapPeersSupplier = () -> clusterService.getNodes()
- .stream()
- .map(ControllerNode::id)
- .filter(id -> !localNodeId.equals(id))
- .filter(id -> clusterService.getState(id).isActive())
- .collect(Collectors.toList());
+ .stream()
+ .map(ControllerNode::id)
+ .filter(id -> !localNodeId.equals(id))
+ .filter(id -> clusterService.getState(id).isActive())
+ .collect(Collectors.toList());
return new EventuallyConsistentMapBuilderImpl<>(
- localNodeId,
- clusterCommunicator,
- persistenceService,
- peersSupplier,
- bootstrapPeersSupplier
+ localNodeId,
+ clusterCommunicator,
+ persistenceService,
+ peersSupplier,
+ bootstrapPeersSupplier
);
}
@Override
public <K, V> ConsistentMapBuilder<K, V> consistentMapBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultConsistentMapBuilder<>(primitiveCreator);
+ return new AtomixConsistentMapBuilder<>(atomix, group.name());
}
@Override
public <V> DocumentTreeBuilder<V> documentTreeBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultDocumentTreeBuilder<>(primitiveCreator);
+ return new AtomixDocumentTreeBuilder<>(atomix, group.name());
}
@Override
public <V> ConsistentTreeMapBuilder<V> consistentTreeMapBuilder() {
- return new DefaultConsistentTreeMapBuilder<>(primitiveCreator);
+ return new AtomixConsistentTreeMapBuilder<>(atomix, group.name());
}
@Override
public <K, V> ConsistentMultimapBuilder<K, V> consistentMultimapBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultConsistentMultimapBuilder<>(primitiveCreator);
+ return new AtomixConsistentMultimapBuilder<>(atomix, group.name());
}
@Override
public <K> AtomicCounterMapBuilder<K> atomicCounterMapBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultAtomicCounterMapBuilder<>(primitiveCreator);
+ return new AtomixAtomicCounterMapBuilder<>(atomix, group.name());
}
@Override
public <E> DistributedSetBuilder<E> setBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultDistributedSetBuilder<>(this::<E, Boolean>consistentMapBuilder);
+ return new AtomixDistributedSetBuilder<>(atomix, group.name());
}
@Override
public AtomicCounterBuilder atomicCounterBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultAtomicCounterBuilder(primitiveCreator);
+ return new AtomixAtomicCounterBuilder(atomix, group.name());
}
@Override
public AtomicIdGeneratorBuilder atomicIdGeneratorBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultAtomicIdGeneratorBuilder(primitiveCreator);
+ return new AtomixAtomicIdGeneratorBuilder(atomix, group.name());
}
@Override
public <V> AtomicValueBuilder<V> atomicValueBuilder() {
checkPermission(STORAGE_WRITE);
- Supplier<ConsistentMapBuilder<String, byte[]>> mapBuilderSupplier =
- () -> this.<String, byte[]>consistentMapBuilder()
- .withName("onos-atomic-values")
- .withSerializer(Serializer.using(KryoNamespaces.BASIC));
- return new DefaultAtomicValueBuilder<>(mapBuilderSupplier);
+ return new AtomixAtomicValueBuilder<>(atomix, group.name());
}
@Override
public TransactionContextBuilder transactionContextBuilder() {
- throw new UnsupportedOperationException();
+ checkPermission(STORAGE_WRITE);
+ return new AtomixTransactionContextBuilder(atomix, group.name());
}
@Override
public LeaderElectorBuilder leaderElectorBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultLeaderElectorBuilder(primitiveCreator);
+ return new AtomixLeaderElectorBuilder(atomix, group.name(), clusterService.getLocalNode().id());
}
@Override
public <T> TopicBuilder<T> topicBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultDistributedTopicBuilder<>(atomicValueBuilder());
+ return new AtomixDistributedTopicBuilder<>(atomix, group.name());
+ }
+
+ @Override
+ public <E> WorkQueueBuilder<E> workQueueBuilder() {
+ checkPermission(STORAGE_WRITE);
+ return new AtomixWorkQueueBuilder<>(atomix, group.name());
}
@Override
public <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer) {
checkPermission(STORAGE_WRITE);
- return primitiveCreator.newWorkQueue(name, serializer);
+ return this.<E>workQueueBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .build();
}
@Override
public <V> AsyncDocumentTree<V> getDocumentTree(String name, Serializer serializer) {
checkPermission(STORAGE_WRITE);
- return primitiveCreator.newAsyncDocumentTree(name, serializer);
+ return this.<V>documentTreeBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .build();
}
@Override
- public <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(
- String name, Serializer serializer) {
+ public <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(String name, Serializer serializer) {
checkPermission(STORAGE_WRITE);
- return primitiveCreator.newAsyncConsistentSetMultimap(name,
- serializer);
+ return this.<K, V>consistentMultimapBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .buildMultimap();
}
@Override
- public <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(
- String name, Serializer serializer) {
+ public <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(String name, Serializer serializer) {
checkPermission(STORAGE_WRITE);
- return primitiveCreator.newAsyncConsistentTreeMap(name, serializer);
+ return this.<V>consistentTreeMapBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .buildTreeMap();
}
@Override
public <T> Topic<T> getTopic(String name, Serializer serializer) {
- AsyncAtomicValue<T> atomicValue = this.<T>atomicValueBuilder()
- .withName("topic-" + name)
- .withSerializer(serializer)
- .build();
- return new DefaultDistributedTopic<>(atomicValue);
+ checkPermission(STORAGE_WRITE);
+ return this.<T>topicBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .build();
}
}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValue.java
deleted file mode 100644
index 33af601..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValue.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import com.google.common.collect.Maps;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AtomicValueEvent;
-import org.onosproject.store.service.AtomicValueEventListener;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Versioned;
-import org.onosproject.utils.MeteringAgent;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Default implementation of a {@code AsyncAtomicValue}.
- *
- * @param <V> value type
- */
-public class DefaultAsyncAtomicValue<V> implements AsyncAtomicValue<V> {
-
- private final String name;
- private final Serializer serializer;
- private final AsyncConsistentMap<String, byte[]> backingMap;
- private final Map<AtomicValueEventListener<V>, MapEventListener<String, byte[]>> listeners =
- Maps.newIdentityHashMap();
- private final MeteringAgent monitor;
-
- private static final String COMPONENT_NAME = "atomicValue";
- private static final String GET = "get";
- private static final String GET_AND_SET = "getAndSet";
- private static final String SET = "set";
- private static final String COMPARE_AND_SET = "compareAndSet";
- private static final String ADD_LISTENER = "addListener";
- private static final String REMOVE_LISTENER = "removeListener";
- private static final String NOTIFY_LISTENER = "notifyListener";
- private static final String DESTROY = "destroy";
-
- public DefaultAsyncAtomicValue(String name, Serializer serializer, AsyncConsistentMap<String, byte[]> backingMap) {
- this.name = checkNotNull(name, "name must not be null");
- this.serializer = checkNotNull(serializer, "serializer must not be null");
- this.backingMap = checkNotNull(backingMap, "backingMap must not be null");
- this.monitor = new MeteringAgent(COMPONENT_NAME, name, true);
- }
-
- //Bypass osgi error with 'activated' MeteringAgent parameter set to false
- DefaultAsyncAtomicValue(String name, Serializer serializer, AsyncConsistentMap<String, byte[]> backingMap,
- MeteringAgent meteringAgent) {
- this.name = checkNotNull(name, "name must not be null");
- this.serializer = checkNotNull(serializer, "serializer must not be null");
- this.backingMap = checkNotNull(backingMap, "backingMap must not be null");
- this.monitor = meteringAgent;
-
- }
-
- @Override
- public String name() {
- return name;
- }
-
- @Override
- public CompletableFuture<Void> destroy() {
- final MeteringAgent.Context newTimer = monitor.startTimer(DESTROY);
- return backingMap.remove(name)
- .whenComplete((r, e) -> newTimer.stop(e))
- .thenApply(v -> null);
- }
-
- @Override
- public CompletableFuture<Boolean> compareAndSet(V expect, V update) {
- final MeteringAgent.Context newTimer = monitor.startTimer(COMPARE_AND_SET);
- return backingMap.replace(name, serializer.encode(expect), serializer.encode(update))
- .whenComplete((r, e) -> newTimer.stop(e));
- }
-
- @Override
- public CompletableFuture<V> get() {
- final MeteringAgent.Context newTimer = monitor.startTimer(GET);
- return backingMap.get(name)
- .thenApply(Versioned::valueOrNull)
- .thenApply(v -> v == null ? null : serializer.<V>decode(v))
- .whenComplete((r, e) -> newTimer.stop(e));
- }
-
- @Override
- public CompletableFuture<V> getAndSet(V value) {
- final MeteringAgent.Context newTimer = monitor.startTimer(GET_AND_SET);
- if (value == null) {
- return backingMap.remove(name)
- .thenApply(Versioned::valueOrNull)
- .thenApply(v -> v == null ? null : serializer.<V>decode(v))
- .whenComplete((r, e) -> newTimer.stop(e));
- }
- return backingMap.put(name, serializer.encode(value))
- .thenApply(Versioned::valueOrNull)
- .thenApply(v -> v == null ? null : serializer.<V>decode(v))
- .whenComplete((r, e) -> newTimer.stop(e));
- }
-
- @Override
- public CompletableFuture<Void> set(V value) {
- final MeteringAgent.Context newTimer = monitor.startTimer(SET);
- if (value == null) {
- return backingMap.remove(name)
- .whenComplete((r, e) -> newTimer.stop(e))
- .thenApply(v -> null);
-
- }
- return backingMap.put(name, serializer.encode(value))
- .whenComplete((r, e) -> newTimer.stop(e))
- .thenApply(v -> null);
- }
-
- @Override
- public CompletableFuture<Void> addListener(AtomicValueEventListener<V> listener) {
- checkNotNull(listener, "listener must not be null");
- final MeteringAgent.Context newTimer = monitor.startTimer(ADD_LISTENER);
- MapEventListener<String, byte[]> mapListener =
- listeners.computeIfAbsent(listener, key -> new InternalMapValueEventListener(listener));
- return backingMap.addListener(mapListener).whenComplete((r, e) -> newTimer.stop(e));
- }
-
- @Override
- public CompletableFuture<Void> removeListener(AtomicValueEventListener<V> listener) {
- checkNotNull(listener, "listener must not be null");
- final MeteringAgent.Context newTimer = monitor.startTimer(REMOVE_LISTENER);
- MapEventListener<String, byte[]> mapListener = listeners.remove(listener);
- if (mapListener != null) {
- return backingMap.removeListener(mapListener)
- .whenComplete((r, e) -> newTimer.stop(e));
- } else {
- newTimer.stop(null);
- return CompletableFuture.completedFuture(null);
- }
- }
-
- private class InternalMapValueEventListener implements MapEventListener<String, byte[]> {
-
- private final AtomicValueEventListener<V> listener;
-
- InternalMapValueEventListener(AtomicValueEventListener<V> listener) {
- this.listener = listener;
- }
-
- @Override
- public void event(MapEvent<String, byte[]> event) {
- if (event.key().equals(name)) {
- final MeteringAgent.Context newTimer = monitor.startTimer(NOTIFY_LISTENER);
- byte[] rawNewValue = Versioned.valueOrNull(event.newValue());
- byte[] rawOldValue = Versioned.valueOrNull(event.oldValue());
-
- try {
- listener.event(new AtomicValueEvent<>(name,
- rawNewValue == null ? null : serializer.decode(rawNewValue),
- rawOldValue == null ? null : serializer.decode(rawOldValue)));
- newTimer.stop(null);
- } catch (Exception e) {
- newTimer.stop(e);
- throw new IllegalStateException(e.getCause());
- }
- }
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSet.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSet.java
deleted file mode 100644
index d79c618..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSet.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.SetEvent;
-import org.onosproject.store.service.SetEventListener;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.onosproject.utils.MeteringAgent;
-
-/**
- * Implementation of {@link AsyncDistributedSet}.
- *
- * @param <E> set entry type
- */
-public class DefaultAsyncDistributedSet<E> implements AsyncDistributedSet<E> {
-
- private static final String CONTAINS = "contains";
- private static final String PRIMITIVE_NAME = "distributedSet";
- private static final String SIZE = "size";
- private static final String IS_EMPTY = "isEmpty";
- private static final String ADD = "add";
- private static final String REMOVE = "remove";
- private static final String CONTAINS_ALL = "containsAll";
- private static final String ADD_ALL = "addAll";
- private static final String RETAIN_ALL = "retainAll";
- private static final String REMOVE_ALL = "removeAll";
- private static final String CLEAR = "clear";
- private static final String GET_AS_IMMUTABLE_SET = "getAsImmutableSet";
-
- private final String name;
- private final AsyncConsistentMap<E, Boolean> backingMap;
- private final Map<SetEventListener<E>, MapEventListener<E, Boolean>> listenerMapping = Maps.newIdentityHashMap();
- private final MeteringAgent monitor;
-
- public DefaultAsyncDistributedSet(AsyncConsistentMap<E, Boolean> backingMap, String name, boolean meteringEnabled) {
- this.backingMap = backingMap;
- this.name = name;
- monitor = new MeteringAgent(PRIMITIVE_NAME, name, meteringEnabled);
- }
-
- @Override
- public String name() {
- return name;
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- final MeteringAgent.Context timer = monitor.startTimer(SIZE);
- return backingMap.size().whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- final MeteringAgent.Context timer = monitor.startTimer(IS_EMPTY);
- return backingMap.isEmpty().whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Boolean> contains(E element) {
- final MeteringAgent.Context timer = monitor.startTimer(CONTAINS);
- return backingMap.containsKey(element).whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Boolean> add(E entry) {
- final MeteringAgent.Context timer = monitor.startTimer(ADD);
- return backingMap.putIfAbsent(entry, true).thenApply(Objects::isNull).whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(E entry) {
- final MeteringAgent.Context timer = monitor.startTimer(REMOVE);
- return backingMap.remove(entry, true).whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Boolean> containsAll(Collection<? extends E> c) {
- final MeteringAgent.Context timer = monitor.startTimer(CONTAINS_ALL);
- return Tools.allOf(c.stream().map(this::contains).collect(Collectors.toList())).thenApply(v ->
- v.stream().reduce(Boolean::logicalAnd).orElse(true)).whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Boolean> addAll(Collection<? extends E> c) {
- final MeteringAgent.Context timer = monitor.startTimer(ADD_ALL);
- return Tools.allOf(c.stream().map(this::add).collect(Collectors.toList())).thenApply(v ->
- v.stream().reduce(Boolean::logicalOr).orElse(false)).whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Boolean> retainAll(Collection<? extends E> c) {
- final MeteringAgent.Context timer = monitor.startTimer(RETAIN_ALL);
- return backingMap.keySet().thenApply(set -> Sets.difference(set, Sets.newHashSet(c)))
- .thenCompose(this::removeAll)
- .whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Boolean> removeAll(Collection<? extends E> c) {
- final MeteringAgent.Context timer = monitor.startTimer(REMOVE_ALL);
- return Tools.allOf(c.stream().map(this::remove).collect(Collectors.toList())).thenApply(v ->
- v.stream().reduce(Boolean::logicalOr).orElse(false)).whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- final MeteringAgent.Context timer = monitor.startTimer(CLEAR);
- return backingMap.clear().whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<? extends Set<E>> getAsImmutableSet() {
- final MeteringAgent.Context timer = monitor.startTimer(GET_AS_IMMUTABLE_SET);
- return backingMap.keySet().thenApply(s -> ImmutableSet.copyOf(s)).whenComplete((r, e) -> timer.stop(null));
- }
-
- @Override
- public CompletableFuture<Void> addListener(SetEventListener<E> listener) {
- MapEventListener<E, Boolean> mapEventListener = mapEvent -> {
- if (mapEvent.type() == MapEvent.Type.INSERT) {
- listener.event(new SetEvent<>(name, SetEvent.Type.ADD, mapEvent.key()));
- } else if (mapEvent.type() == MapEvent.Type.REMOVE) {
- listener.event(new SetEvent<>(name, SetEvent.Type.REMOVE, mapEvent.key()));
- }
- };
- if (listenerMapping.putIfAbsent(listener, mapEventListener) == null) {
- return backingMap.addListener(mapEventListener);
- }
- return CompletableFuture.completedFuture(null);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(SetEventListener<E> listener) {
- MapEventListener<E, Boolean> mapEventListener = listenerMapping.remove(listener);
- if (mapEventListener != null) {
- return backingMap.removeListener(mapEventListener);
- }
- return CompletableFuture.completedFuture(null);
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterBuilder.java
deleted file mode 100644
index 11a0724..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterBuilder.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.AtomicCounterBuilder;
-
-/**
- * Default implementation of AtomicCounterBuilder.
- */
-public class DefaultAtomicCounterBuilder extends AtomicCounterBuilder {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultAtomicCounterBuilder(DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public AsyncAtomicCounter build() {
- return primitiveCreator.newAsyncCounter(name());
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterMapBuilder.java
deleted file mode 100644
index d5a143a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterMapBuilder.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.AtomicCounterMap;
-import org.onosproject.store.service.AtomicCounterMapBuilder;
-
-/**
- * Default {@code AtomicCounterMapBuilder}.
- */
-public class DefaultAtomicCounterMapBuilder<K> extends AtomicCounterMapBuilder<K> {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultAtomicCounterMapBuilder(DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public AsyncAtomicCounterMap<K> buildAsyncMap() {
- return primitiveCreator.newAsyncAtomicCounterMap(name(), serializer());
- }
-
- @Override
- public AtomicCounterMap<K> build() {
- return buildAsyncMap().asAtomicCounterMap();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicIdGeneratorBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicIdGeneratorBuilder.java
deleted file mode 100644
index c9e8486..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicIdGeneratorBuilder.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncAtomicIdGenerator;
-import org.onosproject.store.service.AtomicIdGeneratorBuilder;
-
-/**
- * Default implementation of AtomicIdGeneratorBuilder.
- */
-public class DefaultAtomicIdGeneratorBuilder extends AtomicIdGeneratorBuilder {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultAtomicIdGeneratorBuilder(DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public AsyncAtomicIdGenerator build() {
- return primitiveCreator.newAsyncIdGenerator(name());
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicValueBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicValueBuilder.java
deleted file mode 100644
index 9058189..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicValueBuilder.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.function.Supplier;
-
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AtomicValueBuilder;
-import org.onosproject.store.service.ConsistentMapBuilder;
-import org.onosproject.store.service.Serializer;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Default implementation of AtomicValueBuilder.
- *
- * @param <V> value type
- */
-public class DefaultAtomicValueBuilder<V> extends AtomicValueBuilder<V> {
-
- private ConsistentMapBuilder<String, byte[]> mapBuilder;
-
- public DefaultAtomicValueBuilder(Supplier<ConsistentMapBuilder<String, byte[]>> mapBuilderSupplier) {
- mapBuilder = mapBuilderSupplier.get();
- }
-
- @Override
- public AsyncAtomicValue<V> build() {
- if (compatibilityFunction != null) {
- Serializer serializer = Serializer.using(KryoNamespaces.API, CompatibleValue.class);
-
- AsyncAtomicValue<CompatibleValue<byte[]>> rawValue = new DefaultAsyncAtomicValue<>(
- checkNotNull(name()), serializer, mapBuilder.buildAsyncMap());
-
- AsyncAtomicValue<CompatibleValue<V>> compatibleValue =
- DistributedPrimitives.newTranscodingAtomicValue(
- rawValue,
- value -> value == null ? null :
- new CompatibleValue<byte[]>(serializer().encode(value.value()), value.version()),
- value -> value == null ? null :
- new CompatibleValue<V>(serializer().decode(value.value()), value.version()));
- return DistributedPrimitives.newCompatibleAtomicValue(compatibleValue, compatibilityFunction, version());
- }
- return new DefaultAsyncAtomicValue<>(
- checkNotNull(name()),
- checkNotNull(serializer()),
- mapBuilder.buildAsyncMap());
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMapBuilder.java
deleted file mode 100644
index d6b9441..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMapBuilder.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.ConsistentMap;
-import org.onosproject.store.service.ConsistentMapBuilder;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Default {@link AsyncConsistentMap} builder.
- *
- * @param <K> type for map key
- * @param <V> type for map value
- */
-public class DefaultConsistentMapBuilder<K, V> extends ConsistentMapBuilder<K, V> {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultConsistentMapBuilder(DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public ConsistentMap<K, V> build() {
- return buildAsyncMap().asConsistentMap();
- }
-
- @Override
- public AsyncConsistentMap<K, V> buildAsyncMap() {
- AsyncConsistentMap<K, V> map;
-
- // If a compatibility function is defined, we don't assume CompatibleValue and Version is registered in
- // the user-provided serializer since it's an implementation detail. Instead, we use the user-provided
- // serializer to convert the CompatibleValue value to a raw byte[] and use a separate serializer to encode
- // the CompatibleValue to binary.
- if (compatibilityFunction != null) {
- Serializer serializer = serializer();
-
- // Convert the byte[] value to CompatibleValue<byte[]>
- AsyncConsistentMap<K, CompatibleValue<byte[]>> rawMap = primitiveCreator.newAsyncConsistentMap(
- withSerializer(Serializer.using(KryoNamespaces.API, CompatibleValue.class)));
-
- // Convert the CompatibleValue<byte[]> value to CompatibleValue<V> using the user-provided serializer.
- AsyncConsistentMap<K, CompatibleValue<V>> compatibleMap =
- DistributedPrimitives.newTranscodingMap(
- rawMap,
- key -> key,
- key -> key,
- value -> value == null ? null :
- new CompatibleValue<byte[]>(serializer.encode(value.value()), value.version()),
- value -> value == null ? null :
- new CompatibleValue<V>(serializer.decode(value.value()), value.version()));
- map = DistributedPrimitives.newCompatibleMap(compatibleMap, compatibilityFunction, version());
- } else {
- map = primitiveCreator.newAsyncConsistentMap(name(), serializer());
- }
-
- map = nullValues() ? map : DistributedPrimitives.newNotNullMap(map);
- map = relaxedReadConsistency() ? DistributedPrimitives.newCachingMap(map) : map;
- map = readOnly() ? DistributedPrimitives.newUnmodifiableMap(map) : map;
- return meteringEnabled() ? DistributedPrimitives.newMeteredMap(map) : map;
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMultimapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMultimapBuilder.java
deleted file mode 100644
index 18ea1f1..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMultimapBuilder.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Copyright 2016 Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.ConsistentMultimap;
-import org.onosproject.store.service.ConsistentMultimapBuilder;
-
-/**
- * Default {@link AsyncConsistentMultimap} builder.
- */
-public class DefaultConsistentMultimapBuilder<K, V>
- extends ConsistentMultimapBuilder<K, V> {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultConsistentMultimapBuilder(
- DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public AsyncConsistentMultimap<K, V> buildMultimap() {
- return primitiveCreator.newAsyncConsistentSetMultimap(name(), serializer());
- }
-
- @Override
- public ConsistentMultimap<K, V> build() {
- AsyncConsistentMultimap<K, V> multimap = buildMultimap();
- if (relaxedReadConsistency()) {
- multimap = new CachingAsyncConsistentMultimap<>(multimap);
- }
- return multimap.asMultimap();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentTreeMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentTreeMapBuilder.java
deleted file mode 100644
index b80d93e..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentTreeMapBuilder.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.ConsistentTreeMap;
-import org.onosproject.store.service.ConsistentTreeMapBuilder;
-
-/**
- * Default {@link org.onosproject.store.service.AsyncConsistentTreeMap} builder.
- *
- * @param <V> type for map value
- */
-public class DefaultConsistentTreeMapBuilder<V> extends ConsistentTreeMapBuilder<V> {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultConsistentTreeMapBuilder(DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public AsyncConsistentTreeMap<V> buildTreeMap() {
- return primitiveCreator.newAsyncConsistentTreeMap(name(), serializer());
- }
-
- @Override
- public ConsistentTreeMap<V> build() {
- return buildTreeMap().asTreeMap();
- }
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedDocumentTree.java
deleted file mode 100644
index 8d4f9c3..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedDocumentTree.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.collect.Maps;
-
-/**
- * Default implementation of {@link AsyncDocumentTree}.
- * <p>
- * This implementation delegates execution to a backing tree implemented on top of Atomix framework.
- *
- * @param <V> tree node value type.
- */
-public class DefaultDistributedDocumentTree<V> implements AsyncDocumentTree<V> {
-
- private final String name;
- private final AsyncDocumentTree<byte[]> backingTree;
- private final Serializer serializer;
- private final Map<DocumentTreeListener<V>, InternalBackingDocumentTreeListener> listeners =
- Maps.newIdentityHashMap();
-
- DefaultDistributedDocumentTree(String name, AsyncDocumentTree<byte[]> backingTree, Serializer serializer) {
- this.name = name;
- this.backingTree = backingTree;
- this.serializer = serializer;
- }
-
- @Override
- public String name() {
- return name;
- }
-
- @Override
- public Type primitiveType() {
- return backingTree.primitiveType();
- }
-
- @Override
- public DocumentPath root() {
- return backingTree.root();
- }
-
- @Override
- public CompletableFuture<Map<String, Versioned<V>>> getChildren(DocumentPath path) {
- return backingTree.getChildren(path)
- .thenApply(map -> Maps.transformValues(map, v -> v.map(serializer::decode)));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(DocumentPath path) {
- return backingTree.get(path)
- .thenApply(v -> v == null ? null : v.map(serializer::decode));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
- return backingTree.set(path, serializer.encode(value))
- .thenApply(v -> v == null ? null : v.map(serializer::decode));
- }
-
- @Override
- public CompletableFuture<Boolean> create(DocumentPath path, V value) {
- return backingTree.create(path, serializer.encode(value));
- }
-
- @Override
- public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
- return backingTree.createRecursive(path, serializer.encode(value));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
- return backingTree.replace(path, serializer.encode(newValue), version);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
- return backingTree.replace(path, serializer.encode(newValue), serializer.encode(currentValue));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
- return backingTree.removeNode(path)
- .thenApply(v -> v == null ? null : v.map(serializer::decode));
- }
-
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- return backingTree.begin(transactionId);
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<V>> transactionLog) {
- return backingTree.prepare(transactionLog.map(record -> record.map(serializer::encode)));
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- return backingTree.commit(transactionId);
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- return backingTree.rollback(transactionId);
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<V>> transactionLog) {
- return backingTree.prepareAndCommit(transactionLog.map(record -> record.map(serializer::encode)));
- }
-
-
- @Override
- public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
- synchronized (listeners) {
- InternalBackingDocumentTreeListener backingListener =
- listeners.computeIfAbsent(listener, k -> new InternalBackingDocumentTreeListener(listener));
- return backingTree.addListener(path, backingListener);
- }
- }
-
- @Override
- public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
- synchronized (listeners) {
- InternalBackingDocumentTreeListener backingListener = listeners.remove(listener);
- if (backingListener != null) {
- return backingTree.removeListener(backingListener);
- } else {
- return CompletableFuture.completedFuture(null);
- }
- }
- }
-
- private class InternalBackingDocumentTreeListener implements DocumentTreeListener<byte[]> {
-
- private final DocumentTreeListener<V> listener;
-
- InternalBackingDocumentTreeListener(DocumentTreeListener<V> listener) {
- this.listener = listener;
- }
-
- @Override
- public void event(DocumentTreeEvent<byte[]> event) {
- listener.event(new DocumentTreeEvent<V>(event.path(),
- event.type(),
- event.newValue().map(v -> v.map(serializer::decode)),
- event.oldValue().map(v -> v.map(serializer::decode))));
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedLockBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedLockBuilder.java
deleted file mode 100644
index fffebcc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedLockBuilder.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncDistributedLock;
-import org.onosproject.store.service.DistributedLockBuilder;
-
-/**
- * Default implementation of DistributedLockBuilder.
- */
-public class DefaultDistributedLockBuilder extends DistributedLockBuilder {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultDistributedLockBuilder(DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public AsyncDistributedLock build() {
- return primitiveCreator.newAsyncDistributedLock(name());
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedSetBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedSetBuilder.java
deleted file mode 100644
index a601239..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedSetBuilder.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.function.Supplier;
-
-import org.onosproject.core.ApplicationId;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.ConsistentMapBuilder;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.DistributedSetBuilder;
-
-/**
- * Default distributed set builder.
- *
- * @param <E> type for set elements
- */
-public class DefaultDistributedSetBuilder<E> extends DistributedSetBuilder<E> {
-
- private String name;
- private ConsistentMapBuilder<E, Boolean> mapBuilder;
- private boolean metering = true;
-
- public DefaultDistributedSetBuilder(Supplier<ConsistentMapBuilder<E, Boolean>> mapBuilderSupplier) {
- this.mapBuilder = mapBuilderSupplier.get();
- mapBuilder.withMeteringDisabled();
- }
-
- @Override
- public DistributedSetBuilder<E> withName(String name) {
- mapBuilder.withName(name);
- this.name = name;
- return this;
- }
-
- @Override
- public DistributedSetBuilder<E> withApplicationId(ApplicationId id) {
- mapBuilder.withApplicationId(id);
- return this;
- }
-
- @Override
- public DistributedSetBuilder<E> withPurgeOnUninstall() {
- mapBuilder.withPurgeOnUninstall();
- return this;
- }
-
- @Override
- public DistributedSetBuilder<E> withSerializer(Serializer serializer) {
- mapBuilder.withSerializer(serializer);
- return this;
- }
-
- @Override
- public DistributedSetBuilder<E> withUpdatesDisabled() {
- mapBuilder.withUpdatesDisabled();
- return this;
- }
-
- @Override
- public DistributedSetBuilder<E> withRelaxedReadConsistency() {
- mapBuilder.withRelaxedReadConsistency();
- return this;
- }
-
- @Override
- public DistributedSetBuilder<E> withMeteringDisabled() {
- metering = false;
- return this;
- }
-
- @Override
- public AsyncDistributedSet<E> build() {
- return new DefaultAsyncDistributedSet<E>(mapBuilder.buildAsyncMap(), name, metering);
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopicBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopicBuilder.java
deleted file mode 100644
index 21ca7a1..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopicBuilder.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.function.BiFunction;
-
-import org.onosproject.core.Version;
-import org.onosproject.store.service.AtomicValueBuilder;
-import org.onosproject.store.service.RevisionType;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Topic;
-import org.onosproject.store.service.TopicBuilder;
-
-/**
- * Default topic builder.
- */
-public class DefaultDistributedTopicBuilder<T> extends TopicBuilder<T> {
- private final AtomicValueBuilder<T> valueBuilder;
-
- public DefaultDistributedTopicBuilder(AtomicValueBuilder<T> valueBuilder) {
- this.valueBuilder = valueBuilder;
- }
-
- @Override
- public TopicBuilder<T> withName(String name) {
- valueBuilder.withName(name);
- return this;
- }
-
- @Override
- public TopicBuilder<T> withSerializer(Serializer serializer) {
- valueBuilder.withSerializer(serializer);
- return this;
- }
-
- @Override
- public TopicBuilder<T> withVersion(Version version) {
- valueBuilder.withVersion(version);
- return this;
- }
-
- @Override
- public TopicBuilder<T> withRevisionType(RevisionType revisionType) {
- valueBuilder.withRevisionType(revisionType);
- return this;
- }
-
- @Override
- public TopicBuilder<T> withCompatibilityFunction(BiFunction<T, Version, T> compatibilityFunction) {
- valueBuilder.withCompatibilityFunction(compatibilityFunction);
- return this;
- }
-
- @Override
- public Topic<T> build() {
- return new DefaultDistributedTopic<>(valueBuilder.build());
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedWorkQueue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedWorkQueue.java
deleted file mode 100644
index 11895db..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedWorkQueue.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueue;
-import org.onosproject.store.service.WorkQueueStats;
-
-import com.google.common.collect.Collections2;
-
-/**
- * Default implementation of {@link WorkQueue}.
- *
- * @param <E> task payload type.
- */
-public class DefaultDistributedWorkQueue<E> implements WorkQueue<E> {
-
- private final WorkQueue<byte[]> backingQueue;
- private final Serializer serializer;
-
- public DefaultDistributedWorkQueue(WorkQueue<byte[]> backingQueue, Serializer serializer) {
- this.backingQueue = backingQueue;
- this.serializer = serializer;
- }
-
- @Override
- public String name() {
- return backingQueue.name();
- }
-
- @Override
- public CompletableFuture<Void> addMultiple(Collection<E> items) {
- return backingQueue.addMultiple(items.stream()
- .map(serializer::encode)
- .collect(Collectors.toCollection(ArrayList::new)));
- }
-
- private Collection<Task<E>> decodeCollection(Collection<Task<byte[]>> tasks) {
- return Collections2.transform(tasks, task -> task.map(serializer::decode));
- }
-
- @Override
- public CompletableFuture<Collection<Task<E>>> take(int maxTasks) {
- return backingQueue.take(maxTasks)
- .thenApply(this::decodeCollection);
- }
-
- @Override
- public CompletableFuture<Void> complete(Collection<String> ids) {
- return backingQueue.complete(ids);
- }
-
- @Override
- public CompletableFuture<WorkQueueStats> stats() {
- return backingQueue.stats();
- }
-
- @Override
- public CompletableFuture<Void> registerTaskProcessor(Consumer<E> callback,
- int parallelism,
- Executor executor) {
- Consumer<byte[]> backingQueueCallback = payload -> callback.accept(serializer.decode(payload));
- return backingQueue.registerTaskProcessor(backingQueueCallback, parallelism, executor);
- }
-
- @Override
- public CompletableFuture<Void> stopProcessing() {
- return backingQueue.stopProcessing();
- }
-
- @Override
- public CompletableFuture<Void> destroy() {
- return backingQueue.destroy();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDocumentTreeBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDocumentTreeBuilder.java
deleted file mode 100644
index f0da9ab..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDocumentTreeBuilder.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentTreeBuilder;
-
-/**
- * Default {@link AsyncDocumentTree} builder.
- *
- * @param <V> type for document tree value
- */
-public class DefaultDocumentTreeBuilder<V> extends DocumentTreeBuilder<V> {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultDocumentTreeBuilder(DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public AsyncDocumentTree<V> buildDocumentTree() {
- return primitiveCreator.newAsyncDocumentTree(name(), serializer(), ordering());
- }
-
- //TODO
- /*
- public ConsistentDocumentTree<V> build() {
- return buildDocumentTree().asDocumentTree();
- }
- }*/
- //writing a dummy implementation till we have ConsistentDocumentTree.
- @Deprecated
- @Override
- public AsyncDocumentTree<V> build() {
- AsyncDocumentTree<V> tree = primitiveCreator.newAsyncDocumentTree(name(), serializer(), ordering());
- tree = relaxedReadConsistency() ? DistributedPrimitives.newCachingDocumentTree(tree) : tree;
- return tree;
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultLeaderElectorBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultLeaderElectorBuilder.java
deleted file mode 100644
index 21ddf03..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultLeaderElectorBuilder.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncLeaderElector;
-import org.onosproject.store.service.LeaderElectorBuilder;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Default implementation of {@code LeaderElectorBuilder}.
- */
-public class DefaultLeaderElectorBuilder extends LeaderElectorBuilder {
-
- private final DistributedPrimitiveCreator primitiveCreator;
-
- public DefaultLeaderElectorBuilder(DistributedPrimitiveCreator primitiveCreator) {
- this.primitiveCreator = primitiveCreator;
- }
-
- @Override
- public AsyncLeaderElector build() {
- AsyncLeaderElector leaderElector = primitiveCreator.newAsyncLeaderElector(name(), electionTimeoutMillis(),
- TimeUnit.MILLISECONDS);
- if (relaxedReadConsistency()) {
- leaderElector = new CachingAsyncLeaderElector(leaderElector);
- }
- return leaderElector;
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContext.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContext.java
deleted file mode 100644
index 19ea562..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContext.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.CommitStatus;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionContext;
-import org.onosproject.store.service.TransactionalMap;
-import org.onosproject.utils.MeteringAgent;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkState;
-
-/**
- * Default implementation of transaction context.
- */
-public class DefaultTransactionContext implements TransactionContext {
- private final Logger log = LoggerFactory.getLogger(getClass());
- private final AtomicBoolean isOpen = new AtomicBoolean(false);
- private final TransactionId transactionId;
- private final TransactionCoordinator transactionCoordinator;
- private final MeteringAgent monitor;
-
- public DefaultTransactionContext(TransactionId transactionId, TransactionCoordinator transactionCoordinator) {
- this.transactionId = transactionId;
- this.transactionCoordinator = transactionCoordinator;
- this.monitor = new MeteringAgent("transactionContext", "*", true);
- }
-
- @Override
- public String name() {
- return transactionId.toString();
- }
-
- @Override
- public TransactionId transactionId() {
- return transactionId;
- }
-
- @Override
- public boolean isOpen() {
- return isOpen.get();
- }
-
- @Override
- public void begin() {
- if (isOpen.compareAndSet(false, true)) {
- log.trace("Opened transaction {}", transactionId);
- } else {
- throw new IllegalStateException("TransactionContext is already open");
- }
- }
-
- @Override
- public CompletableFuture<CommitStatus> commit() {
- checkState(isOpen.get(), "Transaction not open");
- final MeteringAgent.Context timer = monitor.startTimer("commit");
- log.debug("Committing transaction {}", transactionId);
- return transactionCoordinator.commit().whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public void abort() {
- if (isOpen.compareAndSet(true, false)) {
- log.debug("Aborted transaction {}", transactionId);
- }
- }
-
- @Override
- public <K, V> TransactionalMap<K, V> getTransactionalMap(String mapName, Serializer serializer) {
- return transactionCoordinator.getTransactionalMap(mapName, serializer);
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("transactionId", transactionId)
- .toString();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContextBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContextBuilder.java
deleted file mode 100644
index ecbb1d2..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContextBuilder.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.TransactionContext;
-import org.onosproject.store.service.TransactionContextBuilder;
-
-/**
- * Default Transaction Context Builder.
- */
-public class DefaultTransactionContextBuilder extends TransactionContextBuilder {
-
- private final TransactionId transactionId;
- private final TransactionManager transactionManager;
-
- public DefaultTransactionContextBuilder(TransactionId transactionId, TransactionManager transactionManager) {
- this.transactionId = transactionId;
- this.transactionManager = transactionManager;
- }
-
- @Override
- public TransactionContext build() {
- return new DefaultTransactionContext(transactionId,
- new TransactionCoordinator(transactionId, transactionManager));
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalMapParticipant.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalMapParticipant.java
deleted file mode 100644
index 14346ce..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalMapParticipant.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Map;
-import java.util.stream.Stream;
-
-import com.google.common.collect.Maps;
-import org.apache.commons.lang3.tuple.Pair;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.service.ConsistentMap;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-/**
- * Repeatable read based map participant.
- */
-public class DefaultTransactionalMapParticipant<K, V> extends TransactionalMapParticipant<K, V> {
- private final Map<K, Versioned<V>> readCache = Maps.newConcurrentMap();
-
- public DefaultTransactionalMapParticipant(
- ConsistentMap<K, V> backingMap, Transaction<MapUpdate<K, V>> transaction) {
- super(backingMap, transaction);
- }
-
- @Override
- protected V read(K key) {
- Versioned<V> value = backingMap.getOrDefault(key, null);
- readCache.put(key, value);
- return value.value();
- }
-
- @Override
- public boolean hasPendingUpdates() {
- return !writeCache.isEmpty() || !deleteSet.isEmpty();
- }
-
- @Override
- protected Stream<MapUpdate<K, V>> records(Version lockVersion) {
- return Stream.concat(deleteStream(), writeStream(lockVersion));
- }
-
- /**
- * Returns a transaction record stream for deleted keys.
- */
- private Stream<MapUpdate<K, V>> deleteStream() {
- return deleteSet.stream()
- .map(key -> Pair.of(key, readCache.get(key)))
- .map(e -> MapUpdate.<K, V>newBuilder()
- .withType(MapUpdate.Type.REMOVE_IF_VERSION_MATCH)
- .withKey(e.getKey())
- .withVersion(e.getValue().version())
- .build());
- }
-
- /**
- * Returns a transaction record stream for updated keys.
- */
- private Stream<MapUpdate<K, V>> writeStream(Version lockVersion) {
- return writeCache.entrySet().stream()
- .map(entry -> {
- Versioned<V> original = readCache.get(entry.getKey());
- return MapUpdate.<K, V>newBuilder()
- .withType(MapUpdate.Type.PUT_IF_VERSION_MATCH)
- .withKey(entry.getKey())
- .withValue(entry.getValue())
- .withVersion(Math.max(original.version(), lockVersion.value()))
- .build();
- });
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalSet.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalSet.java
deleted file mode 100644
index a2bac21..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalSet.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionContext;
-import org.onosproject.store.service.TransactionalMap;
-import org.onosproject.store.service.TransactionalSet;
-
-/**
- * Default TransactionalSet implementation that provides a repeatable reads
- * transaction isolation level.
- *
- * @param <E> element type.
- */
-public class DefaultTransactionalSet<E> implements TransactionalSet<E> {
-
- private TransactionalMap<E, Boolean> map;
-
- // dummy value to associate with an Object in the backing map
- private static final Boolean PRESENT = Boolean.TRUE;
-
- public DefaultTransactionalSet(
- String name,
- TransactionContext txContext,
- Serializer serializer) {
- map = txContext.getTransactionalMap(name, serializer);
- }
-
- @Override
- public boolean add(E e) {
- return map.put(e, PRESENT) == null;
- }
-
- @Override
- public boolean remove(E e) {
- return map.remove(e) != null;
- }
-
- @Override
- public boolean contains(E e) {
- return map.get(e) != null;
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMap.java
deleted file mode 100644
index 193a250..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMap.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Map.Entry;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-/**
- * {@code AsyncConsistentMap} that merely delegates control to
- * another AsyncConsistentMap.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class DelegatingAsyncConsistentMap<K, V>
- extends DelegatingDistributedPrimitive implements AsyncConsistentMap<K, V> {
-
- private final AsyncConsistentMap<K, V> delegateMap;
-
- DelegatingAsyncConsistentMap(AsyncConsistentMap<K, V> delegateMap) {
- super(delegateMap);
- this.delegateMap = delegateMap;
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return delegateMap.size();
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- return delegateMap.containsKey(key);
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- return delegateMap.containsValue(value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(K key) {
- return delegateMap.get(key);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
- return delegateMap.getOrDefault(key, defaultValue);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> computeIf(K key,
- Predicate<? super V> condition,
- BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- return delegateMap.computeIf(key, condition, remappingFunction);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(K key, V value) {
- return delegateMap.put(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
- return delegateMap.putAndGet(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> remove(K key) {
- return delegateMap.remove(key);
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return delegateMap.clear();
- }
-
- @Override
- public CompletableFuture<Set<K>> keySet() {
- return delegateMap.keySet();
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V>>> values() {
- return delegateMap.values();
- }
-
- @Override
- public CompletableFuture<Set<Entry<K, Versioned<V>>>> entrySet() {
- return delegateMap.entrySet();
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
- return delegateMap.putIfAbsent(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return delegateMap.remove(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, long version) {
- return delegateMap.remove(key, version);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(K key, V value) {
- return delegateMap.replace(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
- return delegateMap.replace(key, oldValue, newValue);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
- return delegateMap.replace(key, oldVersion, newValue);
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Entry<K, Versioned<V>>>> iterator() {
- return delegateMap.iterator();
- }
-
- @Override
- public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
- return delegateMap.addListener(listener, executor);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
- return delegateMap.removeListener(listener);
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- return delegateMap.begin(transactionId);
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K, V>> transactionLog) {
- return delegateMap.prepare(transactionLog);
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K, V>> transactionLog) {
- return delegateMap.prepareAndCommit(transactionLog);
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- return delegateMap.commit(transactionId);
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- return delegateMap.rollback(transactionId);
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- delegateMap.addStatusChangeListener(listener);
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- delegateMap.removeStatusChangeListener(listener);
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- return delegateMap.statusChangeListeners();
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("delegateMap", delegateMap)
- .toString();
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(delegateMap);
- }
-
- @Override
- public boolean equals(Object other) {
- if (other instanceof DelegatingAsyncConsistentMap) {
- DelegatingAsyncConsistentMap<K, V> that = (DelegatingAsyncConsistentMap) other;
- return this.delegateMap.equals(that.delegateMap);
- }
- return false;
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMultimap.java
deleted file mode 100644
index 779bb7a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMultimap.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import com.google.common.collect.Multiset;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-
-import java.util.concurrent.Executor;
-
-/**
- * {@code AsyncConsistentMultimap} that merely delegates control to
- * another AsyncConsistentMultimap.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class DelegatingAsyncConsistentMultimap<K, V>
- extends DelegatingDistributedPrimitive implements AsyncConsistentMultimap<K, V> {
-
- private final AsyncConsistentMultimap<K, V> delegateMap;
-
- public DelegatingAsyncConsistentMultimap(
- AsyncConsistentMultimap<K, V> delegateMap) {
- super(delegateMap);
- this.delegateMap = delegateMap;
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return delegateMap.size();
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- return delegateMap.isEmpty();
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- return delegateMap.containsKey(key);
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- return delegateMap.containsValue(value);
- }
-
- @Override
- public CompletableFuture<Boolean> containsEntry(K key, V value) {
- return delegateMap.containsEntry(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> put(K key, V value) {
- return delegateMap.put(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> putAndGet(K key, V value) {
- return delegateMap.putAndGet(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return delegateMap.remove(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> removeAndGet(K key, V value) {
- return delegateMap.removeAndGet(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> removeAll(
- K key, Collection<? extends V> values) {
- return delegateMap.removeAll(key, values);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>>
- removeAll(K key) {
- return delegateMap.removeAll(key);
- }
-
- @Override
- public CompletableFuture<Boolean> putAll(
- K key, Collection<? extends V> values) {
- return delegateMap.putAll(key, values);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>>
- replaceValues(K key, Collection<V> values) {
- return delegateMap.replaceValues(key, values);
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return delegateMap.clear();
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> get(K key) {
- return delegateMap.get(key);
- }
-
- @Override
- public CompletableFuture<Set<K>> keySet() {
- return delegateMap.keySet();
- }
-
- @Override
- public CompletableFuture<Multiset<K>> keys() {
- return delegateMap.keys();
- }
-
- @Override
- public CompletableFuture<Multiset<V>> values() {
- return delegateMap.values();
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Map.Entry<K, V>>> iterator() {
- return delegateMap.iterator();
- }
-
- @Override
- public CompletableFuture<Collection<Map.Entry<K, V>>> entries() {
- return delegateMap.entries();
- }
-
- @Override
- public CompletableFuture<Void> addListener(MultimapEventListener<K, V> listener, Executor executor) {
- return delegateMap.addListener(listener, executor);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MultimapEventListener<K, V> listener) {
- return delegateMap.removeListener(listener);
- }
-
- @Override
- public CompletableFuture<Map<K, Collection<V>>> asMap() {
- return delegateMap.asMap();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentTreeMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentTreeMap.java
deleted file mode 100644
index b3a1f95..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentTreeMap.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Predicate;
-
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * A {@link AsyncConsistentTreeMap} that delegates control to another instance
- * of {@link AsyncConsistentTreeMap}.
- */
-public class DelegatingAsyncConsistentTreeMap<V>
- extends DelegatingDistributedPrimitive implements AsyncConsistentTreeMap<V> {
-
- private final AsyncConsistentTreeMap<V> delegateMap;
-
- DelegatingAsyncConsistentTreeMap(AsyncConsistentTreeMap<V> delegateMap) {
- super(delegateMap);
- this.delegateMap = checkNotNull(delegateMap,
- "delegate map cannot be null");
- }
-
- @Override
- public CompletableFuture<String> firstKey() {
- return delegateMap.firstKey();
- }
-
- @Override
- public CompletableFuture<String> lastKey() {
- return delegateMap.lastKey();
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V>>> ceilingEntry(String key) {
- return delegateMap.ceilingEntry(key);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V>>> floorEntry(String key) {
- return delegateMap.floorEntry(key);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V>>> higherEntry(String key) {
- return delegateMap.higherEntry(key);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V>>> lowerEntry(String key) {
- return delegateMap.lowerEntry(key);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V>>> firstEntry() {
- return delegateMap.firstEntry();
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V>>> lastEntry() {
- return delegateMap.lastEntry();
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V>>> pollFirstEntry() {
- return delegateMap.pollFirstEntry();
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V>>> pollLastEntry() {
- return delegateMap.pollLastEntry();
- }
-
- @Override
- public CompletableFuture<String> lowerKey(String key) {
- return delegateMap.lowerKey(key);
- }
-
- @Override
- public CompletableFuture<String> floorKey(String key) {
- return delegateMap.floorKey(key);
- }
-
- @Override
- public CompletableFuture<String> ceilingKey(String key) {
- return delegateMap.ceilingKey(key);
- }
-
- @Override
- public CompletableFuture<String> higherKey(String key) {
- return delegateMap.higherKey(key);
- }
-
- @Override
- public CompletableFuture<NavigableSet<String>> navigableKeySet() {
- return delegateMap.navigableKeySet();
- }
-
- @Override
- public CompletableFuture<NavigableMap<String, V>> subMap(
- String upperKey,
- String lowerKey,
- boolean inclusiveUpper,
- boolean inclusiveLower) {
- return delegateMap.subMap(upperKey, lowerKey,
- inclusiveUpper, inclusiveLower);
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return delegateMap.size();
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(String key) {
- return delegateMap.containsKey(key);
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- return delegateMap.containsValue(value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(String key) {
- return delegateMap.get(key);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> getOrDefault(String key, V defaultValue) {
- return delegateMap.getOrDefault(key, defaultValue);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> computeIf(
- String key,
- Predicate<? super V> condition,
- BiFunction<? super String, ? super V,
- ? extends V> remappingFunction) {
- return delegateMap.computeIf(key, condition, remappingFunction);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(String key, V value) {
- return delegateMap.put(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(String key, V value) {
- return delegateMap.putAndGet(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> remove(String key) {
- return delegateMap.remove(key);
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return delegateMap.clear();
- }
-
- @Override
- public CompletableFuture<Set<String>> keySet() {
- return delegateMap.keySet();
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V>>> values() {
- return delegateMap.values();
- }
-
- @Override
- public CompletableFuture<Set<Map.Entry<String, Versioned<V>>>> entrySet() {
- return delegateMap.entrySet();
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(String key, V value) {
- return delegateMap.putIfAbsent(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(String key, V value) {
- return delegateMap.remove(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(String key, long version) {
- return delegateMap.remove(key, version);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(String key, V value) {
- return delegateMap.replace(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(String key, V oldValue,
- V newValue) {
- return delegateMap.replace(key, oldValue, newValue);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(String key, long oldVersion,
- V newValue) {
- return delegateMap.replace(key, oldVersion, newValue);
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Map.Entry<String, Versioned<V>>>> iterator() {
- return delegateMap.iterator();
- }
-
- @Override
- public CompletableFuture<Void> addListener(
- MapEventListener<String, V> listener, Executor executor) {
- return delegateMap.addListener(listener, executor);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(
- MapEventListener<String, V> listener) {
- return delegateMap.removeListener(listener);
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- return delegateMap.begin(transactionId);
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<String, V>> transactionLog) {
- return delegateMap.prepare(transactionLog);
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<String, V>> transactionLog) {
- return delegateMap.prepareAndCommit(transactionLog);
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- return delegateMap.commit(transactionId);
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- return delegateMap.rollback(transactionId);
- }
-
- @Override
- public boolean equals(Object other) {
- if (other instanceof DelegatingAsyncConsistentTreeMap) {
- DelegatingAsyncConsistentTreeMap<V> that =
- (DelegatingAsyncConsistentTreeMap) other;
- return this.delegateMap.equals(that.delegateMap);
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(delegateMap);
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncDocumentTree.java
deleted file mode 100644
index 6182c48..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncDocumentTree.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-/**
- * Document tree that delegates to an underlying instance.
- */
-public class DelegatingAsyncDocumentTree<V> extends DelegatingDistributedPrimitive implements AsyncDocumentTree<V> {
- private final AsyncDocumentTree<V> delegateTree;
-
- public DelegatingAsyncDocumentTree(AsyncDocumentTree<V> delegateTree) {
- super(delegateTree);
- this.delegateTree = delegateTree;
- }
-
- @Override
- public DocumentPath root() {
- return delegateTree.root();
- }
-
- @Override
- public CompletableFuture<Map<String, Versioned<V>>> getChildren(DocumentPath path) {
- return delegateTree.getChildren(path);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(DocumentPath path) {
- return delegateTree.get(path);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
- return delegateTree.set(path, value);
- }
-
- @Override
- public CompletableFuture<Boolean> create(DocumentPath path, V value) {
- return delegateTree.create(path, value);
- }
-
- @Override
- public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
- return delegateTree.createRecursive(path, value);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
- return delegateTree.replace(path, newValue, version);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
- return delegateTree.replace(path, newValue, currentValue);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
- return delegateTree.removeNode(path);
- }
-
- @Override
- public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
- return delegateTree.addListener(path, listener);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
- return delegateTree.removeListener(listener);
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- return delegateTree.begin(transactionId);
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<V>> transactionLog) {
- return delegateTree.prepare(transactionLog);
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<V>> transactionLog) {
- return delegateTree.prepareAndCommit(transactionLog);
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- return delegateTree.commit(transactionId);
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- return delegateTree.rollback(transactionId);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("delegateTree", delegateTree)
- .toString();
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(delegateTree);
- }
-
- @Override
- public boolean equals(Object other) {
- if (other instanceof DelegatingAsyncDocumentTree) {
- DelegatingAsyncDocumentTree<V> that = (DelegatingAsyncDocumentTree) other;
- return this.delegateTree.equals(that.delegateTree);
- }
- return false;
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncLeaderElector.java
deleted file mode 100644
index 0235616..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncLeaderElector.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.service.AsyncLeaderElector;
-
-/**
- * Delegating leader elector.
- */
-public class DelegatingAsyncLeaderElector extends DelegatingDistributedPrimitive implements AsyncLeaderElector {
-
- private final AsyncLeaderElector delegateLeaderElector;
-
- public DelegatingAsyncLeaderElector(AsyncLeaderElector delegateLeaderElector) {
- super(delegateLeaderElector);
- this.delegateLeaderElector = delegateLeaderElector;
- }
-
- @Override
- public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
- return delegateLeaderElector.run(topic, nodeId);
- }
-
- @Override
- public CompletableFuture<Void> withdraw(String topic) {
- return delegateLeaderElector.withdraw(topic);
- }
-
- @Override
- public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
- return delegateLeaderElector.anoint(topic, nodeId);
- }
-
- @Override
- public CompletableFuture<Void> evict(NodeId nodeId) {
- return delegateLeaderElector.evict(nodeId);
- }
-
- @Override
- public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
- return delegateLeaderElector.promote(topic, nodeId);
- }
-
- @Override
- public CompletableFuture<Leadership> getLeadership(String topic) {
- return delegateLeaderElector.getLeadership(topic);
- }
-
- @Override
- public CompletableFuture<Map<String, Leadership>> getLeaderships() {
- return delegateLeaderElector.getLeaderships();
- }
-
- @Override
- public CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> consumer) {
- return delegateLeaderElector.addChangeListener(consumer);
- }
-
- @Override
- public CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
- return delegateLeaderElector.removeChangeListener(consumer);
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingDistributedPrimitive.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingDistributedPrimitive.java
deleted file mode 100644
index ff6876f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingDistributedPrimitive.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.core.ApplicationId;
-import org.onosproject.store.service.DistributedPrimitive;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Base class for primitive delegates.
- */
-public abstract class DelegatingDistributedPrimitive implements DistributedPrimitive {
- private final DistributedPrimitive primitive;
-
- public DelegatingDistributedPrimitive(DistributedPrimitive primitive) {
- this.primitive = checkNotNull(primitive);
- }
-
- @Override
- public String name() {
- return primitive.name();
- }
-
- @Override
- public Type primitiveType() {
- return primitive.primitiveType();
- }
-
- @Override
- public ApplicationId applicationId() {
- return primitive.applicationId();
- }
-
- @Override
- public CompletableFuture<Void> destroy() {
- return primitive.destroy();
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- primitive.addStatusChangeListener(listener);
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- primitive.removeStatusChangeListener(listener);
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- return primitive.statusChangeListeners();
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("delegate", primitive)
- .toString();
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(primitive);
- }
-
- @Override
- public boolean equals(Object other) {
- return other instanceof DelegatingDistributedPrimitive
- && primitive.equals(((DelegatingDistributedPrimitive) other).primitive);
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DistributedPrimitives.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DistributedPrimitives.java
deleted file mode 100644
index 9b209ba..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DistributedPrimitives.java
+++ /dev/null
@@ -1,267 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import org.onosproject.core.Version;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.AsyncDocumentTree;
-
-/**
- * Misc utilities for working with {@code DistributedPrimitive}s.
- */
-public final class DistributedPrimitives {
-
- private DistributedPrimitives() {}
-
- /**
- * Creates an instance of {@code AsyncDistributedSet} that is backed by a {@code AsyncConsistentMap}.
- *
- * @param map backing map
- * @return set
- * @param <E> set element type
- */
- public static <E> AsyncDistributedSet<E> newSetFromMap(AsyncConsistentMap<E, Boolean> map) {
- return new DefaultAsyncDistributedSet<>(map, map.name(), true);
- }
-
- /**
- * Creates an instance of {@code AsyncConsistentMap} that records metrics for all its operations.
- *
- * @param map map whose operations are to be metered
- * @return metered map
- * @param <K> map key type
- * @param <V> map value type
- */
- public static <K, V> AsyncConsistentMap<K, V> newMeteredMap(AsyncConsistentMap<K, V> map) {
- return new MeteredAsyncConsistentMap<>(map);
- }
-
- /**
- * Creates an instance of {@code AsyncConsistentMap} that caches entries on get.
- *
- * @param map backing map
- * @return caching map
- * @param <K> map key type
- * @param <V> map value type
- */
- public static <K, V> AsyncConsistentMap<K, V> newCachingMap(AsyncConsistentMap<K, V> map) {
- return new CachingAsyncConsistentMap<>(map);
- }
-
- /**
- * Creates an instance of {@code AsyncConsistentMap} that disallows updates.
- *
- * @param map backing map
- * @return unmodifiable map
- * @param <K> map key type
- * @param <V> map value type
- */
- public static <K, V> AsyncConsistentMap<K, V> newUnmodifiableMap(AsyncConsistentMap<K, V> map) {
- return new UnmodifiableAsyncConsistentMap<>(map);
- }
-
- /**
- * Creates an instance of {@code AsyncConsistentMap} that disallows null values.
- *
- * @param map backing map
- * @return not null map
- * @param <K> map key type
- * @param <V> map value type
- */
- public static <K, V> AsyncConsistentMap<K, V> newNotNullMap(AsyncConsistentMap<K, V> map) {
- return new NotNullAsyncConsistentMap<>(map);
- }
-
- /**
- * Creates an instance of {@code AsyncConsistentMap} that converts values from other versions.
- *
- * @param map backing map
- * @param compatibilityFunction the compatibility function
- * @param version local node version
- * @param <K> map key type
- * @param <V> map value type
- * @return compatible map
- */
- public static <K, V> AsyncConsistentMap<K, V> newCompatibleMap(
- AsyncConsistentMap<K, CompatibleValue<V>> map,
- BiFunction<V, Version, V> compatibilityFunction,
- Version version) {
- Function<V, CompatibleValue<V>> encoder = value -> new CompatibleValue<>(value, version);
- Function<CompatibleValue<V>, V> decoder = value -> {
- if (!value.version().equals(version)) {
- return compatibilityFunction.apply(value.value(), value.version());
- }
- return value.value();
- };
- return new TranscodingAsyncConsistentMap<>(map, k -> k, k -> k, encoder, decoder);
- }
-
- /**
- * Creates an instance of {@code AsyncAtomicValue} that transforms value types.
- *
- * @param value backing value
- * @param valueEncoder transformer for value type of returned value to value type of input value
- * @param valueDecoder transformer for value type of input value to value type of returned value
- * @param <V1> returned value type
- * @param <V2> input value type
- * @return new counter map
- */
- public static <V1, V2> AsyncAtomicValue<V1> newTranscodingAtomicValue(AsyncAtomicValue<V2> value,
- Function<V1, V2> valueEncoder,
- Function<V2, V1> valueDecoder) {
- return new TranscodingAsyncAtomicValue<>(value, valueEncoder, valueDecoder);
- }
-
- /**
- * Creates an instance of {@code AsyncAtomicValue} that converts values from other versions.
- *
- * @param atomicValue backing value
- * @param compatibilityFunction the compatibility function
- * @param version local node version
- * @param <V> value type
- * @return compatible map
- */
- public static <V> AsyncAtomicValue<V> newCompatibleAtomicValue(
- AsyncAtomicValue<CompatibleValue<V>> atomicValue,
- BiFunction<V, Version, V> compatibilityFunction,
- Version version) {
- Function<V, CompatibleValue<V>> encoder = value -> new CompatibleValue<>(value, version);
- Function<CompatibleValue<V>, V> decoder = value -> {
- if (!value.version().equals(version)) {
- return compatibilityFunction.apply(value.value(), value.version());
- }
- return value.value();
- };
- return new TranscodingAsyncAtomicValue<>(atomicValue, encoder, decoder);
- }
-
- /**
- * Creates an instance of {@code AsyncAtomicCounterMap} that transforms key types.
- *
- * @param map backing map
- * @param keyEncoder transformer for key type of returned map to key type of input map
- * @param keyDecoder transformer for key type of input map to key type of returned map
- * @param <K1> returned map key type
- * @param <K2> input map key type
- * @return new counter map
- */
- public static <K1, K2> AsyncAtomicCounterMap<K1> newTranscodingAtomicCounterMap(AsyncAtomicCounterMap<K2> map,
- Function<K1, K2> keyEncoder,
- Function<K2, K1> keyDecoder) {
- return new TranscodingAsyncAtomicCounterMap<>(map, keyEncoder, keyDecoder);
- }
-
- /**
- * Creates an instance of {@code AsyncConsistentMap} that transforms operations inputs and applies them
- * to corresponding operation in a different typed map and returns the output after reverse transforming it.
- *
- * @param map backing map
- * @param keyEncoder transformer for key type of returned map to key type of input map
- * @param keyDecoder transformer for key type of input map to key type of returned map
- * @param valueEncoder transformer for value type of returned map to value type of input map
- * @param valueDecoder transformer for value type of input map to value type of returned map
- * @param <K1> returned map key type
- * @param <K2> input map key type
- * @param <V1> returned map value type
- * @param <V2> input map key type
- * @return new map
- */
- public static <K1, V1, K2, V2> AsyncConsistentMap<K1, V1> newTranscodingMap(AsyncConsistentMap<K2, V2> map,
- Function<K1, K2> keyEncoder,
- Function<K2, K1> keyDecoder,
- Function<V1, V2> valueEncoder,
- Function<V2, V1> valueDecoder) {
- return new TranscodingAsyncConsistentMap<K1, V1, K2, V2>(map,
- keyEncoder,
- keyDecoder,
- valueEncoder,
- valueDecoder);
- }
-
- /**
- * Creates an instance of {@code DistributedTreeMap} that transforms operations inputs and applies them
- * to corresponding operation in a different typed map and returns the output after reverse transforming it.
- *
- * @param map backing map
- * @param valueEncoder transformer for value type of returned map to value type of input map
- * @param valueDecoder transformer for value type of input map to value type of returned map
- * @param <V1> returned map value type
- * @param <V2> input map key type
- * @return new map
- */
- public static <V1, V2> AsyncConsistentTreeMap<V1> newTranscodingTreeMap(
- AsyncConsistentTreeMap<V2> map,
- Function<V1, V2> valueEncoder,
- Function<V2, V1> valueDecoder) {
- return new TranscodingAsyncConsistentTreeMap<>(map,
- valueEncoder,
- valueDecoder);
- }
-
- /**
- * Creates an instance of {@code AsyncConsistentMultimap} that transforms
- * operations inputs and applies them to corresponding operation in a
- * differently typed map and returns the output after reverse transforming
- * it.
- *
- * @param multimap backing multimap
- * @param keyEncoder transformer for key type of returned map to key type
- * of input map
- * @param keyDecoder transformer for key type of input map to key type of
- * returned map
- * @param valueEncoder transformer for value type of returned map to value
- * type of input map
- * @param valueDecoder transformer for value type of input map to value
- * type of returned map
- * @param <K1> returned map key type
- * @param <K2> input map key type
- * @param <V1> returned map value type
- * @param <V2> input map key type
- * @return new map
- */
- public static <K1, V1, K2, V2> AsyncConsistentMultimap<K1, V1>
- newTranscodingMultimap(AsyncConsistentMultimap<K2, V2> multimap,
- Function<K1, K2> keyEncoder,
- Function<K2, K1> keyDecoder,
- Function<V1, V2> valueEncoder,
- Function<V2, V1> valueDecoder) {
- return new TranscodingAsyncConsistentMultimap<>(multimap,
- keyEncoder,
- keyDecoder,
- valueDecoder,
- valueEncoder);
- }
-
- /**
- * Creates an instance of {@code AsyncDocumentTree} that caches values on get.
- *
- * @param tree backing tree
- * @return caching tree
- * @param <V> tree value type
- */
- public static <V> AsyncDocumentTree<V> newCachingDocumentTree(AsyncDocumentTree<V> tree) {
- return new CachingAsyncDocumentTree<V>(tree);
- }
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapBuilderImpl.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapBuilderImpl.java
index df35bd6..da00555 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapBuilderImpl.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapBuilderImpl.java
@@ -15,14 +15,6 @@
*/
package org.onosproject.store.primitives.impl;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.persistence.PersistenceService;
-import org.onosproject.store.Timestamp;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.service.EventuallyConsistentMap;
-import org.onosproject.store.service.EventuallyConsistentMapBuilder;
-
import java.util.Collection;
import java.util.List;
import java.util.concurrent.ExecutorService;
@@ -31,6 +23,14 @@
import java.util.function.BiFunction;
import java.util.function.Supplier;
+import org.onlab.util.KryoNamespace;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.persistence.PersistenceService;
+import org.onosproject.store.Timestamp;
+import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
+import org.onosproject.store.service.EventuallyConsistentMap;
+import org.onosproject.store.service.EventuallyConsistentMapBuilder;
+
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapImpl.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapImpl.java
index 74e567b..762689c 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapImpl.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapImpl.java
@@ -15,6 +15,31 @@
*/
package org.onosproject.store.primitives.impl;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.Timer;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
import com.google.common.collect.Collections2;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
@@ -42,31 +67,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Set;
-import java.util.Timer;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/FederatedDistributedPrimitiveCreator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/FederatedDistributedPrimitiveCreator.java
deleted file mode 100644
index 3fe721f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/FederatedDistributedPrimitiveCreator.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.function.Function;
-
-import com.google.common.base.Charsets;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.google.common.hash.Funnel;
-import com.google.common.hash.Funnels;
-import com.google.common.hash.Hashing;
-import org.onlab.util.HexString;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.AsyncAtomicIdGenerator;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncDistributedLock;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.AsyncLeaderElector;
-import org.onosproject.store.service.AtomicCounterMapOptions;
-import org.onosproject.store.service.AtomicCounterOptions;
-import org.onosproject.store.service.AtomicIdGeneratorOptions;
-import org.onosproject.store.service.AtomicValueOptions;
-import org.onosproject.store.service.ConsistentMapOptions;
-import org.onosproject.store.service.ConsistentMultimapOptions;
-import org.onosproject.store.service.ConsistentTreeMapOptions;
-import org.onosproject.store.service.DistributedLockOptions;
-import org.onosproject.store.service.DistributedSetOptions;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeOptions;
-import org.onosproject.store.service.LeaderElectorOptions;
-import org.onosproject.store.service.WorkQueue;
-import org.onosproject.store.service.WorkQueueOptions;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@code DistributedPrimitiveCreator} that federates responsibility for creating
- * distributed primitives to a collection of other {@link DistributedPrimitiveCreator creators}.
- */
-public class FederatedDistributedPrimitiveCreator implements DistributedPrimitiveCreator {
-
- private static final Funnel<Iterable<? extends CharSequence>> STR_LIST_FUNNEL =
- Funnels.sequentialFunnel(Funnels.unencodedCharsFunnel());
-
- private final TreeMap<PartitionId, DistributedPrimitiveCreator> members;
- private final List<PartitionId> sortedMemberPartitionIds;
- private final int buckets;
-
- public FederatedDistributedPrimitiveCreator(Map<PartitionId, DistributedPrimitiveCreator> members, int buckets) {
- this.members = Maps.newTreeMap();
- this.members.putAll(checkNotNull(members));
- this.sortedMemberPartitionIds = Lists.newArrayList(members.keySet());
- this.buckets = buckets;
- }
-
- @Override
- public <K, V> AsyncConsistentMap<K, V> newAsyncConsistentMap(ConsistentMapOptions options) {
- Map<PartitionId, AsyncConsistentMap<byte[], byte[]>> maps =
- Maps.transformValues(members,
- partition -> DistributedPrimitives.newTranscodingMap(
- partition.<String, byte[]>newAsyncConsistentMap(options.name(), null),
- HexString::toHexString,
- HexString::fromHexString,
- Function.identity(),
- Function.identity()));
- Hasher<byte[]> hasher = key -> {
- int bucket = Math.abs(Hashing.murmur3_32().hashBytes(key).asInt()) % buckets;
- return sortedMemberPartitionIds.get(Hashing.consistentHash(bucket, sortedMemberPartitionIds.size()));
- };
- AsyncConsistentMap<byte[], byte[]> partitionedMap =
- new PartitionedAsyncConsistentMap<>(options.name(), maps, hasher);
- return DistributedPrimitives.newTranscodingMap(partitionedMap,
- key -> options.serializer().encode(key),
- bytes -> options.serializer().decode(bytes),
- value -> value == null ? null : options.serializer().encode(value),
- bytes -> options.serializer().decode(bytes));
- }
-
- @Override
- public <V> AsyncConsistentTreeMap<V> newAsyncConsistentTreeMap(ConsistentTreeMapOptions options) {
- return getCreator(options.name()).newAsyncConsistentTreeMap(options.name(), options.serializer());
- }
-
- @Override
- public <K, V> AsyncConsistentMultimap<K, V> newAsyncConsistentSetMultimap(ConsistentMultimapOptions options) {
- Map<PartitionId, AsyncConsistentMultimap<byte[], byte[]>> maps =
- Maps.transformValues(members,
- partition -> DistributedPrimitives.newTranscodingMultimap(
- partition.<String, byte[]>newAsyncConsistentSetMultimap(options.name(), null),
- HexString::toHexString,
- HexString::fromHexString,
- Function.identity(),
- Function.identity()));
- Hasher<byte[]> hasher = key -> {
- int bucket = Math.abs(Hashing.murmur3_32().hashBytes(key).asInt()) % buckets;
- return sortedMemberPartitionIds.get(Hashing.consistentHash(bucket, sortedMemberPartitionIds.size()));
- };
- AsyncConsistentMultimap<byte[], byte[]> partitionedMap =
- new PartitionedAsyncConsistentMultimap<>(options.name(), maps, hasher);
- return DistributedPrimitives.newTranscodingMultimap(partitionedMap,
- key -> options.serializer().encode(key),
- bytes -> options.serializer().decode(bytes),
- value -> value == null ? null : options.serializer().encode(value),
- bytes -> options.serializer().decode(bytes));
- }
-
- @Override
- public <E> AsyncDistributedSet<E> newAsyncDistributedSet(DistributedSetOptions options) {
- return DistributedPrimitives.newSetFromMap(newAsyncConsistentMap(options.name(), options.serializer()));
- }
-
- @Override
- public <K> AsyncAtomicCounterMap<K> newAsyncAtomicCounterMap(AtomicCounterMapOptions options) {
- return getCreator(options.name()).newAsyncAtomicCounterMap(options);
- }
-
- @Override
- public AsyncAtomicCounter newAsyncCounter(AtomicCounterOptions options) {
- return getCreator(options.name()).newAsyncCounter(options);
- }
-
- @Override
- public AsyncAtomicIdGenerator newAsyncIdGenerator(AtomicIdGeneratorOptions options) {
- return getCreator(options.name()).newAsyncIdGenerator(options);
- }
-
- @Override
- public <V> AsyncAtomicValue<V> newAsyncAtomicValue(AtomicValueOptions options) {
- return getCreator(options.name()).newAsyncAtomicValue(options);
- }
-
- @Override
- public AsyncDistributedLock newAsyncDistributedLock(DistributedLockOptions options) {
- return getCreator(options.name()).newAsyncDistributedLock(options);
- }
-
- @Override
- public AsyncLeaderElector newAsyncLeaderElector(LeaderElectorOptions options) {
- Map<PartitionId, AsyncLeaderElector> leaderElectors =
- Maps.transformValues(members,
- partition -> partition.newAsyncLeaderElector(options));
- Hasher<String> hasher = topic -> {
- int hashCode = Hashing.sha256().hashString(topic, Charsets.UTF_8).asInt();
- return sortedMemberPartitionIds.get(Math.abs(hashCode) % members.size());
- };
- return new PartitionedAsyncLeaderElector(options.name(), leaderElectors, hasher);
- }
-
- @Override
- public <E> WorkQueue<E> newWorkQueue(WorkQueueOptions options) {
- return getCreator(options.name()).newWorkQueue(options);
- }
-
- @Override
- public <V> AsyncDocumentTree<V> newAsyncDocumentTree(DocumentTreeOptions options) {
- Map<PartitionId, AsyncDocumentTree<V>> trees =
- Maps.transformValues(members, part -> part.<V>newAsyncDocumentTree(options));
- Hasher<DocumentPath> hasher = key -> {
- int bucket = (key == null) ? 0 :
- Math.abs(Hashing.murmur3_32()
- .hashObject(key.pathElements(), STR_LIST_FUNNEL)
- .asInt()) % buckets;
- return sortedMemberPartitionIds.get(Hashing.consistentHash(bucket, sortedMemberPartitionIds.size()));
- };
- return new PartitionedAsyncDocumentTree<>(options.name(), trees, hasher);
- }
-
- @Override
- public Set<String> getAsyncConsistentMapNames() {
- return members.values()
- .stream()
- .map(DistributedPrimitiveCreator::getAsyncConsistentMapNames)
- .reduce(Sets::union)
- .orElse(ImmutableSet.of());
- }
-
- @Override
- public Set<String> getAsyncAtomicCounterNames() {
- return members.values()
- .stream()
- .map(DistributedPrimitiveCreator::getAsyncAtomicCounterNames)
- .reduce(Sets::union)
- .orElse(ImmutableSet.of());
- }
-
- @Override
- public Set<String> getWorkQueueNames() {
- return members.values()
- .stream()
- .map(DistributedPrimitiveCreator::getWorkQueueNames)
- .reduce(Sets::union)
- .orElse(ImmutableSet.of());
- }
-
- /**
- * Returns the {@code DistributedPrimitiveCreator} to use for hosting a primitive.
- * @param name primitive name
- * @return primitive creator
- */
- private DistributedPrimitiveCreator getCreator(String name) {
- int hashCode = Hashing.sha256().hashString(name, Charsets.UTF_8).asInt();
- return members.get(sortedMemberPartitionIds.get(Math.abs(hashCode) % members.size()));
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Hasher.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Hasher.java
deleted file mode 100644
index 5714e33..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Hasher.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.cluster.PartitionId;
-
-/**
- * Interface for mapping from an object to {@link PartitionId}.
- *
- * @param <K> object type.
- */
-public interface Hasher<K> {
- /**
- * Returns the {@link PartitionId} to which the specified object maps.
- * @param object object
- * @return partition identifier
- */
- PartitionId hash(K object);
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Managed.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Managed.java
deleted file mode 100644
index 1d5977f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Managed.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.concurrent.CompletableFuture;
-
-/**
- * Interface for types that can be asynchronously opened and closed.
- */
-public interface Managed<T> {
-
- /**
- * Opens the managed object.
- *
- * @return A completable future to be completed once the object has been opened.
- */
- CompletableFuture<Void> open();
-
- /**
- * Closes the managed object.
- *
- * @return A completable future to be completed once the object has been closed.
- */
- CompletableFuture<Void> close();
-
- /**
- * Return {@code true} if the managed object is open.
- * @return {@code true} if open
- */
- boolean isOpen();
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MapValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MapValue.java
index 1c6845d..3175963 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MapValue.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MapValue.java
@@ -15,10 +15,9 @@
*/
package org.onosproject.store.primitives.impl;
-import org.onosproject.store.Timestamp;
-
import com.google.common.base.MoreObjects;
import com.google.common.base.Objects;
+import org.onosproject.store.Timestamp;
import static com.google.common.base.Preconditions.checkNotNull;
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MeteredAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MeteredAsyncConsistentMap.java
deleted file mode 100644
index de53121..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MeteredAsyncConsistentMap.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import com.google.common.collect.Maps;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-import org.onosproject.utils.MeteringAgent;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Predicate;
-
-/**
- * {@link AsyncConsistentMap} that meters all its operations.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class MeteredAsyncConsistentMap<K, V> extends DelegatingAsyncConsistentMap<K, V> {
-
- private static final String PRIMITIVE_NAME = "consistentMap";
- private static final String SIZE = "size";
- private static final String IS_EMPTY = "isEmpty";
- private static final String CONTAINS_KEY = "containsKey";
- private static final String CONTAINS_VALUE = "containsValue";
- private static final String GET = "get";
- private static final String GET_OR_DEFAULT = "getOrDefault";
- private static final String COMPUTE_IF = "computeIf";
- private static final String PUT = "put";
- private static final String PUT_AND_GET = "putAndGet";
- private static final String PUT_IF_ABSENT = "putIfAbsent";
- private static final String REMOVE = "remove";
- private static final String CLEAR = "clear";
- private static final String KEY_SET = "keySet";
- private static final String VALUES = "values";
- private static final String ENTRY_SET = "entrySet";
- private static final String REPLACE = "replace";
- private static final String COMPUTE_IF_ABSENT = "computeIfAbsent";
- private static final String BEGIN = "begin";
- private static final String PREPARE = "prepare";
- private static final String COMMIT = "commit";
- private static final String ROLLBACK = "rollback";
- private static final String PREPARE_AND_COMMIT = "prepareAndCommit";
- private static final String ADD_LISTENER = "addListener";
- private static final String REMOVE_LISTENER = "removeListener";
- private static final String NOTIFY_LISTENER = "notifyListener";
-
- private final Map<MapEventListener<K, V>, InternalMeteredMapEventListener> listeners =
- Maps.newIdentityHashMap();
- private final MeteringAgent monitor;
-
- public MeteredAsyncConsistentMap(AsyncConsistentMap<K, V> backingMap) {
- super(backingMap);
- this.monitor = new MeteringAgent(PRIMITIVE_NAME, backingMap.name(), true);
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- final MeteringAgent.Context timer = monitor.startTimer(SIZE);
- return super.size()
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- final MeteringAgent.Context timer = monitor.startTimer(IS_EMPTY);
- return super.isEmpty()
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- final MeteringAgent.Context timer = monitor.startTimer(CONTAINS_KEY);
- return super.containsKey(key)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- final MeteringAgent.Context timer = monitor.startTimer(CONTAINS_VALUE);
- return super.containsValue(value)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(K key) {
- final MeteringAgent.Context timer = monitor.startTimer(GET);
- return super.get(key)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
- final MeteringAgent.Context timer = monitor.startTimer(GET_OR_DEFAULT);
- return super.getOrDefault(key, defaultValue)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> computeIfAbsent(K key,
- Function<? super K, ? extends V> mappingFunction) {
- final MeteringAgent.Context timer = monitor.startTimer(COMPUTE_IF_ABSENT);
- return super.computeIfAbsent(key, mappingFunction)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> computeIf(K key,
- Predicate<? super V> condition,
- BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- final MeteringAgent.Context timer = monitor.startTimer(COMPUTE_IF);
- return super.computeIf(key, condition, remappingFunction)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(K key, V value) {
- final MeteringAgent.Context timer = monitor.startTimer(PUT);
- return super.put(key, value)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
- final MeteringAgent.Context timer = monitor.startTimer(PUT_AND_GET);
- return super.putAndGet(key, value)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> remove(K key) {
- final MeteringAgent.Context timer = monitor.startTimer(REMOVE);
- return super.remove(key)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- final MeteringAgent.Context timer = monitor.startTimer(CLEAR);
- return super.clear()
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Set<K>> keySet() {
- final MeteringAgent.Context timer = monitor.startTimer(KEY_SET);
- return super.keySet()
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V>>> values() {
- final MeteringAgent.Context timer = monitor.startTimer(VALUES);
- return super.values()
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Set<Entry<K, Versioned<V>>>> entrySet() {
- final MeteringAgent.Context timer = monitor.startTimer(ENTRY_SET);
- return super.entrySet()
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
- final MeteringAgent.Context timer = monitor.startTimer(PUT_IF_ABSENT);
- return super.putIfAbsent(key, value)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- final MeteringAgent.Context timer = monitor.startTimer(REMOVE);
- return super.remove(key, value)
- .whenComplete((r, e) -> timer.stop(e));
-
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, long version) {
- final MeteringAgent.Context timer = monitor.startTimer(REMOVE);
- return super.remove(key, version)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(K key, V value) {
- final MeteringAgent.Context timer = monitor.startTimer(REPLACE);
- return super.replace(key, value)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
- final MeteringAgent.Context timer = monitor.startTimer(REPLACE);
- return super.replace(key, oldValue, newValue)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
- final MeteringAgent.Context timer = monitor.startTimer(REPLACE);
- return super.replace(key, oldVersion, newValue)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
- final MeteringAgent.Context timer = monitor.startTimer(ADD_LISTENER);
- synchronized (listeners) {
- InternalMeteredMapEventListener meteredListener =
- listeners.computeIfAbsent(listener, k -> new InternalMeteredMapEventListener(listener));
- return super.addListener(meteredListener, executor)
- .whenComplete((r, e) -> timer.stop(e));
- }
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
- final MeteringAgent.Context timer = monitor.startTimer(REMOVE_LISTENER);
- InternalMeteredMapEventListener meteredListener = listeners.remove(listener);
- if (meteredListener != null) {
- return super.removeListener(meteredListener)
- .whenComplete((r, e) -> timer.stop(e));
- } else {
- timer.stop(null);
- return CompletableFuture.completedFuture(null);
- }
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- final MeteringAgent.Context timer = monitor.startTimer(BEGIN);
- return super.begin(transactionId)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K, V>> transactionLog) {
- final MeteringAgent.Context timer = monitor.startTimer(PREPARE);
- return super.prepare(transactionLog)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- final MeteringAgent.Context timer = monitor.startTimer(COMMIT);
- return super.commit(transactionId)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- final MeteringAgent.Context timer = monitor.startTimer(ROLLBACK);
- return super.rollback(transactionId)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K, V>> transactionLog) {
- final MeteringAgent.Context timer = monitor.startTimer(PREPARE_AND_COMMIT);
- return super.prepareAndCommit(transactionLog)
- .whenComplete((r, e) -> timer.stop(e));
- }
-
- private class InternalMeteredMapEventListener implements MapEventListener<K, V> {
-
- private final MapEventListener<K, V> listener;
-
- InternalMeteredMapEventListener(MapEventListener<K, V> listener) {
- this.listener = listener;
- }
-
- @Override
- public void event(MapEvent<K, V> event) {
- final MeteringAgent.Context timer = monitor.startTimer(NOTIFY_LISTENER);
- try {
- listener.event(event);
- timer.stop(null);
- } catch (Exception e) {
- timer.stop(e);
- throw new IllegalStateException(e);
- }
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NotNullAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NotNullAsyncConsistentMap.java
deleted file mode 100644
index 9282cbf..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NotNullAsyncConsistentMap.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.Versioned;
-
-/**
- * {@link org.onosproject.store.service.AsyncConsistentMap} that doesn't allow null values.
- */
-public class NotNullAsyncConsistentMap<K, V> extends DelegatingAsyncConsistentMap<K, V> {
- private final AsyncConsistentMap<K, V> delegateMap;
-
- public NotNullAsyncConsistentMap(AsyncConsistentMap<K, V> delegateMap) {
- super(delegateMap);
- this.delegateMap = delegateMap;
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- if (value == null) {
- return CompletableFuture.completedFuture(false);
- }
- return super.containsValue(value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(K key) {
- return super.get(key).thenApply(v -> v != null && v.value() == null ? null : v);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
- return super.getOrDefault(key, defaultValue).thenApply(v -> v != null && v.value() == null ? null : v);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(K key, V value) {
- if (value == null) {
- return super.remove(key);
- }
- return super.put(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
- if (value == null) {
- return super.remove(key).thenApply(v -> null);
- }
- return super.putAndGet(key, value);
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V>>> values() {
- return super.values().thenApply(value -> value.stream()
- .filter(v -> v.value() != null)
- .collect(Collectors.toList()));
- }
-
- @Override
- public CompletableFuture<Set<Map.Entry<K, Versioned<V>>>> entrySet() {
- return super.entrySet().thenApply(entries -> entries.stream()
- .filter(e -> e.getValue().value() != null)
- .collect(Collectors.toSet()));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
- if (value == null) {
- return super.remove(key);
- }
- return super.putIfAbsent(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- if (value == null) {
- return CompletableFuture.completedFuture(false);
- }
- return super.remove(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, long version) {
- return super.remove(key, version);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(K key, V value) {
- if (value == null) {
- return super.remove(key);
- }
- return super.replace(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
- if (oldValue == null) {
- return super.putIfAbsent(key, newValue).thenApply(Objects::isNull);
- } else if (newValue == null) {
- return super.remove(key, oldValue);
- }
- return super.replace(key, oldValue, newValue);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
- return super.replace(key, oldVersion, newValue);
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionManager.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionManager.java
index 28cc69e..f9a8ec0 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionManager.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionManager.java
@@ -17,32 +17,22 @@
package org.onosproject.store.primitives.impl;
import java.util.List;
-import java.util.Map;
import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
+import io.atomix.cluster.MemberId;
+import io.atomix.primitive.partition.PartitionGroup;
+import io.atomix.protocols.raft.MultiRaftProtocol;
import org.apache.felix.scr.annotations.Activate;
import org.apache.felix.scr.annotations.Component;
import org.apache.felix.scr.annotations.Deactivate;
import org.apache.felix.scr.annotations.Reference;
import org.apache.felix.scr.annotations.ReferenceCardinality;
import org.apache.felix.scr.annotations.Service;
-import org.onlab.util.Tools;
-import org.onosproject.cluster.ClusterMetadata;
-import org.onosproject.cluster.ClusterMetadataDiff;
-import org.onosproject.cluster.ClusterMetadataEvent;
-import org.onosproject.cluster.ClusterMetadataEventListener;
-import org.onosproject.cluster.ClusterMetadataService;
-import org.onosproject.cluster.ClusterService;
import org.onosproject.cluster.NodeId;
-import org.onosproject.cluster.PartitionDiff;
import org.onosproject.cluster.PartitionId;
import org.onosproject.event.AbstractListenerManager;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
+import org.onosproject.store.impl.AtomixManager;
import org.onosproject.store.primitives.DistributedPrimitiveCreator;
import org.onosproject.store.primitives.PartitionAdminService;
import org.onosproject.store.primitives.PartitionEvent;
@@ -67,74 +57,52 @@
private final Logger log = getLogger(getClass());
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
- protected ClusterCommunicationService clusterCommunicator;
+ protected AtomixManager atomixManager;
- @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
- protected ClusterMetadataService metadataService;
-
- @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
- protected ClusterService clusterService;
-
- private final Map<PartitionId, StoragePartition> partitions = Maps.newConcurrentMap();
- private final AtomicReference<ClusterMetadata> currentClusterMetadata = new AtomicReference<>();
-
- private final ClusterMetadataEventListener metadataListener = new InternalClusterMetadataListener();
+ private PartitionGroup partitionGroup;
@Activate
public void activate() {
+ partitionGroup = atomixManager.getAtomix().getPartitionService().getPartitionGroup(MultiRaftProtocol.TYPE);
eventDispatcher.addSink(PartitionEvent.class, listenerRegistry);
- currentClusterMetadata.set(metadataService.getClusterMetadata());
-
- metadataService.addListener(metadataListener);
-
- currentClusterMetadata.get()
- .getPartitions()
- .forEach(partition -> partitions.put(partition.getId(), new StoragePartition(
- partition,
- clusterCommunicator,
- clusterService)));
- CompletableFuture.allOf(partitions.values().stream()
- .map(StoragePartition::open)
- .toArray(CompletableFuture[]::new))
- .join();
log.info("Started");
}
@Deactivate
public void deactivate() {
- metadataService.removeListener(metadataListener);
eventDispatcher.removeSink(PartitionEvent.class);
-
- CompletableFuture.allOf(partitions.values().stream()
- .map(StoragePartition::close)
- .toArray(CompletableFuture[]::new))
- .join();
log.info("Stopped");
}
@Override
public int getNumberOfPartitions() {
checkPermission(PARTITION_READ);
- return partitions.size();
+ return partitionGroup.getPartitions().size();
}
@Override
public Set<PartitionId> getAllPartitionIds() {
checkPermission(PARTITION_READ);
- return partitions.keySet();
+ return partitionGroup.getPartitionIds().stream()
+ .map(partitionId -> PartitionId.from(partitionId.id()))
+ .collect(Collectors.toSet());
}
@Override
public DistributedPrimitiveCreator getDistributedPrimitiveCreator(PartitionId partitionId) {
checkPermission(PARTITION_READ);
- return partitions.get(partitionId).client();
+ throw new UnsupportedOperationException();
}
@Override
public Set<NodeId> getConfiguredMembers(PartitionId partitionId) {
checkPermission(PARTITION_READ);
- StoragePartition partition = partitions.get(partitionId);
- return ImmutableSet.copyOf(partition.getMembers());
+ io.atomix.primitive.partition.PartitionId atomixPartitionId =
+ io.atomix.primitive.partition.PartitionId.from(partitionGroup.name(), partitionId.id());
+ return partitionGroup.getPartition(atomixPartitionId).members()
+ .stream()
+ .map(member -> NodeId.nodeId(member.id()))
+ .collect(Collectors.toSet());
}
@Override
@@ -147,36 +115,30 @@
@Override
public List<PartitionInfo> partitionInfo() {
- return partitions.values()
- .stream()
- .flatMap(x -> Tools.stream(x.info()))
- .collect(Collectors.toList());
- }
-
- private void processMetadataUpdate(ClusterMetadata clusterMetadata) {
- ClusterMetadataDiff diffExaminer =
- new ClusterMetadataDiff(currentClusterMetadata.get(), clusterMetadata);
- diffExaminer.partitionDiffs()
- .values()
- .stream()
- .filter(PartitionDiff::hasChanged)
- .forEach(diff -> partitions.get(diff.partitionId()).onUpdate(diff.newValue()));
- currentClusterMetadata.set(clusterMetadata);
- }
-
- private class InternalClusterMetadataListener implements ClusterMetadataEventListener {
- @Override
- public void event(ClusterMetadataEvent event) {
- processMetadataUpdate(event.subject());
- }
+ checkPermission(PARTITION_READ);
+ return partitionGroup.getPartitions()
+ .stream()
+ .map(partition -> {
+ MemberId primary = partition.primary();
+ return new PartitionInfo(
+ PartitionId.from(partition.id().id()),
+ partition.term(),
+ partition.members().stream().map(member -> member.id()).collect(Collectors.toList()),
+ primary != null ? primary.id() : null);
+ })
+ .collect(Collectors.toList());
}
@Override
public List<PartitionClientInfo> partitionClientInfo() {
- return partitions.values()
- .stream()
- .map(StoragePartition::client)
- .map(StoragePartitionClient::clientInfo)
- .collect(Collectors.toList());
+ checkPermission(PARTITION_READ);
+ return partitionGroup.getPartitions()
+ .stream()
+ .map(partition -> new PartitionClientInfo(
+ PartitionId.from(partition.id().id()),
+ partition.members().stream()
+ .map(member -> NodeId.nodeId(member.id()))
+ .collect(Collectors.toList())))
+ .collect(Collectors.toList());
}
}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMap.java
deleted file mode 100644
index 38ba0f3..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMap.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import org.onlab.util.Match;
-import org.onlab.util.Tools;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-
-/**
- * {@link AsyncConsistentMap} that has its entries partitioned horizontally across
- * several {@link AsyncConsistentMap maps}.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class PartitionedAsyncConsistentMap<K, V> implements AsyncConsistentMap<K, V> {
-
- private final String name;
- private final TreeMap<PartitionId, AsyncConsistentMap<K, V>> partitions = Maps.newTreeMap();
- private final Hasher<K> keyHasher;
-
- public PartitionedAsyncConsistentMap(String name,
- Map<PartitionId, AsyncConsistentMap<K, V>> partitions,
- Hasher<K> keyHasher) {
- this.name = name;
- this.partitions.putAll(checkNotNull(partitions));
- this.keyHasher = checkNotNull(keyHasher);
- }
-
- @Override
- public String name() {
- return name;
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return Tools.allOf(getMaps().stream().map(m -> m.size()).collect(Collectors.toList()),
- Math::addExact,
- 0);
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- return size().thenApply(size -> size == 0);
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- return getMap(key).containsKey(key);
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- return Tools.firstOf(getMaps().stream().map(m -> m.containsValue(value)).collect(Collectors.toList()),
- Match.ifValue(true),
- false);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(K key) {
- return getMap(key).get(key);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
- return getMap(key).getOrDefault(key, defaultValue);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> computeIf(K key,
- Predicate<? super V> condition,
- BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- return getMap(key).computeIf(key, condition, remappingFunction);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(K key, V value) {
- return getMap(key).put(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
- return getMap(key).putAndGet(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> remove(K key) {
- return getMap(key).remove(key);
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return CompletableFuture.allOf(getMaps().stream()
- .map(map -> map.clear())
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Set<K>> keySet() {
- return Tools.allOf(getMaps().stream().map(m -> m.keySet()).collect(Collectors.toList()),
- (s1, s2) -> ImmutableSet.<K>builder().addAll(s1).addAll(s2).build(),
- ImmutableSet.of());
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V>>> values() {
- return Tools.allOf(getMaps().stream().map(m -> m.values()).collect(Collectors.toList()),
- (c1, c2) -> ImmutableList.<Versioned<V>>builder().addAll(c1).addAll(c2).build(),
- ImmutableList.of());
- }
-
- @Override
- public CompletableFuture<Set<Entry<K, Versioned<V>>>> entrySet() {
- return Tools.allOf(getMaps().stream().map(m -> m.entrySet()).collect(Collectors.toList()),
- (s1, s2) -> ImmutableSet.<Entry<K, Versioned<V>>>builder().addAll(s1).addAll(s2).build(),
- ImmutableSet.of());
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
- return getMap(key).putIfAbsent(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return getMap(key).remove(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, long version) {
- return getMap(key).remove(key, version);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(K key, V value) {
- return getMap(key).replace(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
- return getMap(key).replace(key, oldValue, newValue);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
- return getMap(key).replace(key, oldVersion, newValue);
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Entry<K, Versioned<V>>>> iterator() {
- return Tools.allOf(getMaps().stream().map(m -> m.iterator()).collect(Collectors.toList()))
- .thenApply(PartitionedMultimapIterator::new);
- }
-
- @Override
- public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
- return CompletableFuture.allOf(getMaps().stream()
- .map(map -> map.addListener(listener, executor))
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
- return CompletableFuture.allOf(getMaps().stream()
- .map(map -> map.removeListener(listener))
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K, V>> transactionLog) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K, V>> transactionLog) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- partitions.values().forEach(map -> map.addStatusChangeListener(listener));
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- partitions.values().forEach(map -> map.removeStatusChangeListener(listener));
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Returns the map (partition) to which the specified key maps.
- * @param key key
- * @return AsyncConsistentMap to which key maps
- */
- private AsyncConsistentMap<K, V> getMap(K key) {
- return partitions.get(keyHasher.hash(key));
- }
-
- /**
- * Returns all the constituent maps.
- * @return collection of maps.
- */
- private Collection<AsyncConsistentMap<K, V>> getMaps() {
- return partitions.values();
- }
-
- private class PartitionedMultimapIterator<K, V> implements AsyncIterator<Map.Entry<K, Versioned<V>>> {
- private final Iterator<AsyncIterator<Entry<K, Versioned<V>>>> iterators;
- private volatile AsyncIterator<Entry<K, Versioned<V>>> iterator;
-
- public PartitionedMultimapIterator(List<AsyncIterator<Entry<K, Versioned<V>>>> iterators) {
- this.iterators = iterators.iterator();
- }
-
- @Override
- public CompletableFuture<Boolean> hasNext() {
- if (iterator == null && iterators.hasNext()) {
- iterator = iterators.next();
- }
- if (iterator == null) {
- return CompletableFuture.completedFuture(false);
- }
- return iterator.hasNext()
- .thenCompose(hasNext -> {
- if (!hasNext) {
- iterator = null;
- return hasNext();
- }
- return CompletableFuture.completedFuture(true);
- });
- }
-
- @Override
- public CompletableFuture<Entry<K, Versioned<V>>> next() {
- if (iterator == null && iterators.hasNext()) {
- iterator = iterators.next();
- }
- if (iterator == null) {
- return Tools.exceptionalFuture(new NoSuchElementException());
- }
- return iterator.next();
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMultimap.java
deleted file mode 100644
index a4e0bf9..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMultimap.java
+++ /dev/null
@@ -1,277 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multiset;
-import com.google.common.collect.Multisets;
-import org.onlab.util.Match;
-import org.onlab.util.Tools;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@link AsyncConsistentMultimap} that has its entries partitioned horizontally across
- * several {@link AsyncConsistentMultimap maps}.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class PartitionedAsyncConsistentMultimap<K, V> implements AsyncConsistentMultimap<K, V> {
-
- private final String name;
- private final TreeMap<PartitionId, AsyncConsistentMultimap<K, V>> partitions = Maps.newTreeMap();
- private final Hasher<K> keyHasher;
-
- public PartitionedAsyncConsistentMultimap(String name,
- Map<PartitionId, AsyncConsistentMultimap<K, V>> partitions,
- Hasher<K> keyHasher) {
- this.name = name;
- this.partitions.putAll(checkNotNull(partitions));
- this.keyHasher = checkNotNull(keyHasher);
- }
-
- @Override
- public String name() {
- return name;
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return Tools.allOf(getMultimaps().stream().map(m -> m.size()).collect(Collectors.toList()),
- Math::addExact,
- 0);
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- return size().thenApply(size -> size == 0);
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- return getMultimap(key).containsKey(key);
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- return Tools.firstOf(getMultimaps().stream().map(m -> m.containsValue(value)).collect(Collectors.toList()),
- Match.ifValue(true),
- false);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> get(K key) {
- return getMultimap(key).get(key);
- }
-
- @Override
- public CompletableFuture<Boolean> containsEntry(K key, V value) {
- return getMultimap(key).containsEntry(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> put(K key, V value) {
- return getMultimap(key).put(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> putAndGet(K key, V value) {
- return getMultimap(key).putAndGet(key, value);
- }
-
- @Override
- public CompletableFuture<Boolean> removeAll(K key, Collection<? extends V> values) {
- return getMultimap(key).removeAll(key, values);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> removeAll(K key) {
- return getMultimap(key).removeAll(key);
- }
-
- @Override
- public CompletableFuture<Boolean> putAll(K key, Collection<? extends V> values) {
- return getMultimap(key).putAll(key, values);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> replaceValues(K key, Collection<V> values) {
- return getMultimap(key).replaceValues(key, values);
- }
-
- @Override
- public CompletableFuture<Map<K, Collection<V>>> asMap() {
- throw new UnsupportedOperationException("Expensive operation.");
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return CompletableFuture.allOf(getMultimaps().stream()
- .map(map -> map.clear())
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Set<K>> keySet() {
- return Tools.allOf(getMultimaps().stream().map(m -> m.keySet()).collect(Collectors.toList()),
- (s1, s2) -> ImmutableSet.<K>builder().addAll(s1).addAll(s2).build(),
- ImmutableSet.of());
- }
-
- @Override
- public CompletableFuture<Multiset<K>> keys() {
- return Tools.allOf(getMultimaps().stream().map(m -> m.keys()).collect(Collectors.toList()))
- .thenApply(results -> results.stream().reduce(Multisets::sum).orElse(HashMultiset.create()));
- }
-
- @Override
- public CompletableFuture<Multiset<V>> values() {
- return Tools.allOf(getMultimaps().stream().map(m -> m.values()).collect(Collectors.toList()))
- .thenApply(results -> results.stream().reduce(Multisets::sum).orElse(HashMultiset.create()));
- }
-
- @Override
- public CompletableFuture<Collection<Entry<K, V>>> entries() {
- return Tools.allOf(getMultimaps().stream().map(m -> m.entries()).collect(Collectors.toList()))
- .thenApply(results -> results.stream().reduce((s1, s2) -> ImmutableList.copyOf(Iterables.concat(s1, s2)))
- .orElse(ImmutableList.of()));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return getMultimap(key).remove(key, value);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V>>> removeAndGet(K key, V value) {
- return getMultimap(key).removeAndGet(key, value);
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Entry<K, V>>> iterator() {
- return Tools.allOf(getMultimaps().stream().map(m -> m.iterator()).collect(Collectors.toList()))
- .thenApply(PartitionedMultimapIterator::new);
- }
-
- @Override
- public CompletableFuture<Void> addListener(MultimapEventListener<K, V> listener, Executor executor) {
- return CompletableFuture.allOf(getMultimaps().stream()
- .map(map -> map.addListener(listener, executor))
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MultimapEventListener<K, V> listener) {
- return CompletableFuture.allOf(getMultimaps().stream()
- .map(map -> map.removeListener(listener))
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- partitions.values().forEach(map -> map.addStatusChangeListener(listener));
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- partitions.values().forEach(map -> map.removeStatusChangeListener(listener));
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- throw new UnsupportedOperationException();
- }
-
- /**
- * Returns the map (partition) to which the specified key maps.
- *
- * @param key key
- * @return AsyncConsistentMap to which key maps
- */
- private AsyncConsistentMultimap<K, V> getMultimap(K key) {
- return partitions.get(keyHasher.hash(key));
- }
-
- /**
- * Returns all the constituent maps.
- *
- * @return collection of maps.
- */
- private Collection<AsyncConsistentMultimap<K, V>> getMultimaps() {
- return partitions.values();
- }
-
- private class PartitionedMultimapIterator<K, V> implements AsyncIterator<Map.Entry<K, V>> {
- private final Iterator<AsyncIterator<Entry<K, V>>> iterators;
- private volatile AsyncIterator<Entry<K, V>> iterator;
-
- public PartitionedMultimapIterator(List<AsyncIterator<Entry<K, V>>> iterators) {
- this.iterators = iterators.iterator();
- }
-
- @Override
- public CompletableFuture<Boolean> hasNext() {
- if (iterator == null && iterators.hasNext()) {
- iterator = iterators.next();
- }
- if (iterator == null) {
- return CompletableFuture.completedFuture(false);
- }
- return iterator.hasNext()
- .thenCompose(hasNext -> {
- if (!hasNext) {
- iterator = null;
- return hasNext();
- }
- return CompletableFuture.completedFuture(true);
- });
- }
-
- @Override
- public CompletableFuture<Entry<K, V>> next() {
- if (iterator == null && iterators.hasNext()) {
- iterator = iterators.next();
- }
- if (iterator == null) {
- return Tools.exceptionalFuture(new NoSuchElementException());
- }
- return iterator.next();
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncDocumentTree.java
deleted file mode 100644
index 2d42d4f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncDocumentTree.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import org.onlab.util.Tools;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Partitioned asynchronous document tree.
- */
-public class PartitionedAsyncDocumentTree<V> implements AsyncDocumentTree<V> {
-
- private final String name;
- private final TreeMap<PartitionId, AsyncDocumentTree<V>> partitions = Maps.newTreeMap();
- private final Hasher<DocumentPath> pathHasher;
-
- public PartitionedAsyncDocumentTree(
- String name,
- Map<PartitionId, AsyncDocumentTree<V>> partitions,
- Hasher<DocumentPath> pathHasher) {
- this.name = name;
- this.partitions.putAll(checkNotNull(partitions));
- this.pathHasher = checkNotNull(pathHasher);
- }
-
- @Override
- public String name() {
- return name;
- }
-
- @Override
- public DocumentPath root() {
- return DocumentPath.ROOT;
- }
-
- /**
- * Returns the document tree (partition) to which the specified path maps.
- *
- * @param path path
- * @return AsyncConsistentMap to which path maps
- */
- private AsyncDocumentTree<V> partition(DocumentPath path) {
- return partitions.get(pathHasher.hash(path));
- }
-
- /**
- * Returns all the constituent trees.
- *
- * @return collection of partitions.
- */
- private Collection<AsyncDocumentTree<V>> partitions() {
- return partitions.values();
- }
-
- @Override
- public CompletableFuture<Map<String, Versioned<V>>> getChildren(DocumentPath path) {
- return Tools.allOf(partitions().stream()
- .map(partition -> partition.getChildren(path).exceptionally(r -> null))
- .collect(Collectors.toList())).thenApply(allChildren -> {
- Map<String, Versioned<V>> children = Maps.newLinkedHashMap();
- allChildren.stream().filter(Objects::nonNull).forEach(children::putAll);
- return children;
- });
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(DocumentPath path) {
- return partition(path).get(path);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
- return partition(path).set(path, value);
- }
-
- @Override
- public CompletableFuture<Boolean> create(DocumentPath path, V value) {
- if (path.parent() == null) {
- // create value on root
- return partition(path).createRecursive(path, value);
- }
- // TODO: This operation is not atomic
- return partition(path.parent()).get(path.parent()).thenCompose(parentValue -> {
- if (parentValue == null) {
- return Tools.exceptionalFuture(new NoSuchDocumentPathException(String.valueOf(path.parent())));
- } else {
- // not atomic: parent did exist at some point, so moving forward
- return partition(path).createRecursive(path, value);
- }
- });
- }
-
- @Override
- public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
- return partition(path).createRecursive(path, value);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
- return partition(path).replace(path, newValue, version);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
- return partition(path).replace(path, newValue, currentValue);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
- return partition(path).removeNode(path);
- }
-
- @Override
- public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
- return CompletableFuture.allOf(partitions().stream()
- .map(map -> map.addListener(path, listener))
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
- return CompletableFuture.allOf(partitions().stream()
- .map(map -> map.removeListener(listener))
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- return partitions().stream()
- .map(p -> p.begin(transactionId))
- // returning lowest Version
- .reduce((f1, f2) -> f1.thenCombine(f2, Tools::min))
- .orElse(Tools.exceptionalFuture(new IllegalStateException("Empty partitions?")));
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<V>> transactionLog) {
- Map<AsyncDocumentTree<V>, List<NodeUpdate<V>>> perPart =
- transactionLog.records().stream()
- .collect(Collectors.groupingBy(nu -> partition(nu.path())));
-
- // must walk all partitions to ensure empty TransactionLog will
- // be issued against no-op partitions in order for commit to succeed
- return partitions().stream()
- .map(p -> p.prepare(new TransactionLog<>(transactionLog.transactionId(),
- transactionLog.version(),
- perPart.getOrDefault(p, ImmutableList.of()))))
- .reduce((f1, f2) -> f1.thenCombine(f2, Boolean::logicalAnd))
- .orElse(CompletableFuture.completedFuture(true));
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<V>> transactionLog) {
- // Note: cannot call prepareAndCommit on each partition,
- // must check all partitions are prepare()-ed first.
- return prepare(transactionLog)
- .thenApply(prepOk -> {
- if (prepOk) {
- commit(transactionLog.transactionId());
- return true;
- } else {
- return false;
- }
- });
-
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- return CompletableFuture.allOf(partitions().stream()
- .map(p -> p.commit(transactionId))
- .toArray(CompletableFuture[]::new)
- );
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- return CompletableFuture.allOf(partitions().stream()
- .map(p -> p.rollback(transactionId))
- .toArray(CompletableFuture[]::new)
- );
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncLeaderElector.java
deleted file mode 100644
index 78bcf82..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncLeaderElector.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.event.Change;
-import org.onosproject.store.service.AsyncLeaderElector;
-import com.google.common.collect.Maps;
-
-/**
- * {@link AsyncLeaderElector} that has its topics partitioned horizontally across
- * several {@link AsyncLeaderElector leader electors}.
- */
-public class PartitionedAsyncLeaderElector implements AsyncLeaderElector {
-
- private final String name;
- private final TreeMap<PartitionId, AsyncLeaderElector> partitions = Maps.newTreeMap();
- private final Hasher<String> topicHasher;
-
- public PartitionedAsyncLeaderElector(String name,
- Map<PartitionId, AsyncLeaderElector> partitions,
- Hasher<String> topicHasher) {
- this.name = name;
- this.partitions.putAll(checkNotNull(partitions));
- this.topicHasher = checkNotNull(topicHasher);
- }
-
- @Override
- public String name() {
- return name;
- }
-
- @Override
- public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
- return getLeaderElector(topic).run(topic, nodeId);
- }
-
- @Override
- public CompletableFuture<Void> withdraw(String topic) {
- return getLeaderElector(topic).withdraw(topic);
- }
-
- @Override
- public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
- return getLeaderElector(topic).anoint(topic, nodeId);
- }
-
- @Override
- public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
- return getLeaderElector(topic).promote(topic, nodeId);
- }
-
- @Override
- public CompletableFuture<Void> evict(NodeId nodeId) {
- return CompletableFuture.allOf(getLeaderElectors().stream()
- .map(le -> le.evict(nodeId))
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Leadership> getLeadership(String topic) {
- return getLeaderElector(topic).getLeadership(topic);
- }
-
- @Override
- public CompletableFuture<Map<String, Leadership>> getLeaderships() {
- Map<String, Leadership> leaderships = Maps.newConcurrentMap();
- return CompletableFuture.allOf(getLeaderElectors().stream()
- .map(le -> le.getLeaderships()
- .thenAccept(m -> leaderships.putAll(m)))
- .toArray(CompletableFuture[]::new))
- .thenApply(v -> leaderships);
- }
-
- @Override
- public CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> listener) {
- return CompletableFuture.allOf(getLeaderElectors().stream()
- .map(map -> map.addChangeListener(listener))
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> listener) {
- return CompletableFuture.allOf(getLeaderElectors().stream()
- .map(map -> map.removeChangeListener(listener))
- .toArray(CompletableFuture[]::new));
- }
-
- /**
- * Returns the leaderElector (partition) to which the specified topic maps.
- * @param topic topic name
- * @return AsyncLeaderElector to which topic maps
- */
- private AsyncLeaderElector getLeaderElector(String topic) {
- return partitions.get(topicHasher.hash(topic));
- }
-
- /**
- * Returns all the constituent leader electors.
- * @return collection of leader electors.
- */
- private Collection<AsyncLeaderElector> getLeaderElectors() {
- return partitions.values();
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- partitions.values().forEach(elector -> elector.addStatusChangeListener(listener));
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- partitions.values().forEach(elector -> elector.removeStatusChangeListener(listener));
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- throw new UnsupportedOperationException();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedTransactionalMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedTransactionalMap.java
deleted file mode 100644
index 4f6d5b6..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedTransactionalMap.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Map;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.service.TransactionalMap;
-
-/**
- * Partitioned transactional map.
- */
-public class PartitionedTransactionalMap<K, V> implements TransactionalMap<K, V> {
- protected final Map<PartitionId, TransactionalMapParticipant<K, V>> partitions;
- protected final Hasher<K> hasher;
-
- public PartitionedTransactionalMap(
- Map<PartitionId, TransactionalMapParticipant<K, V>> partitions, Hasher<K> hasher) {
- this.partitions = partitions;
- this.hasher = hasher;
- }
-
- /**
- * Returns the collection of map partitions.
- *
- * @return a collection of map partitions
- */
- @SuppressWarnings("unchecked")
- Collection<TransactionParticipant> participants() {
- return (Collection) partitions.values();
- }
-
- /**
- * Returns the partition for the given key.
- *
- * @param key the key for which to return the partition
- * @return the partition for the given key
- */
- private TransactionalMap<K, V> partition(K key) {
- return partitions.get(hasher.hash(key));
- }
-
- @Override
- public V get(K key) {
- return partition(key).get(key);
- }
-
- @Override
- public boolean containsKey(K key) {
- return partition(key).containsKey(key);
- }
-
- @Override
- public V put(K key, V value) {
- return partition(key).put(key, value);
- }
-
- @Override
- public V remove(K key) {
- return partition(key).remove(key);
- }
-
- @Override
- public V putIfAbsent(K key, V value) {
- return partition(key).putIfAbsent(key, value);
- }
-
- @Override
- public boolean remove(K key, V value) {
- return partition(key).remove(key, value);
- }
-
- @Override
- public boolean replace(K key, V oldValue, V newValue) {
- return partition(key).replace(key, oldValue, newValue);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(this)
- .add("partitions", partitions.values())
- .toString();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftClientCommunicator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftClientCommunicator.java
deleted file mode 100644
index 45f40ae..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftClientCommunicator.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.protocol.CloseSessionRequest;
-import io.atomix.protocols.raft.protocol.CloseSessionResponse;
-import io.atomix.protocols.raft.protocol.CommandRequest;
-import io.atomix.protocols.raft.protocol.CommandResponse;
-import io.atomix.protocols.raft.protocol.HeartbeatRequest;
-import io.atomix.protocols.raft.protocol.HeartbeatResponse;
-import io.atomix.protocols.raft.protocol.KeepAliveRequest;
-import io.atomix.protocols.raft.protocol.KeepAliveResponse;
-import io.atomix.protocols.raft.protocol.MetadataRequest;
-import io.atomix.protocols.raft.protocol.MetadataResponse;
-import io.atomix.protocols.raft.protocol.OpenSessionRequest;
-import io.atomix.protocols.raft.protocol.OpenSessionResponse;
-import io.atomix.protocols.raft.protocol.PublishRequest;
-import io.atomix.protocols.raft.protocol.QueryRequest;
-import io.atomix.protocols.raft.protocol.QueryResponse;
-import io.atomix.protocols.raft.protocol.RaftClientProtocol;
-import io.atomix.protocols.raft.protocol.ResetRequest;
-import io.atomix.protocols.raft.session.SessionId;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Raft client protocol that uses a cluster communicator.
- */
-public class RaftClientCommunicator extends RaftCommunicator implements RaftClientProtocol {
-
- public RaftClientCommunicator(
- String prefix,
- Serializer serializer,
- ClusterCommunicationService clusterCommunicator) {
- super(new RaftMessageContext(prefix), serializer, clusterCommunicator);
- }
-
- @Override
- public CompletableFuture<OpenSessionResponse> openSession(MemberId memberId, OpenSessionRequest request) {
- return sendAndReceive(context.openSessionSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<CloseSessionResponse> closeSession(MemberId memberId, CloseSessionRequest request) {
- return sendAndReceive(context.closeSessionSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<KeepAliveResponse> keepAlive(MemberId memberId, KeepAliveRequest request) {
- return sendAndReceive(context.keepAliveSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<QueryResponse> query(MemberId memberId, QueryRequest request) {
- return sendAndReceive(context.querySubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<CommandResponse> command(MemberId memberId, CommandRequest request) {
- return sendAndReceive(context.commandSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<MetadataResponse> metadata(MemberId memberId, MetadataRequest request) {
- return sendAndReceive(context.metadataSubject, request, memberId);
- }
-
- @Override
- public void registerHeartbeatHandler(Function<HeartbeatRequest, CompletableFuture<HeartbeatResponse>> function) {
- clusterCommunicator.addSubscriber(context.heartbeatSubject, serializer::decode, function, serializer::encode);
- }
-
- @Override
- public void unregisterHeartbeatHandler() {
- clusterCommunicator.removeSubscriber(context.heartbeatSubject);
- }
-
- @Override
- public void reset(Collection<MemberId> members, ResetRequest request) {
- Set<NodeId> nodes = members.stream().map(m -> NodeId.nodeId(m.id())).collect(Collectors.toSet());
- clusterCommunicator.multicast(
- request,
- context.resetSubject(request.session()),
- serializer::encode,
- nodes);
- }
-
- @Override
- public void registerPublishListener(SessionId sessionId, Consumer<PublishRequest> listener, Executor executor) {
- clusterCommunicator.addSubscriber(
- context.publishSubject(sessionId.id()),
- serializer::decode,
- listener,
- executor);
- }
-
- @Override
- public void unregisterPublishListener(SessionId sessionId) {
- clusterCommunicator.removeSubscriber(context.publishSubject(sessionId.id()));
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftCommunicator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftCommunicator.java
deleted file mode 100644
index 1117ab9..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftCommunicator.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.net.ConnectException;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
-
-import io.atomix.protocols.raft.RaftException;
-import io.atomix.protocols.raft.cluster.MemberId;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.cluster.messaging.MessageSubject;
-import org.onosproject.store.cluster.messaging.MessagingException;
-import org.onosproject.store.service.Serializer;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Abstract base class for Raft protocol client/server.
- */
-public abstract class RaftCommunicator {
- protected final RaftMessageContext context;
- protected final Serializer serializer;
- protected final ClusterCommunicationService clusterCommunicator;
-
- public RaftCommunicator(
- RaftMessageContext context,
- Serializer serializer,
- ClusterCommunicationService clusterCommunicator) {
- this.context = checkNotNull(context, "context cannot be null");
- this.serializer = checkNotNull(serializer, "serializer cannot be null");
- this.clusterCommunicator = checkNotNull(clusterCommunicator, "clusterCommunicator cannot be null");
- }
-
- protected <T, U> CompletableFuture<U> sendAndReceive(MessageSubject subject, T request, MemberId memberId) {
- CompletableFuture<U> future = new CompletableFuture<>();
- clusterCommunicator.<T, U>sendAndReceive(
- request, subject, serializer::encode, serializer::decode, NodeId.nodeId(memberId.id()))
- .whenComplete((result, error) -> {
- if (error == null) {
- future.complete(result);
- } else {
- if (error instanceof CompletionException) {
- error = error.getCause();
- }
- if (error instanceof MessagingException.NoRemoteHandler) {
- error = new ConnectException(error.getMessage());
- } else if (error instanceof MessagingException.RemoteHandlerFailure
- || error instanceof MessagingException.ProtocolException) {
- error = new RaftException.ProtocolException(error.getMessage());
- }
- future.completeExceptionally(error);
- }
- });
- return future;
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftMessageContext.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftMessageContext.java
deleted file mode 100644
index 9f1f1bc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftMessageContext.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.cluster.messaging.MessageSubject;
-
-/**
- * Protocol message context.
- */
-final class RaftMessageContext {
- private final String prefix;
- final MessageSubject heartbeatSubject;
- final MessageSubject openSessionSubject;
- final MessageSubject closeSessionSubject;
- final MessageSubject keepAliveSubject;
- final MessageSubject querySubject;
- final MessageSubject commandSubject;
- final MessageSubject metadataSubject;
- final MessageSubject joinSubject;
- final MessageSubject leaveSubject;
- final MessageSubject configureSubject;
- final MessageSubject reconfigureSubject;
- final MessageSubject installSubject;
- final MessageSubject pollSubject;
- final MessageSubject voteSubject;
- final MessageSubject appendSubject;
- final MessageSubject transferSubject;
-
- RaftMessageContext(String prefix) {
- this.prefix = prefix;
- this.heartbeatSubject = getSubject(prefix, "heartbeat");
- this.openSessionSubject = getSubject(prefix, "open");
- this.closeSessionSubject = getSubject(prefix, "close");
- this.keepAliveSubject = getSubject(prefix, "keep-alive");
- this.querySubject = getSubject(prefix, "query");
- this.commandSubject = getSubject(prefix, "command");
- this.metadataSubject = getSubject(prefix, "metadata");
- this.joinSubject = getSubject(prefix, "join");
- this.leaveSubject = getSubject(prefix, "leave");
- this.configureSubject = getSubject(prefix, "configure");
- this.reconfigureSubject = getSubject(prefix, "reconfigure");
- this.installSubject = getSubject(prefix, "install");
- this.pollSubject = getSubject(prefix, "poll");
- this.voteSubject = getSubject(prefix, "vote");
- this.appendSubject = getSubject(prefix, "append");
- this.transferSubject = getSubject(prefix, "transfer");
- }
-
- private static MessageSubject getSubject(String prefix, String type) {
- if (prefix == null) {
- return new MessageSubject(type);
- } else {
- return new MessageSubject(String.format("%s-%s", prefix, type));
- }
- }
-
- /**
- * Returns the publish subject for the given session.
- *
- * @param sessionId the session for which to return the publish subject
- * @return the publish subject for the given session
- */
- MessageSubject publishSubject(long sessionId) {
- if (prefix == null) {
- return new MessageSubject(String.format("publish-%d", sessionId));
- } else {
- return new MessageSubject(String.format("%s-publish-%d", prefix, sessionId));
- }
- }
-
- /**
- * Returns the reset subject for the given session.
- *
- * @param sessionId the session for which to return the reset subject
- * @return the reset subject for the given session
- */
- MessageSubject resetSubject(long sessionId) {
- if (prefix == null) {
- return new MessageSubject(String.format("reset-%d", sessionId));
- } else {
- return new MessageSubject(String.format("%s-reset-%d", prefix, sessionId));
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftServerCommunicator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftServerCommunicator.java
deleted file mode 100644
index f58db45..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftServerCommunicator.java
+++ /dev/null
@@ -1,324 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.protocol.AppendRequest;
-import io.atomix.protocols.raft.protocol.AppendResponse;
-import io.atomix.protocols.raft.protocol.CloseSessionRequest;
-import io.atomix.protocols.raft.protocol.CloseSessionResponse;
-import io.atomix.protocols.raft.protocol.CommandRequest;
-import io.atomix.protocols.raft.protocol.CommandResponse;
-import io.atomix.protocols.raft.protocol.ConfigureRequest;
-import io.atomix.protocols.raft.protocol.ConfigureResponse;
-import io.atomix.protocols.raft.protocol.HeartbeatRequest;
-import io.atomix.protocols.raft.protocol.HeartbeatResponse;
-import io.atomix.protocols.raft.protocol.InstallRequest;
-import io.atomix.protocols.raft.protocol.InstallResponse;
-import io.atomix.protocols.raft.protocol.JoinRequest;
-import io.atomix.protocols.raft.protocol.JoinResponse;
-import io.atomix.protocols.raft.protocol.KeepAliveRequest;
-import io.atomix.protocols.raft.protocol.KeepAliveResponse;
-import io.atomix.protocols.raft.protocol.LeaveRequest;
-import io.atomix.protocols.raft.protocol.LeaveResponse;
-import io.atomix.protocols.raft.protocol.MetadataRequest;
-import io.atomix.protocols.raft.protocol.MetadataResponse;
-import io.atomix.protocols.raft.protocol.OpenSessionRequest;
-import io.atomix.protocols.raft.protocol.OpenSessionResponse;
-import io.atomix.protocols.raft.protocol.PollRequest;
-import io.atomix.protocols.raft.protocol.PollResponse;
-import io.atomix.protocols.raft.protocol.PublishRequest;
-import io.atomix.protocols.raft.protocol.QueryRequest;
-import io.atomix.protocols.raft.protocol.QueryResponse;
-import io.atomix.protocols.raft.protocol.RaftServerProtocol;
-import io.atomix.protocols.raft.protocol.ReconfigureRequest;
-import io.atomix.protocols.raft.protocol.ReconfigureResponse;
-import io.atomix.protocols.raft.protocol.ResetRequest;
-import io.atomix.protocols.raft.protocol.TransferRequest;
-import io.atomix.protocols.raft.protocol.TransferResponse;
-import io.atomix.protocols.raft.protocol.VoteRequest;
-import io.atomix.protocols.raft.protocol.VoteResponse;
-import io.atomix.protocols.raft.session.SessionId;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Raft server protocol that uses a {@link ClusterCommunicationService}.
- */
-public class RaftServerCommunicator extends RaftCommunicator implements RaftServerProtocol {
-
- public RaftServerCommunicator(
- String prefix,
- Serializer serializer,
- ClusterCommunicationService clusterCommunicator) {
- super(new RaftMessageContext(prefix), serializer, clusterCommunicator);
- }
-
- @Override
- public CompletableFuture<OpenSessionResponse> openSession(MemberId memberId, OpenSessionRequest request) {
- return sendAndReceive(context.openSessionSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<CloseSessionResponse> closeSession(MemberId memberId, CloseSessionRequest request) {
- return sendAndReceive(context.closeSessionSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<KeepAliveResponse> keepAlive(MemberId memberId, KeepAliveRequest request) {
- return sendAndReceive(context.keepAliveSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<QueryResponse> query(MemberId memberId, QueryRequest request) {
- return sendAndReceive(context.querySubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<CommandResponse> command(MemberId memberId, CommandRequest request) {
- return sendAndReceive(context.commandSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<MetadataResponse> metadata(MemberId memberId, MetadataRequest request) {
- return sendAndReceive(context.metadataSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<JoinResponse> join(MemberId memberId, JoinRequest request) {
- return sendAndReceive(context.joinSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<LeaveResponse> leave(MemberId memberId, LeaveRequest request) {
- return sendAndReceive(context.leaveSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<ConfigureResponse> configure(MemberId memberId, ConfigureRequest request) {
- return sendAndReceive(context.configureSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<ReconfigureResponse> reconfigure(MemberId memberId, ReconfigureRequest request) {
- return sendAndReceive(context.reconfigureSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<InstallResponse> install(MemberId memberId, InstallRequest request) {
- return sendAndReceive(context.installSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<PollResponse> poll(MemberId memberId, PollRequest request) {
- return sendAndReceive(context.pollSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<VoteResponse> vote(MemberId memberId, VoteRequest request) {
- return sendAndReceive(context.voteSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<AppendResponse> append(MemberId memberId, AppendRequest request) {
- return sendAndReceive(context.appendSubject, request, memberId);
- }
-
- @Override
- public CompletableFuture<TransferResponse> transfer(MemberId memberId, TransferRequest request) {
- return sendAndReceive(context.transferSubject, request, memberId);
- }
-
- @Override
- public void publish(MemberId memberId, PublishRequest request) {
- clusterCommunicator.unicast(request,
- context.publishSubject(request.session()), serializer::encode, NodeId.nodeId(memberId.id()));
- }
-
- @Override
- public CompletableFuture<HeartbeatResponse> heartbeat(MemberId memberId, HeartbeatRequest request) {
- return sendAndReceive(context.heartbeatSubject, request, memberId);
- }
-
- @Override
- public void registerOpenSessionHandler(
- Function<OpenSessionRequest, CompletableFuture<OpenSessionResponse>> handler) {
- clusterCommunicator.addSubscriber(context.openSessionSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterOpenSessionHandler() {
- clusterCommunicator.removeSubscriber(context.openSessionSubject);
- }
-
- @Override
- public void registerCloseSessionHandler(
- Function<CloseSessionRequest, CompletableFuture<CloseSessionResponse>> handler) {
- clusterCommunicator.addSubscriber(context.closeSessionSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterCloseSessionHandler() {
- clusterCommunicator.removeSubscriber(context.closeSessionSubject);
- }
-
- @Override
- public void registerKeepAliveHandler(Function<KeepAliveRequest, CompletableFuture<KeepAliveResponse>> handler) {
- clusterCommunicator.addSubscriber(context.keepAliveSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterKeepAliveHandler() {
- clusterCommunicator.removeSubscriber(context.keepAliveSubject);
- }
-
- @Override
- public void registerQueryHandler(Function<QueryRequest, CompletableFuture<QueryResponse>> handler) {
- clusterCommunicator.addSubscriber(context.querySubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterQueryHandler() {
- clusterCommunicator.removeSubscriber(context.querySubject);
- }
-
- @Override
- public void registerCommandHandler(Function<CommandRequest, CompletableFuture<CommandResponse>> handler) {
- clusterCommunicator.addSubscriber(context.commandSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterCommandHandler() {
- clusterCommunicator.removeSubscriber(context.commandSubject);
- }
-
- @Override
- public void registerMetadataHandler(Function<MetadataRequest, CompletableFuture<MetadataResponse>> handler) {
- clusterCommunicator.addSubscriber(context.metadataSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterMetadataHandler() {
- clusterCommunicator.removeSubscriber(context.metadataSubject);
- }
-
- @Override
- public void registerJoinHandler(Function<JoinRequest, CompletableFuture<JoinResponse>> handler) {
- clusterCommunicator.addSubscriber(context.joinSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterJoinHandler() {
- clusterCommunicator.removeSubscriber(context.joinSubject);
- }
-
- @Override
- public void registerLeaveHandler(Function<LeaveRequest, CompletableFuture<LeaveResponse>> handler) {
- clusterCommunicator.addSubscriber(context.leaveSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterLeaveHandler() {
- clusterCommunicator.removeSubscriber(context.leaveSubject);
- }
-
- @Override
- public void registerConfigureHandler(Function<ConfigureRequest, CompletableFuture<ConfigureResponse>> handler) {
- clusterCommunicator.addSubscriber(context.configureSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterConfigureHandler() {
- clusterCommunicator.removeSubscriber(context.configureSubject);
- }
-
- @Override
- public void registerReconfigureHandler(
- Function<ReconfigureRequest, CompletableFuture<ReconfigureResponse>> handler) {
- clusterCommunicator.addSubscriber(context.reconfigureSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterReconfigureHandler() {
- clusterCommunicator.removeSubscriber(context.reconfigureSubject);
- }
-
- @Override
- public void registerInstallHandler(Function<InstallRequest, CompletableFuture<InstallResponse>> handler) {
- clusterCommunicator.addSubscriber(context.installSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterInstallHandler() {
- clusterCommunicator.removeSubscriber(context.installSubject);
- }
-
- @Override
- public void registerPollHandler(Function<PollRequest, CompletableFuture<PollResponse>> handler) {
- clusterCommunicator.addSubscriber(context.pollSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterPollHandler() {
- clusterCommunicator.removeSubscriber(context.pollSubject);
- }
-
- @Override
- public void registerVoteHandler(Function<VoteRequest, CompletableFuture<VoteResponse>> handler) {
- clusterCommunicator.addSubscriber(context.voteSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterVoteHandler() {
- clusterCommunicator.removeSubscriber(context.voteSubject);
- }
-
- @Override
- public void registerAppendHandler(Function<AppendRequest, CompletableFuture<AppendResponse>> handler) {
- clusterCommunicator.addSubscriber(context.appendSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterAppendHandler() {
- clusterCommunicator.removeSubscriber(context.appendSubject);
- }
-
- @Override
- public void registerTransferHandler(Function<TransferRequest, CompletableFuture<TransferResponse>> handler) {
- clusterCommunicator.addSubscriber(context.transferSubject, serializer::decode, handler, serializer::encode);
- }
-
- @Override
- public void unregisterTransferHandler() {
- clusterCommunicator.removeSubscriber(context.transferSubject);
- }
-
- @Override
- public void registerResetListener(SessionId sessionId, Consumer<ResetRequest> listener, Executor executor) {
- clusterCommunicator.addSubscriber(context.resetSubject(sessionId.id()), serializer::decode, listener, executor);
- }
-
- @Override
- public void unregisterResetListener(SessionId sessionId) {
- clusterCommunicator.removeSubscriber(context.resetSubject(sessionId.id()));
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageManager.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageManager.java
index 719a96e..dd71e38 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageManager.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageManager.java
@@ -18,11 +18,17 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
-import java.util.UUID;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import com.google.common.collect.Maps;
+import io.atomix.core.Atomix;
+import io.atomix.core.counter.AtomicCounter;
+import io.atomix.core.counter.AtomicCounterType;
+import io.atomix.core.map.AtomicMapType;
+import io.atomix.core.workqueue.WorkQueueType;
+import io.atomix.primitive.partition.PartitionGroup;
+import io.atomix.protocols.raft.MultiRaftProtocol;
import org.apache.felix.scr.annotations.Activate;
import org.apache.felix.scr.annotations.Component;
import org.apache.felix.scr.annotations.Deactivate;
@@ -34,15 +40,12 @@
import org.onosproject.cluster.Member;
import org.onosproject.cluster.MembershipService;
import org.onosproject.cluster.NodeId;
-import org.onosproject.cluster.PartitionId;
import org.onosproject.persistence.PersistenceService;
import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
+import org.onosproject.store.impl.AtomixManager;
import org.onosproject.store.primitives.PartitionAdminService;
-import org.onosproject.store.primitives.PartitionService;
import org.onosproject.store.primitives.TransactionId;
import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicValue;
import org.onosproject.store.service.AsyncConsistentMultimap;
import org.onosproject.store.service.AsyncConsistentTreeMap;
import org.onosproject.store.service.AsyncDocumentTree;
@@ -50,7 +53,6 @@
import org.onosproject.store.service.AtomicCounterMapBuilder;
import org.onosproject.store.service.AtomicIdGeneratorBuilder;
import org.onosproject.store.service.AtomicValueBuilder;
-import org.onosproject.store.service.ConsistentMap;
import org.onosproject.store.service.ConsistentMapBuilder;
import org.onosproject.store.service.ConsistentMultimapBuilder;
import org.onosproject.store.service.ConsistentTreeMapBuilder;
@@ -68,6 +70,7 @@
import org.onosproject.store.service.TopicBuilder;
import org.onosproject.store.service.TransactionContextBuilder;
import org.onosproject.store.service.WorkQueue;
+import org.onosproject.store.service.WorkQueueBuilder;
import org.onosproject.store.service.WorkQueueStats;
import org.slf4j.Logger;
@@ -82,8 +85,6 @@
@Component(immediate = true)
public class StorageManager implements StorageService, StorageAdminService {
- private static final int BUCKETS = 128;
-
private final Logger log = getLogger(getClass());
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
@@ -96,26 +97,21 @@
protected PersistenceService persistenceService;
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
- protected PartitionService partitionService;
-
- @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
protected PartitionAdminService partitionAdminService;
@Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
protected MembershipService membershipService;
- private final Supplier<TransactionId> transactionIdGenerator =
- () -> TransactionId.from(UUID.randomUUID().toString());
- private DistributedPrimitiveCreator federatedPrimitiveCreator;
- private TransactionManager transactionManager;
+ @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+ protected AtomixManager atomixManager;
+
+ private Atomix atomix;
+ private PartitionGroup group;
@Activate
public void activate() {
- Map<PartitionId, DistributedPrimitiveCreator> partitionMap = Maps.newHashMap();
- partitionService.getAllPartitionIds().stream()
- .forEach(id -> partitionMap.put(id, partitionService.getDistributedPrimitiveCreator(id)));
- federatedPrimitiveCreator = new FederatedDistributedPrimitiveCreator(partitionMap, BUCKETS);
- transactionManager = new TransactionManager(this, partitionService, BUCKETS);
+ atomix = atomixManager.getAtomix();
+ group = atomix.getPartitionService().getPartitionGroup(MultiRaftProtocol.TYPE);
log.info("Started");
}
@@ -137,175 +133,212 @@
// Use the MembershipService to provide peers for the map that are isolated within the current version.
Supplier<List<NodeId>> peersSupplier = () -> membershipService.getMembers().stream()
- .map(Member::nodeId)
- .filter(nodeId -> !nodeId.equals(localNodeId))
- .filter(id -> clusterService.getState(id).isActive())
- .collect(Collectors.toList());
+ .map(Member::nodeId)
+ .filter(nodeId -> !nodeId.equals(localNodeId))
+ .filter(id -> clusterService.getState(id).isActive())
+ .collect(Collectors.toList());
// If this is the first node in its version, bootstrap from the previous version. Otherwise, bootstrap the
// map from members isolated within the current version.
Supplier<List<NodeId>> bootstrapPeersSupplier = () -> {
if (membershipService.getMembers().size() == 1) {
return clusterService.getNodes()
- .stream()
- .map(ControllerNode::id)
- .filter(id -> !localNodeId.equals(id))
- .filter(id -> clusterService.getState(id).isActive())
- .collect(Collectors.toList());
+ .stream()
+ .map(ControllerNode::id)
+ .filter(id -> !localNodeId.equals(id))
+ .filter(id -> clusterService.getState(id).isActive())
+ .collect(Collectors.toList());
} else {
return membershipService.getMembers()
- .stream()
- .map(Member::nodeId)
- .filter(id -> !localNodeId.equals(id))
- .filter(id -> clusterService.getState(id).isActive())
- .collect(Collectors.toList());
+ .stream()
+ .map(Member::nodeId)
+ .filter(id -> !localNodeId.equals(id))
+ .filter(id -> clusterService.getState(id).isActive())
+ .collect(Collectors.toList());
}
};
return new EventuallyConsistentMapBuilderImpl<>(
- localNodeId,
- clusterCommunicator,
- persistenceService,
- peersSupplier,
- bootstrapPeersSupplier
+ localNodeId,
+ clusterCommunicator,
+ persistenceService,
+ peersSupplier,
+ bootstrapPeersSupplier
);
}
@Override
public <K, V> ConsistentMapBuilder<K, V> consistentMapBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultConsistentMapBuilder<>(federatedPrimitiveCreator);
+ return new AtomixConsistentMapBuilder<>(atomix, group.name());
}
@Override
public <V> DocumentTreeBuilder<V> documentTreeBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultDocumentTreeBuilder<V>(federatedPrimitiveCreator);
+ return new AtomixDocumentTreeBuilder<V>(atomix, group.name());
}
@Override
public <V> ConsistentTreeMapBuilder<V> consistentTreeMapBuilder() {
- return new DefaultConsistentTreeMapBuilder<V>(
- federatedPrimitiveCreator);
+ return new AtomixConsistentTreeMapBuilder<>(atomix, group.name());
}
@Override
public <K, V> ConsistentMultimapBuilder<K, V> consistentMultimapBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultConsistentMultimapBuilder<K, V>(
- federatedPrimitiveCreator);
+ return new AtomixConsistentMultimapBuilder<>(atomix, group.name());
}
@Override
public <K> AtomicCounterMapBuilder<K> atomicCounterMapBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultAtomicCounterMapBuilder<>(federatedPrimitiveCreator);
+ return new AtomixAtomicCounterMapBuilder<>(atomix, group.name());
}
@Override
public <E> DistributedSetBuilder<E> setBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultDistributedSetBuilder<>(() -> this.<E, Boolean>consistentMapBuilder());
+ return new AtomixDistributedSetBuilder<>(atomix, group.name());
}
@Override
public AtomicCounterBuilder atomicCounterBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultAtomicCounterBuilder(federatedPrimitiveCreator);
+ return new AtomixAtomicCounterBuilder(atomix, group.name());
}
@Override
public AtomicIdGeneratorBuilder atomicIdGeneratorBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultAtomicIdGeneratorBuilder(federatedPrimitiveCreator);
+ return new AtomixAtomicIdGeneratorBuilder(atomix, group.name());
}
@Override
public <V> AtomicValueBuilder<V> atomicValueBuilder() {
checkPermission(STORAGE_WRITE);
- Supplier<ConsistentMapBuilder<String, byte[]>> mapBuilderSupplier =
- () -> this.<String, byte[]>consistentMapBuilder()
- .withName("onos-atomic-values")
- .withSerializer(Serializer.using(KryoNamespaces.BASIC));
- return new DefaultAtomicValueBuilder<>(mapBuilderSupplier);
+ return new AtomixAtomicValueBuilder<>(atomix, group.name());
}
@Override
public TransactionContextBuilder transactionContextBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultTransactionContextBuilder(transactionIdGenerator.get(), transactionManager);
+ return new AtomixTransactionContextBuilder(atomix, group.name());
}
@Override
public DistributedLockBuilder lockBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultDistributedLockBuilder(federatedPrimitiveCreator);
+ return new AtomixDistributedLockBuilder(atomix, group.name());
}
@Override
public LeaderElectorBuilder leaderElectorBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultLeaderElectorBuilder(federatedPrimitiveCreator);
+ return new AtomixLeaderElectorBuilder(atomix, group.name(), clusterService.getLocalNode().id());
}
@Override
public <T> TopicBuilder<T> topicBuilder() {
checkPermission(STORAGE_WRITE);
- return new DefaultDistributedTopicBuilder<>(atomicValueBuilder());
+ return new AtomixDistributedTopicBuilder<>(atomix, group.name());
+ }
+
+ @Override
+ public <E> WorkQueueBuilder<E> workQueueBuilder() {
+ checkPermission(STORAGE_WRITE);
+ return new AtomixWorkQueueBuilder<>(atomix, group.name());
}
@Override
public <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer) {
checkPermission(STORAGE_WRITE);
- return federatedPrimitiveCreator.newWorkQueue(name, serializer);
+ return this.<E>workQueueBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .build();
}
@Override
public <V> AsyncDocumentTree<V> getDocumentTree(String name, Serializer serializer) {
checkPermission(STORAGE_WRITE);
- return federatedPrimitiveCreator.newAsyncDocumentTree(name, serializer);
+ return this.<V>documentTreeBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .build();
}
@Override
- public <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(
- String name, Serializer serializer) {
+ public <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(String name, Serializer serializer) {
checkPermission(STORAGE_WRITE);
- return federatedPrimitiveCreator.newAsyncConsistentSetMultimap(name,
- serializer);
+ return new AtomixConsistentMultimapBuilder<K, V>(atomix, group.name())
+ .withName(name)
+ .withSerializer(serializer)
+ .buildMultimap();
}
@Override
- public <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(
- String name, Serializer serializer) {
+ public <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(String name, Serializer serializer) {
checkPermission(STORAGE_WRITE);
- return federatedPrimitiveCreator.newAsyncConsistentTreeMap(name,
- serializer);
+ return this.<V>consistentTreeMapBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .buildTreeMap();
+ }
+
+ @Override
+ public <T> Topic<T> getTopic(String name, Serializer serializer) {
+ checkPermission(STORAGE_WRITE);
+ return this.<T>topicBuilder()
+ .withName(name)
+ .withSerializer(serializer)
+ .build();
}
@Override
public List<MapInfo> getMapInfo() {
- return listMapInfo(federatedPrimitiveCreator);
+ Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
+ return atomix.getPrimitives(AtomicMapType.instance())
+ .stream()
+ .map(info -> {
+ io.atomix.core.map.AtomicMap<String, byte[]> map =
+ atomix.<String, byte[]>atomicMapBuilder(info.name())
+ .withSerializer(new AtomixSerializerAdapter(serializer))
+ .build();
+ int size = map.size();
+ map.close();
+ return new MapInfo(info.name(), size);
+ }).collect(Collectors.toList());
}
@Override
public Map<String, Long> getCounters() {
- Map<String, Long> counters = Maps.newConcurrentMap();
- federatedPrimitiveCreator.getAsyncAtomicCounterNames()
- .forEach(name -> counters.put(name,
- federatedPrimitiveCreator.newAsyncCounter(name).asAtomicCounter().get()));
- return counters;
+ return atomix.getPrimitives(AtomicCounterType.instance())
+ .stream()
+ .map(info -> {
+ AtomicCounter counter = atomix.atomicCounterBuilder(info.name()).build();
+ long value = counter.get();
+ counter.close();
+ return Maps.immutableEntry(info.name(), value);
+ }).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
}
@Override
public Map<String, WorkQueueStats> getQueueStats() {
- Map<String, WorkQueueStats> workQueueStats = Maps.newConcurrentMap();
- federatedPrimitiveCreator.getWorkQueueNames()
- .forEach(name -> workQueueStats.put(name,
- federatedPrimitiveCreator.newWorkQueue(name,
- Serializer.using(KryoNamespaces.BASIC))
- .stats()
- .join()));
- return workQueueStats;
+ Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
+ return atomix.getPrimitives(WorkQueueType.instance())
+ .stream()
+ .map(info -> {
+ io.atomix.core.workqueue.WorkQueue queue = atomix.workQueueBuilder(info.name())
+ .withSerializer(new AtomixSerializerAdapter(serializer))
+ .build();
+ io.atomix.core.workqueue.WorkQueueStats stats = queue.stats();
+ return Maps.immutableEntry(info.name(), WorkQueueStats.builder()
+ .withTotalCompleted(stats.totalCompleted())
+ .withTotalInProgress(stats.totalInProgress())
+ .withTotalPending(stats.totalPending())
+ .build());
+ }).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
}
@Override
@@ -315,27 +348,9 @@
@Override
public Collection<TransactionId> getPendingTransactions() {
- return transactionManager.getPendingTransactions();
- }
-
- private List<MapInfo> listMapInfo(DistributedPrimitiveCreator creator) {
- Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
- return creator.getAsyncConsistentMapNames()
- .stream()
- .map(name -> {
- ConsistentMap<String, byte[]> map =
- creator.<String, byte[]>newAsyncConsistentMap(name, serializer)
- .asConsistentMap();
- return new MapInfo(name, map.size());
- }).collect(Collectors.toList());
- }
-
- @Override
- public <T> Topic<T> getTopic(String name, Serializer serializer) {
- AsyncAtomicValue<T> atomicValue = this.<T>atomicValueBuilder()
- .withName("topic-" + name)
- .withSerializer(serializer)
- .build();
- return new DefaultDistributedTopic<>(atomicValue);
+ return atomix.getTransactionService().getActiveTransactions()
+ .stream()
+ .map(transactionId -> TransactionId.from(transactionId.id()))
+ .collect(Collectors.toList());
}
}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageNamespaces.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageNamespaces.java
deleted file mode 100644
index 4eaac99..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageNamespaces.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-
-import io.atomix.protocols.raft.RaftError;
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.cluster.RaftMember;
-import io.atomix.protocols.raft.cluster.impl.DefaultRaftMember;
-import io.atomix.protocols.raft.event.RaftEvent;
-import io.atomix.protocols.raft.event.impl.DefaultEventType;
-import io.atomix.protocols.raft.operation.OperationType;
-import io.atomix.protocols.raft.operation.RaftOperation;
-import io.atomix.protocols.raft.operation.impl.DefaultOperationId;
-import io.atomix.protocols.raft.protocol.AppendRequest;
-import io.atomix.protocols.raft.protocol.AppendResponse;
-import io.atomix.protocols.raft.protocol.CloseSessionRequest;
-import io.atomix.protocols.raft.protocol.CloseSessionResponse;
-import io.atomix.protocols.raft.protocol.CommandRequest;
-import io.atomix.protocols.raft.protocol.CommandResponse;
-import io.atomix.protocols.raft.protocol.ConfigureRequest;
-import io.atomix.protocols.raft.protocol.ConfigureResponse;
-import io.atomix.protocols.raft.protocol.HeartbeatRequest;
-import io.atomix.protocols.raft.protocol.HeartbeatResponse;
-import io.atomix.protocols.raft.protocol.InstallRequest;
-import io.atomix.protocols.raft.protocol.InstallResponse;
-import io.atomix.protocols.raft.protocol.JoinRequest;
-import io.atomix.protocols.raft.protocol.JoinResponse;
-import io.atomix.protocols.raft.protocol.KeepAliveRequest;
-import io.atomix.protocols.raft.protocol.KeepAliveResponse;
-import io.atomix.protocols.raft.protocol.LeaveRequest;
-import io.atomix.protocols.raft.protocol.LeaveResponse;
-import io.atomix.protocols.raft.protocol.MetadataRequest;
-import io.atomix.protocols.raft.protocol.MetadataResponse;
-import io.atomix.protocols.raft.protocol.OpenSessionRequest;
-import io.atomix.protocols.raft.protocol.OpenSessionResponse;
-import io.atomix.protocols.raft.protocol.PollRequest;
-import io.atomix.protocols.raft.protocol.PollResponse;
-import io.atomix.protocols.raft.protocol.PublishRequest;
-import io.atomix.protocols.raft.protocol.QueryRequest;
-import io.atomix.protocols.raft.protocol.QueryResponse;
-import io.atomix.protocols.raft.protocol.RaftResponse;
-import io.atomix.protocols.raft.protocol.ReconfigureRequest;
-import io.atomix.protocols.raft.protocol.ReconfigureResponse;
-import io.atomix.protocols.raft.protocol.ResetRequest;
-import io.atomix.protocols.raft.protocol.VoteRequest;
-import io.atomix.protocols.raft.protocol.VoteResponse;
-import io.atomix.protocols.raft.service.PropagationStrategy;
-import io.atomix.protocols.raft.session.RaftSessionMetadata;
-import io.atomix.protocols.raft.session.SessionId;
-import io.atomix.protocols.raft.storage.log.entry.CloseSessionEntry;
-import io.atomix.protocols.raft.storage.log.entry.CommandEntry;
-import io.atomix.protocols.raft.storage.log.entry.ConfigurationEntry;
-import io.atomix.protocols.raft.storage.log.entry.InitializeEntry;
-import io.atomix.protocols.raft.storage.log.entry.KeepAliveEntry;
-import io.atomix.protocols.raft.storage.log.entry.MetadataEntry;
-import io.atomix.protocols.raft.storage.log.entry.OpenSessionEntry;
-import io.atomix.protocols.raft.storage.log.entry.QueryEntry;
-import io.atomix.protocols.raft.storage.system.Configuration;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Storage serializer namespaces.
- */
-public final class StorageNamespaces {
-
- /**
- * Raft protocol namespace.
- */
- public static final KryoNamespace RAFT_PROTOCOL = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(OpenSessionRequest.class)
- .register(OpenSessionResponse.class)
- .register(CloseSessionRequest.class)
- .register(CloseSessionResponse.class)
- .register(KeepAliveRequest.class)
- .register(KeepAliveResponse.class)
- .register(HeartbeatRequest.class)
- .register(HeartbeatResponse.class)
- .register(QueryRequest.class)
- .register(QueryResponse.class)
- .register(CommandRequest.class)
- .register(CommandResponse.class)
- .register(MetadataRequest.class)
- .register(MetadataResponse.class)
- .register(JoinRequest.class)
- .register(JoinResponse.class)
- .register(LeaveRequest.class)
- .register(LeaveResponse.class)
- .register(ConfigureRequest.class)
- .register(ConfigureResponse.class)
- .register(ReconfigureRequest.class)
- .register(ReconfigureResponse.class)
- .register(InstallRequest.class)
- .register(InstallResponse.class)
- .register(PollRequest.class)
- .register(PollResponse.class)
- .register(VoteRequest.class)
- .register(VoteResponse.class)
- .register(AppendRequest.class)
- .register(AppendResponse.class)
- .register(PublishRequest.class)
- .register(ResetRequest.class)
- .register(RaftResponse.Status.class)
- .register(RaftError.class)
- .register(RaftError.Type.class)
- .register(RaftOperation.class)
- .register(DefaultOperationId.class)
- .register(OperationType.class)
- .register(RaftEvent.class)
- .register(DefaultEventType.class)
- .register(RaftSessionMetadata.class)
- .register(CloseSessionEntry.class)
- .register(CommandEntry.class)
- .register(ConfigurationEntry.class)
- .register(InitializeEntry.class)
- .register(KeepAliveEntry.class)
- .register(MetadataEntry.class)
- .register(OpenSessionEntry.class)
- .register(QueryEntry.class)
- .register(ReadConsistency.class)
- .register(ArrayList.class)
- .register(LinkedList.class)
- .register(Collections.emptyList().getClass())
- .register(HashSet.class)
- .register(DefaultRaftMember.class)
- .register(MemberId.class)
- .register(SessionId.class)
- .register(RaftMember.Type.class)
- .register(Instant.class)
- .register(Configuration.class)
- .register(PropagationStrategy.class)
- .build("RaftProtocol");
-
- /**
- * Raft storage namespace.
- */
- public static final KryoNamespace RAFT_STORAGE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 100)
- .register(CloseSessionEntry.class)
- .register(CommandEntry.class)
- .register(ConfigurationEntry.class)
- .register(InitializeEntry.class)
- .register(KeepAliveEntry.class)
- .register(MetadataEntry.class)
- .register(OpenSessionEntry.class)
- .register(QueryEntry.class)
- .register(RaftOperation.class)
- .register(DefaultOperationId.class)
- .register(OperationType.class)
- .register(ReadConsistency.class)
- .register(ArrayList.class)
- .register(HashSet.class)
- .register(DefaultRaftMember.class)
- .register(MemberId.class)
- .register(RaftMember.Type.class)
- .register(Instant.class)
- .register(Configuration.class)
- .register(PropagationStrategy.class)
- .build("RaftStorage");
-
- private StorageNamespaces() {
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartition.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartition.java
deleted file mode 100644
index 2a29114..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartition.java
+++ /dev/null
@@ -1,284 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.io.File;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Collections2;
-import com.google.common.collect.ImmutableMap;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.service.RaftService;
-import org.onosproject.cluster.ClusterService;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.cluster.Partition;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.core.Version;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapService;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapService;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapService;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapService;
-import org.onosproject.store.primitives.resources.impl.AtomixCounterService;
-import org.onosproject.store.primitives.resources.impl.AtomixDistributedLockService;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeService;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorService;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueService;
-import org.onosproject.store.service.DistributedPrimitive;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.PartitionInfo;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Storage partition.
- */
-public class StoragePartition implements Managed<StoragePartition> {
-
- static final String PARTITIONS_DIR =
- System.getProperty("karaf.data") + "/db/partitions/";
-
- protected final AtomicBoolean isOpened = new AtomicBoolean(false);
- protected final ClusterCommunicationService clusterCommunicator;
- protected Partition partition;
- protected NodeId localNodeId;
- protected StoragePartitionServer server;
- protected StoragePartitionClient client;
-
- public static final Map<String, Supplier<RaftService>> RAFT_SERVICES =
- ImmutableMap.<String, Supplier<RaftService>>builder()
- .put(DistributedPrimitive.Type.CONSISTENT_MAP.name(), AtomixConsistentMapService::new)
- .put(DistributedPrimitive.Type.CONSISTENT_TREEMAP.name(), AtomixConsistentTreeMapService::new)
- .put(DistributedPrimitive.Type.CONSISTENT_MULTIMAP.name(), AtomixConsistentSetMultimapService::new)
- .put(DistributedPrimitive.Type.COUNTER_MAP.name(), AtomixAtomicCounterMapService::new)
- .put(DistributedPrimitive.Type.COUNTER.name(), AtomixCounterService::new)
- .put(DistributedPrimitive.Type.LEADER_ELECTOR.name(), AtomixLeaderElectorService::new)
- .put(DistributedPrimitive.Type.WORK_QUEUE.name(), AtomixWorkQueueService::new)
- .put(DistributedPrimitive.Type.DOCUMENT_TREE.name(),
- () -> new AtomixDocumentTreeService(Ordering.NATURAL))
- .put(String.format("%s-%s", DistributedPrimitive.Type.DOCUMENT_TREE.name(), Ordering.NATURAL),
- () -> new AtomixDocumentTreeService(Ordering.NATURAL))
- .put(String.format("%s-%s", DistributedPrimitive.Type.DOCUMENT_TREE.name(), Ordering.INSERTION),
- () -> new AtomixDocumentTreeService(Ordering.INSERTION))
- .put(DistributedPrimitive.Type.LOCK.name(), AtomixDistributedLockService::new)
- .build();
-
- public StoragePartition(
- Partition partition,
- ClusterCommunicationService clusterCommunicator,
- ClusterService clusterService) {
- this.partition = partition;
- this.clusterCommunicator = clusterCommunicator;
- this.localNodeId = clusterService.getLocalNode().id();
- }
-
- /**
- * Returns the partition client instance.
- * @return client
- */
- public StoragePartitionClient client() {
- return client;
- }
-
- @Override
- public CompletableFuture<Void> open() {
- if (partition.getMembers().contains(localNodeId)) {
- return openServer()
- .thenCompose(v -> openClient())
- .thenAccept(v -> isOpened.set(true))
- .thenApply(v -> null);
- }
- return openClient()
- .thenAccept(v -> isOpened.set(true))
- .thenApply(v -> null);
- }
-
- @Override
- public CompletableFuture<Void> close() {
- // We do not explicitly close the server and instead let the cluster
- // deal with this as an unclean exit.
- return closeClient();
- }
-
- /**
- * Deletes the partition.
- *
- * @return future to be completed once the partition has been deleted
- */
- public CompletableFuture<Void> delete() {
- return closeServer().thenCompose(v -> closeClient()).thenRun(() -> deleteServer());
- }
-
- /**
- * Returns the partition data folder.
- *
- * @return the partition data folder
- */
- public File getDataFolder() {
- return new File(PARTITIONS_DIR + partition.getId());
- }
-
- /**
- * Returns the partition name.
- *
- * @return the partition name
- */
- public String getName() {
- return partition.getId().toString();
- }
-
- /**
- * Returns the identifier of the {@link Partition partition} associated with this instance.
- *
- * @return partition identifier
- */
- public PartitionId getId() {
- return partition.getId();
- }
-
- /**
- * Returns the partition version.
- *
- * @return the partition version
- */
- public Version getVersion() {
- return partition.getVersion();
- }
-
- /**
- * Returns the identifiers of partition members.
- * @return partition member instance ids
- */
- public Collection<NodeId> getMembers() {
- return partition.getMembers();
- }
-
- /**
- * Returns the {@link MemberId identifiers} of partition members.
- * @return partition member identifiers
- */
- public Collection<MemberId> getMemberIds() {
- return Collections2.transform(getMembers(), n -> MemberId.from(n.id()));
- }
-
- /**
- * Attempts to rejoin the partition.
- * @return future that is completed after the operation is complete
- */
- protected CompletableFuture<Void> openServer() {
- StoragePartitionServer server = new StoragePartitionServer(
- this,
- MemberId.from(localNodeId.id()),
- clusterCommunicator);
- return server.open().thenRun(() -> this.server = server);
- }
-
- /**
- * Attempts to join the partition as a new member.
- * @return future that is completed after the operation is complete
- */
- private CompletableFuture<Void> joinCluster() {
- Set<NodeId> otherMembers = partition.getMembers()
- .stream()
- .filter(nodeId -> !nodeId.equals(localNodeId))
- .collect(Collectors.toSet());
- StoragePartitionServer server = new StoragePartitionServer(this,
- MemberId.from(localNodeId.id()),
- clusterCommunicator);
- return server.join(Collections2.transform(otherMembers, n -> MemberId.from(n.id())))
- .thenRun(() -> this.server = server);
- }
-
- private CompletableFuture<StoragePartitionClient> openClient() {
- client = new StoragePartitionClient(this,
- MemberId.from(localNodeId.id()),
- new RaftClientCommunicator(
- String.format("partition-%s-%s", partition.getId(), partition.getVersion()),
- Serializer.using(StorageNamespaces.RAFT_PROTOCOL),
- clusterCommunicator));
- return client.open().thenApply(v -> client);
- }
-
- /**
- * Closes the partition server if it was previously opened.
- * @return future that is completed when the operation completes
- */
- public CompletableFuture<Void> leaveCluster() {
- return server != null
- ? server.closeAndExit().thenRun(() -> server.delete())
- : CompletableFuture.completedFuture(null);
- }
-
- @Override
- public boolean isOpen() {
- return isOpened.get();
- }
-
- private CompletableFuture<Void> closeServer() {
- if (server != null) {
- return server.close();
- }
- return CompletableFuture.completedFuture(null);
- }
-
- private void deleteServer() {
- if (server != null) {
- server.delete();
- }
- }
-
- private CompletableFuture<Void> closeClient() {
- if (client != null) {
- return client.close();
- }
- return CompletableFuture.completedFuture(null);
- }
-
- /**
- * Returns the partition information if this partition is locally managed i.e.
- * this node is a active member of the partition.
- * @return partition info
- */
- public Optional<PartitionInfo> info() {
- return server != null && server.isOpen() ? Optional.of(server.info()) : Optional.empty();
- }
-
- /**
- * Process updates to partitions and handles joining or leaving a partition.
- * @param newValue new Partition
- */
- void onUpdate(Partition newValue) {
- boolean wasPresent = partition.getMembers().contains(localNodeId);
- boolean isPresent = newValue.getMembers().contains(localNodeId);
- this.partition = newValue;
- if ((wasPresent && isPresent) || (!wasPresent && !isPresent)) {
- // no action needed
- return;
- }
- // Only need to do action if our membership changed
- if (wasPresent) {
- leaveCluster();
- } else {
- joinCluster();
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionClient.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionClient.java
deleted file mode 100644
index 5fa7383..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionClient.java
+++ /dev/null
@@ -1,436 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.time.Duration;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import com.google.common.base.Suppliers;
-import io.atomix.protocols.raft.RaftClient;
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.protocol.RaftClientProtocol;
-import io.atomix.protocols.raft.proxy.CommunicationStrategy;
-import io.atomix.protocols.raft.service.PropagationStrategy;
-import io.atomix.protocols.raft.session.RaftSessionMetadata;
-import org.onlab.util.HexString;
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMap;
-import org.onosproject.store.primitives.resources.impl.AtomixCounter;
-import org.onosproject.store.primitives.resources.impl.AtomixDistributedLock;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTree;
-import org.onosproject.store.primitives.resources.impl.AtomixIdGenerator;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElector;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueue;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.AsyncAtomicIdGenerator;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncDistributedLock;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.AsyncLeaderElector;
-import org.onosproject.store.service.AtomicCounterMapOptions;
-import org.onosproject.store.service.AtomicCounterOptions;
-import org.onosproject.store.service.AtomicIdGeneratorOptions;
-import org.onosproject.store.service.AtomicValueOptions;
-import org.onosproject.store.service.ConsistentMapOptions;
-import org.onosproject.store.service.ConsistentMultimapOptions;
-import org.onosproject.store.service.ConsistentTreeMapOptions;
-import org.onosproject.store.service.DistributedLockOptions;
-import org.onosproject.store.service.DistributedPrimitive;
-import org.onosproject.store.service.DistributedSetOptions;
-import org.onosproject.store.service.DocumentTreeOptions;
-import org.onosproject.store.service.LeaderElectorOptions;
-import org.onosproject.store.service.PartitionClientInfo;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.WorkQueue;
-import org.onosproject.store.service.WorkQueueOptions;
-import org.slf4j.Logger;
-
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * StoragePartition client.
- */
-public class StoragePartitionClient implements DistributedPrimitiveCreator, Managed<StoragePartitionClient> {
-
- private static final int MAX_RETRIES = 8;
- private static final String ATOMIC_VALUES_CONSISTENT_MAP_NAME = "onos-atomic-values";
-
- private static final String MIN_TIMEOUT_PROPERTY = "onos.cluster.raft.client.minTimeoutMillis";
- private static final String MAX_TIMEOUT_PROPERTY = "onos.cluster.raft.client.maxTimeoutMillis";
-
- private static final Duration MIN_TIMEOUT;
- private static final Duration MAX_TIMEOUT;
-
- private static final long DEFAULT_MIN_TIMEOUT_MILLIS = 5000;
- private static final long DEFAULT_MAX_TIMEOUT_MILLIS = 30000;
-
- static {
- Duration minTimeout;
- try {
- minTimeout = Duration.ofMillis(Long.parseLong(
- System.getProperty(MIN_TIMEOUT_PROPERTY,
- String.valueOf(DEFAULT_MIN_TIMEOUT_MILLIS))));
- } catch (NumberFormatException e) {
- minTimeout = Duration.ofMillis(DEFAULT_MIN_TIMEOUT_MILLIS);
- }
- MIN_TIMEOUT = minTimeout;
-
- Duration maxTimeout;
- try {
- maxTimeout = Duration.ofMillis(Long.parseLong(
- System.getProperty(MAX_TIMEOUT_PROPERTY,
- String.valueOf(DEFAULT_MAX_TIMEOUT_MILLIS))));
- } catch (NumberFormatException e) {
- maxTimeout = Duration.ofMillis(DEFAULT_MAX_TIMEOUT_MILLIS);
- }
- MAX_TIMEOUT = maxTimeout;
- }
-
- private final Logger log = getLogger(getClass());
-
- private final StoragePartition partition;
- private final MemberId localMemberId;
- private final RaftClientProtocol protocol;
- private RaftClient client;
- private final com.google.common.base.Supplier<AsyncConsistentMap<String, byte[]>> onosAtomicValuesMap =
- Suppliers.memoize(() -> newAsyncConsistentMap(ATOMIC_VALUES_CONSISTENT_MAP_NAME,
- Serializer.using(KryoNamespaces.BASIC)));
-
- public StoragePartitionClient(StoragePartition partition, MemberId localMemberId, RaftClientProtocol protocol) {
- this.partition = partition;
- this.localMemberId = localMemberId;
- this.protocol = protocol;
- }
-
- @Override
- public CompletableFuture<Void> open() {
- synchronized (StoragePartitionClient.this) {
- client = newRaftClient(protocol);
- }
- return client.connect(partition.getMemberIds()).whenComplete((r, e) -> {
- if (e == null) {
- log.info("Successfully started client for partition {}", partition.getId());
- } else {
- log.info("Failed to start client for partition {}", partition.getId(), e);
- }
- }).thenApply(v -> null);
- }
-
- @Override
- public CompletableFuture<Void> close() {
- return client != null ? client.close() : CompletableFuture.completedFuture(null);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <K, V> AsyncConsistentMap<K, V> newAsyncConsistentMap(ConsistentMapOptions options) {
- AtomixConsistentMap rawMap =
- new AtomixConsistentMap(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(DistributedPrimitive.Type.CONSISTENT_MAP.name())
- .withReadConsistency(ReadConsistency.SEQUENTIAL)
- .withCommunicationStrategy(CommunicationStrategy.ANY)
- .withMinTimeout(MIN_TIMEOUT)
- .withMaxTimeout(MAX_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
-
- if (options.serializer() != null) {
- return DistributedPrimitives.newTranscodingMap(rawMap,
- key -> HexString.toHexString(options.serializer().encode(key)),
- string -> options.serializer().decode(HexString.fromHexString(string)),
- value -> value == null ? null : options.serializer().encode(value),
- bytes -> options.serializer().decode(bytes));
- }
- return (AsyncConsistentMap<K, V>) rawMap;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <V> AsyncConsistentTreeMap<V> newAsyncConsistentTreeMap(ConsistentTreeMapOptions options) {
- AtomixConsistentTreeMap rawMap =
- new AtomixConsistentTreeMap(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(DistributedPrimitive.Type.CONSISTENT_TREEMAP.name())
- .withReadConsistency(ReadConsistency.SEQUENTIAL)
- .withCommunicationStrategy(CommunicationStrategy.ANY)
- .withMinTimeout(MIN_TIMEOUT)
- .withMaxTimeout(MAX_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
-
- if (options.serializer() != null) {
- return DistributedPrimitives.newTranscodingTreeMap(
- rawMap,
- value -> value == null ? null : options.serializer().encode(value),
- bytes -> options.serializer().decode(bytes));
- }
- return (AsyncConsistentTreeMap<V>) rawMap;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <K, V> AsyncConsistentMultimap<K, V> newAsyncConsistentSetMultimap(ConsistentMultimapOptions options) {
- AtomixConsistentSetMultimap rawMap =
- new AtomixConsistentSetMultimap(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(DistributedPrimitive.Type.CONSISTENT_MULTIMAP.name())
- .withReadConsistency(ReadConsistency.SEQUENTIAL)
- .withCommunicationStrategy(CommunicationStrategy.ANY)
- .withMinTimeout(MIN_TIMEOUT)
- .withMaxTimeout(MAX_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
-
- if (options.serializer() != null) {
- return DistributedPrimitives.newTranscodingMultimap(
- rawMap,
- key -> HexString.toHexString(options.serializer().encode(key)),
- string -> options.serializer().decode(HexString.fromHexString(string)),
- value -> options.serializer().encode(value),
- bytes -> options.serializer().decode(bytes));
- }
- return (AsyncConsistentMultimap<K, V>) rawMap;
- }
-
- @Override
- public <E> AsyncDistributedSet<E> newAsyncDistributedSet(DistributedSetOptions options) {
- return DistributedPrimitives.newSetFromMap(newAsyncConsistentMap(options.name(), options.serializer()));
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <K> AsyncAtomicCounterMap<K> newAsyncAtomicCounterMap(AtomicCounterMapOptions options) {
- AtomixAtomicCounterMap rawMap = new AtomixAtomicCounterMap(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(DistributedPrimitive.Type.COUNTER_MAP.name())
- .withReadConsistency(ReadConsistency.LINEARIZABLE_LEASE)
- .withCommunicationStrategy(CommunicationStrategy.LEADER)
- .withMinTimeout(MIN_TIMEOUT)
- .withMaxTimeout(MAX_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
-
- if (options.serializer() != null) {
- return DistributedPrimitives.newTranscodingAtomicCounterMap(
- rawMap,
- key -> HexString.toHexString(options.serializer().encode(key)),
- string -> options.serializer().decode(HexString.fromHexString(string)));
- }
- return (AsyncAtomicCounterMap<K>) rawMap;
- }
-
- @Override
- public AsyncAtomicCounter newAsyncCounter(AtomicCounterOptions options) {
- return new AtomixCounter(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(DistributedPrimitive.Type.COUNTER.name())
- .withReadConsistency(ReadConsistency.LINEARIZABLE_LEASE)
- .withCommunicationStrategy(CommunicationStrategy.LEADER)
- .withMinTimeout(MIN_TIMEOUT)
- .withMaxTimeout(MAX_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
- }
-
- @Override
- public AsyncAtomicIdGenerator newAsyncIdGenerator(AtomicIdGeneratorOptions options) {
- return new AtomixIdGenerator(newAsyncCounter(options.name()));
- }
-
- @Override
- public <V> AsyncAtomicValue<V> newAsyncAtomicValue(AtomicValueOptions options) {
- return new DefaultAsyncAtomicValue<>(options.name(), options.serializer(), onosAtomicValuesMap.get());
- }
-
- @Override
- public <E> WorkQueue<E> newWorkQueue(WorkQueueOptions options) {
- AtomixWorkQueue atomixWorkQueue = new AtomixWorkQueue(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(DistributedPrimitive.Type.WORK_QUEUE.name())
- .withReadConsistency(ReadConsistency.LINEARIZABLE_LEASE)
- .withCommunicationStrategy(CommunicationStrategy.LEADER)
- .withMinTimeout(MIN_TIMEOUT)
- .withMaxTimeout(MAX_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
- return new DefaultDistributedWorkQueue<>(atomixWorkQueue, options.serializer());
- }
-
- @Override
- public <V> AsyncDocumentTree<V> newAsyncDocumentTree(DocumentTreeOptions options) {
- String serviceType = String.format("%s-%s", DistributedPrimitive.Type.DOCUMENT_TREE.name(), options.ordering());
- AtomixDocumentTree atomixDocumentTree = new AtomixDocumentTree(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(serviceType)
- .withReadConsistency(ReadConsistency.SEQUENTIAL)
- .withCommunicationStrategy(CommunicationStrategy.ANY)
- .withMinTimeout(MIN_TIMEOUT)
- .withMaxTimeout(MAX_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
- return new DefaultDistributedDocumentTree<>(options.name(), atomixDocumentTree, options.serializer());
- }
-
- @Override
- public AsyncDistributedLock newAsyncDistributedLock(DistributedLockOptions options) {
- return new AtomixDistributedLock(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(DistributedPrimitive.Type.LOCK.name())
- .withReadConsistency(ReadConsistency.LINEARIZABLE)
- .withCommunicationStrategy(CommunicationStrategy.LEADER)
- .withMinTimeout(MIN_TIMEOUT)
- .withMaxTimeout(MIN_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
- }
-
- @Override
- public AsyncLeaderElector newAsyncLeaderElector(LeaderElectorOptions options) {
- return new AtomixLeaderElector(client.newProxyBuilder()
- .withName(options.name())
- .withServiceType(DistributedPrimitive.Type.LEADER_ELECTOR.name())
- .withReadConsistency(ReadConsistency.LINEARIZABLE)
- .withCommunicationStrategy(CommunicationStrategy.LEADER)
- .withMinTimeout(Duration.ofMillis(options.electionTimeoutMillis()))
- .withMaxTimeout(MIN_TIMEOUT)
- .withMaxRetries(MAX_RETRIES)
- .withRevision(options.version() != null && options.revisionType() != null
- ? options.version().toInt() : 1)
- .withPropagationStrategy(options.revisionType() != null
- ? PropagationStrategy.valueOf(options.revisionType().name())
- : PropagationStrategy.NONE)
- .build()
- .open()
- .join());
- }
-
- @Override
- public Set<String> getAsyncConsistentMapNames() {
- return client.metadata().getSessions(DistributedPrimitive.Type.CONSISTENT_MAP.name())
- .join()
- .stream()
- .map(RaftSessionMetadata::serviceName)
- .collect(Collectors.toSet());
- }
-
- @Override
- public Set<String> getAsyncAtomicCounterNames() {
- return client.metadata().getSessions(DistributedPrimitive.Type.COUNTER.name())
- .join()
- .stream()
- .map(RaftSessionMetadata::serviceName)
- .collect(Collectors.toSet());
- }
-
- @Override
- public Set<String> getWorkQueueNames() {
- return client.metadata().getSessions(DistributedPrimitive.Type.WORK_QUEUE.name())
- .join()
- .stream()
- .map(RaftSessionMetadata::serviceName)
- .collect(Collectors.toSet());
- }
-
- @Override
- public boolean isOpen() {
- return client != null;
- }
-
- /**
- * Returns the {@link PartitionClientInfo information} for this client.
- * @return partition client information
- */
- public PartitionClientInfo clientInfo() {
- return new PartitionClientInfo(partition.getId(), partition.getMembers());
- }
-
- private RaftClient newRaftClient(RaftClientProtocol protocol) {
- return RaftClient.newBuilder()
- .withClientId("partition-" + partition.getId())
- .withMemberId(localMemberId)
- .withProtocol(protocol)
- .build();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionDetails.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionDetails.java
deleted file mode 100644
index 1bed89d..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionDetails.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Set;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import io.atomix.protocols.raft.cluster.RaftMember;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.service.PartitionInfo;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableSet;
-
-/**
- * Operational details for a {@code StoragePartition}.
- */
-public class StoragePartitionDetails {
-
- private final PartitionId partitionId;
- private final Set<RaftMember> activeMembers;
- private final Set<RaftMember> configuredMembers;
- private final RaftMember leader;
- private final long leaderTerm;
-
- public StoragePartitionDetails(PartitionId partitionId,
- Collection<RaftMember> activeMembers,
- Collection<RaftMember> configuredMembers,
- RaftMember leader,
- long leaderTerm) {
- this.partitionId = partitionId;
- this.activeMembers = ImmutableSet.copyOf(activeMembers);
- this.configuredMembers = ImmutableSet.copyOf(configuredMembers);
- this.leader = leader;
- this.leaderTerm = leaderTerm;
- }
-
- /**
- * Returns the set of active members.
- * @return active members
- */
- public Set<RaftMember> activeMembers() {
- return activeMembers;
- }
-
- /**
- * Returns the set of configured members.
- * @return configured members
- */
- public Set<RaftMember> configuredMembers() {
- return configuredMembers;
- }
-
- /**
- * Returns the partition leader.
- * @return leader
- */
- public RaftMember leader() {
- return leader;
- }
-
- /**
- * Returns the partition leader term.
- * @return leader term
- */
- public long leaderTerm() {
- return leaderTerm;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("activeMembers", activeMembers)
- .add("configuredMembers", configuredMembers)
- .add("leader", leader)
- .add("leaderTerm", leaderTerm)
- .toString();
- }
-
- /**
- * Returns the details as an instance of {@code PartitionInfo}.
- * @return partition info
- */
- public PartitionInfo toPartitionInfo() {
- Function<RaftMember, String> memberToString =
- m -> m == null ? "none" : m.memberId().toString();
- return new PartitionInfo(partitionId,
- leaderTerm,
- activeMembers.stream().map(memberToString).collect(Collectors.toList()),
- memberToString.apply(leader));
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionServer.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionServer.java
deleted file mode 100644
index f2600e8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionServer.java
+++ /dev/null
@@ -1,340 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.FileVisitResult;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.SimpleFileVisitor;
-import java.nio.file.attribute.BasicFileAttributes;
-import java.time.Duration;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import io.atomix.protocols.raft.RaftServer;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.storage.StorageLevel;
-import org.onosproject.cluster.Partition;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.primitives.resources.impl.AtomixSerializerAdapter;
-import org.onosproject.store.service.PartitionInfo;
-import org.onosproject.store.service.Serializer;
-import org.slf4j.Logger;
-
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * {@link StoragePartition} server.
- */
-public class StoragePartitionServer implements Managed<StoragePartitionServer> {
-
- private final Logger log = getLogger(getClass());
-
- private static final String ELECTION_TIMEOUT_MILLIS_PROPERTY = "onos.cluster.raft.electionTimeoutMillis";
- private static final String ELECTION_THRESHOLD_PROPERTY = "onos.cluster.raft.electionFailureThreshold";
- private static final String SESSION_THRESHOLD_PROPERTY = "onos.cluster.raft.sessionFailureThreshold";
- private static final String HEARTBEAT_INTERVAL_MILLIS_PROPERTY = "onos.cluster.raft.heartbeatIntervalMillis";
- private static final String MAX_SEGMENT_SIZE_PROPERTY = "onos.cluster.raft.storage.maxSegmentSize";
- private static final String STORAGE_LEVEL_PROPERTY = "onos.cluster.raft.storage.level";
- private static final String FLUSH_ON_COMMIT_PROPERTY = "onos.cluster.raft.storage.flushOnCommit";
-
- private static final long ELECTION_TIMEOUT_MILLIS;
- private static final int ELECTION_THRESHOLD;
- private static final int SESSION_THRESHOLD;
- private static final long HEARTBEAT_INTERVAL_MILLIS;
- private static final int MAX_SEGMENT_SIZE;
- private static final StorageLevel STORAGE_LEVEL;
- private static final boolean FLUSH_ON_COMMIT;
-
- private static final int DEFAULT_MAX_SEGMENT_SIZE = 1024 * 1024 * 64;
- private static final long DEFAULT_ELECTION_TIMEOUT_MILLIS = 2500;
- private static final int DEFAULT_ELECTION_THRESHOLD = 12;
- private static final int DEFAULT_SESSION_THRESHOLD = 10;
- private static final long DEFAULT_HEARTBEAT_INTERVAL_MILLIS = 500;
- private static final StorageLevel DEFAULT_STORAGE_LEVEL = StorageLevel.MAPPED;
- private static final boolean DEFAULT_FLUSH_ON_COMMIT = false;
-
- static {
- int maxSegmentSize;
- try {
- maxSegmentSize = Integer.parseInt(System.getProperty(
- MAX_SEGMENT_SIZE_PROPERTY,
- String.valueOf(DEFAULT_MAX_SEGMENT_SIZE)));
- } catch (NumberFormatException e) {
- maxSegmentSize = DEFAULT_MAX_SEGMENT_SIZE;
- }
- MAX_SEGMENT_SIZE = maxSegmentSize;
-
- long electionTimeoutMillis;
- try {
- electionTimeoutMillis = Long.parseLong(System.getProperty(
- ELECTION_TIMEOUT_MILLIS_PROPERTY,
- String.valueOf(DEFAULT_ELECTION_TIMEOUT_MILLIS)));
- } catch (NumberFormatException e) {
- electionTimeoutMillis = DEFAULT_ELECTION_TIMEOUT_MILLIS;
- }
- ELECTION_TIMEOUT_MILLIS = electionTimeoutMillis;
-
- int electionFailureThreshold;
- try {
- electionFailureThreshold = Integer.parseInt(System.getProperty(
- ELECTION_THRESHOLD_PROPERTY,
- String.valueOf(DEFAULT_ELECTION_THRESHOLD)));
- } catch (NumberFormatException e) {
- electionFailureThreshold = DEFAULT_ELECTION_THRESHOLD;
- }
- ELECTION_THRESHOLD = electionFailureThreshold;
-
- int sessionFailureThreshold;
- try {
- sessionFailureThreshold = Integer.parseInt(System.getProperty(
- SESSION_THRESHOLD_PROPERTY,
- String.valueOf(DEFAULT_SESSION_THRESHOLD)));
- } catch (NumberFormatException e) {
- sessionFailureThreshold = DEFAULT_SESSION_THRESHOLD;
- }
- SESSION_THRESHOLD = sessionFailureThreshold;
-
- long heartbeatIntervalMillis;
- try {
- heartbeatIntervalMillis = Long.parseLong(System.getProperty(
- HEARTBEAT_INTERVAL_MILLIS_PROPERTY,
- String.valueOf(DEFAULT_HEARTBEAT_INTERVAL_MILLIS)));
- } catch (NumberFormatException e) {
- heartbeatIntervalMillis = DEFAULT_HEARTBEAT_INTERVAL_MILLIS;
- }
- HEARTBEAT_INTERVAL_MILLIS = heartbeatIntervalMillis;
-
- StorageLevel storageLevel;
- try {
- storageLevel = StorageLevel.valueOf(System.getProperty(
- STORAGE_LEVEL_PROPERTY,
- DEFAULT_STORAGE_LEVEL.name()).toUpperCase());
- } catch (IllegalArgumentException e) {
- storageLevel = DEFAULT_STORAGE_LEVEL;
- }
- STORAGE_LEVEL = storageLevel;
-
- boolean flushOnCommit;
- try {
- flushOnCommit = Boolean.parseBoolean(System.getProperty(
- FLUSH_ON_COMMIT_PROPERTY,
- String.valueOf(DEFAULT_FLUSH_ON_COMMIT)));
- } catch (Exception e) {
- flushOnCommit = DEFAULT_FLUSH_ON_COMMIT;
- }
- FLUSH_ON_COMMIT = flushOnCommit;
- }
-
- private final MemberId localMemberId;
- private final StoragePartition partition;
- private final ClusterCommunicationService clusterCommunicator;
- private RaftServer server;
-
- public StoragePartitionServer(
- StoragePartition partition,
- MemberId localMemberId,
- ClusterCommunicationService clusterCommunicator) {
- this.partition = partition;
- this.localMemberId = localMemberId;
- this.clusterCommunicator = clusterCommunicator;
- }
-
- @Override
- public CompletableFuture<Void> open() {
- log.info("Starting server for partition {} ({})", partition.getId(), partition.getVersion());
- CompletableFuture<RaftServer> serverOpenFuture;
- if (partition.getMemberIds().contains(localMemberId)) {
- if (server != null && server.isRunning()) {
- return CompletableFuture.completedFuture(null);
- }
- synchronized (this) {
- server = buildServer();
- }
- serverOpenFuture = server.bootstrap(partition.getMemberIds());
- } else {
- serverOpenFuture = CompletableFuture.completedFuture(null);
- }
- return serverOpenFuture.whenComplete((r, e) -> {
- if (e == null) {
- log.info("Successfully started server for partition {} ({})",
- partition.getId(), partition.getVersion());
- } else {
- log.info("Failed to start server for partition {} ({})",
- partition.getId(), partition.getVersion(), e);
- }
- }).thenApply(v -> null);
- }
-
- @Override
- public CompletableFuture<Void> close() {
- return server.shutdown();
- }
-
- /**
- * Closes the server and exits the partition.
- * @return future that is completed when the operation is complete
- */
- public CompletableFuture<Void> closeAndExit() {
- return server.leave();
- }
-
- /**
- * Deletes the server.
- */
- public void delete() {
- try {
- Files.walkFileTree(partition.getDataFolder().toPath(), new SimpleFileVisitor<Path>() {
- @Override
- public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
- Files.delete(file);
- return FileVisitResult.CONTINUE;
- }
- @Override
- public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException {
- Files.delete(dir);
- return FileVisitResult.CONTINUE;
- }
- });
- } catch (IOException e) {
- log.error("Failed to delete partition: {}", e);
- }
- }
-
- /**
- * Forks the existing partition into a new partition.
- *
- * @param fromPartition the partition from which to fork the server
- * @return future to be completed once the fork operation is complete
- */
- public CompletableFuture<Void> fork(Partition fromPartition) {
- log.info("Forking server for partition {} ({}->{})",
- partition.getId(), fromPartition.getVersion(), partition.getVersion());
- RaftServer.Builder builder = RaftServer.newBuilder(localMemberId)
- .withName(String.format("partition-%s", fromPartition.getId()))
- .withProtocol(new RaftServerCommunicator(
- String.format("partition-%s-%s", fromPartition.getId(), fromPartition.getVersion()),
- Serializer.using(StorageNamespaces.RAFT_PROTOCOL),
- clusterCommunicator))
- .withElectionTimeout(Duration.ofMillis(ELECTION_TIMEOUT_MILLIS))
- .withHeartbeatInterval(Duration.ofMillis(HEARTBEAT_INTERVAL_MILLIS))
- .withElectionThreshold(ELECTION_THRESHOLD)
- .withSessionFailureThreshold(SESSION_THRESHOLD)
- .withStorage(RaftStorage.newBuilder()
- .withPrefix(String.format("partition-%s", partition.getId()))
- .withStorageLevel(STORAGE_LEVEL)
- .withFlushOnCommit(FLUSH_ON_COMMIT)
- .withSerializer(new AtomixSerializerAdapter(Serializer.using(StorageNamespaces.RAFT_STORAGE)))
- .withDirectory(partition.getDataFolder())
- .withMaxSegmentSize(MAX_SEGMENT_SIZE)
- .build());
- StoragePartition.RAFT_SERVICES.forEach(builder::addService);
- RaftServer server = builder.build();
-
- // Create a collection of members currently in the source partition.
- Collection<MemberId> members = fromPartition.getMembers()
- .stream()
- .map(id -> MemberId.from(id.id()))
- .collect(Collectors.toList());
-
- // If this node is a member of the partition, join the partition. Otherwise, listen to the partition.
- CompletableFuture<RaftServer> future = members.contains(localMemberId)
- ? server.bootstrap(members) : server.listen(members);
-
- // TODO: We should leave the cluster for nodes that aren't normally members to ensure the source
- // cluster's configuration is kept consistent for rolling back upgrades, but Atomix deletes configuration
- // files when a node leaves the cluster so we can't do that here.
- return future.thenCompose(v -> server.shutdown())
- .thenCompose(v -> {
- // Delete the cluster configuration file from the forked partition.
- try {
- Files.delete(new File(
- partition.getDataFolder(),
- String.format("partition-%s.conf", partition.getId())).toPath());
- } catch (IOException e) {
- log.error("Failed to delete partition configuration: {}", e);
- }
-
- // Build and bootstrap a new server.
- this.server = buildServer();
- return this.server.bootstrap();
- }).whenComplete((r, e) -> {
- if (e == null) {
- log.info("Successfully forked server for partition {} ({}->{})",
- partition.getId(), fromPartition.getVersion(), partition.getVersion());
- } else {
- log.info("Failed to fork server for partition {} ({}->{})",
- partition.getId(), fromPartition.getVersion(), partition.getVersion(), e);
- }
- }).thenApply(v -> null);
- }
-
- private RaftServer buildServer() {
- RaftServer.Builder builder = RaftServer.newBuilder(localMemberId)
- .withName(String.format("partition-%s", partition.getId()))
- .withProtocol(new RaftServerCommunicator(
- String.format("partition-%s-%s", partition.getId(), partition.getVersion()),
- Serializer.using(StorageNamespaces.RAFT_PROTOCOL),
- clusterCommunicator))
- .withElectionTimeout(Duration.ofMillis(ELECTION_TIMEOUT_MILLIS))
- .withHeartbeatInterval(Duration.ofMillis(HEARTBEAT_INTERVAL_MILLIS))
- .withElectionThreshold(ELECTION_THRESHOLD)
- .withSessionFailureThreshold(SESSION_THRESHOLD)
- .withStorage(RaftStorage.newBuilder()
- .withPrefix(String.format("partition-%s", partition.getId()))
- .withStorageLevel(STORAGE_LEVEL)
- .withFlushOnCommit(FLUSH_ON_COMMIT)
- .withSerializer(new AtomixSerializerAdapter(Serializer.using(StorageNamespaces.RAFT_STORAGE)))
- .withDirectory(partition.getDataFolder())
- .withMaxSegmentSize(MAX_SEGMENT_SIZE)
- .build());
- StoragePartition.RAFT_SERVICES.forEach(builder::addService);
- return builder.build();
- }
-
- public CompletableFuture<Void> join(Collection<MemberId> otherMembers) {
- log.info("Joining partition {} ({})", partition.getId(), partition.getName());
- server = buildServer();
- return server.join(otherMembers).whenComplete((r, e) -> {
- if (e == null) {
- log.info("Successfully joined partition {} ({})", partition.getId(), partition.getName());
- } else {
- log.info("Failed to join partition {} ({})", partition.getId(), partition.getName(), e);
- }
- }).thenApply(v -> null);
- }
-
- @Override
- public boolean isOpen() {
- return server.isRunning();
- }
-
- /**
- * Returns the partition information.
- * @return partition info
- */
- public PartitionInfo info() {
- return new StoragePartitionDetails(partition.getId(),
- server.cluster().getMembers(),
- server.cluster().getMembers(),
- server.cluster().getLeader(),
- server.cluster().getTerm()).toPartitionInfo();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Transaction.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Transaction.java
deleted file mode 100644
index 28074e2..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Transaction.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Transactional;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.TransactionContext;
-import org.onosproject.store.service.TransactionException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkState;
-
-/**
- * Manages a transaction within the context of a single primitive.
- * <p>
- * The {@code Transaction} object is used to manage the transaction for a single partition primitive that implements
- * the {@link Transactional} interface. It's used as a proxy for {@link TransactionContext}s to manage the transaction
- * as it relates to a single piece of atomic state.
- */
-public class Transaction<T> {
-
- /**
- * Transaction state.
- * <p>
- * The transaction state is used to indicate the phase within which the transaction is currently running.
- */
- enum State {
-
- /**
- * Active transaction state.
- * <p>
- * The {@code ACTIVE} state represents a transaction in progress. Active transactions may or may not affect
- * concurrently running transactions depending on the transaction's isolation level.
- */
- ACTIVE,
-
- /**
- * Preparing transaction state.
- * <p>
- * Once a transaction commitment begins, it enters the {@code PREPARING} phase of the two-phase commit protocol.
- */
- PREPARING,
-
- /**
- * Prepared transaction state.
- * <p>
- * Once the first phase of the two-phase commit protocol is complete, the transaction's state is set to
- * {@code PREPARED}.
- */
- PREPARED,
-
- /**
- * Committing transaction state.
- * <p>
- * The {@code COMMITTING} state represents a transaction within the second phase of the two-phase commit
- * protocol.
- */
- COMMITTING,
-
- /**
- * Committed transaction state.
- * <p>
- * Once the second phase of the two-phase commit protocol is complete, the transaction's state is set to
- * {@code COMMITTED}.
- */
- COMMITTED,
-
- /**
- * Rolling back transaction state.
- * <p>
- * In the event of a two-phase lock failure, when the transaction is rolled back it will enter the
- * {@code ROLLING_BACK} state while the rollback is in progress.
- */
- ROLLING_BACK,
-
- /**
- * Rolled back transaction state.
- * <p>
- * Once a transaction has been rolled back, it will enter the {@code ROLLED_BACK} state.
- */
- ROLLED_BACK,
- }
-
- private static final String TX_OPEN_ERROR = "transaction already open";
- private static final String TX_CLOSED_ERROR = "transaction not open";
- private static final String TX_INACTIVE_ERROR = "transaction is not active";
- private static final String TX_UNPREPARED_ERROR = "transaction has not been prepared";
-
- protected final Logger log = LoggerFactory.getLogger(getClass());
- protected final TransactionId transactionId;
- protected final Transactional<T> transactionalObject;
- private final AtomicBoolean open = new AtomicBoolean();
- private volatile State state = State.ACTIVE;
- private volatile Version lock;
-
- public Transaction(TransactionId transactionId, Transactional<T> transactionalObject) {
- this.transactionId = transactionId;
- this.transactionalObject = transactionalObject;
- }
-
- /**
- * Returns the transaction identifier.
- *
- * @return the transaction identifier
- */
- public TransactionId transactionId() {
- return transactionId;
- }
-
- /**
- * Returns the current transaction state.
- *
- * @return the current transaction state
- */
- public State state() {
- return state;
- }
-
- /**
- * Returns a boolean indicating whether the transaction is open.
- *
- * @return indicates whether the transaction is open
- */
- public boolean isOpen() {
- return open.get();
- }
-
- /**
- * Opens the transaction, throwing an {@link IllegalStateException} if it's already open.
- */
- protected void open() {
- if (!open.compareAndSet(false, true)) {
- throw new IllegalStateException(TX_OPEN_ERROR);
- }
- }
-
- /**
- * Checks that the transaction is open and throws an {@link IllegalStateException} if not.
- */
- protected void checkOpen() {
- checkState(isOpen(), TX_CLOSED_ERROR);
- }
-
- /**
- * Checks that the transaction state is {@code ACTIVE} and throws an {@link IllegalStateException} if not.
- */
- protected void checkActive() {
- checkState(state == State.ACTIVE, TX_INACTIVE_ERROR);
- }
-
- /**
- * Checks that the transaction state is {@code PREPARED} and throws an {@link IllegalStateException} if not.
- */
- protected void checkPrepared() {
- checkState(state == State.PREPARED, TX_UNPREPARED_ERROR);
- }
-
- /**
- * Updates the transaction state.
- *
- * @param state the updated transaction state
- */
- protected void setState(State state) {
- this.state = state;
- }
-
- /**
- * Begins the transaction.
- * <p>
- * Locks are acquired when the transaction is begun to prevent concurrent transactions from operating on the shared
- * resource to which this transaction relates.
- *
- * @return a completable future to be completed once the transaction has been started
- */
- public CompletableFuture<Version> begin() {
- log.debug("Beginning transaction {} for {}", transactionId, transactionalObject);
- open();
- return transactionalObject.begin(transactionId).thenApply(lock -> {
- this.lock = lock;
- log.trace("Transaction lock acquired: {}", lock);
- return lock;
- });
- }
-
- /**
- * Prepares the transaction.
- * <p>
- * When preparing the transaction, the given list of updates for the shared resource will be prepared, and
- * concurrent modification checks will be performed. The returned future may be completed with a
- * {@link TransactionException} if a concurrent modification is detected for an isolation level that does
- * not allow such modifications.
- *
- * @param updates the transaction updates
- * @return a completable future to be completed once the transaction has been prepared
- */
- public CompletableFuture<Boolean> prepare(List<T> updates) {
- checkOpen();
- checkActive();
- log.debug("Preparing transaction {} for {}", transactionId, transactionalObject);
- Version lock = this.lock;
- checkState(lock != null, TX_INACTIVE_ERROR);
- setState(State.PREPARING);
- return transactionalObject.prepare(new TransactionLog<T>(transactionId, lock.value(), updates))
- .thenApply(succeeded -> {
- setState(State.PREPARED);
- return succeeded;
- });
- }
-
- /**
- * Prepares and commits the transaction in a single atomic operation.
- * <p>
- * Both the prepare and commit phases of the protocol must be executed within a single atomic operation. This method
- * is used to optimize committing transactions that operate only on a single partition within a single primitive.
- *
- * @param updates the transaction updates
- * @return a completable future to be completed once the transaction has been prepared
- */
- public CompletableFuture<Boolean> prepareAndCommit(List<T> updates) {
- checkOpen();
- checkActive();
- log.debug("Preparing and committing transaction {} for {}", transactionId, transactionalObject);
- Version lock = this.lock;
- checkState(lock != null, TX_INACTIVE_ERROR);
- setState(State.PREPARING);
- return transactionalObject.prepareAndCommit(new TransactionLog<T>(transactionId, lock.value(), updates))
- .thenApply(succeeded -> {
- setState(State.COMMITTED);
- return succeeded;
- });
- }
-
- /**
- * Commits the transaction.
- * <p>
- * Performs the second phase of the two-phase commit protocol, committing the previously
- * {@link #prepare(List) prepared} updates.
- *
- * @return a completable future to be completed once the transaction has been committed
- */
- public CompletableFuture<Void> commit() {
- checkOpen();
- checkPrepared();
- log.debug("Committing transaction {} for {}", transactionId, transactionalObject);
- setState(State.COMMITTING);
- return transactionalObject.commit(transactionId).thenRun(() -> {
- setState(State.COMMITTED);
- });
- }
-
- /**
- * Rolls back the transaction.
- * <p>
- * Rolls back the first phase of the two-phase commit protocol, cancelling prepared updates.
- *
- * @return a completable future to be completed once the transaction has been rolled back
- */
- public CompletableFuture<Void> rollback() {
- checkOpen();
- checkPrepared();
- log.debug("Rolling back transaction {} for {}", transactionId, transactionalObject);
- setState(State.ROLLING_BACK);
- return transactionalObject.rollback(transactionId).thenRun(() -> {
- setState(State.ROLLED_BACK);
- });
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("transactionId", transactionId)
- .add("state", state)
- .toString();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionCoordinator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionCoordinator.java
deleted file mode 100644
index 23d4399..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionCoordinator.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Sets;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.CommitStatus;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionalMap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-/**
- * Transaction coordinator.
- */
-public class TransactionCoordinator {
- private final Logger log = LoggerFactory.getLogger(getClass());
- protected final TransactionId transactionId;
- protected final TransactionManager transactionManager;
- protected final Set<TransactionParticipant> transactionParticipants = Sets.newConcurrentHashSet();
-
- public TransactionCoordinator(TransactionId transactionId, TransactionManager transactionManager) {
- this.transactionId = transactionId;
- this.transactionManager = transactionManager;
- }
-
- /**
- * Returns a transactional map for this transaction.
- *
- * @param name the transactional map name
- * @param serializer the serializer
- * @param <K> key type
- * @param <V> value type
- * @return a transactional map for this transaction
- */
- public <K, V> TransactionalMap<K, V> getTransactionalMap(String name, Serializer serializer) {
- PartitionedTransactionalMap<K, V> map = transactionManager.getTransactionalMap(name, serializer, this);
- transactionParticipants.addAll(map.participants());
- return map;
- }
-
- /**
- * Commits the transaction.
- *
- * @return the transaction commit status
- */
- public CompletableFuture<CommitStatus> commit() {
- long totalParticipants = transactionParticipants.stream()
- .filter(TransactionParticipant::hasPendingUpdates)
- .count();
-
- if (totalParticipants == 0) {
- log.debug("No transaction participants, skipping commit", totalParticipants);
- return CompletableFuture.completedFuture(CommitStatus.SUCCESS);
- } else if (totalParticipants == 1) {
- log.debug("Committing transaction {} via 1 participant", transactionId);
- return transactionParticipants.stream()
- .filter(TransactionParticipant::hasPendingUpdates)
- .findFirst()
- .get()
- .prepareAndCommit()
- .thenApply(v -> v ? CommitStatus.SUCCESS : CommitStatus.FAILURE);
- } else {
- log.debug("Committing transaction {} via {} participants", transactionId, totalParticipants);
- Set<TransactionParticipant> transactionParticipants = this.transactionParticipants.stream()
- .filter(TransactionParticipant::hasPendingUpdates)
- .collect(Collectors.toSet());
-
- CompletableFuture<CommitStatus> status = transactionManager.updateState(
- transactionId, Transaction.State.PREPARING)
- .thenCompose(v -> prepare(transactionParticipants))
- .thenCompose(result -> result
- ? transactionManager.updateState(transactionId, Transaction.State.COMMITTING)
- .thenCompose(v -> commit(transactionParticipants))
- .thenApply(v -> CommitStatus.SUCCESS)
- : transactionManager.updateState(transactionId, Transaction.State.ROLLING_BACK)
- .thenCompose(v -> rollback(transactionParticipants))
- .thenApply(v -> CommitStatus.FAILURE));
- return status.thenCompose(v -> transactionManager.remove(transactionId).thenApply(u -> v));
- }
- }
-
- /**
- * Performs the prepare phase of the two-phase commit protocol for the given transaction participants.
- *
- * @param transactionParticipants the transaction participants for which to prepare the transaction
- * @return a completable future indicating whether <em>all</em> prepares succeeded
- */
- protected CompletableFuture<Boolean> prepare(Set<TransactionParticipant> transactionParticipants) {
- log.trace("Preparing transaction {} via {}", transactionId, transactionParticipants);
- return Tools.allOf(transactionParticipants.stream()
- .map(TransactionParticipant::prepare)
- .collect(Collectors.toList()))
- .thenApply(list -> list.stream().reduce(Boolean::logicalAnd).orElse(true));
- }
-
- /**
- * Performs the commit phase of the two-phase commit protocol for the given transaction participants.
- *
- * @param transactionParticipants the transaction participants for which to commit the transaction
- * @return a completable future to be completed once the commits are complete
- */
- protected CompletableFuture<Void> commit(Set<TransactionParticipant> transactionParticipants) {
- log.trace("Committing transaction {} via {}", transactionId, transactionParticipants);
- return CompletableFuture.allOf(transactionParticipants.stream()
- .map(TransactionParticipant::commit)
- .toArray(CompletableFuture[]::new));
- }
-
- /**
- * Rolls back transactions for the given participants.
- *
- * @param transactionParticipants the transaction participants for which to roll back the transaction
- * @return a completable future to be completed once the rollbacks are complete
- */
- protected CompletableFuture<Void> rollback(Set<TransactionParticipant> transactionParticipants) {
- log.trace("Rolling back transaction {} via {}", transactionId, transactionParticipants);
- return CompletableFuture.allOf(transactionParticipants.stream()
- .map(TransactionParticipant::rollback)
- .toArray(CompletableFuture[]::new));
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("transactionId", transactionId)
- .add("participants", transactionParticipants)
- .toString();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionManager.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionManager.java
deleted file mode 100644
index 7d8d4c4..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionManager.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.hash.Hashing;
-import com.google.common.util.concurrent.Futures;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.PartitionService;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.StorageService;
-import org.onosproject.store.service.TransactionException;
-
-/**
- * Transaction manager for managing state shared across multiple transactions.
- */
-public class TransactionManager {
- private static final int DEFAULT_CACHE_SIZE = 100;
-
- private final PartitionService partitionService;
- private final List<PartitionId> sortedPartitions;
- private final AsyncConsistentMap<TransactionId, Transaction.State> transactions;
- private final long cacheSize;
- private final int buckets;
- private final Map<PartitionId, Cache<String, CachedMap>> partitionCache = Maps.newConcurrentMap();
-
- public TransactionManager(StorageService storageService, PartitionService partitionService, int buckets) {
- this(storageService, partitionService, DEFAULT_CACHE_SIZE, buckets);
- }
-
- public TransactionManager(
- StorageService storageService,
- PartitionService partitionService,
- int cacheSize,
- int buckets) {
- this.partitionService = partitionService;
- this.cacheSize = cacheSize;
- this.buckets = buckets;
- this.transactions = storageService.<TransactionId, Transaction.State>consistentMapBuilder()
- .withName("onos-transactions")
- .withSerializer(Serializer.using(KryoNamespaces.API,
- Transaction.class,
- Transaction.State.class))
- .buildAsyncMap();
- this.sortedPartitions = Lists.newArrayList(partitionService.getAllPartitionIds());
- Collections.sort(sortedPartitions);
- }
-
- /**
- * Returns the collection of currently pending transactions.
- *
- * @return a collection of currently pending transactions
- */
- public Collection<TransactionId> getPendingTransactions() {
- return Futures.getUnchecked(transactions.keySet());
- }
-
- /**
- * Returns a partitioned transactional map for use within a transaction context.
- * <p>
- * The transaction coordinator will return a map that takes advantage of caching that's shared across transaction
- * contexts.
- *
- * @param name the map name
- * @param serializer the map serializer
- * @param transactionCoordinator the transaction coordinator for which the map is being created
- * @param <K> key type
- * @param <V> value type
- * @return a partitioned transactional map
- */
- <K, V> PartitionedTransactionalMap<K, V> getTransactionalMap(
- String name,
- Serializer serializer,
- TransactionCoordinator transactionCoordinator) {
- Map<PartitionId, TransactionalMapParticipant<K, V>> partitions = new HashMap<>();
- for (PartitionId partitionId : partitionService.getAllPartitionIds()) {
- partitions.put(partitionId, getTransactionalMapPartition(
- name, partitionId, serializer, transactionCoordinator));
- }
-
- Hasher<K> hasher = key -> {
- int bucket = Math.abs(Hashing.murmur3_32().hashBytes(serializer.encode(key)).asInt()) % buckets;
- int partition = Hashing.consistentHash(bucket, sortedPartitions.size());
- return sortedPartitions.get(partition);
- };
- return new PartitionedTransactionalMap<>(partitions, hasher);
- }
-
- @SuppressWarnings("unchecked")
- private <K, V> TransactionalMapParticipant<K, V> getTransactionalMapPartition(
- String mapName,
- PartitionId partitionId,
- Serializer serializer,
- TransactionCoordinator transactionCoordinator) {
- Cache<String, CachedMap> mapCache = partitionCache.computeIfAbsent(partitionId, p ->
- CacheBuilder.newBuilder().maximumSize(cacheSize / partitionService.getNumberOfPartitions()).build());
- try {
- CachedMap<K, V> cachedMap = mapCache.get(mapName,
- () -> new CachedMap<>(partitionService.getDistributedPrimitiveCreator(partitionId)
- .newAsyncConsistentMap(mapName, serializer)));
-
- Transaction<MapUpdate<K, V>> transaction = new Transaction<>(
- transactionCoordinator.transactionId,
- cachedMap.baseMap);
- return new DefaultTransactionalMapParticipant<>(cachedMap.cachedMap.asConsistentMap(), transaction);
- } catch (ExecutionException e) {
- throw new TransactionException(e);
- }
- }
-
- /**
- * Updates the state of a transaction in the transaction registry.
- *
- * @param transactionId the transaction identifier
- * @param state the state of the transaction
- * @return a completable future to be completed once the transaction state has been updated in the registry
- */
- CompletableFuture<Void> updateState(TransactionId transactionId, Transaction.State state) {
- return transactions.put(transactionId, state).thenApply(v -> null);
- }
-
- /**
- * Removes the given transaction from the transaction registry.
- *
- * @param transactionId the transaction identifier
- * @return a completable future to be completed once the transaction state has been removed from the registry
- */
- CompletableFuture<Void> remove(TransactionId transactionId) {
- return transactions.remove(transactionId).thenApply(v -> null);
- }
-
- private static class CachedMap<K, V> {
- private final AsyncConsistentMap<K, V> baseMap;
- private final AsyncConsistentMap<K, V> cachedMap;
-
- public CachedMap(AsyncConsistentMap<K, V> baseMap) {
- this.baseMap = baseMap;
- this.cachedMap = DistributedPrimitives.newCachingMap(baseMap);
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionParticipant.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionParticipant.java
deleted file mode 100644
index 02da9d9..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionParticipant.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.concurrent.CompletableFuture;
-
-/**
- * Participant in a two-phase commit protocol.
- */
-public interface TransactionParticipant {
-
- /**
- * Returns a boolean indicating whether the participant has pending updates.
- *
- * @return indicates whether the participant has pending updates
- */
- boolean hasPendingUpdates();
-
- /**
- * Executes the prepare phase.
- *
- * @return {@code true} is successful; {@code false} otherwise
- */
- CompletableFuture<Boolean> prepare();
-
- /**
- * Attempts to execute the commit phase for previously prepared transaction.
- *
- * @return future that is completed when the operation completes
- */
- CompletableFuture<Void> commit();
-
- /**
- * Executes the prepare and commit phases atomically.
- *
- * @return {@code true} is successful; {@code false} otherwise
- */
- CompletableFuture<Boolean> prepareAndCommit();
-
- /**
- * Attempts to execute the rollback phase for previously prepared transaction.
- *
- * @return future that is completed when the operation completes
- */
- CompletableFuture<Void> rollback();
-
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionalMapParticipant.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionalMapParticipant.java
deleted file mode 100644
index 3cb87a4..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionalMapParticipant.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.service.ConsistentMap;
-import org.onosproject.store.service.DistributedPrimitive;
-import org.onosproject.store.service.TransactionException;
-import org.onosproject.store.service.TransactionalMap;
-import org.onosproject.store.service.Version;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-/**
- * Base class for participants within a single {@link TransactionalMap}.
- * <p>
- * This class provides the basic functionality required by transactional map participants and provides methods
- * for defining operations specific to individual isolation levels.
- *
- * @param <K> key type
- * @param <V> value type.
- */
-public abstract class TransactionalMapParticipant<K, V> implements TransactionalMap<K, V>, TransactionParticipant {
- private static final String TX_CLOSED_ERROR = "Transaction is closed";
- private static final String ERROR_NULL_VALUE = "Null values are not allowed";
- private static final String ERROR_NULL_KEY = "Null key is not allowed";
-
- protected final ConsistentMap<K, V> backingMap;
- protected final Transaction<MapUpdate<K, V>> transaction;
- protected final Map<K, V> writeCache = Maps.newConcurrentMap();
- protected final Set<K> deleteSet = Sets.newConcurrentHashSet();
- protected volatile Version lock;
-
- protected TransactionalMapParticipant(
- ConsistentMap<K, V> backingMap,
- Transaction<MapUpdate<K, V>> transaction) {
- this.backingMap = backingMap;
- this.transaction = transaction;
- }
-
- /**
- * Starts the transaction for this partition when a read occurs.
- * <p>
- * Acquiring a pessimistic lock at the start of the transaction ensures that underlying cached maps have been
- * synchronized prior to a read.
- */
- private void beginTransaction() {
- if (lock == null) {
- synchronized (this) {
- if (lock == null) {
- try {
- lock = transaction.begin()
- .get(DistributedPrimitive.DEFAULT_OPERATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new TransactionException.Interrupted();
- } catch (TimeoutException e) {
- throw new TransactionException.Timeout();
- } catch (ExecutionException e) {
- throw new TransactionException(e.getCause());
- }
- }
- }
- }
- }
-
- @Override
- public V get(K key) {
- // Start the transaction for this primitive/partition if necessary.
- beginTransaction();
-
- checkState(transaction.isOpen(), TX_CLOSED_ERROR);
- checkNotNull(key, ERROR_NULL_KEY);
-
- if (deleteSet.contains(key)) {
- return null;
- }
-
- V latest = writeCache.get(key);
- if (latest != null) {
- return latest;
- } else {
- return read(key);
- }
- }
-
- /**
- * Executes a get operation based on the transaction isolation level.
- *
- * @param key the key to look up
- * @return the value
- */
- protected abstract V read(K key);
-
- @Override
- public boolean containsKey(K key) {
- return get(key) != null;
- }
-
- @Override
- public V put(K key, V value) {
- checkNotNull(value, ERROR_NULL_VALUE);
-
- V latest = get(key);
- writeCache.put(key, value);
- deleteSet.remove(key);
- return latest;
- }
-
- @Override
- public V remove(K key) {
- V latest = get(key);
- if (latest != null) {
- writeCache.remove(key);
- deleteSet.add(key);
- }
- return latest;
- }
-
- @Override
- public boolean remove(K key, V value) {
- checkNotNull(value, ERROR_NULL_VALUE);
-
- V latest = get(key);
- if (Objects.equal(value, latest)) {
- remove(key);
- return true;
- }
- return false;
- }
-
- @Override
- public boolean replace(K key, V oldValue, V newValue) {
- checkNotNull(oldValue, ERROR_NULL_VALUE);
- checkNotNull(newValue, ERROR_NULL_VALUE);
-
- V latest = get(key);
- if (Objects.equal(oldValue, latest)) {
- put(key, newValue);
- return true;
- }
- return false;
- }
-
- @Override
- public V putIfAbsent(K key, V value) {
- checkNotNull(value, ERROR_NULL_VALUE);
-
- V latest = get(key);
- if (latest == null) {
- put(key, value);
- }
- return latest;
- }
-
- @Override
- public CompletableFuture<Boolean> prepare() {
- return transaction.prepare(log(lock));
- }
-
- @Override
- public CompletableFuture<Void> commit() {
- return transaction.commit();
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit() {
- return transaction.prepareAndCommit(log(lock));
- }
-
- @Override
- public CompletableFuture<Void> rollback() {
- return transaction.rollback();
- }
-
- /**
- * Returns a list of updates performed within this map partition.
- *
- * @param lockVersion the global transaction lock version
- * @return a list of map updates
- */
- protected List<MapUpdate<K, V>> log(Version lockVersion) {
- return records(lockVersion).collect(Collectors.toList());
- }
-
- /**
- * Returns a stream of updates performed within this map partition.
- *
- * @param lockVersion the global transaction lock version
- * @return a stream of map updates
- */
- protected abstract Stream<MapUpdate<K, V>> records(Version lockVersion);
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(this)
- .add("backingMap", backingMap)
- .toString();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicCounterMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicCounterMap.java
deleted file mode 100644
index 550465d..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicCounterMap.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Function;
-
-/**
- * An {@code AsyncAtomicCounterMap} that transcodes keys.
- */
-public class TranscodingAsyncAtomicCounterMap<K1, K2> implements AsyncAtomicCounterMap<K1> {
- private final AsyncAtomicCounterMap<K2> backingMap;
- private final Function<K1, K2> keyEncoder;
- private final Function<K2, K1> keyDecoder;
-
- public TranscodingAsyncAtomicCounterMap(AsyncAtomicCounterMap<K2> backingMap,
- Function<K1, K2> keyEncoder, Function<K2, K1> keyDecoder) {
- this.backingMap = backingMap;
- this.keyEncoder = k -> k == null ? null : keyEncoder.apply(k);
- this.keyDecoder = k -> k == null ? null : keyDecoder.apply(k);
- }
-
- @Override
- public String name() {
- return backingMap.name();
- }
-
- @Override
- public CompletableFuture<Long> incrementAndGet(K1 key) {
- try {
- return backingMap.incrementAndGet(keyEncoder.apply(key));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> decrementAndGet(K1 key) {
- try {
- return backingMap.decrementAndGet(keyEncoder.apply(key));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> getAndIncrement(K1 key) {
- try {
- return backingMap.getAndIncrement(keyEncoder.apply(key));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> getAndDecrement(K1 key) {
- try {
- return backingMap.getAndDecrement(keyEncoder.apply(key));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> addAndGet(K1 key, long delta) {
- try {
- return backingMap.addAndGet(keyEncoder.apply(key), delta);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> getAndAdd(K1 key, long delta) {
- try {
- return backingMap.getAndAdd(keyEncoder.apply(key), delta);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> get(K1 key) {
- try {
- return backingMap.get(keyEncoder.apply(key));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> put(K1 key, long newValue) {
- try {
- return backingMap.put(keyEncoder.apply(key), newValue);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> putIfAbsent(K1 key, long newValue) {
- try {
- return backingMap.putIfAbsent(keyEncoder.apply(key), newValue);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K1 key, long expectedOldValue, long newValue) {
- try {
- return backingMap.replace(keyEncoder.apply(key), expectedOldValue, newValue);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Long> remove(K1 key) {
- try {
- return backingMap.remove(keyEncoder.apply(key));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K1 key, long value) {
- try {
- return backingMap.remove(keyEncoder.apply(key), value);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- try {
- return backingMap.size();
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- try {
- return backingMap.isEmpty();
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- try {
- return backingMap.clear();
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicValue.java
deleted file mode 100644
index 24dd35d..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicValue.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import com.google.common.collect.Maps;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AtomicValueEvent;
-import org.onosproject.store.service.AtomicValueEventListener;
-
-/**
- * An {@code AsyncAtomicValue} that transcodes values.
- */
-public class TranscodingAsyncAtomicValue<V1, V2> implements AsyncAtomicValue<V1> {
- private final AsyncAtomicValue<V2> backingValue;
- private final Function<V1, V2> valueEncoder;
- private final Function<V2, V1> valueDecoder;
- private final Map<AtomicValueEventListener<V1>, InternalValueEventListener> listeners = Maps.newIdentityHashMap();
-
- public TranscodingAsyncAtomicValue(
- AsyncAtomicValue<V2> backingValue, Function<V1, V2> valueEncoder, Function<V2, V1> valueDecoder) {
- this.backingValue = backingValue;
- this.valueEncoder = k -> k == null ? null : valueEncoder.apply(k);
- this.valueDecoder = k -> k == null ? null : valueDecoder.apply(k);
- }
-
- @Override
- public String name() {
- return backingValue.name();
- }
-
- @Override
- public CompletableFuture<Boolean> compareAndSet(V1 expect, V1 update) {
- return backingValue.compareAndSet(valueEncoder.apply(expect), valueEncoder.apply(update));
- }
-
- @Override
- public CompletableFuture<V1> get() {
- return backingValue.get().thenApply(valueDecoder);
- }
-
- @Override
- public CompletableFuture<V1> getAndSet(V1 value) {
- return backingValue.getAndSet(valueEncoder.apply(value)).thenApply(valueDecoder);
- }
-
- @Override
- public CompletableFuture<Void> set(V1 value) {
- return backingValue.set(valueEncoder.apply(value));
- }
-
- @Override
- public CompletableFuture<Void> addListener(AtomicValueEventListener<V1> listener) {
- synchronized (listeners) {
- InternalValueEventListener backingMapListener =
- listeners.computeIfAbsent(listener, k -> new InternalValueEventListener(listener));
- return backingValue.addListener(backingMapListener);
- }
- }
-
- @Override
- public CompletableFuture<Void> removeListener(AtomicValueEventListener<V1> listener) {
- synchronized (listeners) {
- InternalValueEventListener backingMapListener = listeners.remove(listener);
- if (backingMapListener != null) {
- return backingValue.removeListener(backingMapListener);
- } else {
- return CompletableFuture.completedFuture(null);
- }
- }
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- backingValue.addStatusChangeListener(listener);
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- backingValue.removeStatusChangeListener(listener);
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- return backingValue.statusChangeListeners();
- }
-
- private class InternalValueEventListener implements AtomicValueEventListener<V2> {
- private final AtomicValueEventListener<V1> listener;
-
- InternalValueEventListener(AtomicValueEventListener<V1> listener) {
- this.listener = listener;
- }
-
- @Override
- public void event(AtomicValueEvent<V2> event) {
- listener.event(new AtomicValueEvent<>(
- event.name(),
- event.newValue() != null ? valueDecoder.apply(event.newValue()) : null,
- event.oldValue() != null ? valueDecoder.apply(event.oldValue()) : null));
- }
- }
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMap.java
deleted file mode 100644
index d68ada5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMap.java
+++ /dev/null
@@ -1,374 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import com.google.common.collect.Maps;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-/**
- * An {@code AsyncConsistentMap} that maps its operations to operations on a
- * differently typed {@code AsyncConsistentMap} by transcoding operation inputs and outputs.
- *
- * @param <K2> key type of other map
- * @param <V2> value type of other map
- * @param <K1> key type of this map
- * @param <V1> value type of this map
- */
-public class TranscodingAsyncConsistentMap<K1, V1, K2, V2> implements AsyncConsistentMap<K1, V1> {
-
- private final AsyncConsistentMap<K2, V2> backingMap;
- private final Function<K1, K2> keyEncoder;
- private final Function<K2, K1> keyDecoder;
- private final Function<V2, V1> valueDecoder;
- private final Function<V1, V2> valueEncoder;
- private final Function<Versioned<V2>, Versioned<V1>> versionedValueTransform;
- private final Map<MapEventListener<K1, V1>, InternalBackingMapEventListener> listeners =
- Maps.newIdentityHashMap();
-
- public TranscodingAsyncConsistentMap(AsyncConsistentMap<K2, V2> backingMap,
- Function<K1, K2> keyEncoder,
- Function<K2, K1> keyDecoder,
- Function<V1, V2> valueEncoder,
- Function<V2, V1> valueDecoder) {
- this.backingMap = backingMap;
- this.keyEncoder = k -> k == null ? null : keyEncoder.apply(k);
- this.keyDecoder = k -> k == null ? null : keyDecoder.apply(k);
- this.valueEncoder = v -> v == null ? null : valueEncoder.apply(v);
- this.valueDecoder = v -> v == null ? null : valueDecoder.apply(v);
- this.versionedValueTransform = v -> v == null ? null : v.map(valueDecoder);
- }
-
- @Override
- public String name() {
- return backingMap.name();
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return backingMap.size();
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K1 key) {
- try {
- return backingMap.containsKey(keyEncoder.apply(key));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V1 value) {
- try {
- return backingMap.containsValue(valueEncoder.apply(value));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> get(K1 key) {
- try {
- return backingMap.get(keyEncoder.apply(key)).thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> getOrDefault(K1 key, V1 defaultValue) {
- try {
- return backingMap.getOrDefault(keyEncoder.apply(key), valueEncoder.apply(defaultValue))
- .thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> computeIf(K1 key,
- Predicate<? super V1> condition,
- BiFunction<? super K1, ? super V1, ? extends V1> remappingFunction) {
- try {
- return backingMap.computeIf(keyEncoder.apply(key),
- v -> condition.test(valueDecoder.apply(v)),
- (k, v) -> valueEncoder.apply(remappingFunction.apply(keyDecoder.apply(k),
- valueDecoder.apply(v))))
- .thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> put(K1 key, V1 value) {
- try {
- return backingMap.put(keyEncoder.apply(key), valueEncoder.apply(value))
- .thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> putAndGet(K1 key, V1 value) {
- try {
- return backingMap.putAndGet(keyEncoder.apply(key), valueEncoder.apply(value))
- .thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> remove(K1 key) {
- try {
- return backingMap.remove(keyEncoder.apply(key)).thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return backingMap.clear();
- }
-
- @Override
- public CompletableFuture<Set<K1>> keySet() {
- return backingMap.keySet()
- .thenApply(s -> s.stream().map(keyDecoder).collect(Collectors.toSet()));
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V1>>> values() {
- return backingMap.values()
- .thenApply(c -> c.stream().map(versionedValueTransform).collect(Collectors.toList()));
- }
-
- @Override
- public CompletableFuture<Set<Entry<K1, Versioned<V1>>>> entrySet() {
- return backingMap.entrySet()
- .thenApply(s -> s.stream()
- .map(e -> Maps.immutableEntry(keyDecoder.apply(e.getKey()),
- versionedValueTransform.apply(e.getValue())))
- .collect(Collectors.toSet()));
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> putIfAbsent(K1 key, V1 value) {
- try {
- return backingMap.putIfAbsent(keyEncoder.apply(key), valueEncoder.apply(value))
- .thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K1 key, V1 value) {
- try {
- return backingMap.remove(keyEncoder.apply(key), valueEncoder.apply(value));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K1 key, long version) {
- try {
- return backingMap.remove(keyEncoder.apply(key), version);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> replace(K1 key, V1 value) {
- try {
- return backingMap.replace(keyEncoder.apply(key), valueEncoder.apply(value))
- .thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K1 key, V1 oldValue, V1 newValue) {
- try {
- return backingMap.replace(keyEncoder.apply(key),
- valueEncoder.apply(oldValue),
- valueEncoder.apply(newValue));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K1 key, long oldVersion, V1 newValue) {
- try {
- return backingMap.replace(keyEncoder.apply(key), oldVersion, valueEncoder.apply(newValue));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Entry<K1, Versioned<V1>>>> iterator() {
- return backingMap.iterator().thenApply(TranscodingIterator::new);
- }
-
- @Override
- public CompletableFuture<Void> addListener(MapEventListener<K1, V1> listener, Executor executor) {
- synchronized (listeners) {
- InternalBackingMapEventListener backingMapListener =
- listeners.computeIfAbsent(listener, k -> new InternalBackingMapEventListener(listener));
- return backingMap.addListener(backingMapListener, executor);
- }
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MapEventListener<K1, V1> listener) {
- synchronized (listeners) {
- InternalBackingMapEventListener backingMapListener = listeners.remove(listener);
- if (backingMapListener != null) {
- return backingMap.removeListener(backingMapListener);
- } else {
- return CompletableFuture.completedFuture(null);
- }
- }
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- try {
- return backingMap.begin(transactionId);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K1, V1>> transactionLog) {
- try {
- return backingMap.prepare(transactionLog.map(record -> record.map(keyEncoder, valueEncoder)));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K1, V1>> transactionLog) {
- try {
- return backingMap.prepareAndCommit(transactionLog.map(record -> record.map(keyEncoder, valueEncoder)));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- try {
- return backingMap.commit(transactionId);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- try {
- return backingMap.rollback(transactionId);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- backingMap.addStatusChangeListener(listener);
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- backingMap.removeStatusChangeListener(listener);
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- return backingMap.statusChangeListeners();
- }
-
- private class TranscodingIterator implements AsyncIterator<Entry<K1, Versioned<V1>>> {
- private final AsyncIterator<Map.Entry<K2, Versioned<V2>>> iterator;
-
- public TranscodingIterator(AsyncIterator<Map.Entry<K2, Versioned<V2>>> iterator) {
- this.iterator = iterator;
- }
-
- @Override
- public CompletableFuture<Boolean> hasNext() {
- return iterator.hasNext();
- }
-
- @Override
- public CompletableFuture<Map.Entry<K1, Versioned<V1>>> next() {
- return iterator.next().thenApply(entry ->
- Maps.immutableEntry(keyDecoder.apply(entry.getKey()), entry.getValue().map(valueDecoder)));
- }
- }
-
- private class InternalBackingMapEventListener implements MapEventListener<K2, V2> {
-
- private final MapEventListener<K1, V1> listener;
-
- InternalBackingMapEventListener(MapEventListener<K1, V1> listener) {
- this.listener = listener;
- }
-
- @Override
- public void event(MapEvent<K2, V2> event) {
- listener.event(new MapEvent<K1, V1>(
- event.type(),
- event.name(),
- keyDecoder.apply(event.key()),
- event.newValue() != null ? event.newValue().map(valueDecoder) : null,
- event.oldValue() != null ? event.oldValue().map(valueDecoder) : null));
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMultimap.java
deleted file mode 100644
index 6fab8c8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMultimap.java
+++ /dev/null
@@ -1,381 +0,0 @@
-/*
- * Copyright 2016 Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiConsumer;
-import java.util.function.BinaryOperator;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collector;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableMultiset;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multiset;
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-
-/**
- * An {@link AsyncConsistentMultimap} that maps its operation to operations to
- * a differently typed {@link AsyncConsistentMultimap} by transcoding operation
- * inputs and outputs while maintaining version numbers.
- *
- * @param <K2> key type of other map
- * @param <V2> value type of other map
- * @param <K1> key type of this map
- * @param <V1> value type of this map
- */
-public class TranscodingAsyncConsistentMultimap<K1, V1, K2, V2>
- implements AsyncConsistentMultimap<K1, V1> {
-
- private final AsyncConsistentMultimap<K2, V2> backingMap;
- private final Function<K1, K2> keyEncoder;
- private final Function<K2, K1> keyDecoder;
- private final Function<V2, V1> valueDecoder;
- private final Function<V1, V2> valueEncoder;
- private final Function<? extends Versioned<V2>,
- ? extends Versioned<V1>> versionedValueTransform;
- private final Function<Versioned<Collection<? extends V2>>,
- Versioned<Collection<? extends V1>>> versionedValueCollectionDecode;
- private final Function<Collection<? extends V1>, Collection<V2>>
- valueCollectionEncode;
- private final Map<MultimapEventListener<K1, V1>, InternalBackingMultimapEventListener> listeners =
- Maps.newIdentityHashMap();
-
- public TranscodingAsyncConsistentMultimap(
- AsyncConsistentMultimap<K2, V2> backingMap,
- Function<K1, K2> keyEncoder,
- Function<K2, K1> keyDecoder,
- Function<V2, V1> valueDecoder,
- Function<V1, V2> valueEncoder) {
- this.backingMap = backingMap;
- this.keyEncoder = k -> k == null ? null : keyEncoder.apply(k);
- this.keyDecoder = k -> k == null ? null : keyDecoder.apply(k);
- this.valueDecoder = v -> v == null ? null : valueDecoder.apply(v);
- this.valueEncoder = v -> v == null ? null : valueEncoder.apply(v);
- this.versionedValueTransform = v -> v == null ? null :
- v.map(valueDecoder);
- this.versionedValueCollectionDecode = v -> v == null ? null :
- new Versioned<>(
- v.value()
- .stream()
- .map(valueDecoder)
- .collect(Collectors.toSet()),
- v.version(),
- v.creationTime());
- this.valueCollectionEncode = v -> v == null ? null :
- v.stream().map(valueEncoder).collect(Collectors.toSet());
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return backingMap.size();
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- return backingMap.isEmpty();
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K1 key) {
- try {
- return backingMap.containsKey(keyEncoder.apply(key));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V1 value) {
- try {
- return backingMap.containsValue(valueEncoder.apply(value));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> containsEntry(K1 key, V1 value) {
- try {
- return backingMap.containsEntry(keyEncoder.apply(key),
- valueEncoder.apply(value));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> put(K1 key, V1 value) {
- try {
- return backingMap.put(keyEncoder.apply(key),
- valueEncoder.apply(value));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V1>>> putAndGet(K1 key, V1 value) {
- try {
- return backingMap.putAndGet(keyEncoder.apply(key), valueEncoder.apply(value))
- .thenApply(versionedValueCollectionDecode);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K1 key, V1 value) {
- try {
- return backingMap.remove(keyEncoder.apply(key), valueEncoder
- .apply(value));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V1>>> removeAndGet(K1 key, V1 value) {
- try {
- return backingMap.removeAndGet(keyEncoder.apply(key), valueEncoder.apply(value))
- .thenApply(versionedValueCollectionDecode);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean> removeAll(
- K1 key, Collection<? extends V1> values) {
- try {
- return backingMap.removeAll(
- keyEncoder.apply(key),
- values.stream().map(valueEncoder).collect(
- Collectors.toSet()));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V1>>>
- removeAll(K1 key) {
- try {
- return backingMap.removeAll(keyEncoder.apply(key))
- .thenApply(versionedValueCollectionDecode);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Boolean>
- putAll(K1 key, Collection<? extends V1> values) {
- try {
- return backingMap.putAll(keyEncoder.apply(key),
- valueCollectionEncode.apply(values));
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V1>>>
- replaceValues(K1 key, Collection<V1> values) {
- try {
- return backingMap.replaceValues(keyEncoder.apply(key),
- valueCollectionEncode.apply(values))
- .thenApply(versionedValueCollectionDecode);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return backingMap.clear();
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends V1>>> get(K1 key) {
- try {
- return backingMap.get(keyEncoder.apply(key))
- .thenApply(versionedValueCollectionDecode);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Set<K1>> keySet() {
- return backingMap.keySet().thenApply(s -> s.stream()
- .map(keyDecoder)
- .collect(Collectors.toSet()));
- }
-
- @Override
- public CompletableFuture<Multiset<K1>> keys() {
- return backingMap.keys().thenApply(s -> s.stream().map(keyDecoder)
- .collect(new MultisetCollector<>()));
- }
-
- @Override
- public CompletableFuture<Multiset<V1>> values() {
- return backingMap.values().thenApply(s ->
- s.stream().map(valueDecoder).collect(new MultisetCollector<>()));
- }
-
- @Override
- public CompletableFuture<Collection<Map.Entry<K1, V1>>> entries() {
- return backingMap.entries().thenApply(s -> s.stream()
- .map(e -> Maps.immutableEntry(keyDecoder.apply(e.getKey()),
- valueDecoder.apply(e.getValue())))
- .collect(Collectors.toSet()));
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Map.Entry<K1, V1>>> iterator() {
- return backingMap.iterator().thenApply(TranscodingIterator::new);
- }
-
- @Override
- public CompletableFuture<Map<K1, Collection<V1>>> asMap() {
- throw new UnsupportedOperationException("Unsupported operation.");
- }
-
- @Override
- public String name() {
- return backingMap.name();
- }
-
- @Override
- public CompletableFuture<Void> addListener(MultimapEventListener<K1, V1> listener, Executor executor) {
- synchronized (listeners) {
- InternalBackingMultimapEventListener backingMapListener =
- listeners.computeIfAbsent(listener, k -> new InternalBackingMultimapEventListener(listener));
- return backingMap.addListener(backingMapListener, executor);
- }
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MultimapEventListener<K1, V1> listener) {
- synchronized (listeners) {
- InternalBackingMultimapEventListener backingMapListener = listeners.remove(listener);
- if (backingMapListener != null) {
- return backingMap.removeListener(backingMapListener);
- } else {
- return CompletableFuture.completedFuture(null);
- }
- }
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- backingMap.addStatusChangeListener(listener);
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- backingMap.removeStatusChangeListener(listener);
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- return backingMap.statusChangeListeners();
- }
-
- private class MultisetCollector<T> implements Collector<T,
- ImmutableMultiset.Builder<T>,
- Multiset<T>> {
-
- @Override
- public Supplier<ImmutableMultiset.Builder<T>> supplier() {
- return ImmutableMultiset::builder;
- }
-
- @Override
- public BiConsumer<ImmutableMultiset.Builder<T>, T> accumulator() {
- return ((builder, t) -> builder.add(t));
- }
-
- @Override
- public BinaryOperator<ImmutableMultiset.Builder<T>> combiner() {
- return (a, b) -> {
- a.addAll(b.build());
- return a;
- };
- }
-
- @Override
- public Function<ImmutableMultiset.Builder<T>, Multiset<T>> finisher() {
- return ImmutableMultiset.Builder::build;
- }
-
- @Override
- public Set<Characteristics> characteristics() {
- return EnumSet.of(Characteristics.UNORDERED);
- }
- }
-
- private class TranscodingIterator implements AsyncIterator<Map.Entry<K1, V1>> {
- private final AsyncIterator<Map.Entry<K2, V2>> iterator;
-
- public TranscodingIterator(AsyncIterator<Map.Entry<K2, V2>> iterator) {
- this.iterator = iterator;
- }
-
- @Override
- public CompletableFuture<Boolean> hasNext() {
- return iterator.hasNext();
- }
-
- @Override
- public CompletableFuture<Map.Entry<K1, V1>> next() {
- return iterator.next().thenApply(entry ->
- Maps.immutableEntry(keyDecoder.apply(entry.getKey()), valueDecoder.apply(entry.getValue())));
- }
- }
-
- private class InternalBackingMultimapEventListener implements MultimapEventListener<K2, V2> {
-
- private final MultimapEventListener<K1, V1> listener;
-
- InternalBackingMultimapEventListener(MultimapEventListener<K1, V1> listener) {
- this.listener = listener;
- }
-
- @Override
- public void event(MultimapEvent<K2, V2> event) {
- listener.event(new MultimapEvent(event.name(),
- keyDecoder.apply(event.key()),
- valueDecoder.apply(event.newValue()),
- valueDecoder.apply(event.oldValue())));
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentTreeMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentTreeMap.java
deleted file mode 100644
index d8e686e..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentTreeMap.java
+++ /dev/null
@@ -1,437 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import com.google.common.collect.Maps;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-/**
- * An {@code AsyncConsistentTreeMap} that maps its operations to operations on
- * a differently typed {@code AsyncConsistentTreeMap} by transcoding operation
- * inputs and outputs.
- *
- * @param <V2> value type of other map
- * @param <V1> value type of this map
- */
-public class TranscodingAsyncConsistentTreeMap<V1, V2>
- implements AsyncConsistentTreeMap<V1> {
- private final AsyncConsistentTreeMap<V2> backingMap;
- private final Function<V2, V1> valueDecoder;
- private final Function<V1, V2> valueEncoder;
- private final Function<Versioned<V2>, Versioned<V1>>
- versionedValueTransform;
- private final Map<MapEventListener<String, V1>,
- TranscodingAsyncConsistentTreeMap.InternalBackingMapEventListener>
- listeners = Maps.newIdentityHashMap();
-
- public TranscodingAsyncConsistentTreeMap(
- AsyncConsistentTreeMap<V2> backingMap,
- Function<V1, V2> valueEncoder,
- Function<V2, V1> valueDecoder) {
- this.backingMap = backingMap;
- this.valueEncoder = v -> v == null ? null : valueEncoder.apply(v);
- this.valueDecoder = v -> v == null ? null : valueDecoder.apply(v);
- this.versionedValueTransform = v -> v == null ? null :
- v.map(valueDecoder);
- }
-
- @Override
- public CompletableFuture<String> firstKey() {
- return backingMap.firstKey();
- }
-
- @Override
- public CompletableFuture<String> lastKey() {
- return backingMap.lastKey();
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>>
- ceilingEntry(String key) {
- return backingMap.ceilingEntry(key)
- .thenApply(
- entry ->
- Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform
- .apply(entry.getValue())));
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>>
- floorEntry(String key) {
- return backingMap.floorEntry(key)
- .thenApply(
- entry ->
- Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform
- .apply(entry.getValue())));
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>>
- higherEntry(String key) {
- return backingMap
- .higherEntry(key)
- .thenApply(entry ->
- Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform
- .apply(entry.getValue())));
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>>
- lowerEntry(String key) {
- return backingMap.lowerEntry(key).thenApply(
- entry ->
- Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform
- .apply(entry.getValue())));
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>>
- firstEntry() {
- return backingMap.firstEntry()
- .thenApply(entry ->
- Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform
- .apply(entry.getValue())));
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>>
- lastEntry() {
- return backingMap.lastEntry()
- .thenApply(
- entry -> Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform
- .apply(entry.getValue())));
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>>
- pollFirstEntry() {
- return backingMap.pollFirstEntry()
- .thenApply(
- entry -> Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform
- .apply(entry.getValue())));
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>>
- pollLastEntry() {
- return backingMap.pollLastEntry()
- .thenApply(entry -> Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform.apply(entry.getValue())));
- }
-
- @Override
- public CompletableFuture<String> lowerKey(String key) {
- return backingMap.lowerKey(key);
- }
-
- @Override
- public CompletableFuture<String> floorKey(String key) {
- return backingMap.floorKey(key);
- }
-
- @Override
- public CompletableFuture<String> ceilingKey(String key) {
- return backingMap.ceilingKey(key);
- }
-
- @Override
- public CompletableFuture<String> higherKey(String key) {
- return backingMap.higherKey(key);
- }
-
- @Override
- public CompletableFuture<NavigableSet<String>> navigableKeySet() {
- return backingMap.navigableKeySet();
- }
-
- @Override
- public CompletableFuture<NavigableMap<String, V1>> subMap(
- String upperKey,
- String lowerKey,
- boolean inclusiveUpper,
- boolean inclusiveLower) {
- throw new UnsupportedOperationException("This operation is not yet" +
- "supported.");
- }
-
- @Override
- public String name() {
- return backingMap.name();
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return backingMap.size();
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(String key) {
- return backingMap.containsKey(key);
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V1 value) {
- return backingMap.containsValue(valueEncoder.apply(value));
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> get(String key) {
- return backingMap.get(key).thenApply(versionedValueTransform);
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> getOrDefault(String key, V1 defaultValue) {
- return backingMap.getOrDefault(key, valueEncoder.apply(defaultValue)).thenApply(versionedValueTransform);
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> computeIf(
- String key, Predicate<? super V1> condition,
- BiFunction<? super String, ? super V1, ? extends V1>
- remappingFunction) {
- try {
- return backingMap
- .computeIf(
- key,
- v -> condition.test(valueDecoder.apply(v)),
- (k, v) -> valueEncoder
- .apply(
- remappingFunction.apply(
- key,
- valueDecoder.apply(v))))
- .thenApply(versionedValueTransform);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> put(String key, V1 value) {
- return backingMap.put(key, valueEncoder.apply(value))
- .thenApply(versionedValueTransform);
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> putAndGet(String key, V1 value) {
- return backingMap.putAndGet(key, valueEncoder.apply(value))
- .thenApply(versionedValueTransform);
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> remove(String key) {
- return backingMap.remove(key).thenApply(versionedValueTransform);
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return backingMap.clear();
- }
-
- @Override
- public CompletableFuture<Set<String>> keySet() {
- return backingMap.keySet();
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V1>>> values() {
- return backingMap.values().thenApply(valueSet -> valueSet.stream()
- .map(versionedValueTransform).collect(Collectors.toSet()));
- }
-
- @Override
- public CompletableFuture<Set<Map.Entry<String, Versioned<V1>>>>
- entrySet() {
- return backingMap.entrySet()
- .thenApply(
- entries -> entries
- .stream()
- .map(entry ->
- Maps.immutableEntry(
- entry.getKey(),
- versionedValueTransform
- .apply(entry.getValue())
- ))
- .collect(Collectors.toSet()));
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> putIfAbsent(String key, V1 value) {
- return backingMap.putIfAbsent(key, valueEncoder.apply(value))
- .thenApply(versionedValueTransform);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(String key, V1 value) {
- return backingMap.remove(key, valueEncoder.apply(value));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(String key, long version) {
- return backingMap.remove(key, version);
- }
-
- @Override
- public CompletableFuture<Versioned<V1>> replace(String key, V1 value) {
- return backingMap.replace(key, valueEncoder.apply(value))
- .thenApply(versionedValueTransform);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(String key, V1 oldValue,
- V1 newValue) {
- return backingMap.replace(key, valueEncoder.apply(oldValue),
- valueEncoder.apply(newValue));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(String key, long oldVersion,
- V1 newValue) {
- return backingMap.replace(key, oldVersion,
- valueEncoder.apply(newValue));
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Map.Entry<String, Versioned<V1>>>> iterator() {
- return backingMap.iterator().thenApply(TranscodingIterator::new);
- }
-
- @Override
- public CompletableFuture<Void> addListener(
- MapEventListener<String, V1> listener,
- Executor executor) {
- InternalBackingMapEventListener backingMapEventListener =
- listeners.computeIfAbsent(
- listener,
- k -> new InternalBackingMapEventListener(listener));
- return backingMap.addListener(backingMapEventListener, executor);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(
- MapEventListener<String, V1> listener) {
- InternalBackingMapEventListener backingMapEventListener =
- listeners.remove(listener);
- if (backingMapEventListener == null) {
- return CompletableFuture.completedFuture(null);
- } else {
- return backingMap.removeListener(backingMapEventListener);
- }
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- throw new UnsupportedOperationException("This operation is not yet supported.");
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<String, V1>> transactionLog) {
- throw new UnsupportedOperationException("This operation is not yet supported.");
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<String, V1>> transactionLog) {
- throw new UnsupportedOperationException("This operation is not yet supported.");
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- throw new UnsupportedOperationException("This operation is not yet supported.");
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- throw new UnsupportedOperationException("This operation is not yet supported.");
- }
-
- private class TranscodingIterator implements AsyncIterator<Map.Entry<String, Versioned<V1>>> {
- private final AsyncIterator<Map.Entry<String, Versioned<V2>>> iterator;
-
- public TranscodingIterator(AsyncIterator<Map.Entry<String, Versioned<V2>>> iterator) {
- this.iterator = iterator;
- }
-
- @Override
- public CompletableFuture<Boolean> hasNext() {
- return iterator.hasNext();
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<V1>>> next() {
- return iterator.next().thenApply(entry ->
- Maps.immutableEntry(entry.getKey(), entry.getValue().map(valueDecoder)));
- }
- }
-
- private class InternalBackingMapEventListener
- implements MapEventListener<String, V2> {
-
- private final MapEventListener<String, V1> listener;
-
- InternalBackingMapEventListener(
- MapEventListener<String, V1> listener) {
- this.listener = listener;
- }
-
- @Override
- public void event(MapEvent<String, V2> event) {
- listener.event(new MapEvent<String, V1>(
- event.type(),
- event.name(),
- event.key(),
- event.newValue() != null ?
- event.newValue().map(valueDecoder) : null,
- event.oldValue() != null ?
- event.oldValue().map(valueDecoder) : null));
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UnmodifiableAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UnmodifiableAsyncConsistentMap.java
deleted file mode 100644
index 65378b9..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UnmodifiableAsyncConsistentMap.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.function.BiFunction;
-import java.util.function.Predicate;
-
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.Versioned;
-
-/**
- * An unmodifiable {@link AsyncConsistentMap}.
- * <p>
- * Any attempt to update the map through this instance will cause the
- * operation to be completed with an {@link UnsupportedOperationException}.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class UnmodifiableAsyncConsistentMap<K, V> extends DelegatingAsyncConsistentMap<K, V> {
-
- private static final String ERROR_MSG = "map updates are not allowed";
-
- public UnmodifiableAsyncConsistentMap(AsyncConsistentMap<K, V> backingMap) {
- super(backingMap);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> computeIf(K key,
- Predicate<? super V> condition,
- BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(""));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(K key, V value) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> remove(K key) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, long version) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(K key, V value) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
- return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateEntry.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateEntry.java
index 42bf1f9..680795c 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateEntry.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateEntry.java
@@ -15,10 +15,10 @@
*/
package org.onosproject.store.primitives.impl;
-import static com.google.common.base.Preconditions.checkNotNull;
-
import com.google.common.base.MoreObjects;
+import static com.google.common.base.Preconditions.checkNotNull;
+
/**
* Describes a single update event in an EventuallyConsistentMap.
*/
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateRequest.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateRequest.java
index 7563469..cb3db2f 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateRequest.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateRequest.java
@@ -15,12 +15,12 @@
*/
package org.onosproject.store.primitives.impl;
+import java.util.Set;
+
import com.google.common.base.MoreObjects;
import com.google.common.collect.ImmutableSet;
import org.onosproject.cluster.NodeId;
-import java.util.Set;
-
import static com.google.common.base.Preconditions.checkNotNull;
/**
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/package-info.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/package-info.java
index 21824a7..e151e54 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/package-info.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/package-info.java
@@ -15,6 +15,6 @@
*/
/**
- * Implementation classes for various Distributed primitives.
+ * Implementation classes for store utilities.
*/
package org.onosproject.store.primitives.impl;
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AbstractRaftPrimitive.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AbstractRaftPrimitive.java
deleted file mode 100644
index 17b8ef8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AbstractRaftPrimitive.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-import java.util.Set;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onosproject.store.service.DistributedPrimitive;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Abstract base class for primitives that interact with Raft replicated state machines via proxy.
- */
-public abstract class AbstractRaftPrimitive implements DistributedPrimitive {
- private final Function<RaftProxy.State, Status> mapper = state -> {
- switch (state) {
- case CONNECTED:
- return Status.ACTIVE;
- case SUSPENDED:
- return Status.SUSPENDED;
- case CLOSED:
- return Status.INACTIVE;
- default:
- throw new IllegalStateException("Unknown state " + state);
- }
- };
-
- protected final RaftProxy proxy;
- private final Set<Consumer<Status>> statusChangeListeners = Sets.newCopyOnWriteArraySet();
-
- public AbstractRaftPrimitive(RaftProxy proxy) {
- this.proxy = checkNotNull(proxy, "proxy cannot be null");
- proxy.addStateChangeListener(this::onStateChange);
- }
-
- @Override
- public String name() {
- return proxy.name();
- }
-
- /**
- * Handles a Raft session state change.
- *
- * @param state the updated Raft session state
- */
- private void onStateChange(RaftProxy.State state) {
- statusChangeListeners.forEach(listener -> listener.accept(mapper.apply(state)));
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- statusChangeListeners.add(listener);
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- statusChangeListeners.remove(listener);
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- return ImmutableSet.copyOf(statusChangeListeners);
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("proxy", proxy)
- .toString();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMap.java
deleted file mode 100644
index 9d25e26..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMap.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.concurrent.CompletableFuture;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.AddAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.DecrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndAdd;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndDecrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndIncrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.IncrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PutIfAbsent;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Replace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.ADD_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.DECREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_DECREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_INCREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.INCREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT_IF_ABSENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REMOVE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.SIZE;
-
-/**
- * {@code AsyncAtomicCounterMap} implementation backed by Atomix.
- */
-public class AtomixAtomicCounterMap extends AbstractRaftPrimitive implements AsyncAtomicCounterMap<String> {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixAtomicCounterMapOperations.NAMESPACE)
- .build());
-
- public AtomixAtomicCounterMap(RaftProxy proxy) {
- super(proxy);
- }
-
- @Override
- public CompletableFuture<Long> incrementAndGet(String key) {
- return proxy.invoke(INCREMENT_AND_GET, SERIALIZER::encode, new IncrementAndGet(key), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> decrementAndGet(String key) {
- return proxy.invoke(DECREMENT_AND_GET, SERIALIZER::encode, new DecrementAndGet(key), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> getAndIncrement(String key) {
- return proxy.invoke(GET_AND_INCREMENT, SERIALIZER::encode, new GetAndIncrement(key), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> getAndDecrement(String key) {
- return proxy.invoke(GET_AND_DECREMENT, SERIALIZER::encode, new GetAndDecrement(key), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> addAndGet(String key, long delta) {
- return proxy.invoke(ADD_AND_GET, SERIALIZER::encode, new AddAndGet(key, delta), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> getAndAdd(String key, long delta) {
- return proxy.invoke(GET_AND_ADD, SERIALIZER::encode, new GetAndAdd(key, delta), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> get(String key) {
- return proxy.invoke(GET, SERIALIZER::encode, new Get(key), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> put(String key, long newValue) {
- return proxy.invoke(PUT, SERIALIZER::encode, new Put(key, newValue), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> putIfAbsent(String key, long newValue) {
- return proxy.invoke(PUT_IF_ABSENT, SERIALIZER::encode, new PutIfAbsent(key, newValue), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> replace(String key, long expectedOldValue, long newValue) {
- return proxy.invoke(
- REPLACE,
- SERIALIZER::encode,
- new Replace(key, expectedOldValue, newValue),
- SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> remove(String key) {
- return proxy.invoke(REMOVE, SERIALIZER::encode, new Remove(key), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(String key, long value) {
- return proxy.invoke(REMOVE_VALUE, SERIALIZER::encode, new RemoveValue(key, value), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return proxy.invoke(SIZE, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- return proxy.invoke(IS_EMPTY, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return proxy.invoke(CLEAR);
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapOperations.java
deleted file mode 100644
index 7b2cba2..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapOperations.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Atomic counter map commands.
- */
-public enum AtomixAtomicCounterMapOperations implements OperationId {
- PUT(OperationType.COMMAND),
- PUT_IF_ABSENT(OperationType.COMMAND),
- GET(OperationType.QUERY),
- REPLACE(OperationType.COMMAND),
- REMOVE(OperationType.COMMAND),
- REMOVE_VALUE(OperationType.COMMAND),
- GET_AND_INCREMENT(OperationType.COMMAND),
- GET_AND_DECREMENT(OperationType.COMMAND),
- INCREMENT_AND_GET(OperationType.COMMAND),
- DECREMENT_AND_GET(OperationType.COMMAND),
- ADD_AND_GET(OperationType.COMMAND),
- GET_AND_ADD(OperationType.COMMAND),
- SIZE(OperationType.QUERY),
- IS_EMPTY(OperationType.QUERY),
- CLEAR(OperationType.COMMAND);
-
- private final OperationType type;
-
- AtomixAtomicCounterMapOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(IncrementAndGet.class)
- .register(DecrementAndGet.class)
- .register(GetAndIncrement.class)
- .register(GetAndDecrement.class)
- .register(AddAndGet.class)
- .register(GetAndAdd.class)
- .register(Get.class)
- .register(Put.class)
- .register(PutIfAbsent.class)
- .register(Replace.class)
- .register(Remove.class)
- .register(RemoveValue.class)
- .build("AtomixAtomicCounterMapOperations");
-
- public abstract static class AtomicCounterMapOperation<V> {
- }
-
- public abstract static class KeyOperation extends AtomicCounterMapOperation {
- private String key;
-
- public KeyOperation() {
- }
-
- public KeyOperation(String key) {
- this.key = key;
- }
-
- public String key() {
- return key;
- }
- }
-
- public static class KeyValueOperation extends KeyOperation {
- private long value;
-
- public KeyValueOperation() {
- }
-
- public KeyValueOperation(String key, long value) {
- super(key);
- this.value = value;
- }
-
- public long value() {
- return value;
- }
- }
-
- public static class Get extends KeyOperation {
- public Get() {
- }
-
- public Get(String key) {
- super(key);
- }
- }
-
- public static class Put extends KeyValueOperation {
- public Put() {
- }
-
- public Put(String key, long value) {
- super(key, value);
- }
- }
-
- public static class PutIfAbsent extends KeyValueOperation {
- public PutIfAbsent() {
- }
-
- public PutIfAbsent(String key, long value) {
- super(key, value);
- }
- }
-
- public static class Replace extends KeyOperation {
- private long replace;
- private long value;
-
- public Replace() {
- }
-
- public Replace(String key, long replace, long value) {
- super(key);
- this.replace = replace;
- this.value = value;
- }
-
- public long replace() {
- return replace;
- }
-
- public long value() {
- return value;
- }
- }
-
- public static class Remove extends KeyOperation {
- public Remove() {
- }
-
- public Remove(String key) {
- super(key);
- }
- }
-
- public static class RemoveValue extends KeyValueOperation {
- public RemoveValue() {
- }
-
- public RemoveValue(String key, long value) {
- super(key, value);
- }
- }
-
- public static class IncrementAndGet extends KeyOperation {
- public IncrementAndGet() {
- }
-
- public IncrementAndGet(String key) {
- super(key);
- }
- }
-
- public static class DecrementAndGet extends KeyOperation {
- public DecrementAndGet(String key) {
- super(key);
- }
-
- public DecrementAndGet() {
- }
- }
-
- public static class GetAndIncrement extends KeyOperation {
- public GetAndIncrement() {
- }
-
- public GetAndIncrement(String key) {
- super(key);
- }
- }
-
- public static class GetAndDecrement extends KeyOperation {
- public GetAndDecrement() {
- }
-
- public GetAndDecrement(String key) {
- super(key);
- }
- }
-
- public abstract static class DeltaOperation extends KeyOperation {
- private long delta;
-
- public DeltaOperation() {
- }
-
- public DeltaOperation(String key, long delta) {
- super(key);
- this.delta = delta;
- }
-
- public long delta() {
- return delta;
- }
- }
-
- public static class AddAndGet extends DeltaOperation {
- public AddAndGet() {
- }
-
- public AddAndGet(String key, long delta) {
- super(key, delta);
- }
- }
-
- public static class GetAndAdd extends DeltaOperation {
- public GetAndAdd() {
- }
-
- public GetAndAdd(String key, long delta) {
- super(key, delta);
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapService.java
deleted file mode 100644
index 354db26..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapService.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.AddAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.DecrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndAdd;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndDecrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndIncrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.IncrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PutIfAbsent;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Replace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.ADD_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.DECREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_DECREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_INCREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.INCREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT_IF_ABSENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REMOVE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.SIZE;
-
-/**
- * Atomic counter map state for Atomix.
- * <p>
- * The counter map state is implemented as a snapshottable state machine. Snapshots are necessary
- * since incremental compaction is impractical for counters where the value of a counter is the sum
- * of all its increments. Note that this snapshotting large state machines may risk blocking of the
- * Raft cluster with the current implementation of snapshotting in Copycat.
- */
-public class AtomixAtomicCounterMapService extends AbstractRaftService {
-
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixAtomicCounterMapOperations.NAMESPACE)
- .build());
-
- private Map<String, Long> map = new HashMap<>();
-
- @Override
- protected void configure(RaftServiceExecutor executor) {
- executor.register(PUT, SERIALIZER::decode, this::put, SERIALIZER::encode);
- executor.register(PUT_IF_ABSENT, SERIALIZER::decode, this::putIfAbsent, SERIALIZER::encode);
- executor.register(GET, SERIALIZER::decode, this::get, SERIALIZER::encode);
- executor.register(REPLACE, SERIALIZER::decode, this::replace, SERIALIZER::encode);
- executor.register(REMOVE, SERIALIZER::decode, this::remove, SERIALIZER::encode);
- executor.register(REMOVE_VALUE, SERIALIZER::decode, this::removeValue, SERIALIZER::encode);
- executor.register(GET_AND_INCREMENT, SERIALIZER::decode, this::getAndIncrement, SERIALIZER::encode);
- executor.register(GET_AND_DECREMENT, SERIALIZER::decode, this::getAndDecrement, SERIALIZER::encode);
- executor.register(INCREMENT_AND_GET, SERIALIZER::decode, this::incrementAndGet, SERIALIZER::encode);
- executor.register(DECREMENT_AND_GET, SERIALIZER::decode, this::decrementAndGet, SERIALIZER::encode);
- executor.register(ADD_AND_GET, SERIALIZER::decode, this::addAndGet, SERIALIZER::encode);
- executor.register(GET_AND_ADD, SERIALIZER::decode, this::getAndAdd, SERIALIZER::encode);
- executor.register(SIZE, this::size, SERIALIZER::encode);
- executor.register(IS_EMPTY, this::isEmpty, SERIALIZER::encode);
- executor.register(CLEAR, this::clear);
- }
-
- @Override
- public void snapshot(SnapshotWriter writer) {
- writer.writeObject(map, SERIALIZER::encode);
- }
-
- @Override
- public void install(SnapshotReader reader) {
- map = reader.readObject(SERIALIZER::decode);
- }
-
- /**
- * Returns the primitive value for the given primitive wrapper.
- */
- private long primitive(Long value) {
- if (value != null) {
- return value;
- } else {
- return 0;
- }
- }
-
- /**
- * Handles a {@link Put} command which implements {@link AtomixAtomicCounterMap#put(String, long)}.
- *
- * @param commit put commit
- * @return put result
- */
- protected long put(Commit<Put> commit) {
- return primitive(map.put(commit.value().key(), commit.value().value()));
- }
-
- /**
- * Handles a {@link PutIfAbsent} command which implements {@link AtomixAtomicCounterMap#putIfAbsent(String, long)}.
- *
- * @param commit putIfAbsent commit
- * @return putIfAbsent result
- */
- protected long putIfAbsent(Commit<PutIfAbsent> commit) {
- return primitive(map.putIfAbsent(commit.value().key(), commit.value().value()));
- }
-
- /**
- * Handles a {@link Get} query which implements {@link AtomixAtomicCounterMap#get(String)}}.
- *
- * @param commit get commit
- * @return get result
- */
- protected long get(Commit<Get> commit) {
- return primitive(map.get(commit.value().key()));
- }
-
- /**
- * Handles a {@link Replace} command which implements {@link AtomixAtomicCounterMap#replace(String, long, long)}.
- *
- * @param commit replace commit
- * @return replace result
- */
- protected boolean replace(Commit<Replace> commit) {
- Long value = map.get(commit.value().key());
- if (value == null) {
- if (commit.value().replace() == 0) {
- map.put(commit.value().key(), commit.value().value());
- return true;
- } else {
- return false;
- }
- } else if (value == commit.value().replace()) {
- map.put(commit.value().key(), commit.value().value());
- return true;
- }
- return false;
- }
-
- /**
- * Handles a {@link Remove} command which implements {@link AtomixAtomicCounterMap#remove(String)}.
- *
- * @param commit remove commit
- * @return remove result
- */
- protected long remove(Commit<Remove> commit) {
- return primitive(map.remove(commit.value().key()));
- }
-
- /**
- * Handles a {@link RemoveValue} command which implements {@link AtomixAtomicCounterMap#remove(String, long)}.
- *
- * @param commit removeValue commit
- * @return removeValue result
- */
- protected boolean removeValue(Commit<RemoveValue> commit) {
- Long value = map.get(commit.value().key());
- if (value == null) {
- if (commit.value().value() == 0) {
- map.remove(commit.value().key());
- return true;
- }
- return false;
- } else if (value == commit.value().value()) {
- map.remove(commit.value().key());
- return true;
- }
- return false;
- }
-
- /**
- * Handles a {@link GetAndIncrement} command which implements
- * {@link AtomixAtomicCounterMap#getAndIncrement(String)}.
- *
- * @param commit getAndIncrement commit
- * @return getAndIncrement result
- */
- protected long getAndIncrement(Commit<GetAndIncrement> commit) {
- long value = primitive(map.get(commit.value().key()));
- map.put(commit.value().key(), value + 1);
- return value;
- }
-
- /**
- * Handles a {@link GetAndDecrement} command which implements
- * {@link AtomixAtomicCounterMap#getAndDecrement(String)}.
- *
- * @param commit getAndDecrement commit
- * @return getAndDecrement result
- */
- protected long getAndDecrement(Commit<GetAndDecrement> commit) {
- long value = primitive(map.get(commit.value().key()));
- map.put(commit.value().key(), value - 1);
- return value;
- }
-
- /**
- * Handles a {@link IncrementAndGet} command which implements
- * {@link AtomixAtomicCounterMap#incrementAndGet(String)}.
- *
- * @param commit incrementAndGet commit
- * @return incrementAndGet result
- */
- protected long incrementAndGet(Commit<IncrementAndGet> commit) {
- long value = primitive(map.get(commit.value().key()));
- map.put(commit.value().key(), ++value);
- return value;
- }
-
- /**
- * Handles a {@link DecrementAndGet} command which implements
- * {@link AtomixAtomicCounterMap#decrementAndGet(String)}.
- *
- * @param commit decrementAndGet commit
- * @return decrementAndGet result
- */
- protected long decrementAndGet(Commit<DecrementAndGet> commit) {
- long value = primitive(map.get(commit.value().key()));
- map.put(commit.value().key(), --value);
- return value;
- }
-
- /**
- * Handles a {@link AddAndGet} command which implements {@link AtomixAtomicCounterMap#addAndGet(String, long)}.
- *
- * @param commit addAndGet commit
- * @return addAndGet result
- */
- protected long addAndGet(Commit<AddAndGet> commit) {
- long value = primitive(map.get(commit.value().key()));
- value += commit.value().delta();
- map.put(commit.value().key(), value);
- return value;
- }
-
- /**
- * Handles a {@link GetAndAdd} command which implements {@link AtomixAtomicCounterMap#getAndAdd(String, long)}.
- *
- * @param commit getAndAdd commit
- * @return getAndAdd result
- */
- protected long getAndAdd(Commit<GetAndAdd> commit) {
- long value = primitive(map.get(commit.value().key()));
- map.put(commit.value().key(), value + commit.value().delta());
- return value;
- }
-
- /**
- * Handles a {@code Size} query which implements {@link AtomixAtomicCounterMap#size()}.
- *
- * @param commit size commit
- * @return size result
- */
- protected int size(Commit<Void> commit) {
- return map.size();
- }
-
- /**
- * Handles an {@code IsEmpty} query which implements {@link AtomixAtomicCounterMap#isEmpty()}.
- *
- * @param commit isEmpty commit
- * @return isEmpty result
- */
- protected boolean isEmpty(Commit<Void> commit) {
- return map.isEmpty();
- }
-
- /**
- * Handles a {@code Clear} command which implements {@link AtomixAtomicCounterMap#clear()}.
- *
- * @param commit clear commit
- */
- protected void clear(Commit<Void> commit) {
- map.clear();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMap.java
deleted file mode 100644
index 0514333..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMap.java
+++ /dev/null
@@ -1,530 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.ConcurrentModificationException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Predicate;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ContainsKey;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ContainsValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GetOrDefault;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.RemoveVersion;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Replace;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ReplaceValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ReplaceVersion;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionRollback;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.ConsistentMapException;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.BEGIN;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CLOSE_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CONTAINS_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CONTAINS_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ENTRY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET_OR_DEFAULT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IteratorBatch;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IteratorPosition;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.KEY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PREPARE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PREPARE_AND_COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT_IF_ABSENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_VERSION;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE_VERSION;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ROLLBACK;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.SIZE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.VALUES;
-
-/**
- * Distributed resource providing the {@link AsyncConsistentMap} primitive.
- */
-public class AtomixConsistentMap extends AbstractRaftPrimitive implements AsyncConsistentMap<String, byte[]> {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixConsistentMapOperations.NAMESPACE)
- .register(AtomixConsistentMapEvents.NAMESPACE)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 100)
- .build());
-
- private final Map<MapEventListener<String, byte[]>, Executor> mapEventListeners = new ConcurrentHashMap<>();
-
- public AtomixConsistentMap(RaftProxy proxy) {
- super(proxy);
- proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleEvent);
- proxy.addStateChangeListener(state -> {
- if (state == RaftProxy.State.CONNECTED && isListening()) {
- proxy.invoke(ADD_LISTENER);
- }
- });
- }
-
- protected Serializer serializer() {
- return SERIALIZER;
- }
-
- private void handleEvent(List<MapEvent<String, byte[]>> events) {
- events.forEach(event ->
- mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event))));
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- return proxy.invoke(IS_EMPTY, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return proxy.invoke(SIZE, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(String key) {
- return proxy.invoke(CONTAINS_KEY, serializer()::encode, new ContainsKey(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(byte[] value) {
- return proxy.invoke(CONTAINS_VALUE, serializer()::encode, new ContainsValue(value), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Versioned<byte[]>> get(String key) {
- return proxy.invoke(GET, serializer()::encode, new Get(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Versioned<byte[]>> getOrDefault(String key, byte[] defaultValue) {
- return proxy.invoke(
- GET_OR_DEFAULT,
- serializer()::encode,
- new GetOrDefault(key, defaultValue),
- serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Set<String>> keySet() {
- return proxy.invoke(KEY_SET, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<byte[]>>> values() {
- return proxy.invoke(VALUES, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Set<Entry<String, Versioned<byte[]>>>> entrySet() {
- return proxy.invoke(ENTRY_SET, serializer()::decode);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Versioned<byte[]>> put(String key, byte[] value) {
- return proxy.<Put, MapEntryUpdateResult<String, byte[]>>invoke(
- PUT,
- serializer()::encode,
- new Put(key, value),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.result());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Versioned<byte[]>> putAndGet(String key, byte[] value) {
- return proxy.<Put, MapEntryUpdateResult<String, byte[]>>invoke(
- PUT_AND_GET,
- serializer()::encode,
- new Put(key, value),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.result());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Versioned<byte[]>> putIfAbsent(String key, byte[] value) {
- return proxy.<Put, MapEntryUpdateResult<String, byte[]>>invoke(
- PUT_IF_ABSENT,
- serializer()::encode,
- new Put(key, value),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.result());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Versioned<byte[]>> remove(String key) {
- return proxy.<Remove, MapEntryUpdateResult<String, byte[]>>invoke(
- REMOVE,
- serializer()::encode,
- new Remove(key),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.result());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Boolean> remove(String key, byte[] value) {
- return proxy.<RemoveValue, MapEntryUpdateResult<String, byte[]>>invoke(
- REMOVE_VALUE,
- serializer()::encode,
- new RemoveValue(key, value),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.updated());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Boolean> remove(String key, long version) {
- return proxy.<RemoveVersion, MapEntryUpdateResult<String, byte[]>>invoke(
- REMOVE_VERSION,
- serializer()::encode,
- new RemoveVersion(key, version),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.updated());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Versioned<byte[]>> replace(String key, byte[] value) {
- return proxy.<Replace, MapEntryUpdateResult<String, byte[]>>invoke(
- REPLACE,
- serializer()::encode,
- new Replace(key, value),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.result());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Boolean> replace(String key, byte[] oldValue, byte[] newValue) {
- return proxy.<ReplaceValue, MapEntryUpdateResult<String, byte[]>>invoke(
- REPLACE_VALUE,
- serializer()::encode,
- new ReplaceValue(key, oldValue, newValue),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.updated());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Boolean> replace(String key, long oldVersion, byte[] newValue) {
- return proxy.<ReplaceVersion, MapEntryUpdateResult<String, byte[]>>invoke(
- REPLACE_VERSION,
- serializer()::encode,
- new ReplaceVersion(key, oldVersion, newValue),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> v.updated());
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return proxy.<MapEntryUpdateResult.Status>invoke(CLEAR, serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenApply(v -> null);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public CompletableFuture<Versioned<byte[]>> computeIf(String key,
- Predicate<? super byte[]> condition,
- BiFunction<? super String, ? super byte[], ? extends byte[]> remappingFunction) {
- return get(key).thenCompose(r1 -> {
- byte[] existingValue = r1 == null ? null : r1.value();
- // if the condition evaluates to false, return existing value.
- if (!condition.test(existingValue)) {
- return CompletableFuture.completedFuture(r1);
- }
-
- byte[] computedValue;
- try {
- computedValue = remappingFunction.apply(key, existingValue);
- } catch (Exception e) {
- return Tools.exceptionalFuture(e);
- }
-
- if (computedValue == null && r1 == null) {
- return CompletableFuture.completedFuture(null);
- }
-
- if (r1 == null) {
- return proxy.<Put, MapEntryUpdateResult<String, byte[]>>invoke(
- PUT_IF_ABSENT,
- serializer()::encode,
- new Put(key, computedValue),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenCompose(r -> checkLocked(r))
- .thenApply(result -> new Versioned<>(computedValue, result.version()));
- } else if (computedValue == null) {
- return proxy.<RemoveVersion, MapEntryUpdateResult<String, byte[]>>invoke(
- REMOVE_VERSION,
- serializer()::encode,
- new RemoveVersion(key, r1.version()),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenCompose(r -> checkLocked(r))
- .thenApply(v -> null);
- } else {
- return proxy.<ReplaceVersion, MapEntryUpdateResult<String, byte[]>>invoke(
- REPLACE_VERSION,
- serializer()::encode,
- new ReplaceVersion(key, r1.version(), computedValue),
- serializer()::decode)
- .whenComplete((r, e) -> throwIfLocked(r))
- .thenCompose(r -> checkLocked(r))
- .thenApply(result -> result.status() == MapEntryUpdateResult.Status.OK
- ? new Versioned(computedValue, result.version()) : result.result());
- }
- });
- }
-
- private CompletableFuture<MapEntryUpdateResult<String, byte[]>> checkLocked(
- MapEntryUpdateResult<String, byte[]> result) {
- if (result.status() == MapEntryUpdateResult.Status.PRECONDITION_FAILED ||
- result.status() == MapEntryUpdateResult.Status.WRITE_LOCK) {
- return Tools.exceptionalFuture(new ConsistentMapException.ConcurrentModification());
- }
- return CompletableFuture.completedFuture(result);
- }
-
- @Override
- public synchronized CompletableFuture<Void> addListener(MapEventListener<String, byte[]> listener,
- Executor executor) {
- if (mapEventListeners.isEmpty()) {
- return proxy.invoke(ADD_LISTENER).thenRun(() -> mapEventListeners.put(listener, executor));
- } else {
- mapEventListeners.put(listener, executor);
- return CompletableFuture.completedFuture(null);
- }
- }
-
- @Override
- public synchronized CompletableFuture<Void> removeListener(MapEventListener<String, byte[]> listener) {
- if (mapEventListeners.remove(listener) != null && mapEventListeners.isEmpty()) {
- return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
- }
- return CompletableFuture.completedFuture(null);
- }
-
- private void throwIfLocked(MapEntryUpdateResult<String, byte[]> result) {
- if (result != null) {
- throwIfLocked(result.status());
- }
- }
-
- private void throwIfLocked(MapEntryUpdateResult.Status status) {
- if (status == MapEntryUpdateResult.Status.WRITE_LOCK) {
- throw new ConcurrentModificationException("Cannot update map: Another transaction in progress");
- }
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- return proxy.<TransactionBegin, Long>invoke(
- BEGIN,
- serializer()::encode,
- new TransactionBegin(transactionId),
- serializer()::decode)
- .thenApply(Version::new);
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
- return proxy.<TransactionPrepare, PrepareResult>invoke(
- PREPARE,
- serializer()::encode,
- new TransactionPrepare(transactionLog),
- serializer()::decode)
- .thenApply(v -> v == PrepareResult.OK);
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
- return proxy.<TransactionPrepareAndCommit, PrepareResult>invoke(
- PREPARE_AND_COMMIT,
- serializer()::encode,
- new TransactionPrepareAndCommit(transactionLog),
- serializer()::decode)
- .thenApply(v -> v == PrepareResult.OK);
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- return proxy.<TransactionCommit, CommitResult>invoke(
- COMMIT,
- serializer()::encode,
- new TransactionCommit(transactionId),
- serializer()::decode)
- .thenApply(v -> null);
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- return proxy.invoke(
- ROLLBACK,
- serializer()::encode,
- new TransactionRollback(transactionId),
- serializer()::decode)
- .thenApply(v -> null);
- }
-
- private boolean isListening() {
- return !mapEventListeners.isEmpty();
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Entry<String, Versioned<byte[]>>>> iterator() {
- return proxy.<Long>invoke(OPEN_ITERATOR, SERIALIZER::decode)
- .thenApply(ConsistentMultimapIterator::new);
- }
-
- /**
- * Consistent multimap iterator.
- */
- private class ConsistentMultimapIterator implements AsyncIterator<Map.Entry<String, Versioned<byte[]>>> {
- private final long id;
- private volatile CompletableFuture<IteratorBatch> batch;
- private volatile CompletableFuture<Void> closeFuture;
-
- ConsistentMultimapIterator(long id) {
- this.id = id;
- this.batch = CompletableFuture.completedFuture(
- new IteratorBatch(0, Collections.emptyList()));
- }
-
- /**
- * Returns the current batch iterator or lazily fetches the next batch from the cluster.
- *
- * @return the next batch iterator
- */
- private CompletableFuture<Iterator<Entry<String, Versioned<byte[]>>>> batch() {
- return batch.thenCompose(iterator -> {
- if (iterator != null && !iterator.hasNext()) {
- batch = fetch(iterator.position());
- return batch.thenApply(Function.identity());
- }
- return CompletableFuture.completedFuture(iterator);
- });
- }
-
- /**
- * Fetches the next batch of entries from the cluster.
- *
- * @param position the position from which to fetch the next batch
- * @return the next batch of entries from the cluster
- */
- private CompletableFuture<IteratorBatch> fetch(int position) {
- return proxy.<IteratorPosition, IteratorBatch>invoke(
- NEXT,
- SERIALIZER::encode,
- new IteratorPosition(id, position),
- SERIALIZER::decode)
- .thenCompose(batch -> {
- if (batch == null) {
- return close().thenApply(v -> null);
- }
- return CompletableFuture.completedFuture(batch);
- });
- }
-
- /**
- * Closes the iterator.
- *
- * @return future to be completed once the iterator has been closed
- */
- private CompletableFuture<Void> close() {
- if (closeFuture == null) {
- synchronized (this) {
- if (closeFuture == null) {
- closeFuture = proxy.invoke(CLOSE_ITERATOR, SERIALIZER::encode, id);
- }
- }
- }
- return closeFuture;
- }
-
- @Override
- public CompletableFuture<Boolean> hasNext() {
- return batch().thenApply(iterator -> iterator != null && iterator.hasNext());
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> next() {
- return batch().thenCompose(iterator -> {
- if (iterator == null) {
- return Tools.exceptionalFuture(new NoSuchElementException());
- }
- return CompletableFuture.completedFuture(iterator.next());
- });
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapEvents.java
deleted file mode 100644
index a0429da..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapEvents.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.MapEvent;
-
-/**
- * Atomix consistent map events.
- */
-public enum AtomixConsistentMapEvents implements EventType {
- CHANGE;
-
- @Override
- public String id() {
- return name();
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
- .register(MapEvent.class)
- .register(MapEvent.Type.class)
- .register(byte[].class)
- .build("AtomixConsistentMapEvents");
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapOperations.java
deleted file mode 100644
index 4f131ef..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapOperations.java
+++ /dev/null
@@ -1,648 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import io.atomix.utils.ArraySizeHashPrinter;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@link AtomixConsistentMap} resource state machine operations.
- */
-public enum AtomixConsistentMapOperations implements OperationId {
- IS_EMPTY(OperationType.QUERY),
- SIZE(OperationType.QUERY),
- CONTAINS_KEY(OperationType.QUERY),
- CONTAINS_VALUE(OperationType.QUERY),
- GET(OperationType.QUERY),
- GET_OR_DEFAULT(OperationType.QUERY),
- KEY_SET(OperationType.QUERY),
- VALUES(OperationType.QUERY),
- ENTRY_SET(OperationType.QUERY),
- PUT(OperationType.COMMAND),
- PUT_IF_ABSENT(OperationType.COMMAND),
- PUT_AND_GET(OperationType.COMMAND),
- REMOVE(OperationType.COMMAND),
- REMOVE_VALUE(OperationType.COMMAND),
- REMOVE_VERSION(OperationType.COMMAND),
- REPLACE(OperationType.COMMAND),
- REPLACE_VALUE(OperationType.COMMAND),
- REPLACE_VERSION(OperationType.COMMAND),
- CLEAR(OperationType.COMMAND),
- ADD_LISTENER(OperationType.COMMAND),
- REMOVE_LISTENER(OperationType.COMMAND),
- BEGIN(OperationType.COMMAND),
- PREPARE(OperationType.COMMAND),
- PREPARE_AND_COMMIT(OperationType.COMMAND),
- COMMIT(OperationType.COMMAND),
- ROLLBACK(OperationType.COMMAND),
- OPEN_ITERATOR(OperationType.COMMAND),
- NEXT(OperationType.QUERY),
- CLOSE_ITERATOR(OperationType.COMMAND);
-
- private final OperationType type;
-
- AtomixConsistentMapOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(ContainsKey.class)
- .register(ContainsValue.class)
- .register(Get.class)
- .register(GetOrDefault.class)
- .register(Put.class)
- .register(Remove.class)
- .register(RemoveValue.class)
- .register(RemoveVersion.class)
- .register(Replace.class)
- .register(ReplaceValue.class)
- .register(ReplaceVersion.class)
- .register(TransactionBegin.class)
- .register(TransactionPrepare.class)
- .register(TransactionPrepareAndCommit.class)
- .register(TransactionCommit.class)
- .register(TransactionRollback.class)
- .register(TransactionId.class)
- .register(TransactionLog.class)
- .register(MapUpdate.class)
- .register(MapUpdate.Type.class)
- .register(PrepareResult.class)
- .register(CommitResult.class)
- .register(RollbackResult.class)
- .register(Match.class)
- .register(MapEntryUpdateResult.class)
- .register(MapEntryUpdateResult.Status.class)
- .register(Versioned.class)
- .register(byte[].class)
- .register(Maps.immutableEntry("", "").getClass())
- .register(IteratorBatch.class)
- .register(IteratorPosition.class)
- .build("AtomixConsistentMapOperations");
-
- /**
- * Abstract map command.
- */
- @SuppressWarnings("serial")
- public abstract static class MapOperation {
- @Override
- public String toString() {
- return toStringHelper(getClass())
- .toString();
- }
- }
-
- /**
- * Abstract key-based query.
- */
- @SuppressWarnings("serial")
- public abstract static class KeyOperation extends MapOperation {
- protected String key;
-
- public KeyOperation() {
- }
-
- public KeyOperation(String key) {
- this.key = checkNotNull(key, "key cannot be null");
- }
-
- /**
- * Returns the key.
- * @return key
- */
- public String key() {
- return key;
- }
-
- @Override
- public String toString() {
- return toStringHelper(getClass())
- .add("key", key)
- .toString();
- }
- }
-
- /**
- * Abstract value-based query.
- */
- @SuppressWarnings("serial")
- public abstract static class ValueOperation extends MapOperation {
- protected byte[] value;
-
- public ValueOperation() {
- }
-
- public ValueOperation(byte[] value) {
- this.value = value;
- }
-
- /**
- * Returns the value.
- * @return value
- */
- public byte[] value() {
- return value;
- }
-
- @Override
- public String toString() {
- return toStringHelper(getClass())
- .add("value", value)
- .toString();
- }
- }
-
- /**
- * Abstract key/value operation.
- */
- @SuppressWarnings("serial")
- public abstract static class KeyValueOperation extends KeyOperation {
- protected byte[] value;
-
- public KeyValueOperation() {
- }
-
- public KeyValueOperation(String key, byte[] value) {
- super(key);
- this.value = value;
- }
-
- /**
- * Returns the value.
- * @return value
- */
- public byte[] value() {
- return value;
- }
-
- @Override
- public String toString() {
- return toStringHelper(getClass())
- .add("key", key)
- .add("value", ArraySizeHashPrinter.of(value))
- .toString();
- }
- }
-
- /**
- * Abstract key/version operation.
- */
- @SuppressWarnings("serial")
- public abstract static class KeyVersionOperation extends KeyOperation {
- protected long version;
-
- public KeyVersionOperation() {
- }
-
- public KeyVersionOperation(String key, long version) {
- super(key);
- this.version = version;
- }
-
- /**
- * Returns the version.
- * @return version
- */
- public long version() {
- return version;
- }
-
- @Override
- public String toString() {
- return toStringHelper(getClass())
- .add("key", key)
- .add("version", version)
- .toString();
- }
- }
-
- /**
- * Contains key command.
- */
- @SuppressWarnings("serial")
- public static class ContainsKey extends KeyOperation {
- public ContainsKey() {
- }
-
- public ContainsKey(String key) {
- super(key);
- }
- }
-
- /**
- * Contains value command.
- */
- @SuppressWarnings("serial")
- public static class ContainsValue extends ValueOperation {
- public ContainsValue() {
- }
-
- public ContainsValue(byte[] value) {
- super(value);
- }
- }
-
- /**
- * Map put operation.
- */
- public static class Put extends KeyValueOperation {
- public Put() {
- }
-
- public Put(String key, byte[] value) {
- super(key, value);
- }
- }
-
- /**
- * Remove operation.
- */
- public static class Remove extends KeyOperation {
- public Remove() {
- }
-
- public Remove(String key) {
- super(key);
- }
- }
-
- /**
- * Remove if value match operation.
- */
- public static class RemoveValue extends KeyValueOperation {
- public RemoveValue() {
- }
-
- public RemoveValue(String key, byte[] value) {
- super(key, value);
- }
- }
-
- /**
- * Remove if version match operation.
- */
- public static class RemoveVersion extends KeyVersionOperation {
- public RemoveVersion() {
- }
-
- public RemoveVersion(String key, long version) {
- super(key, version);
- }
- }
-
- /**
- * Replace operation.
- */
- public static class Replace extends KeyValueOperation {
- public Replace() {
- }
-
- public Replace(String key, byte[] value) {
- super(key, value);
- }
- }
-
- /**
- * Replace by value operation.
- */
- public static class ReplaceValue extends KeyOperation {
- private byte[] oldValue;
- private byte[] newValue;
-
- public ReplaceValue() {
- }
-
- public ReplaceValue(String key, byte[] oldValue, byte[] newValue) {
- super(key);
- this.oldValue = oldValue;
- this.newValue = newValue;
- }
-
- public byte[] oldValue() {
- return oldValue;
- }
-
- public byte[] newValue() {
- return newValue;
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("key", key)
- .add("oldValue", ArraySizeHashPrinter.of(oldValue))
- .add("newValue", ArraySizeHashPrinter.of(newValue))
- .toString();
- }
- }
-
- /**
- * Replace by version operation.
- */
- public static class ReplaceVersion extends KeyOperation {
- private long oldVersion;
- private byte[] newValue;
-
- public ReplaceVersion() {
- }
-
- public ReplaceVersion(String key, long oldVersion, byte[] newValue) {
- super(key);
- this.oldVersion = oldVersion;
- this.newValue = newValue;
- }
-
- public long oldVersion() {
- return oldVersion;
- }
-
- public byte[] newValue() {
- return newValue;
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("key", key)
- .add("oldVersion", oldVersion)
- .add("newValue", ArraySizeHashPrinter.of(newValue))
- .toString();
- }
- }
-
- /**
- * Transaction begin command.
- */
- public static class TransactionBegin extends MapOperation {
- private TransactionId transactionId;
-
- public TransactionBegin() {
- }
-
- public TransactionBegin(TransactionId transactionId) {
- this.transactionId = transactionId;
- }
-
- public TransactionId transactionId() {
- return transactionId;
- }
- }
-
- /**
- * Map prepare command.
- */
- @SuppressWarnings("serial")
- public static class TransactionPrepare extends MapOperation {
- private TransactionLog<MapUpdate<String, byte[]>> transactionLog;
-
- public TransactionPrepare() {
- }
-
- public TransactionPrepare(TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
- this.transactionLog = transactionLog;
- }
-
- public TransactionLog<MapUpdate<String, byte[]>> transactionLog() {
- return transactionLog;
- }
-
- @Override
- public String toString() {
- return toStringHelper(getClass())
- .add("transactionLog", transactionLog)
- .toString();
- }
- }
-
- /**
- * Map prepareAndCommit command.
- */
- @SuppressWarnings("serial")
- public static class TransactionPrepareAndCommit extends TransactionPrepare {
- public TransactionPrepareAndCommit() {
- }
-
- public TransactionPrepareAndCommit(TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
- super(transactionLog);
- }
- }
-
- /**
- * Map transaction commit command.
- */
- @SuppressWarnings("serial")
- public static class TransactionCommit extends MapOperation {
- private TransactionId transactionId;
-
- public TransactionCommit() {
- }
-
- public TransactionCommit(TransactionId transactionId) {
- this.transactionId = transactionId;
- }
-
- /**
- * Returns the transaction identifier.
- * @return transaction id
- */
- public TransactionId transactionId() {
- return transactionId;
- }
-
- @Override
- public String toString() {
- return toStringHelper(getClass())
- .add("transactionId", transactionId)
- .toString();
- }
- }
-
- /**
- * Map transaction rollback command.
- */
- @SuppressWarnings("serial")
- public static class TransactionRollback extends MapOperation {
- private TransactionId transactionId;
-
- public TransactionRollback() {
- }
-
- public TransactionRollback(TransactionId transactionId) {
- this.transactionId = transactionId;
- }
-
- /**
- * Returns the transaction identifier.
- * @return transaction id
- */
- public TransactionId transactionId() {
- return transactionId;
- }
-
- @Override
- public String toString() {
- return toStringHelper(getClass())
- .add("transactionId", transactionId)
- .toString();
- }
- }
-
- /**
- * Get query.
- */
- @SuppressWarnings("serial")
- public static class Get extends KeyOperation {
- public Get() {
- }
-
- public Get(String key) {
- super(key);
- }
- }
-
- /**
- * Get or default query.
- */
- @SuppressWarnings("serial")
- public static class GetOrDefault extends KeyOperation {
- private byte[] defaultValue;
-
- public GetOrDefault() {
- }
-
- public GetOrDefault(String key, byte[] defaultValue) {
- super(key);
- this.defaultValue = defaultValue;
- }
-
- /**
- * Returns the default value.
- *
- * @return the default value
- */
- public byte[] defaultValue() {
- return defaultValue;
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("key", key)
- .add("defaultValue", ArraySizeHashPrinter.of(defaultValue))
- .toString();
- }
- }
-
- /**
- * Iterator position.
- */
- public static class IteratorPosition {
- private long iteratorId;
- private int position;
-
- private IteratorPosition() {
- }
-
- public IteratorPosition(long iteratorId, int position) {
- this.iteratorId = iteratorId;
- this.position = position;
- }
-
- public long iteratorId() {
- return iteratorId;
- }
-
- public int position() {
- return position;
- }
- }
-
- /**
- * Iterator batch.
- */
- public static class IteratorBatch implements Iterator<Map.Entry<String, Versioned<byte[]>>> {
- private int position;
- private Collection<Map.Entry<String, Versioned<byte[]>>> entries;
- private transient volatile Iterator<Map.Entry<String, Versioned<byte[]>>> iterator;
-
- private IteratorBatch() {
- }
-
- public IteratorBatch(int position, Collection<Map.Entry<String, Versioned<byte[]>>> entries) {
- this.position = position;
- this.entries = entries;
- }
-
- public int position() {
- return position;
- }
-
- public Collection<Map.Entry<String, Versioned<byte[]>>> entries() {
- return entries;
- }
-
- private Iterator<Map.Entry<String, Versioned<byte[]>>> iterator() {
- Iterator<Map.Entry<String, Versioned<byte[]>>> iterator = this.iterator;
- if (iterator == null) {
- synchronized (entries) {
- iterator = this.iterator;
- if (iterator == null) {
- iterator = entries.iterator();
- this.iterator = iterator;
- }
- }
- }
- return iterator;
- }
-
- @Override
- public boolean hasNext() {
- return iterator().hasNext();
- }
-
- @Override
- public Map.Entry<String, Versioned<byte[]>> next() {
- return iterator().next();
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapService.java
deleted file mode 100644
index 0fcf3b5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapService.java
+++ /dev/null
@@ -1,1131 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ContainsKey;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ContainsValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GetOrDefault;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.RemoveVersion;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Replace;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ReplaceValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ReplaceVersion;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionRollback;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkState;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.BEGIN;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CLOSE_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CONTAINS_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CONTAINS_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ENTRY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET_OR_DEFAULT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IteratorBatch;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IteratorPosition;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.KEY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PREPARE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PREPARE_AND_COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT_IF_ABSENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_VERSION;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE_VERSION;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ROLLBACK;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.SIZE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.VALUES;
-
-/**
- * State Machine for {@link AtomixConsistentMap} resource.
- */
-public class AtomixConsistentMapService extends AbstractRaftService {
-
- private static final int MAX_ITERATOR_BATCH_SIZE = 1024 * 32;
-
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixConsistentMapOperations.NAMESPACE)
- .register(AtomixConsistentMapEvents.NAMESPACE)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 100)
- .register(TransactionScope.class)
- .register(TransactionLog.class)
- .register(TransactionId.class)
- .register(MapEntryValue.class)
- .register(MapEntryValue.Type.class)
- .register(new HashMap().keySet().getClass())
- .build());
-
- protected Map<Long, RaftSession> listeners = new LinkedHashMap<>();
- private Map<String, MapEntryValue> map;
- protected Set<String> preparedKeys = Sets.newHashSet();
- private Map<Long, IteratorContext> iterators = Maps.newHashMap();
- protected Map<TransactionId, TransactionScope> activeTransactions = Maps.newHashMap();
- protected long currentVersion;
-
- public AtomixConsistentMapService() {
- map = createMap();
- }
-
- protected Map<String, MapEntryValue> createMap() {
- return Maps.newConcurrentMap();
- }
-
- protected Map<String, MapEntryValue> entries() {
- return map;
- }
-
- protected Serializer serializer() {
- return SERIALIZER;
- }
-
- @Override
- public void snapshot(SnapshotWriter writer) {
- writer.writeObject(Sets.newHashSet(listeners.keySet()), serializer()::encode);
- writer.writeObject(preparedKeys, serializer()::encode);
- writer.writeObject(entries(), serializer()::encode);
- writer.writeObject(activeTransactions, serializer()::encode);
- writer.writeLong(currentVersion);
-
- Map<Long, Long> iterators = Maps.newHashMap();
- this.iterators.forEach((id, context) -> iterators.put(id, context.sessionId));
- writer.writeObject(iterators, serializer()::encode);
- }
-
- @Override
- public void install(SnapshotReader reader) {
- listeners = new LinkedHashMap<>();
- for (Long sessionId : reader.<Set<Long>>readObject(serializer()::decode)) {
- listeners.put(sessionId, sessions().getSession(sessionId));
- }
- preparedKeys = reader.readObject(serializer()::decode);
- map = reader.readObject(serializer()::decode);
- activeTransactions = reader.readObject(serializer()::decode);
- currentVersion = reader.readLong();
-
- Map<Long, Long> iterators = reader.readObject(serializer()::decode);
- this.iterators = Maps.newHashMap();
- iterators.forEach((id, session) ->
- this.iterators.put(id, new IteratorContext(session, entries().entrySet().iterator())));
- }
-
- @Override
- protected void configure(RaftServiceExecutor executor) {
- // Listeners
- executor.register(ADD_LISTENER, (Commit<Void> c) -> listen(c.session()));
- executor.register(REMOVE_LISTENER, (Commit<Void> c) -> unlisten(c.session()));
- // Queries
- executor.register(CONTAINS_KEY, serializer()::decode, this::containsKey, serializer()::encode);
- executor.register(CONTAINS_VALUE, serializer()::decode, this::containsValue, serializer()::encode);
- executor.register(ENTRY_SET, (Commit<Void> c) -> entrySet(), serializer()::encode);
- executor.register(GET, serializer()::decode, this::get, serializer()::encode);
- executor.register(GET_OR_DEFAULT, serializer()::decode, this::getOrDefault, serializer()::encode);
- executor.register(IS_EMPTY, (Commit<Void> c) -> isEmpty(), serializer()::encode);
- executor.register(KEY_SET, (Commit<Void> c) -> keySet(), serializer()::encode);
- executor.register(SIZE, (Commit<Void> c) -> size(), serializer()::encode);
- executor.register(VALUES, (Commit<Void> c) -> values(), serializer()::encode);
- // Commands
- executor.register(PUT, serializer()::decode, this::put, serializer()::encode);
- executor.register(PUT_IF_ABSENT, serializer()::decode, this::putIfAbsent, serializer()::encode);
- executor.register(PUT_AND_GET, serializer()::decode, this::putAndGet, serializer()::encode);
- executor.register(REMOVE, serializer()::decode, this::remove, serializer()::encode);
- executor.register(REMOVE_VALUE, serializer()::decode, this::removeValue, serializer()::encode);
- executor.register(REMOVE_VERSION, serializer()::decode, this::removeVersion, serializer()::encode);
- executor.register(REPLACE, serializer()::decode, this::replace, serializer()::encode);
- executor.register(REPLACE_VALUE, serializer()::decode, this::replaceValue, serializer()::encode);
- executor.register(REPLACE_VERSION, serializer()::decode, this::replaceVersion, serializer()::encode);
- executor.register(CLEAR, (Commit<Void> c) -> clear(), serializer()::encode);
- executor.register(BEGIN, serializer()::decode, this::begin, serializer()::encode);
- executor.register(PREPARE, serializer()::decode, this::prepare, serializer()::encode);
- executor.register(PREPARE_AND_COMMIT, serializer()::decode, this::prepareAndCommit, serializer()::encode);
- executor.register(COMMIT, serializer()::decode, this::commit, serializer()::encode);
- executor.register(ROLLBACK, serializer()::decode, this::rollback, serializer()::encode);
- executor.register(OPEN_ITERATOR, this::openIterator, serializer()::encode);
- executor.register(NEXT, serializer()::decode, this::next, serializer()::encode);
- executor.register(CLOSE_ITERATOR, serializer()::decode, this::closeIterator);
- }
-
- /**
- * Handles a contains key commit.
- *
- * @param commit containsKey commit
- * @return {@code true} if map contains key
- */
- protected boolean containsKey(Commit<? extends ContainsKey> commit) {
- MapEntryValue value = entries().get(commit.value().key());
- return value != null && value.type() != MapEntryValue.Type.TOMBSTONE;
- }
-
- /**
- * Handles a contains value commit.
- *
- * @param commit containsValue commit
- * @return {@code true} if map contains value
- */
- protected boolean containsValue(Commit<? extends ContainsValue> commit) {
- Match<byte[]> valueMatch = Match.ifValue(commit.value().value());
- return entries().values().stream()
- .filter(value -> value.type() != MapEntryValue.Type.TOMBSTONE)
- .anyMatch(value -> valueMatch.matches(value.value()));
- }
-
- /**
- * Handles a get commit.
- *
- * @param commit get commit
- * @return value mapped to key
- */
- protected Versioned<byte[]> get(Commit<? extends Get> commit) {
- return toVersioned(entries().get(commit.value().key()));
- }
-
- /**
- * Handles a get or default commit.
- *
- * @param commit get or default commit
- * @return value mapped to key
- */
- protected Versioned<byte[]> getOrDefault(Commit<? extends GetOrDefault> commit) {
- MapEntryValue value = entries().get(commit.value().key());
- if (value == null) {
- return new Versioned<>(commit.value().defaultValue(), 0);
- } else if (value.type() == MapEntryValue.Type.TOMBSTONE) {
- return new Versioned<>(commit.value().defaultValue(), value.version);
- } else {
- return new Versioned<>(value.value(), value.version);
- }
- }
-
- /**
- * Handles a size commit.
- *
- * @return number of entries in map
- */
- protected int size() {
- return (int) entries().values().stream()
- .filter(value -> value.type() != MapEntryValue.Type.TOMBSTONE)
- .count();
- }
-
- /**
- * Handles an is empty commit.
- *
- * @return {@code true} if map is empty
- */
- protected boolean isEmpty() {
- return entries().values().stream()
- .noneMatch(value -> value.type() != MapEntryValue.Type.TOMBSTONE);
- }
-
- /**
- * Handles a keySet commit.
- *
- * @return set of keys in map
- */
- protected Set<String> keySet() {
- return entries().entrySet().stream()
- .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
- .map(Map.Entry::getKey)
- .collect(Collectors.toSet());
- }
-
- /**
- * Handles a values commit.
- *
- * @return collection of values in map
- */
- protected Collection<Versioned<byte[]>> values() {
- return entries().entrySet().stream()
- .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
- .map(entry -> toVersioned(entry.getValue()))
- .collect(Collectors.toList());
- }
-
- /**
- * Handles a entry set commit.
- *
- * @return set of map entries
- */
- protected Set<Map.Entry<String, Versioned<byte[]>>> entrySet() {
- return entries().entrySet().stream()
- .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
- .map(e -> Maps.immutableEntry(e.getKey(), toVersioned(e.getValue())))
- .collect(Collectors.toSet());
- }
-
- /**
- * Returns a boolean indicating whether the given MapEntryValues are equal.
- *
- * @param oldValue the first value to compare
- * @param newValue the second value to compare
- * @return indicates whether the two values are equal
- */
- protected boolean valuesEqual(MapEntryValue oldValue, MapEntryValue newValue) {
- return (oldValue == null && newValue == null)
- || (oldValue != null && newValue != null && valuesEqual(oldValue.value(), newValue.value()));
- }
-
- /**
- * Returns a boolean indicating whether the given entry values are equal.
- *
- * @param oldValue the first value to compare
- * @param newValue the second value to compare
- * @return indicates whether the two values are equal
- */
- protected boolean valuesEqual(byte[] oldValue, byte[] newValue) {
- return (oldValue == null && newValue == null)
- || (oldValue != null && newValue != null && Arrays.equals(oldValue, newValue));
- }
-
- /**
- * Returns a boolean indicating whether the given MapEntryValue is null or a tombstone.
- *
- * @param value the value to check
- * @return indicates whether the given value is null or is a tombstone
- */
- protected boolean valueIsNull(MapEntryValue value) {
- return value == null || value.type() == MapEntryValue.Type.TOMBSTONE;
- }
-
- /**
- * Handles a put commit.
- *
- * @param commit put commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> put(Commit<? extends Put> commit) {
- String key = commit.value().key();
- MapEntryValue oldValue = entries().get(key);
- MapEntryValue newValue = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().value());
-
- // If the value is null or a tombstone, this is an insert.
- // Otherwise, only update the value if it has changed to reduce the number of events.
- if (valueIsNull(oldValue)) {
- // If the key has been locked by a transaction, return a WRITE_LOCK error.
- if (preparedKeys.contains(key)) {
- return new MapEntryUpdateResult<>(
- MapEntryUpdateResult.Status.WRITE_LOCK,
- commit.index(),
- key,
- toVersioned(oldValue));
- }
- entries().put(commit.value().key(),
- new MapEntryValue(MapEntryValue.Type.VALUE, newValue.version(), newValue.value()));
- Versioned<byte[]> result = toVersioned(oldValue);
- publish(new MapEvent<>(MapEvent.Type.INSERT, "", key, toVersioned(newValue), result));
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, result);
- } else if (!valuesEqual(oldValue, newValue)) {
- // If the key has been locked by a transaction, return a WRITE_LOCK error.
- if (preparedKeys.contains(key)) {
- return new MapEntryUpdateResult<>(
- MapEntryUpdateResult.Status.WRITE_LOCK,
- commit.index(),
- key,
- toVersioned(oldValue));
- }
- entries().put(commit.value().key(),
- new MapEntryValue(MapEntryValue.Type.VALUE, newValue.version(), newValue.value()));
- Versioned<byte[]> result = toVersioned(oldValue);
- publish(new MapEvent<>(MapEvent.Type.UPDATE, "", key, toVersioned(newValue), result));
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, result);
- }
- // If the value hasn't changed, return a NOOP result.
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.NOOP, commit.index(), key, toVersioned(oldValue));
- }
-
- /**
- * Handles a putIfAbsent commit.
- *
- * @param commit putIfAbsent commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> putIfAbsent(Commit<? extends Put> commit) {
- String key = commit.value().key();
- MapEntryValue oldValue = entries().get(key);
-
- // If the value is null, this is an INSERT.
- if (valueIsNull(oldValue)) {
- // If the key has been locked by a transaction, return a WRITE_LOCK error.
- if (preparedKeys.contains(key)) {
- return new MapEntryUpdateResult<>(
- MapEntryUpdateResult.Status.WRITE_LOCK,
- commit.index(),
- key,
- toVersioned(oldValue));
- }
- MapEntryValue newValue = new MapEntryValue(
- MapEntryValue.Type.VALUE,
- commit.index(),
- commit.value().value());
- entries().put(commit.value().key(), newValue);
- Versioned<byte[]> result = toVersioned(newValue);
- publish(new MapEvent<>(MapEvent.Type.INSERT, "", key, result, null));
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, null);
- }
- return new MapEntryUpdateResult<>(
- MapEntryUpdateResult.Status.PRECONDITION_FAILED,
- commit.index(),
- key,
- toVersioned(oldValue));
- }
-
- /**
- * Handles a putAndGet commit.
- *
- * @param commit putAndGet commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> putAndGet(Commit<? extends Put> commit) {
- String key = commit.value().key();
- MapEntryValue oldValue = entries().get(key);
- MapEntryValue newValue = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().value());
-
- // If the value is null or a tombstone, this is an insert.
- // Otherwise, only update the value if it has changed to reduce the number of events.
- if (valueIsNull(oldValue)) {
- // If the key has been locked by a transaction, return a WRITE_LOCK error.
- if (preparedKeys.contains(key)) {
- return new MapEntryUpdateResult<>(
- MapEntryUpdateResult.Status.WRITE_LOCK,
- commit.index(),
- key,
- toVersioned(oldValue));
- }
- entries().put(commit.value().key(), newValue);
- Versioned<byte[]> result = toVersioned(newValue);
- publish(new MapEvent<>(MapEvent.Type.INSERT, "", key, result, null));
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, result);
- } else if (!valuesEqual(oldValue, newValue)) {
- // If the key has been locked by a transaction, return a WRITE_LOCK error.
- if (preparedKeys.contains(key)) {
- return new MapEntryUpdateResult<>(
- MapEntryUpdateResult.Status.WRITE_LOCK,
- commit.index(),
- key,
- toVersioned(oldValue));
- }
- entries().put(commit.value().key(), newValue);
- Versioned<byte[]> result = toVersioned(newValue);
- publish(new MapEvent<>(MapEvent.Type.UPDATE, "", key, result, toVersioned(oldValue)));
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, result);
- }
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.NOOP, commit.index(), key, toVersioned(oldValue));
- }
-
- /**
- * Handles a remove commit.
- *
- * @param index the commit index
- * @param key the key to remove
- * @param predicate predicate to determine whether to remove the entry
- * @return map entry update result
- */
- private MapEntryUpdateResult<String, byte[]> removeIf(long index, String key, Predicate<MapEntryValue> predicate) {
- MapEntryValue value = entries().get(key);
-
- // If the value does not exist or doesn't match the predicate, return a PRECONDITION_FAILED error.
- if (valueIsNull(value) || !predicate.test(value)) {
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.PRECONDITION_FAILED, index, key, null);
- }
-
- // If the key has been locked by a transaction, return a WRITE_LOCK error.
- if (preparedKeys.contains(key)) {
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.WRITE_LOCK, index, key, null);
- }
-
- // If no transactions are active, remove the key. Otherwise, replace it with a tombstone.
- if (activeTransactions.isEmpty()) {
- entries().remove(key);
- } else {
- entries().put(key, new MapEntryValue(MapEntryValue.Type.TOMBSTONE, index, null));
- }
-
- Versioned<byte[]> result = toVersioned(value);
- publish(new MapEvent<>(MapEvent.Type.REMOVE, "", key, null, result));
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, index, key, result);
- }
-
- /**
- * Handles a remove commit.
- *
- * @param commit remove commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> remove(Commit<? extends Remove> commit) {
- return removeIf(commit.index(), commit.value().key(), v -> true);
- }
-
- /**
- * Handles a removeValue commit.
- *
- * @param commit removeValue commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> removeValue(Commit<? extends RemoveValue> commit) {
- return removeIf(commit.index(), commit.value().key(), v ->
- valuesEqual(v, new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().value())));
- }
-
- /**
- * Handles a removeVersion commit.
- *
- * @param commit removeVersion commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> removeVersion(Commit<? extends RemoveVersion> commit) {
- return removeIf(commit.index(), commit.value().key(), v -> v.version() == commit.value().version());
- }
-
- /**
- * Handles a replace commit.
- *
- * @param index the commit index
- * @param key the key to replace
- * @param newValue the value with which to replace the key
- * @param predicate a predicate to determine whether to replace the key
- * @return map entry update result
- */
- private MapEntryUpdateResult<String, byte[]> replaceIf(
- long index, String key, MapEntryValue newValue, Predicate<MapEntryValue> predicate) {
- MapEntryValue oldValue = entries().get(key);
-
- // If the key is not set or the current value doesn't match the predicate, return a PRECONDITION_FAILED error.
- if (valueIsNull(oldValue) || !predicate.test(oldValue)) {
- return new MapEntryUpdateResult<>(
- MapEntryUpdateResult.Status.PRECONDITION_FAILED,
- index,
- key,
- toVersioned(oldValue));
- }
-
- // If the key has been locked by a transaction, return a WRITE_LOCK error.
- if (preparedKeys.contains(key)) {
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.WRITE_LOCK, index, key, null);
- }
-
- entries().put(key, newValue);
- Versioned<byte[]> result = toVersioned(oldValue);
- publish(new MapEvent<>(MapEvent.Type.UPDATE, "", key, toVersioned(newValue), result));
- return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, index, key, result);
- }
-
- /**
- * Handles a replace commit.
- *
- * @param commit replace commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> replace(Commit<? extends Replace> commit) {
- MapEntryValue value = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().value());
- return replaceIf(commit.index(), commit.value().key(), value, v -> true);
- }
-
- /**
- * Handles a replaceValue commit.
- *
- * @param commit replaceValue commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> replaceValue(Commit<? extends ReplaceValue> commit) {
- MapEntryValue value = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().newValue());
- return replaceIf(commit.index(), commit.value().key(), value,
- v -> valuesEqual(v.value(), commit.value().oldValue()));
- }
-
- /**
- * Handles a replaceVersion commit.
- *
- * @param commit replaceVersion commit
- * @return map entry update result
- */
- protected MapEntryUpdateResult<String, byte[]> replaceVersion(Commit<? extends ReplaceVersion> commit) {
- MapEntryValue value = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().newValue());
- return replaceIf(commit.index(), commit.value().key(), value,
- v -> v.version() == commit.value().oldVersion());
- }
-
- /**
- * Handles a clear commit.
- *
- * @return clear result
- */
- protected MapEntryUpdateResult.Status clear() {
- Iterator<Map.Entry<String, MapEntryValue>> iterator = entries().entrySet().iterator();
- Map<String, MapEntryValue> entriesToAdd = new HashMap<>();
- while (iterator.hasNext()) {
- Map.Entry<String, MapEntryValue> entry = iterator.next();
- String key = entry.getKey();
- MapEntryValue value = entry.getValue();
- if (!valueIsNull(value)) {
- Versioned<byte[]> removedValue = new Versioned<>(value.value(), value.version());
- publish(new MapEvent<>(MapEvent.Type.REMOVE, "", key, null, removedValue));
- if (activeTransactions.isEmpty()) {
- iterator.remove();
- } else {
- entriesToAdd.put(key, new MapEntryValue(MapEntryValue.Type.TOMBSTONE, value.version, null));
- }
- }
- }
- entries().putAll(entriesToAdd);
- return MapEntryUpdateResult.Status.OK;
- }
-
- /**
- * Handles an open iterator commit.
- *
- * @param commit the open iterator commit
- * @return iterator identifier
- */
- protected long openIterator(Commit<Void> commit) {
- iterators.put(commit.index(), new IteratorContext(
- commit.session().sessionId().id(),
- entries().entrySet().iterator()));
- return commit.index();
- }
-
- /**
- * Handles an iterator next commit.
- *
- * @param commit the next commit
- * @return a list of entries to iterate
- */
- protected IteratorBatch next(Commit<IteratorPosition> commit) {
- final long iteratorId = commit.value().iteratorId();
- final int position = commit.value().position();
-
- IteratorContext context = iterators.get(iteratorId);
- if (context == null) {
- return null;
- }
-
- List<Map.Entry<String, Versioned<byte[]>>> entries = new ArrayList<>();
- int size = 0;
- while (context.iterator.hasNext()) {
- context.position++;
- if (context.position > position) {
- Map.Entry<String, MapEntryValue> entry = context.iterator.next();
- String key = entry.getKey();
- Versioned<byte[]> value = toVersioned(entry.getValue());
- size += key.length();
- size += value.value() != null ? value.value().length : 0;
- entries.add(Maps.immutableEntry(key, value));
-
- if (size >= MAX_ITERATOR_BATCH_SIZE) {
- break;
- }
- }
- }
-
- if (entries.isEmpty()) {
- return null;
- }
- return new IteratorBatch(context.position, entries);
- }
-
- /**
- * Handles a close iterator commit.
- *
- * @param commit the close iterator commit
- */
- protected void closeIterator(Commit<Long> commit) {
- iterators.remove(commit.value());
- }
-
- /**
- * Handles a listen commit.
- *
- * @param session listen session
- */
- protected void listen(RaftSession session) {
- listeners.put(session.sessionId().id(), session);
- }
-
- /**
- * Handles an unlisten commit.
- *
- * @param session unlisten session
- */
- protected void unlisten(RaftSession session) {
- listeners.remove(session.sessionId().id());
- }
-
- /**
- * Handles a begin commit.
- *
- * @param commit transaction begin commit
- * @return transaction state version
- */
- protected long begin(Commit<? extends TransactionBegin> commit) {
- long version = commit.index();
- activeTransactions.put(commit.value().transactionId(), new TransactionScope(version));
- return version;
- }
-
- /**
- * Handles an prepare and commit commit.
- *
- * @param commit transaction prepare and commit commit
- * @return prepare result
- */
- protected PrepareResult prepareAndCommit(Commit<? extends TransactionPrepareAndCommit> commit) {
- TransactionId transactionId = commit.value().transactionLog().transactionId();
- PrepareResult prepareResult = prepare(commit);
- TransactionScope transactionScope = activeTransactions.remove(transactionId);
- if (prepareResult == PrepareResult.OK) {
- this.currentVersion = commit.index();
- transactionScope = transactionScope.prepared(commit);
- commitTransaction(transactionScope);
- }
- discardTombstones();
- return prepareResult;
- }
-
- /**
- * Handles an prepare commit.
- *
- * @param commit transaction prepare commit
- * @return prepare result
- */
- protected PrepareResult prepare(Commit<? extends TransactionPrepare> commit) {
- try {
- TransactionLog<MapUpdate<String, byte[]>> transactionLog = commit.value().transactionLog();
-
- // Iterate through records in the transaction log and perform isolation checks.
- for (MapUpdate<String, byte[]> record : transactionLog.records()) {
- String key = record.key();
-
- // If the record is a VERSION_MATCH then check that the record's version matches the current
- // version of the state machine.
- if (record.type() == MapUpdate.Type.VERSION_MATCH && key == null) {
- if (record.version() > currentVersion) {
- return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
- } else {
- continue;
- }
- }
-
- // If the prepared keys already contains the key contained within the record, that indicates a
- // conflict with a concurrent transaction.
- if (preparedKeys.contains(key)) {
- return PrepareResult.CONCURRENT_TRANSACTION;
- }
-
- // Read the existing value from the map.
- MapEntryValue existingValue = entries().get(key);
-
- // Note: if the existing value is null, that means the key has not changed during the transaction,
- // otherwise a tombstone would have been retained.
- if (existingValue == null) {
- // If the value is null, ensure the version is equal to the transaction version.
- if (record.version() != transactionLog.version()) {
- return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
- }
- } else {
- // If the value is non-null, compare the current version with the record version.
- if (existingValue.version() > record.version()) {
- return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
- }
- }
- }
-
- // No violations detected. Mark modified keys locked for transactions.
- transactionLog.records().forEach(record -> {
- if (record.type() != MapUpdate.Type.VERSION_MATCH) {
- preparedKeys.add(record.key());
- }
- });
-
- // Update the transaction scope. If the transaction scope is not set on this node, that indicates the
- // coordinator is communicating with another node. Transactions assume that the client is communicating
- // with a single leader in order to limit the overhead of retaining tombstones.
- TransactionScope transactionScope = activeTransactions.get(transactionLog.transactionId());
- if (transactionScope == null) {
- activeTransactions.put(
- transactionLog.transactionId(),
- new TransactionScope(transactionLog.version(), commit.value().transactionLog()));
- return PrepareResult.PARTIAL_FAILURE;
- } else {
- activeTransactions.put(
- transactionLog.transactionId(),
- transactionScope.prepared(commit));
- return PrepareResult.OK;
- }
- } catch (Exception e) {
- logger().warn("Failure applying {}", commit, e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Handles an commit commit (ha!).
- *
- * @param commit transaction commit commit
- * @return commit result
- */
- protected CommitResult commit(Commit<? extends TransactionCommit> commit) {
- TransactionId transactionId = commit.value().transactionId();
- TransactionScope transactionScope = activeTransactions.remove(transactionId);
- if (transactionScope == null) {
- return CommitResult.UNKNOWN_TRANSACTION_ID;
- }
-
- try {
- this.currentVersion = commit.index();
- return commitTransaction(transactionScope);
- } catch (Exception e) {
- logger().warn("Failure applying {}", commit, e);
- throw new IllegalStateException(e);
- } finally {
- discardTombstones();
- }
- }
-
- /**
- * Applies committed operations to the state machine.
- */
- private CommitResult commitTransaction(TransactionScope transactionScope) {
- TransactionLog<MapUpdate<String, byte[]>> transactionLog = transactionScope.transactionLog();
- boolean retainTombstones = !activeTransactions.isEmpty();
-
- List<MapEvent<String, byte[]>> eventsToPublish = Lists.newArrayList();
- for (MapUpdate<String, byte[]> record : transactionLog.records()) {
- if (record.type() == MapUpdate.Type.VERSION_MATCH) {
- continue;
- }
-
- String key = record.key();
- checkState(preparedKeys.remove(key), "key is not prepared");
-
- if (record.type() == MapUpdate.Type.LOCK) {
- continue;
- }
-
- MapEntryValue previousValue = entries().remove(key);
- MapEntryValue newValue = null;
-
- // If the record is not a delete, create a transactional commit.
- if (record.type() != MapUpdate.Type.REMOVE_IF_VERSION_MATCH) {
- newValue = new MapEntryValue(MapEntryValue.Type.VALUE, currentVersion, record.value());
- } else if (retainTombstones) {
- // For deletes, if tombstones need to be retained then create and store a tombstone commit.
- newValue = new MapEntryValue(MapEntryValue.Type.TOMBSTONE, currentVersion, null);
- }
-
- MapEvent<String, byte[]> event;
- if (newValue != null) {
- entries().put(key, newValue);
- if (!valueIsNull(newValue)) {
- if (!valueIsNull(previousValue)) {
- event = new MapEvent<>(
- MapEvent.Type.UPDATE,
- "",
- key,
- toVersioned(newValue),
- toVersioned(previousValue));
- } else {
- event = new MapEvent<>(
- MapEvent.Type.INSERT,
- "",
- key,
- toVersioned(newValue),
- null);
- }
- } else {
- event = new MapEvent<>(
- MapEvent.Type.REMOVE,
- "",
- key,
- null,
- toVersioned(previousValue));
- }
- } else {
- event = new MapEvent<>(
- MapEvent.Type.REMOVE,
- "",
- key,
- null,
- toVersioned(previousValue));
- }
- eventsToPublish.add(event);
- }
- publish(eventsToPublish);
- return CommitResult.OK;
- }
-
- /**
- * Handles an rollback commit (ha!).
- *
- * @param commit transaction rollback commit
- * @return rollback result
- */
- protected RollbackResult rollback(Commit<? extends TransactionRollback> commit) {
- TransactionId transactionId = commit.value().transactionId();
- TransactionScope transactionScope = activeTransactions.remove(transactionId);
- if (transactionScope == null) {
- return RollbackResult.UNKNOWN_TRANSACTION_ID;
- } else if (!transactionScope.isPrepared()) {
- discardTombstones();
- return RollbackResult.OK;
- } else {
- try {
- transactionScope.transactionLog().records()
- .forEach(record -> {
- if (record.type() != MapUpdate.Type.VERSION_MATCH) {
- preparedKeys.remove(record.key());
- }
- });
- return RollbackResult.OK;
- } finally {
- discardTombstones();
- }
- }
-
- }
-
- /**
- * Discards tombstones no longer needed by active transactions.
- */
- private void discardTombstones() {
- if (activeTransactions.isEmpty()) {
- Iterator<Map.Entry<String, MapEntryValue>> iterator = entries().entrySet().iterator();
- while (iterator.hasNext()) {
- MapEntryValue value = iterator.next().getValue();
- if (value.type() == MapEntryValue.Type.TOMBSTONE) {
- iterator.remove();
- }
- }
- } else {
- long lowWaterMark = activeTransactions.values().stream()
- .mapToLong(TransactionScope::version)
- .min().getAsLong();
- Iterator<Map.Entry<String, MapEntryValue>> iterator = entries().entrySet().iterator();
- while (iterator.hasNext()) {
- MapEntryValue value = iterator.next().getValue();
- if (value.type() == MapEntryValue.Type.TOMBSTONE && value.version < lowWaterMark) {
- iterator.remove();
- }
- }
- }
- }
-
- /**
- * Utility for turning a {@code MapEntryValue} to {@code Versioned}.
- *
- * @param value map entry value
- * @return versioned instance
- */
- protected Versioned<byte[]> toVersioned(MapEntryValue value) {
- return value != null && value.type() != MapEntryValue.Type.TOMBSTONE
- ? new Versioned<>(value.value(), value.version()) : null;
- }
-
- /**
- * Publishes an event to listeners.
- *
- * @param event event to publish
- */
- private void publish(MapEvent<String, byte[]> event) {
- publish(Lists.newArrayList(event));
- }
-
- /**
- * Publishes events to listeners.
- *
- * @param events list of map event to publish
- */
- private void publish(List<MapEvent<String, byte[]>> events) {
- listeners.values().forEach(session -> {
- session.publish(CHANGE, serializer()::encode, events);
- });
- }
-
- @Override
- public void onExpire(RaftSession session) {
- closeListener(session.sessionId().id());
- }
-
- @Override
- public void onClose(RaftSession session) {
- closeListener(session.sessionId().id());
- }
-
- private void closeListener(Long sessionId) {
- listeners.remove(sessionId);
- }
-
- /**
- * Interface implemented by map values.
- */
- protected static class MapEntryValue {
- protected final Type type;
- protected final long version;
- protected final byte[] value;
-
- MapEntryValue(Type type, long version, byte[] value) {
- this.type = type;
- this.version = version;
- this.value = value;
- }
-
- /**
- * Returns the value type.
- *
- * @return the value type
- */
- Type type() {
- return type;
- }
-
- /**
- * Returns the version of the value.
- *
- * @return version
- */
- long version() {
- return version;
- }
-
- /**
- * Returns the raw {@code byte[]}.
- *
- * @return raw value
- */
- byte[] value() {
- return value;
- }
-
- /**
- * Value type.
- */
- enum Type {
- VALUE,
- TOMBSTONE,
- }
- }
-
- /**
- * Map transaction scope.
- */
- protected static final class TransactionScope {
- private final long version;
- private final TransactionLog<MapUpdate<String, byte[]>> transactionLog;
-
- private TransactionScope(long version) {
- this(version, null);
- }
-
- private TransactionScope(long version, TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
- this.version = version;
- this.transactionLog = transactionLog;
- }
-
- /**
- * Returns the transaction version.
- *
- * @return the transaction version
- */
- long version() {
- return version;
- }
-
- /**
- * Returns whether this is a prepared transaction scope.
- *
- * @return whether this is a prepared transaction scope
- */
- boolean isPrepared() {
- return transactionLog != null;
- }
-
- /**
- * Returns the transaction commit log.
- *
- * @return the transaction commit log
- */
- TransactionLog<MapUpdate<String, byte[]>> transactionLog() {
- checkState(isPrepared());
- return transactionLog;
- }
-
- /**
- * Returns a new transaction scope with a prepare commit.
- *
- * @param commit the prepare commit
- * @return new transaction scope updated with the prepare commit
- */
- TransactionScope prepared(Commit<? extends TransactionPrepare> commit) {
- return new TransactionScope(version, commit.value().transactionLog());
- }
- }
-
- private static class IteratorContext {
- private final long sessionId;
- private final Iterator<Map.Entry<String, MapEntryValue>> iterator;
- private int position = 0;
-
- IteratorContext(long sessionId, Iterator<Map.Entry<String, MapEntryValue>> iterator) {
- this.sessionId = sessionId;
- this.iterator = iterator;
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimap.java
deleted file mode 100644
index b04c9dd..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimap.java
+++ /dev/null
@@ -1,354 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.ConcurrentModificationException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.function.Function;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multiset;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Tools;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CLOSE_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsValue;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ENTRIES;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Get;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IteratorBatch;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IteratorPosition;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.KEYS;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.KEY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.MultiRemove;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Put;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_ALL;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.RemoveAll;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Replace;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.SIZE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.VALUES;
-
-
-/**
- * Set based implementation of the {@link AsyncConsistentMultimap}.
- * <p>
- * Note: this implementation does not allow null entries or duplicate entries.
- */
-public class AtomixConsistentSetMultimap
- extends AbstractRaftPrimitive
- implements AsyncConsistentMultimap<String, byte[]> {
-
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixConsistentSetMultimapOperations.NAMESPACE)
- .register(AtomixConsistentSetMultimapEvents.NAMESPACE)
- .build());
-
- private final Map<MultimapEventListener<String, byte[]>, Executor> mapEventListeners = new ConcurrentHashMap<>();
-
- public AtomixConsistentSetMultimap(RaftProxy proxy) {
- super(proxy);
- proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleChange);
- proxy.addStateChangeListener(state -> {
- if (state == RaftProxy.State.CONNECTED && isListening()) {
- proxy.invoke(ADD_LISTENER);
- }
- });
- }
-
- private void handleChange(List<MultimapEvent<String, byte[]>> events) {
- events.forEach(event ->
- mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event))));
- }
-
- @Override
- public CompletableFuture<Integer> size() {
- return proxy.invoke(SIZE, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> isEmpty() {
- return proxy.invoke(IS_EMPTY, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(String key) {
- return proxy.invoke(CONTAINS_KEY, SERIALIZER::encode, new ContainsKey(key), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(byte[] value) {
- return proxy.invoke(CONTAINS_VALUE, SERIALIZER::encode, new ContainsValue(value), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> containsEntry(String key, byte[] value) {
- return proxy.invoke(CONTAINS_ENTRY, SERIALIZER::encode, new ContainsEntry(key, value), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> put(String key, byte[] value) {
- return proxy.invoke(
- PUT,
- SERIALIZER::encode,
- new Put(key, Lists.newArrayList(value), null),
- SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends byte[]>>> putAndGet(String key, byte[] value) {
- return proxy.invoke(
- PUT_AND_GET,
- SERIALIZER::encode,
- new Put(key, Lists.newArrayList(value), null),
- SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> remove(String key, byte[] value) {
- return proxy.invoke(REMOVE, SERIALIZER::encode, new MultiRemove(key,
- Lists.newArrayList(value),
- null), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends byte[]>>> removeAndGet(String key, byte[] value) {
- return proxy.invoke(REMOVE_AND_GET, SERIALIZER::encode, new MultiRemove(key,
- Lists.newArrayList(value),
- null), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> removeAll(String key, Collection<? extends byte[]> values) {
- return proxy.invoke(
- REMOVE,
- SERIALIZER::encode,
- new MultiRemove(key, (Collection<byte[]>) values, null),
- SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends byte[]>>> removeAll(String key) {
- return proxy.invoke(REMOVE_ALL, SERIALIZER::encode, new RemoveAll(key, null), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> putAll(
- String key, Collection<? extends byte[]> values) {
- return proxy.invoke(PUT, SERIALIZER::encode, new Put(key, values, null), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends byte[]>>> replaceValues(
- String key, Collection<byte[]> values) {
- return proxy.invoke(
- REPLACE,
- SERIALIZER::encode,
- new Replace(key, values, null),
- SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- return proxy.invoke(CLEAR);
- }
-
- @Override
- public CompletableFuture<Versioned<Collection<? extends byte[]>>> get(String key) {
- return proxy.invoke(GET, SERIALIZER::encode, new Get(key), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Set<String>> keySet() {
- return proxy.invoke(KEY_SET, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Multiset<String>> keys() {
- return proxy.invoke(KEYS, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Multiset<byte[]>> values() {
- return proxy.invoke(VALUES, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Collection<Map.Entry<String, byte[]>>> entries() {
- return proxy.invoke(ENTRIES, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<AsyncIterator<Map.Entry<String, byte[]>>> iterator() {
- return proxy.<Long>invoke(OPEN_ITERATOR, SERIALIZER::decode).thenApply(ConsistentMultimapIterator::new);
- }
-
- /**
- * Consistent multimap iterator.
- */
- private class ConsistentMultimapIterator implements AsyncIterator<Map.Entry<String, byte[]>> {
- private final long id;
- private volatile CompletableFuture<IteratorBatch> batch;
- private volatile CompletableFuture<Void> closeFuture;
-
- ConsistentMultimapIterator(long id) {
- this.id = id;
- this.batch = CompletableFuture.completedFuture(
- new IteratorBatch(0, Collections.emptyList()));
- }
-
- /**
- * Returns the current batch iterator or lazily fetches the next batch from the cluster.
- *
- * @return the next batch iterator
- */
- private CompletableFuture<Iterator<Map.Entry<String, byte[]>>> batch() {
- return batch.thenCompose(iterator -> {
- if (iterator != null && !iterator.hasNext()) {
- batch = fetch(iterator.position());
- return batch.thenApply(Function.identity());
- }
- return CompletableFuture.completedFuture(iterator);
- });
- }
-
- /**
- * Fetches the next batch of entries from the cluster.
- *
- * @param position the position from which to fetch the next batch
- * @return the next batch of entries from the cluster
- */
- private CompletableFuture<IteratorBatch> fetch(int position) {
- return proxy.<IteratorPosition, IteratorBatch>invoke(
- NEXT,
- SERIALIZER::encode,
- new IteratorPosition(id, position),
- SERIALIZER::decode)
- .thenCompose(batch -> {
- if (batch == null) {
- return close().thenApply(v -> null);
- }
- return CompletableFuture.completedFuture(batch);
- });
- }
-
- /**
- * Closes the iterator.
- *
- * @return future to be completed once the iterator has been closed
- */
- private CompletableFuture<Void> close() {
- if (closeFuture == null) {
- synchronized (this) {
- if (closeFuture == null) {
- closeFuture = proxy.invoke(CLOSE_ITERATOR, SERIALIZER::encode, id);
- }
- }
- }
- return closeFuture;
- }
-
- @Override
- public CompletableFuture<Boolean> hasNext() {
- return batch().thenApply(iterator -> iterator != null && iterator.hasNext());
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, byte[]>> next() {
- return batch().thenCompose(iterator -> {
- if (iterator == null) {
- return Tools.exceptionalFuture(new NoSuchElementException());
- }
- return CompletableFuture.completedFuture(iterator.next());
- });
- }
- }
-
- @Override
- public CompletableFuture<Void> addListener(MultimapEventListener<String, byte[]> listener, Executor executor) {
- if (mapEventListeners.isEmpty()) {
- return proxy.invoke(ADD_LISTENER).thenRun(() -> mapEventListeners.put(listener, executor));
- } else {
- mapEventListeners.put(listener, executor);
- return CompletableFuture.completedFuture(null);
- }
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MultimapEventListener<String, byte[]> listener) {
- if (mapEventListeners.remove(listener) != null && mapEventListeners.isEmpty()) {
- return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
- }
- return CompletableFuture.completedFuture(null);
- }
-
- @Override
- public CompletableFuture<Map<String, Collection<byte[]>>> asMap() {
- throw new UnsupportedOperationException("Expensive operation.");
- }
-
- /**
- * Helper to check if there was a lock based issue.
- *
- * @param status the status of an update result
- */
- private void throwIfLocked(MapEntryUpdateResult.Status status) {
- if (status == MapEntryUpdateResult.Status.WRITE_LOCK) {
- throw new ConcurrentModificationException("Cannot update map: " +
- "Another transaction " +
- "in progress");
- }
- }
-
- private boolean isListening() {
- return !mapEventListeners.isEmpty();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapEvents.java
deleted file mode 100644
index 83908ae..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapEvents.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.MultimapEvent;
-
-/**
- * Atomix consistent set multimap events.
- */
-public enum AtomixConsistentSetMultimapEvents implements EventType {
- CHANGE,
- ENTRY;
-
- @Override
- public String id() {
- return name();
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
- .register(MultimapEvent.class)
- .register(MultimapEvent.Type.class)
- .register(byte[].class)
- .build("AtomixConsistentSetMultimapEvents");
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapOperations.java
deleted file mode 100644
index 4d38d71..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapOperations.java
+++ /dev/null
@@ -1,464 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * AsyncConsistentMultimap state machine commands.
- */
-public enum AtomixConsistentSetMultimapOperations implements OperationId {
- GET(OperationType.QUERY),
- SIZE(OperationType.QUERY),
- IS_EMPTY(OperationType.QUERY),
- CONTAINS_KEY(OperationType.QUERY),
- CONTAINS_VALUE(OperationType.QUERY),
- CONTAINS_ENTRY(OperationType.QUERY),
- KEY_SET(OperationType.QUERY),
- KEYS(OperationType.QUERY),
- VALUES(OperationType.QUERY),
- ENTRIES(OperationType.QUERY),
- PUT(OperationType.COMMAND),
- PUT_AND_GET(OperationType.COMMAND),
- REMOVE(OperationType.COMMAND),
- REMOVE_AND_GET(OperationType.COMMAND),
- REMOVE_ALL(OperationType.COMMAND),
- REPLACE(OperationType.COMMAND),
- CLEAR(OperationType.COMMAND),
- ADD_LISTENER(OperationType.COMMAND),
- REMOVE_LISTENER(OperationType.COMMAND),
- OPEN_ITERATOR(OperationType.COMMAND),
- NEXT(OperationType.QUERY),
- CLOSE_ITERATOR(OperationType.COMMAND);
-
- private final OperationType type;
-
- AtomixConsistentSetMultimapOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(ContainsEntry.class)
- .register(ContainsKey.class)
- .register(ContainsValue.class)
- .register(Get.class)
- .register(MultiRemove.class)
- .register(Put.class)
- .register(RemoveAll.class)
- .register(Replace.class)
- .register(Match.class)
- .register(Versioned.class)
- .register(ArrayList.class)
- .register(Maps.immutableEntry("", "").getClass())
- .register(IteratorBatch.class)
- .register(IteratorPosition.class)
- .build("AtomixConsistentSetMultimapOperations");
-
- /**
- * Abstract multimap command.
- */
- @SuppressWarnings("serial")
- public abstract static class MultimapOperation {
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .toString();
- }
- }
-
- /**
- * Abstract key-based multimap query.
- */
- @SuppressWarnings("serial")
- public abstract static class KeyOperation extends MultimapOperation {
- protected String key;
-
- public KeyOperation() {
- }
-
- public KeyOperation(String key) {
- this.key = checkNotNull(key);
- }
-
- public String key() {
- return key;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("key", key)
- .toString();
- }
- }
-
- /**
- * Abstract value-based query.
- */
- @SuppressWarnings("serial")
- public abstract static class ValueOperation extends MultimapOperation {
- protected byte[] value;
-
- public ValueOperation() {
- }
-
- public ValueOperation(byte[] value) {
- this.value = checkNotNull(value);
- }
-
- /**
- * Returns the value.
- *
- * @return value.
- */
- public byte[] value() {
- return value;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("value", value)
- .toString();
- }
- }
-
- /**
- * Contains key query.
- */
- @SuppressWarnings("serial")
- public static class ContainsKey extends KeyOperation {
- public ContainsKey() {
- }
-
- public ContainsKey(String key) {
- super(key);
- }
- }
-
- /**
- * Contains value query.
- */
- @SuppressWarnings("serial")
- public static class ContainsValue extends ValueOperation {
- public ContainsValue() {
- }
-
- public ContainsValue(byte[] value) {
- super(value);
- }
- }
-
- /**
- * Contains entry query.
- */
- @SuppressWarnings("serial")
- public static class ContainsEntry extends MultimapOperation {
- protected String key;
- protected byte[] value;
-
- public ContainsEntry() {
- }
-
- public ContainsEntry(String key, byte[] value) {
- this.key = checkNotNull(key);
- this.value = checkNotNull(value);
- }
-
- public String key() {
- return key;
- }
-
- public byte[] value() {
- return value;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("key", key)
- .add("value", value)
- .toString();
- }
- }
-
- /**
- * Remove command, backs remove and removeAll's that return booleans.
- */
- @SuppressWarnings("serial")
- public static class RemoveAll extends MultimapOperation {
- private String key;
- private Match<Long> versionMatch;
-
- public RemoveAll() {
- }
-
- public RemoveAll(String key, Match<Long> versionMatch) {
- this.key = checkNotNull(key);
- this.versionMatch = versionMatch;
- }
-
- public String key() {
- return this.key;
- }
-
- public Match<Long> versionMatch() {
- return versionMatch;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("key", key)
- .add("versionMatch", versionMatch)
- .toString();
- }
- }
-
- /**
- * Remove command, backs remove and removeAll's that return booleans.
- */
- @SuppressWarnings("serial")
- public static class MultiRemove extends MultimapOperation {
- private String key;
- private Collection<byte[]> values;
- private Match<Long> versionMatch;
-
- public MultiRemove() {
- }
-
- public MultiRemove(String key, Collection<byte[]> valueMatches,
- Match<Long> versionMatch) {
- this.key = checkNotNull(key);
- this.values = valueMatches;
- this.versionMatch = versionMatch;
- }
-
- public String key() {
- return this.key;
- }
-
- public Collection<byte[]> values() {
- return values;
- }
-
- public Match<Long> versionMatch() {
- return versionMatch;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("key", key)
- .add("values", values)
- .add("versionMatch", versionMatch)
- .toString();
- }
- }
-
- /**
- * Command to back the put and putAll methods.
- */
- @SuppressWarnings("serial")
- public static class Put extends MultimapOperation {
- private String key;
- private Collection<? extends byte[]> values;
- private Match<Long> versionMatch;
-
- public Put() {
- }
-
- public Put(String key, Collection<? extends byte[]> values, Match<Long> versionMatch) {
- this.key = checkNotNull(key);
- this.values = values;
- this.versionMatch = versionMatch;
- }
-
- public String key() {
- return key;
- }
-
- public Collection<? extends byte[]> values() {
- return values;
- }
-
- public Match<Long> versionMatch() {
- return versionMatch;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("key", key)
- .add("values", values)
- .add("versionMatch", versionMatch)
- .toString();
- }
- }
-
- /**
- * Replace command, returns the collection that was replaced.
- */
- @SuppressWarnings("serial")
- public static class Replace extends MultimapOperation {
- private String key;
- private Collection<byte[]> values;
- private Match<Long> versionMatch;
-
- public Replace() {
- }
-
- public Replace(String key, Collection<byte[]> values,
- Match<Long> versionMatch) {
- this.key = checkNotNull(key);
- this.values = values;
- this.versionMatch = versionMatch;
- }
-
- public String key() {
- return this.key;
- }
-
- public Match<Long> versionMatch() {
- return versionMatch;
- }
-
- public Collection<byte[]> values() {
- return values;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("key", key)
- .add("values", values)
- .add("versionMatch", versionMatch)
- .toString();
- }
- }
-
- /**
- * Get value query.
- */
- public static class Get extends KeyOperation {
- public Get() {
- }
-
- public Get(String key) {
- super(key);
- }
- }
-
- /**
- * Iterator position.
- */
- public static class IteratorPosition {
- private long iteratorId;
- private int position;
-
- private IteratorPosition() {
- }
-
- public IteratorPosition(long iteratorId, int position) {
- this.iteratorId = iteratorId;
- this.position = position;
- }
-
- public long iteratorId() {
- return iteratorId;
- }
-
- public int position() {
- return position;
- }
- }
-
- /**
- * Iterator batch.
- */
- public static class IteratorBatch implements Iterator<Map.Entry<String, byte[]>> {
- private int position;
- private Collection<Map.Entry<String, byte[]>> entries;
- private transient volatile Iterator<Map.Entry<String, byte[]>> iterator;
-
- private IteratorBatch() {
- }
-
- public IteratorBatch(int position, Collection<Map.Entry<String, byte[]>> entries) {
- this.position = position;
- this.entries = entries;
- }
-
- public int position() {
- return position;
- }
-
- public Collection<Map.Entry<String, byte[]>> entries() {
- return entries;
- }
-
- private Iterator<Map.Entry<String, byte[]>> iterator() {
- Iterator<Map.Entry<String, byte[]>> iterator = this.iterator;
- if (iterator == null) {
- synchronized (entries) {
- iterator = this.iterator;
- if (iterator == null) {
- iterator = entries.iterator();
- this.iterator = iterator;
- }
- }
- }
- return iterator;
- }
-
- @Override
- public boolean hasNext() {
- return iterator().hasNext();
- }
-
- @Override
- public Map.Entry<String, byte[]> next() {
- return iterator().next();
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapService.java
deleted file mode 100644
index fd2e676..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapService.java
+++ /dev/null
@@ -1,884 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.BiConsumer;
-import java.util.function.BinaryOperator;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collector;
-import java.util.stream.Collectors;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multiset;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CLOSE_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsValue;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ENTRIES;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Get;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IteratorBatch;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IteratorPosition;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.KEYS;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.KEY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.MultiRemove;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.MultimapOperation;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Put;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_ALL;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.RemoveAll;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Replace;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.SIZE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.VALUES;
-
-/**
- * State Machine for {@link AtomixConsistentSetMultimap} resource.
- */
-public class AtomixConsistentSetMultimapService extends AbstractRaftService {
- private static final int MAX_ITERATOR_BATCH_SIZE = 1024 * 32;
-
- private final Serializer serializer = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixConsistentSetMultimapOperations.NAMESPACE)
- .register(AtomixConsistentSetMultimapEvents.NAMESPACE)
- .register(ByteArrayComparator.class)
- .register(new HashMap().keySet().getClass())
- .register(TreeSet.class)
- .register(new com.esotericsoftware.kryo.Serializer<NonTransactionalCommit>() {
- @Override
- public void write(Kryo kryo, Output output, NonTransactionalCommit object) {
- kryo.writeClassAndObject(output, object.valueSet);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public NonTransactionalCommit read(Kryo kryo, Input input, Class<NonTransactionalCommit> type) {
- NonTransactionalCommit commit = new NonTransactionalCommit();
- commit.valueSet.addAll((Collection<byte[]>) kryo.readClassAndObject(input));
- return commit;
- }
- }, NonTransactionalCommit.class)
- .build());
-
- private AtomicLong globalVersion = new AtomicLong(1);
- private Map<Long, RaftSession> listeners = new LinkedHashMap<>();
- private Map<String, MapEntryValue> backingMap = Maps.newConcurrentMap();
- private Map<Long, IteratorContext> iterators = Maps.newHashMap();
-
- @Override
- public void snapshot(SnapshotWriter writer) {
- writer.writeLong(globalVersion.get());
- writer.writeObject(Sets.newHashSet(listeners.keySet()), serializer::encode);
- writer.writeObject(backingMap, serializer::encode);
-
- Map<Long, Long> iterators = Maps.newHashMap();
- this.iterators.forEach((id, context) -> iterators.put(id, context.sessionId));
- writer.writeObject(iterators, serializer::encode);
- }
-
- @Override
- public void install(SnapshotReader reader) {
- globalVersion = new AtomicLong(reader.readLong());
-
- listeners = new LinkedHashMap<>();
- for (Long sessionId : reader.<Set<Long>>readObject(serializer::decode)) {
- listeners.put(sessionId, sessions().getSession(sessionId));
- }
-
- backingMap = reader.readObject(serializer::decode);
-
- Map<Long, Long> iterators = reader.readObject(serializer::decode);
- this.iterators = Maps.newHashMap();
- iterators.forEach((id, session) ->
- this.iterators.put(id, new IteratorContext(session, backingMap.entrySet().iterator())));
- }
-
- @Override
- protected void configure(RaftServiceExecutor executor) {
- executor.register(SIZE, this::size, serializer::encode);
- executor.register(IS_EMPTY, this::isEmpty, serializer::encode);
- executor.register(CONTAINS_KEY, serializer::decode, this::containsKey, serializer::encode);
- executor.register(CONTAINS_VALUE, serializer::decode, this::containsValue, serializer::encode);
- executor.register(CONTAINS_ENTRY, serializer::decode, this::containsEntry, serializer::encode);
- executor.register(CLEAR, this::clear);
- executor.register(KEY_SET, this::keySet, serializer::encode);
- executor.register(KEYS, this::keys, serializer::encode);
- executor.register(VALUES, this::values, serializer::encode);
- executor.register(ENTRIES, this::entries, serializer::encode);
- executor.register(GET, serializer::decode, this::get, serializer::encode);
- executor.register(REMOVE_ALL, serializer::decode, this::removeAll, serializer::encode);
- executor.register(REMOVE, serializer::decode, this::multiRemove, serializer::encode);
- executor.register(REMOVE_AND_GET, serializer::decode, this::removeAndGet, serializer::encode);
- executor.register(PUT, serializer::decode, this::put, serializer::encode);
- executor.register(PUT_AND_GET, serializer::decode, this::putAndGet, serializer::encode);
- executor.register(REPLACE, serializer::decode, this::replace, serializer::encode);
- executor.register(ADD_LISTENER, this::listen);
- executor.register(REMOVE_LISTENER, this::unlisten);
- executor.register(OPEN_ITERATOR, this::openIterator, serializer::encode);
- executor.register(NEXT, serializer::decode, this::next, serializer::encode);
- executor.register(CLOSE_ITERATOR, serializer::decode, this::closeIterator);
- }
-
- @Override
- public void onExpire(RaftSession session) {
- listeners.remove(session.sessionId().id());
- iterators.entrySet().removeIf(entry -> entry.getValue().sessionId == session.sessionId().id());
- }
-
- @Override
- public void onClose(RaftSession session) {
- listeners.remove(session.sessionId().id());
- iterators.entrySet().removeIf(entry -> entry.getValue().sessionId == session.sessionId().id());
- }
-
- /**
- * Handles a Size commit.
- *
- * @param commit Size commit
- * @return number of unique key value pairs in the multimap
- */
- protected int size(Commit<Void> commit) {
- return backingMap.values()
- .stream()
- .map(valueCollection -> valueCollection.values().size())
- .collect(Collectors.summingInt(size -> size));
- }
-
- /**
- * Handles an IsEmpty commit.
- *
- * @param commit IsEmpty commit
- * @return true if the multimap contains no key-value pairs, else false
- */
- protected boolean isEmpty(Commit<Void> commit) {
- return backingMap.isEmpty();
- }
-
- /**
- * Handles a contains key commit.
- *
- * @param commit ContainsKey commit
- * @return returns true if the key is in the multimap, else false
- */
- protected boolean containsKey(Commit<? extends ContainsKey> commit) {
- return backingMap.containsKey(commit.value().key());
- }
-
- /**
- * Handles a ContainsValue commit.
- *
- * @param commit ContainsValue commit
- * @return true if the value is in the multimap, else false
- */
- protected boolean containsValue(Commit<? extends ContainsValue> commit) {
- if (backingMap.values().isEmpty()) {
- return false;
- }
- Match<byte[]> match = Match.ifValue(commit.value().value());
- return backingMap
- .values()
- .stream()
- .anyMatch(valueList ->
- valueList
- .values()
- .stream()
- .anyMatch(byteValue ->
- match.matches(byteValue)));
- }
-
- /**
- * Handles a ContainsEntry commit.
- *
- * @param commit ContainsEntry commit
- * @return true if the key-value pair exists, else false
- */
- protected boolean containsEntry(Commit<? extends ContainsEntry> commit) {
- MapEntryValue entryValue =
- backingMap.get(commit.value().key());
- if (entryValue == null) {
- return false;
- } else {
- Match valueMatch = Match.ifValue(commit.value().value());
- return entryValue
- .values()
- .stream()
- .anyMatch(byteValue -> valueMatch.matches(byteValue));
- }
- }
-
- /**
- * Handles a Clear commit.
- *
- * @param commit Clear commit
- */
- protected void clear(Commit<Void> commit) {
- backingMap.clear();
- }
-
- /**
- * Handles a KeySet commit.
- *
- * @param commit KeySet commit
- * @return a set of all keys in the multimap
- */
- protected Set<String> keySet(Commit<Void> commit) {
- return ImmutableSet.copyOf(backingMap.keySet());
- }
-
- /**
- * Handles a Keys commit.
- *
- * @param commit Keys commit
- * @return a multiset of keys with each key included an equal number of
- * times to the total key-value pairs in which that key participates
- */
- protected Multiset<String> keys(Commit<Void> commit) {
- Multiset keys = HashMultiset.create();
- backingMap.forEach((key, mapEntryValue) -> {
- keys.add(key, mapEntryValue.values().size());
- });
- return keys;
- }
-
- /**
- * Handles a Values commit.
- *
- * @param commit Values commit
- * @return the set of values in the multimap with duplicates included
- */
- protected Multiset<byte[]> values(Commit<Void> commit) {
- return backingMap
- .values()
- .stream()
- .collect(new HashMultisetValueCollector());
- }
-
- /**
- * Handles an Entries commit.
- *
- * @param commit Entries commit
- * @return a set of all key-value pairs in the multimap
- */
- protected Collection<Map.Entry<String, byte[]>> entries(Commit<Void> commit) {
- return backingMap
- .entrySet()
- .stream()
- .collect(new EntrySetCollector());
- }
-
- /**
- * Handles a Get commit.
- *
- * @param commit Get commit
- * @return the collection of values associated with the key or an empty
- * list if none exist
- */
- protected Versioned<Collection<? extends byte[]>> get(Commit<? extends Get> commit) {
- return toVersioned(backingMap.get(commit.value().key()));
- }
-
- /**
- * Handles a removeAll commit, and returns the previous mapping.
- *
- * @param commit removeAll commit
- * @return collection of removed values
- */
- protected Versioned<Collection<? extends byte[]>> removeAll(Commit<? extends RemoveAll> commit) {
- String key = commit.value().key();
-
- if (!backingMap.containsKey(key)) {
- return new Versioned<>(Sets.newHashSet(), -1);
- }
-
- Versioned<Collection<? extends byte[]>> removedValues =
- backingMap.get(key).addCommit(commit);
- publish(removedValues.value().stream()
- .map(value -> new MultimapEvent<String, byte[]>(
- "", key, null, value))
- .collect(Collectors.toList()));
- return removedValues;
- }
-
- /**
- * Handles a multiRemove commit, returns true if the remove results in any
- * change.
- * @param commit multiRemove commit
- * @return true if any change results, else false
- */
- protected boolean multiRemove(Commit<? extends MultiRemove> commit) {
- String key = commit.value().key();
-
- if (!backingMap.containsKey(key)) {
- return false;
- }
-
- Versioned<Collection<? extends byte[]>> removedValues = backingMap
- .get(key)
- .addCommit(commit);
-
- if (removedValues != null) {
- if (removedValues.value().isEmpty()) {
- backingMap.remove(key);
- }
-
- publish(removedValues.value().stream()
- .map(value -> new MultimapEvent<String, byte[]>(
- "", key, null, value))
- .collect(Collectors.toList()));
- return true;
- }
-
- return false;
- }
-
- /**
- * Handles a removeAndGet commit.
- *
- * @param commit multiRemove commit
- * @return the updated values or null if the values are empty
- */
- protected Versioned<Collection<? extends byte[]>> removeAndGet(Commit<? extends MultiRemove> commit) {
- String key = commit.value().key();
-
- if (!backingMap.containsKey(key)) {
- return null;
- }
-
- Versioned<Collection<? extends byte[]>> removedValues = backingMap
- .get(key)
- .addCommit(commit);
-
- if (removedValues != null) {
- if (removedValues.value().isEmpty()) {
- backingMap.remove(key);
- }
-
- publish(removedValues.value().stream()
- .map(value -> new MultimapEvent<String, byte[]>(
- "", key, null, value))
- .collect(Collectors.toList()));
- }
-
- return toVersioned(backingMap.get(key));
- }
-
- /**
- * Handles a put commit, returns true if any change results from this
- * commit.
- * @param commit a put commit
- * @return true if this commit results in a change, else false
- */
- protected boolean put(Commit<? extends Put> commit) {
- String key = commit.value().key();
- if (commit.value().values().isEmpty()) {
- return false;
- }
- if (!backingMap.containsKey(key)) {
- backingMap.put(key, new NonTransactionalCommit());
- }
-
- Versioned<Collection<? extends byte[]>> addedValues = backingMap
- .get(key)
- .addCommit(commit);
-
- if (addedValues != null) {
- publish(addedValues.value().stream()
- .map(value -> new MultimapEvent<String, byte[]>(
- "", key, value, null))
- .collect(Collectors.toList()));
- return true;
- }
-
- return false;
- }
-
- /**
- * Handles a putAndGet commit.
- *
- * @param commit a put commit
- * @return the updated values
- */
- protected Versioned<Collection<? extends byte[]>> putAndGet(Commit<? extends Put> commit) {
- String key = commit.value().key();
- if (commit.value().values().isEmpty()) {
- return null;
- }
- if (!backingMap.containsKey(key)) {
- backingMap.put(key, new NonTransactionalCommit());
- }
-
- Versioned<Collection<? extends byte[]>> addedValues = backingMap
- .get(key)
- .addCommit(commit);
-
- if (addedValues != null) {
- publish(addedValues.value().stream()
- .map(value -> new MultimapEvent<String, byte[]>(
- "", key, value, null))
- .collect(Collectors.toList()));
- }
-
- return toVersioned(backingMap.get(key));
- }
-
- protected Versioned<Collection<? extends byte[]>> replace(
- Commit<? extends Replace> commit) {
- String key = commit.value().key();
- if (!backingMap.containsKey(key)) {
- backingMap.put(key, new NonTransactionalCommit());
- }
-
- Versioned<Collection<? extends byte[]>> values = backingMap.get(commit.value().key()).addCommit(commit);
- if (values != null) {
- Set<byte[]> addedValues = Sets.newTreeSet(new ByteArrayComparator());
- addedValues.addAll(commit.value().values());
-
- Set<byte[]> removedValues = Sets.newTreeSet(new ByteArrayComparator());
- removedValues.addAll(values.value());
-
- List<MultimapEvent<String, byte[]>> events = Lists.newArrayList();
- Sets.difference(removedValues, addedValues)
- .forEach(value -> events.add(new MultimapEvent<>("", key, null, value)));
- Sets.difference(addedValues, removedValues)
- .forEach(value -> events.add(new MultimapEvent<>("", key, value, null)));
-
- publish(events);
- }
- return values;
- }
-
- /**
- * Handles a listen commit.
- *
- * @param commit listen commit
- */
- protected void listen(Commit<Void> commit) {
- listeners.put(commit.session().sessionId().id(), commit.session());
- }
-
- /**
- * Handles an unlisten commit.
- *
- * @param commit unlisten commit
- */
- protected void unlisten(Commit<Void> commit) {
- listeners.remove(commit.session().sessionId().id());
- }
-
- /**
- * Handles an open iterator commit.
- *
- * @param commit the open iterator commit
- * @return iterator identifier
- */
- protected long openIterator(Commit<Void> commit) {
- iterators.put(commit.index(), new IteratorContext(
- commit.session().sessionId().id(),
- backingMap.entrySet().iterator()));
- return commit.index();
- }
-
- /**
- * Handles an iterator next commit.
- *
- * @param commit the next commit
- * @return a list of entries to iterate
- */
- protected IteratorBatch next(Commit<IteratorPosition> commit) {
- final long iteratorId = commit.value().iteratorId();
- final int position = commit.value().position();
-
- IteratorContext context = iterators.get(iteratorId);
- if (context == null) {
- return null;
- }
-
- List<Map.Entry<String, byte[]>> entries = new ArrayList<>();
- int size = 0;
- while (context.iterator.hasNext()) {
- context.position++;
- if (context.position > position) {
- Map.Entry<String, MapEntryValue> entry = context.iterator.next();
- String key = entry.getKey();
- int keySize = key.length();
- for (byte[] value : entry.getValue().values()) {
- entries.add(Maps.immutableEntry(key, value));
- size += keySize;
- size += value.length;
- }
-
- if (size >= MAX_ITERATOR_BATCH_SIZE) {
- break;
- }
- }
- }
-
- if (entries.isEmpty()) {
- return null;
- }
- return new IteratorBatch(context.position, entries);
- }
-
- /**
- * Handles a close iterator commit.
- *
- * @param commit the close iterator commit
- */
- protected void closeIterator(Commit<Long> commit) {
- iterators.remove(commit.value());
- }
-
- /**
- * Publishes events to listeners.
- *
- * @param events list of map event to publish
- */
- private void publish(List<MultimapEvent<String, byte[]>> events) {
- listeners.values().forEach(session -> session.publish(CHANGE, serializer::encode, events));
- }
-
- private interface MapEntryValue {
-
- /**
- * Returns the list of raw {@code byte[]'s}.
- *
- * @return list of raw values
- */
- Collection<? extends byte[]> values();
-
- /**
- * Returns the version of the value.
- *
- * @return version
- */
- long version();
-
- /**
- * Add a new commit and modifies the set of values accordingly.
- * In the case of a replace or removeAll it returns the set of removed
- * values. In the case of put or multiRemove it returns null for no
- * change and a set of the added or removed values respectively if a
- * change resulted.
- *
- * @param commit the commit to be added
- */
- Versioned<Collection<? extends byte[]>> addCommit(
- Commit<? extends MultimapOperation> commit);
- }
-
- private class NonTransactionalCommit implements MapEntryValue {
- private long version;
- private final TreeSet<byte[]> valueSet = Sets.newTreeSet(new ByteArrayComparator());
-
- public NonTransactionalCommit() {
- //Set the version to current it will only be updated once this is
- // populated
- this.version = globalVersion.get();
- }
-
- @Override
- public Collection<? extends byte[]> values() {
- return ImmutableSet.copyOf(valueSet);
- }
-
- @Override
- public long version() {
- return version;
- }
-
- @Override
- public Versioned<Collection<? extends byte[]>> addCommit(
- Commit<? extends MultimapOperation> commit) {
- Preconditions.checkNotNull(commit);
- Preconditions.checkNotNull(commit.value());
- Versioned<Collection<? extends byte[]>> retVersion;
-
- if (commit.value() instanceof Put) {
- //Using a treeset here sanitizes the input, removing duplicates
- Set<byte[]> valuesToAdd =
- Sets.newTreeSet(new ByteArrayComparator());
- ((Put) commit.value()).values().forEach(value -> {
- if (!valueSet.contains(value)) {
- valuesToAdd.add(value);
- }
- });
- if (valuesToAdd.isEmpty()) {
- //Do not increment or add the commit if no change resulted
- return null;
- }
- retVersion = new Versioned<>(valuesToAdd, version);
- valuesToAdd.forEach(value -> valueSet.add(value));
- version++;
- return retVersion;
-
- } else if (commit.value() instanceof Replace) {
- //Will this work?? Need to check before check-in!
- Set<byte[]> removedValues = Sets.newHashSet();
- removedValues.addAll(valueSet);
- retVersion = new Versioned<>(removedValues, version);
- valueSet.clear();
- Set<byte[]> valuesToAdd =
- Sets.newTreeSet(new ByteArrayComparator());
- ((Replace) commit.value()).values().forEach(value -> {
- valuesToAdd.add(value);
- });
- if (valuesToAdd.isEmpty()) {
- version = globalVersion.incrementAndGet();
- backingMap.remove(((Replace) commit.value()).key());
- return retVersion;
- }
- valuesToAdd.forEach(value -> valueSet.add(value));
- version = globalVersion.incrementAndGet();
- return retVersion;
-
- } else if (commit.value() instanceof RemoveAll) {
- Set<byte[]> removed = Sets.newHashSet();
- //We can assume here that values only appear once and so we
- //do not need to sanitize the return for duplicates.
- removed.addAll(valueSet);
- retVersion = new Versioned<>(removed, version);
- valueSet.clear();
- //In the case of a removeAll all commits will be removed and
- //unlike the multiRemove case we do not need to consider
- //dependencies among additive and removal commits.
-
- //Save the key for use after the commit is closed
- String key = ((RemoveAll) commit.value()).key();
- version = globalVersion.incrementAndGet();
- backingMap.remove(key);
- return retVersion;
-
- } else if (commit.value() instanceof MultiRemove) {
- //Must first calculate how many commits the removal depends on.
- //At this time we also sanitize the removal set by adding to a
- //set with proper handling of byte[] equality.
- Set<byte[]> removed = Sets.newHashSet();
- ((MultiRemove) commit.value()).values().forEach(value -> {
- if (valueSet.contains(value)) {
- removed.add(value);
- }
- });
- //If there is nothing to be removed no action should be taken.
- if (removed.isEmpty()) {
- return null;
- }
- //Save key in case countdown results in closing the commit.
- String removedKey = ((MultiRemove) commit.value()).key();
- removed.forEach(removedValue -> {
- valueSet.remove(removedValue);
- });
- //The version is updated locally as well as globally even if
- //this object will be removed from the map in case any other
- //party still holds a reference to this object.
- retVersion = new Versioned<>(removed, version);
- version = globalVersion.incrementAndGet();
- if (valueSet.isEmpty()) {
- backingMap.remove(removedKey);
- }
- return retVersion;
-
- } else {
- throw new IllegalArgumentException();
- }
- }
- }
-
- /**
- * A collector that creates MapEntryValues and creates a multiset of all
- * values in the map an equal number of times to the number of sets in
- * which they participate.
- */
- private class HashMultisetValueCollector implements
- Collector<MapEntryValue,
- HashMultiset<byte[]>,
- HashMultiset<byte[]>> {
-
- @Override
- public Supplier<HashMultiset<byte[]>> supplier() {
- return HashMultiset::create;
- }
-
- @Override
- public BiConsumer<HashMultiset<byte[]>, MapEntryValue> accumulator() {
- return (multiset, mapEntryValue) ->
- multiset.addAll(mapEntryValue.values());
- }
-
- @Override
- public BinaryOperator<HashMultiset<byte[]>> combiner() {
- return (setOne, setTwo) -> {
- setOne.addAll(setTwo);
- return setOne;
- };
- }
-
- @Override
- public Function<HashMultiset<byte[]>,
- HashMultiset<byte[]>> finisher() {
- return Function.identity();
- }
-
- @Override
- public Set<Characteristics> characteristics() {
- return EnumSet.of(Characteristics.UNORDERED);
- }
- }
-
- /**
- * A collector that creates Entries of {@code <String, MapEntryValue>} and
- * creates a set of entries all key value pairs in the map.
- */
- private class EntrySetCollector implements
- Collector<Map.Entry<String, MapEntryValue>,
- Set<Map.Entry<String, byte[]>>,
- Set<Map.Entry<String, byte[]>>> {
- private Set<Map.Entry<String, byte[]>> set = null;
-
- @Override
- public Supplier<Set<Map.Entry<String, byte[]>>> supplier() {
- return () -> {
- if (set == null) {
- set = Sets.newHashSet();
- }
- return set;
- };
- }
-
- @Override
- public BiConsumer<Set<Map.Entry<String, byte[]>>,
- Map.Entry<String, MapEntryValue>> accumulator() {
- return (set, entry) -> {
- entry
- .getValue()
- .values()
- .forEach(byteValue ->
- set.add(Maps.immutableEntry(entry.getKey(),
- byteValue)));
- };
- }
-
- @Override
- public BinaryOperator<Set<Map.Entry<String, byte[]>>> combiner() {
- return (setOne, setTwo) -> {
- setOne.addAll(setTwo);
- return setOne;
- };
- }
-
- @Override
- public Function<Set<Map.Entry<String, byte[]>>,
- Set<Map.Entry<String, byte[]>>> finisher() {
- return (unused) -> set;
- }
-
- @Override
- public Set<Characteristics> characteristics() {
- return EnumSet.of(Characteristics.UNORDERED);
- }
- }
- /**
- * Utility for turning a {@code MapEntryValue} to {@code Versioned}.
- * @param value map entry value
- * @return versioned instance or an empty list versioned -1 if argument is
- * null
- */
- private Versioned<Collection<? extends byte[]>> toVersioned(
- MapEntryValue value) {
- return value == null ? new Versioned<>(Lists.newArrayList(), -1) :
- new Versioned<>(value.values(),
- value.version());
- }
-
- private static class ByteArrayComparator implements Comparator<byte[]> {
-
- @Override
- public int compare(byte[] o1, byte[] o2) {
- if (Arrays.equals(o1, o2)) {
- return 0;
- } else {
- for (int i = 0; i < o1.length && i < o2.length; i++) {
- if (o1[i] < o2[i]) {
- return -1;
- } else if (o1[i] > o2[i]) {
- return 1;
- }
- }
- return o1.length > o2.length ? 1 : -1;
- }
- }
- }
-
- private static class IteratorContext {
- private final long sessionId;
- private final Iterator<Map.Entry<String, MapEntryValue>> iterator;
- private int position = 0;
-
- IteratorContext(long sessionId, Iterator<Map.Entry<String, MapEntryValue>> iterator) {
- this.sessionId = sessionId;
- this.iterator = iterator;
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMap.java
deleted file mode 100644
index 25c4d84..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMap.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FloorEntry;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HigherEntry;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LowerEntry;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CEILING_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CEILING_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CeilingEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CeilingKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FIRST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FIRST_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FLOOR_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FLOOR_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FloorKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HIGHER_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HIGHER_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HigherKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LAST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LAST_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LOWER_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LOWER_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LowerKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.POLL_FIRST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.POLL_LAST_ENTRY;
-
-/**
- * Implementation of {@link AsyncConsistentTreeMap}.
- */
-public class AtomixConsistentTreeMap extends AtomixConsistentMap implements AsyncConsistentTreeMap<byte[]> {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixConsistentMapOperations.NAMESPACE)
- .register(AtomixConsistentTreeMapOperations.NAMESPACE)
- .register(AtomixConsistentMapEvents.NAMESPACE)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 150)
- .register(AtomixConsistentMapService.TransactionScope.class)
- .register(TransactionLog.class)
- .register(TransactionId.class)
- .register(AtomixConsistentMapService.MapEntryValue.class)
- .register(AtomixConsistentMapService.MapEntryValue.Type.class)
- .register(new HashMap().keySet().getClass())
- .register(TreeMap.class)
- .build());
-
- public AtomixConsistentTreeMap(RaftProxy proxy) {
- super(proxy);
- }
-
- @Override
- protected Serializer serializer() {
- return SERIALIZER;
- }
-
- @Override
- public CompletableFuture<String> firstKey() {
- return proxy.invoke(FIRST_KEY, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<String> lastKey() {
- return proxy.invoke(LAST_KEY, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> ceilingEntry(String key) {
- return proxy.invoke(CEILING_ENTRY, serializer()::encode, new CeilingEntry(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> floorEntry(String key) {
- return proxy.invoke(FLOOR_ENTRY, serializer()::encode, new FloorEntry(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> higherEntry(
- String key) {
- return proxy.invoke(HIGHER_ENTRY, serializer()::encode, new HigherEntry(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> lowerEntry(
- String key) {
- return proxy.invoke(LOWER_ENTRY, serializer()::encode, new LowerEntry(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> firstEntry() {
- return proxy.invoke(FIRST_ENTRY, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> lastEntry() {
- return proxy.invoke(LAST_ENTRY, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> pollFirstEntry() {
- return proxy.invoke(POLL_FIRST_ENTRY, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> pollLastEntry() {
- return proxy.invoke(POLL_LAST_ENTRY, serializer()::decode);
- }
-
- @Override
- public CompletableFuture<String> lowerKey(String key) {
- return proxy.invoke(LOWER_KEY, serializer()::encode, new LowerKey(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<String> floorKey(String key) {
- return proxy.invoke(FLOOR_KEY, serializer()::encode, new FloorKey(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<String> ceilingKey(String key) {
- return proxy.invoke(CEILING_KEY, serializer()::encode, new CeilingKey(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<String> higherKey(String key) {
- return proxy.invoke(HIGHER_KEY, serializer()::encode, new HigherKey(key), serializer()::decode);
- }
-
- @Override
- public CompletableFuture<NavigableSet<String>> navigableKeySet() {
- throw new UnsupportedOperationException("This operation is not yet supported.");
- }
-
- @Override
- public CompletableFuture<NavigableMap<String, byte[]>> subMap(
- String upperKey, String lowerKey, boolean inclusiveUpper,
- boolean inclusiveLower) {
- throw new UnsupportedOperationException("This operation is not yet supported.");
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapOperations.java
deleted file mode 100644
index 74ae7eb..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapOperations.java
+++ /dev/null
@@ -1,415 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.AbstractMap;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@link org.onosproject.store.service.AsyncConsistentTreeMap} Resource
- * state machine operations.
- */
-public enum AtomixConsistentTreeMapOperations implements OperationId {
- SUB_MAP(OperationType.QUERY),
- FIRST_KEY(OperationType.QUERY),
- LAST_KEY(OperationType.QUERY),
- FIRST_ENTRY(OperationType.QUERY),
- LAST_ENTRY(OperationType.QUERY),
- POLL_FIRST_ENTRY(OperationType.QUERY),
- POLL_LAST_ENTRY(OperationType.QUERY),
- LOWER_ENTRY(OperationType.QUERY),
- LOWER_KEY(OperationType.QUERY),
- FLOOR_ENTRY(OperationType.QUERY),
- FLOOR_KEY(OperationType.QUERY),
- CEILING_ENTRY(OperationType.QUERY),
- CEILING_KEY(OperationType.QUERY),
- HIGHER_ENTRY(OperationType.QUERY),
- HIGHER_KEY(OperationType.QUERY);
-
- private final OperationType type;
-
- AtomixConsistentTreeMapOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 100)
- .register(LowerKey.class)
- .register(LowerEntry.class)
- .register(HigherKey.class)
- .register(HigherEntry.class)
- .register(FloorKey.class)
- .register(FloorEntry.class)
- .register(CeilingKey.class)
- .register(CeilingEntry.class)
- .register(Versioned.class)
- .register(AbstractMap.SimpleImmutableEntry.class)
- .register(Maps.immutableEntry("", "").getClass())
- .build("AtomixConsistentTreeMapOperations");
-
- /**
- * Abstract treeMap command.
- */
- @SuppressWarnings("serial")
- public abstract static class TreeOperation {
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .toString();
- }
- }
-
- /**
- * Abstract key-based query.
- */
- @SuppressWarnings("serial")
- public abstract static class KeyOperation extends TreeOperation {
- protected String key;
-
- public KeyOperation(String key) {
- this.key = checkNotNull(key);
- }
-
- public KeyOperation() {
- }
-
- public String key() {
- return key;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("key", key)
- .toString();
- }
- }
-
- /**
- * Abstract value-based query.
- */
- @SuppressWarnings("serial")
- public abstract static class ValueOperation extends TreeOperation {
- protected byte[] value;
-
- public ValueOperation() {}
-
- public ValueOperation(byte[] value) {
- this.value = checkNotNull(value);
- }
-
- public byte[] value() {
- return value;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("value", value)
- .toString();
- }
- }
-
- /**
- * Contains key command.
- */
- @SuppressWarnings("serial")
- public static class ContainsKey extends KeyOperation {
-
- public ContainsKey(String key) {
- super(key);
- }
-
- public ContainsKey() {
- }
- }
- /**
- * Contains value command.
- */
- @SuppressWarnings("serial")
- public static class ContainsValue extends ValueOperation {
- public ContainsValue() {
- }
-
- public ContainsValue(byte[] value) {
- super(value);
- }
-
- }
-
- /**
- * AsyncConsistentTreeMap update command.
- */
- @SuppressWarnings("serial")
- public static class UpdateAndGet extends TreeOperation {
- private String key;
- private byte[] value;
- private Match<byte[]> valueMatch;
- private Match<Long> versionMatch;
- public UpdateAndGet() {
- }
-
- public UpdateAndGet(String key,
- byte[] value,
- Match<byte[]> valueMatch,
- Match<Long> versionMatch) {
- this.key = key;
- this.value = value;
- this.valueMatch = valueMatch;
- this.versionMatch = versionMatch;
- }
-
- public String key() {
- return this.key;
- }
-
- public byte[] value() {
- return this.value;
- }
-
- public Match<byte[]> valueMatch() {
- return this.valueMatch;
- }
-
- public Match<Long> versionMatch() {
- return this.versionMatch;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("key", key)
- .add("value", value)
- .add("valueMatch", valueMatch)
- .add("versionMatch", versionMatch)
- .toString();
- }
- }
-
- /**
- * Get query.
- */
- @SuppressWarnings("serial")
- public static class Get extends KeyOperation {
- public Get() {
- }
-
- public Get(String key) {
- super(key);
- }
- }
-
- /**
- * Get or default query.
- */
- @SuppressWarnings("serial")
- public static class GetOrDefault extends KeyOperation {
- private byte[] defaultValue;
-
- public GetOrDefault() {
- }
-
- public GetOrDefault(String key, byte[] defaultValue) {
- super(key);
- this.defaultValue = defaultValue;
- }
-
- /**
- * Returns the default value.
- *
- * @return the default value
- */
- public byte[] defaultValue() {
- return defaultValue;
- }
- }
-
- /**
- * Query returns the entry associated with the largest key less than the
- * passed in key.
- */
- @SuppressWarnings("serial")
- public static class LowerEntry extends KeyOperation {
- public LowerEntry() {
- }
-
- public LowerEntry(String key) {
- super(key);
- }
- }
-
- /**
- * Query returns the largest key less than the specified key.
- */
- @SuppressWarnings("serial")
- public static class LowerKey extends KeyOperation {
- public LowerKey() {
- }
-
- public LowerKey(String key) {
- super(key);
- }
- }
-
- /**
- * Query returns the entry associated with the largest key smaller than or
- * equal to the specified key.
- */
- @SuppressWarnings("serial")
- public static class FloorEntry extends KeyOperation {
- public FloorEntry() {
- }
-
- public FloorEntry(String key) {
- super(key);
- }
- }
-
- /**
- * Query returns the largest key smaller than or equal to the passed in
- * key.
- */
- @SuppressWarnings("serial")
- public static class FloorKey extends KeyOperation {
- public FloorKey() {
- }
-
- public FloorKey(String key) {
- super(key);
- }
- }
-
- /**
- * Returns the entry associated with the smallest key larger than or equal
- * to the specified key.
- */
- @SuppressWarnings("serial")
- public static class CeilingEntry extends KeyOperation {
- public CeilingEntry() {
- }
-
- public CeilingEntry(String key) {
- super(key);
- }
- }
-
- /**
- * Returns the smallest key larger than or equal to the specified key.
- */
- @SuppressWarnings("serial")
- public static class CeilingKey extends KeyOperation {
- public CeilingKey() {
- }
-
- public CeilingKey(String key) {
- super(key);
- }
- }
-
- /**
- * Returns the entry associated with the smallest key larger than the
- * specified key.
- */
- @SuppressWarnings("serial")
- public static class HigherEntry extends KeyOperation {
- public HigherEntry() {
- }
-
- public HigherEntry(String key) {
- super(key);
- }
- }
-
- /**
- * Returns the smallest key larger than the specified key.
- */
- @SuppressWarnings("serial")
- public static class HigherKey extends KeyOperation {
- public HigherKey() {
- }
-
- public HigherKey(String key) {
- super(key);
- }
- }
-
- @SuppressWarnings("serial")
- public static class SubMap<K, V> extends TreeOperation {
- private K fromKey;
- private K toKey;
- private boolean inclusiveFrom;
- private boolean inclusiveTo;
-
- public SubMap() {
- }
-
- public SubMap(K fromKey, K toKey, boolean inclusiveFrom,
- boolean inclusiveTo) {
- this.fromKey = fromKey;
- this.toKey = toKey;
- this.inclusiveFrom = inclusiveFrom;
- this.inclusiveTo = inclusiveTo;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("getFromKey", fromKey)
- .add("getToKey", toKey)
- .add("inclusiveFromBound", inclusiveFrom)
- .add("inclusiveToBound", inclusiveTo)
- .toString();
- }
-
- public K fromKey() {
- return fromKey;
- }
-
- public K toKey() {
- return toKey;
- }
-
- public boolean isInclusiveFrom() {
- return inclusiveFrom;
- }
-
- public boolean isInclusiveTo() {
- return inclusiveTo;
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapService.java
deleted file mode 100644
index 89c8da6..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapService.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CEILING_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CEILING_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CeilingEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CeilingKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FIRST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FIRST_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FLOOR_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FLOOR_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FloorEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FloorKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HIGHER_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HIGHER_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HigherEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HigherKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LAST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LAST_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LOWER_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LOWER_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LowerEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LowerKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.POLL_FIRST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.POLL_LAST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.SUB_MAP;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.SubMap;
-
-/**
- * State machine corresponding to {@link AtomixConsistentTreeMap} backed by a
- * {@link TreeMap}.
- */
-public class AtomixConsistentTreeMapService extends AtomixConsistentMapService {
-
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixConsistentMapOperations.NAMESPACE)
- .register(AtomixConsistentTreeMapOperations.NAMESPACE)
- .register(AtomixConsistentMapEvents.NAMESPACE)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 150)
- .register(TransactionScope.class)
- .register(TransactionLog.class)
- .register(TransactionId.class)
- .register(MapEntryValue.class)
- .register(MapEntryValue.Type.class)
- .register(new HashMap().keySet().getClass())
- .register(ConcurrentSkipListMap.class)
- .build());
-
- @Override
- protected NavigableMap<String, MapEntryValue> createMap() {
- return new ConcurrentSkipListMap<>();
- }
-
- @Override
- protected NavigableMap<String, MapEntryValue> entries() {
- return (NavigableMap<String, MapEntryValue>) super.entries();
- }
-
- @Override
- protected Serializer serializer() {
- return SERIALIZER;
- }
-
- @Override
- public void configure(RaftServiceExecutor executor) {
- super.configure(executor);
- executor.register(SUB_MAP, serializer()::decode, this::subMap, serializer()::encode);
- executor.register(FIRST_KEY, (Commit<Void> c) -> firstKey(), serializer()::encode);
- executor.register(LAST_KEY, (Commit<Void> c) -> lastKey(), serializer()::encode);
- executor.register(FIRST_ENTRY, (Commit<Void> c) -> firstEntry(), serializer()::encode);
- executor.register(LAST_ENTRY, (Commit<Void> c) -> lastEntry(), serializer()::encode);
- executor.register(POLL_FIRST_ENTRY, (Commit<Void> c) -> pollFirstEntry(), serializer()::encode);
- executor.register(POLL_LAST_ENTRY, (Commit<Void> c) -> pollLastEntry(), serializer()::encode);
- executor.register(LOWER_ENTRY, serializer()::decode, this::lowerEntry, serializer()::encode);
- executor.register(LOWER_KEY, serializer()::decode, this::lowerKey, serializer()::encode);
- executor.register(FLOOR_ENTRY, serializer()::decode, this::floorEntry, serializer()::encode);
- executor.register(FLOOR_KEY, serializer()::decode, this::floorKey, serializer()::encode);
- executor.register(CEILING_ENTRY, serializer()::decode, this::ceilingEntry, serializer()::encode);
- executor.register(CEILING_KEY, serializer()::decode, this::ceilingKey, serializer()::encode);
- executor.register(HIGHER_ENTRY, serializer()::decode, this::higherEntry, serializer()::encode);
- executor.register(HIGHER_KEY, serializer()::decode, this::higherKey, serializer()::encode);
- }
-
- protected NavigableMap<String, MapEntryValue> subMap(
- Commit<? extends SubMap> commit) {
- // Do not support this until lazy communication is possible. At present
- // it transmits up to the entire map.
- SubMap<String, MapEntryValue> subMap = commit.value();
- return entries().subMap(subMap.fromKey(), subMap.isInclusiveFrom(),
- subMap.toKey(), subMap.isInclusiveTo());
- }
-
- protected String firstKey() {
- return isEmpty() ? null : entries().firstKey();
- }
-
- protected String lastKey() {
- return isEmpty() ? null : entries().lastKey();
- }
-
- protected Map.Entry<String, Versioned<byte[]>> higherEntry(Commit<? extends HigherEntry> commit) {
- return isEmpty() ? null : toVersionedEntry(entries().higherEntry(commit.value().key()));
- }
-
- protected Map.Entry<String, Versioned<byte[]>> firstEntry() {
- return isEmpty() ? null : toVersionedEntry(entries().firstEntry());
- }
-
- protected Map.Entry<String, Versioned<byte[]>> lastEntry() {
- return isEmpty() ? null : toVersionedEntry(entries().lastEntry());
- }
-
- protected Map.Entry<String, Versioned<byte[]>> pollFirstEntry() {
- return toVersionedEntry(entries().pollFirstEntry());
- }
-
- protected Map.Entry<String, Versioned<byte[]>> pollLastEntry() {
- return toVersionedEntry(entries().pollLastEntry());
- }
-
- protected Map.Entry<String, Versioned<byte[]>> lowerEntry(Commit<? extends LowerEntry> commit) {
- return toVersionedEntry(entries().lowerEntry(commit.value().key()));
- }
-
- protected String lowerKey(Commit<? extends LowerKey> commit) {
- return entries().lowerKey(commit.value().key());
- }
-
- protected Map.Entry<String, Versioned<byte[]>> floorEntry(Commit<? extends FloorEntry> commit) {
- return toVersionedEntry(entries().floorEntry(commit.value().key()));
- }
-
- protected String floorKey(Commit<? extends FloorKey> commit) {
- return entries().floorKey(commit.value().key());
- }
-
- protected Map.Entry<String, Versioned<byte[]>> ceilingEntry(Commit<CeilingEntry> commit) {
- return toVersionedEntry(entries().ceilingEntry(commit.value().key()));
- }
-
- protected String ceilingKey(Commit<CeilingKey> commit) {
- return entries().ceilingKey(commit.value().key());
- }
-
- protected String higherKey(Commit<HigherKey> commit) {
- return entries().higherKey(commit.value().key());
- }
-
- private Map.Entry<String, Versioned<byte[]>> toVersionedEntry(
- Map.Entry<String, MapEntryValue> entry) {
- return entry == null || valueIsNull(entry.getValue())
- ? null : Maps.immutableEntry(entry.getKey(), toVersioned(entry.getValue()));
- }
-
- @Override
- public void onExpire(RaftSession session) {
- closeListener(session.sessionId().id());
- }
-
- @Override
- public void onClose(RaftSession session) {
- closeListener(session.sessionId().id());
- }
-
- private void closeListener(Long sessionId) {
- listeners.remove(sessionId);
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounter.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounter.java
deleted file mode 100644
index 9c579d6..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounter.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.concurrent.CompletableFuture;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.ADD_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.AddAndGet;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.COMPARE_AND_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.CompareAndSet;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET_AND_ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET_AND_INCREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GetAndAdd;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.INCREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.Set;
-
-/**
- * Atomix counter implementation.
- */
-public class AtomixCounter extends AbstractRaftPrimitive implements AsyncAtomicCounter {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixCounterOperations.NAMESPACE)
- .build());
-
- public AtomixCounter(RaftProxy proxy) {
- super(proxy);
- }
-
- private long nullOrZero(Long value) {
- return value != null ? value : 0;
- }
-
- @Override
- public CompletableFuture<Long> get() {
- return proxy.<Long>invoke(GET, SERIALIZER::decode).thenApply(this::nullOrZero);
- }
-
- @Override
- public CompletableFuture<Void> set(long value) {
- return proxy.invoke(SET, SERIALIZER::encode, new Set(value));
- }
-
- @Override
- public CompletableFuture<Boolean> compareAndSet(long expectedValue, long updateValue) {
- return proxy.invoke(COMPARE_AND_SET, SERIALIZER::encode,
- new CompareAndSet(expectedValue, updateValue), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> addAndGet(long delta) {
- return proxy.invoke(ADD_AND_GET, SERIALIZER::encode, new AddAndGet(delta), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> getAndAdd(long delta) {
- return proxy.invoke(GET_AND_ADD, SERIALIZER::encode, new GetAndAdd(delta), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> incrementAndGet() {
- return proxy.invoke(INCREMENT_AND_GET, SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Long> getAndIncrement() {
- return proxy.invoke(GET_AND_INCREMENT, SERIALIZER::decode);
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterOperations.java
deleted file mode 100644
index 67ff936..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterOperations.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Counter commands.
- */
-public enum AtomixCounterOperations implements OperationId {
- SET(OperationType.COMMAND),
- COMPARE_AND_SET(OperationType.COMMAND),
- INCREMENT_AND_GET(OperationType.COMMAND),
- GET_AND_INCREMENT(OperationType.COMMAND),
- ADD_AND_GET(OperationType.COMMAND),
- GET_AND_ADD(OperationType.COMMAND),
- GET(OperationType.QUERY);
-
- private final OperationType type;
-
- AtomixCounterOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(Get.class)
- .register(Set.class)
- .register(CompareAndSet.class)
- .register(AddAndGet.class)
- .register(GetAndAdd.class)
- .build("AtomixCounterOperations");
-
- /**
- * Abstract value command.
- */
- public abstract static class ValueOperation {
- }
-
- /**
- * Get query.
- */
- public static class Get extends ValueOperation {
- }
-
- /**
- * Set command.
- */
- public static class Set extends ValueOperation {
- private Long value;
-
- public Set() {
- }
-
- public Set(Long value) {
- this.value = value;
- }
-
- /**
- * Returns the command value.
- *
- * @return The command value.
- */
- public Long value() {
- return value;
- }
-
- @Override
- public String toString() {
- return String.format("%s[value=%s]", getClass().getSimpleName(), value);
- }
- }
-
- /**
- * Compare and set command.
- */
- public static class CompareAndSet extends ValueOperation {
- private Long expect;
- private Long update;
-
- public CompareAndSet() {
- }
-
- public CompareAndSet(Long expect, Long update) {
- this.expect = expect;
- this.update = update;
- }
-
- /**
- * Returns the expected value.
- *
- * @return The expected value.
- */
- public Long expect() {
- return expect;
- }
-
- /**
- * Returns the updated value.
- *
- * @return The updated value.
- */
- public Long update() {
- return update;
- }
-
- @Override
- public String toString() {
- return String.format("%s[expect=%s, update=%s]", getClass().getSimpleName(), expect, update);
- }
- }
-
- /**
- * Delta command.
- */
- public abstract static class DeltaOperation extends ValueOperation {
- private long delta;
-
- public DeltaOperation() {
- }
-
- public DeltaOperation(long delta) {
- this.delta = delta;
- }
-
- /**
- * Returns the delta.
- *
- * @return The delta.
- */
- public long delta() {
- return delta;
- }
- }
-
- /**
- * Get and add command.
- */
- public static class GetAndAdd extends DeltaOperation {
- public GetAndAdd() {
- }
-
- public GetAndAdd(long delta) {
- super(delta);
- }
- }
-
- /**
- * Add and get command.
- */
- public static class AddAndGet extends DeltaOperation {
- public AddAndGet() {
- }
-
- public AddAndGet(long delta) {
- super(delta);
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterService.java
deleted file mode 100644
index a56478b..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterService.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Objects;
-
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.ADD_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.AddAndGet;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.COMPARE_AND_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.CompareAndSet;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET_AND_ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET_AND_INCREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GetAndAdd;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.INCREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.Set;
-
-/**
- * Atomix long state.
- */
-public class AtomixCounterService extends AbstractRaftService {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixCounterOperations.NAMESPACE)
- .build());
-
- private Long value = 0L;
-
- @Override
- protected void configure(RaftServiceExecutor executor) {
- executor.register(SET, SERIALIZER::decode, this::set);
- executor.register(GET, this::get, SERIALIZER::encode);
- executor.register(COMPARE_AND_SET, SERIALIZER::decode, this::compareAndSet, SERIALIZER::encode);
- executor.register(INCREMENT_AND_GET, this::incrementAndGet, SERIALIZER::encode);
- executor.register(GET_AND_INCREMENT, this::getAndIncrement, SERIALIZER::encode);
- executor.register(ADD_AND_GET, SERIALIZER::decode, this::addAndGet, SERIALIZER::encode);
- executor.register(GET_AND_ADD, SERIALIZER::decode, this::getAndAdd, SERIALIZER::encode);
- }
-
- @Override
- public void snapshot(SnapshotWriter writer) {
- writer.writeLong(value);
- }
-
- @Override
- public void install(SnapshotReader reader) {
- value = reader.readLong();
- }
-
- /**
- * Handles a set commit.
- *
- * @param commit the commit to handle
- */
- protected void set(Commit<Set> commit) {
- value = commit.value().value();
- }
-
- /**
- * Handles a get commit.
- *
- * @param commit the commit to handle
- * @return counter value
- */
- protected Long get(Commit<Void> commit) {
- return value;
- }
-
- /**
- * Handles a compare and set commit.
- *
- * @param commit the commit to handle
- * @return counter value
- */
- protected boolean compareAndSet(Commit<CompareAndSet> commit) {
- if (Objects.equals(value, commit.value().expect())) {
- value = commit.value().update();
- return true;
- }
- return false;
- }
-
- /**
- * Handles an increment and get commit.
- *
- * @param commit the commit to handle
- * @return counter value
- */
- protected long incrementAndGet(Commit<Void> commit) {
- Long oldValue = value;
- value = oldValue + 1;
- return value;
- }
-
- /**
- * Handles a get and increment commit.
- *
- * @param commit the commit to handle
- * @return counter value
- */
- protected long getAndIncrement(Commit<Void> commit) {
- Long oldValue = value;
- value = oldValue + 1;
- return oldValue;
- }
-
- /**
- * Handles an add and get commit.
- *
- * @param commit the commit to handle
- * @return counter value
- */
- protected long addAndGet(Commit<AddAndGet> commit) {
- Long oldValue = value;
- value = oldValue + commit.value().delta();
- return value;
- }
-
- /**
- * Handles a get and add commit.
- *
- * @param commit the commit to handle
- * @return counter value
- */
- protected long getAndAdd(Commit<GetAndAdd> commit) {
- Long oldValue = value;
- value = oldValue + commit.value().delta();
- return oldValue;
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLock.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLock.java
deleted file mode 100644
index c234be3..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLock.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.time.Duration;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Consumer;
-
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.OrderedExecutor;
-import org.onlab.util.SharedScheduledExecutors;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncDistributedLock;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Version;
-
-import static org.onlab.util.Tools.orderedFuture;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockEvents.FAILED;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockEvents.LOCKED;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.LOCK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.Lock;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.UNLOCK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.Unlock;
-
-/**
- * Atomix lock implementation.
- * <p>
- * This {@link org.onosproject.store.service.DistributedLock} implementation uses a {@link RaftProxy} to interact
- * with a {@link AtomixDistributedLockService} replicated state machine.
- */
-public class AtomixDistributedLock extends AbstractRaftPrimitive implements AsyncDistributedLock {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixDistributedLockOperations.NAMESPACE)
- .register(AtomixDistributedLockEvents.NAMESPACE)
- .build());
-
- private final ScheduledExecutorService scheduledExecutor;
- private final Executor orderedExecutor;
- private final Map<Integer, LockAttempt> attempts = Maps.newConcurrentMap();
- private final AtomicInteger id = new AtomicInteger();
- private final AtomicInteger lock = new AtomicInteger();
-
- public AtomixDistributedLock(RaftProxy proxy) {
- super(proxy);
- this.scheduledExecutor = SharedScheduledExecutors.getPoolThreadExecutor();
- this.orderedExecutor = new OrderedExecutor(scheduledExecutor);
- proxy.addEventListener(LOCKED, SERIALIZER::decode, this::handleLocked);
- proxy.addEventListener(FAILED, SERIALIZER::decode, this::handleFailed);
- }
-
- /**
- * Handles a {@code LOCKED} event.
- *
- * @param event the event to handle
- */
- private void handleLocked(LockEvent event) {
- // Remove the LockAttempt from the attempts map and complete it with the lock version if it exists.
- // If the attempt no longer exists, it likely was expired by a client-side timer.
- LockAttempt attempt = attempts.remove(event.id());
- if (attempt != null) {
- attempt.complete(new Version(event.version()));
- }
- }
-
- /**
- * Handles a {@code FAILED} event.
- *
- * @param event the event to handle
- */
- private void handleFailed(LockEvent event) {
- // Remove the LockAttempt from the attempts map and complete it with a null value if it exists.
- // If the attempt no longer exists, it likely was expired by a client-side timer.
- LockAttempt attempt = attempts.remove(event.id());
- if (attempt != null) {
- attempt.complete(null);
- }
- }
-
- @Override
- public CompletableFuture<Version> lock() {
- // Create and register a new attempt and invoke the LOCK operation on the replicated state machine.
- LockAttempt attempt = new LockAttempt();
- proxy.invoke(LOCK, SERIALIZER::encode, new Lock(attempt.id(), -1)).whenComplete((result, error) -> {
- if (error != null) {
- attempt.completeExceptionally(error);
- }
- });
-
- // Return an ordered future that can safely be blocked inside the executor thread.
- return orderedFuture(attempt, orderedExecutor, scheduledExecutor);
- }
-
- @Override
- public CompletableFuture<Optional<Version>> tryLock() {
- // If the proxy is currently disconnected from the cluster, we can just fail the lock attempt here.
- RaftProxy.State state = proxy.getState();
- if (state != RaftProxy.State.CONNECTED) {
- return CompletableFuture.completedFuture(Optional.empty());
- }
-
- // Create and register a new attempt and invoke the LOCK operation on teh replicated state machine with
- // a 0 timeout. The timeout will cause the state machine to immediately reject the request if the lock is
- // already owned by another process.
- LockAttempt attempt = new LockAttempt();
- proxy.invoke(LOCK, SERIALIZER::encode, new Lock(attempt.id(), 0)).whenComplete((result, error) -> {
- if (error != null) {
- attempt.completeExceptionally(error);
- }
- });
-
- // Return an ordered future that can safely be blocked inside the executor thread.
- return orderedFuture(attempt, orderedExecutor, scheduledExecutor)
- .thenApply(Optional::ofNullable);
- }
-
- @Override
- public CompletableFuture<Optional<Version>> tryLock(Duration timeout) {
- // Create a lock attempt with a client-side timeout and fail the lock if the timer expires.
- // Because time does not progress at the same rate on different nodes, we can't guarantee that
- // the lock won't be granted to this process after it's expired here. Thus, if this timer expires and
- // we fail the lock on the client, we also still need to send an UNLOCK command to the cluster in case it's
- // later granted by the cluster. Note that the semantics of the Raft client will guarantee this operation
- // occurs after any prior LOCK attempt, and the Raft client will retry the UNLOCK request until successful.
- // Additionally, sending the unique lock ID with the command ensures we won't accidentally unlock a different
- // lock call also granted to this process.
- LockAttempt attempt = new LockAttempt(timeout, a -> {
- a.complete(null);
- proxy.invoke(UNLOCK, SERIALIZER::encode, new Unlock(a.id()));
- });
-
- // Invoke the LOCK operation on the replicated state machine with the given timeout. If the lock is currently
- // held by another process, the state machine will add the attempt to a queue and publish a FAILED event if
- // the timer expires before this process can be granted the lock. If the client cannot reach the Raft cluster,
- // the client-side timer will expire the attempt.
- proxy.invoke(LOCK, SERIALIZER::encode, new Lock(attempt.id(), timeout.toMillis()))
- .whenComplete((result, error) -> {
- if (error != null) {
- attempt.completeExceptionally(error);
- }
- });
-
- // Return an ordered future that can safely be blocked inside the executor thread.
- return orderedFuture(attempt, orderedExecutor, scheduledExecutor)
- .thenApply(Optional::ofNullable);
- }
-
- @Override
- public CompletableFuture<Void> unlock() {
- // Use the current lock ID to ensure we only unlock the lock currently held by this process.
- int lock = this.lock.getAndSet(0);
- if (lock != 0) {
- return orderedFuture(
- proxy.invoke(UNLOCK, SERIALIZER::encode, new Unlock(lock)),
- orderedExecutor,
- scheduledExecutor);
- }
- return CompletableFuture.completedFuture(null);
- }
-
- /**
- * Closes the lock.
- *
- * @return a future to be completed once the lock has been closed
- */
- public CompletableFuture<Void> close() {
- return proxy.close();
- }
-
- /**
- * Lock attempt.
- */
- private class LockAttempt extends CompletableFuture<Version> {
- private final int id;
- private final ScheduledFuture<?> scheduledFuture;
-
- LockAttempt() {
- this(null, null);
- }
-
- LockAttempt(Duration duration, Consumer<LockAttempt> callback) {
- this.id = AtomixDistributedLock.this.id.incrementAndGet();
- this.scheduledFuture = duration != null && callback != null
- ? scheduledExecutor.schedule(() -> callback.accept(this), duration.toMillis(), TimeUnit.MILLISECONDS)
- : null;
- attempts.put(id, this);
- }
-
- /**
- * Returns the lock attempt ID.
- *
- * @return the lock attempt ID
- */
- int id() {
- return id;
- }
-
- @Override
- public boolean complete(Version version) {
- if (isDone()) {
- return super.complete(null);
- }
- cancel();
- if (version != null) {
- lock.set(id);
- return super.complete(version);
- } else {
- return super.complete(null);
- }
- }
-
- @Override
- public boolean completeExceptionally(Throwable ex) {
- cancel();
- return super.completeExceptionally(ex);
- }
-
- private void cancel() {
- if (scheduledFuture != null) {
- scheduledFuture.cancel(false);
- }
- attempts.remove(id);
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockEvents.java
deleted file mode 100644
index e6d2b2a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockEvents.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Raft value events.
- */
-public enum AtomixDistributedLockEvents implements EventType {
- LOCKED("lock"),
- FAILED("fail");
-
- private final String id;
-
- AtomixDistributedLockEvents(String id) {
- this.id = id;
- }
-
- @Override
- public String id() {
- return id;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
- .register(LockEvent.class)
- .register(byte[].class)
- .build(AtomixDistributedLockEvents.class.getSimpleName());
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockOperations.java
deleted file mode 100644
index 5cfbc84..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockOperations.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-/**
- * {@link org.onosproject.store.service.DistributedLock} operations.
- * <p>
- * WARNING: Do not refactor enum values. Only add to them.
- * Changing values risk breaking the ability to backup/restore/upgrade clusters.
- */
-public enum AtomixDistributedLockOperations implements OperationId {
- LOCK(OperationType.COMMAND),
- UNLOCK(OperationType.COMMAND);
-
- private final OperationType type;
-
- AtomixDistributedLockOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(Lock.class)
- .register(Unlock.class)
- .build(AtomixDistributedLockOperations.class.getSimpleName());
-
- /**
- * Abstract lock operation.
- */
- public abstract static class LockOperation {
- @Override
- public String toString() {
- return toStringHelper(this).toString();
- }
- }
-
- /**
- * Lock command.
- */
- public static class Lock extends LockOperation {
- private final int id;
- private final long timeout;
-
- public Lock() {
- this(0, 0);
- }
-
- public Lock(int id, long timeout) {
- this.id = id;
- this.timeout = timeout;
- }
-
- /**
- * Returns the lock identifier.
- *
- * @return the lock identifier
- */
- public int id() {
- return id;
- }
-
- /**
- * Returns the lock attempt timeout.
- *
- * @return the lock attempt timeout
- */
- public long timeout() {
- return timeout;
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("id", id)
- .add("timeout", timeout)
- .toString();
- }
- }
-
- /**
- * Unlock command.
- */
- public static class Unlock extends LockOperation {
- private final int id;
-
- public Unlock(int id) {
- this.id = id;
- }
-
- /**
- * Returns the lock identifier.
- *
- * @return the lock identifier
- */
- public int id() {
- return id;
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("id", id)
- .toString();
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockService.java
deleted file mode 100644
index e730ffd..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockService.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.time.Duration;
-import java.util.ArrayDeque;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Queue;
-
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.utils.concurrent.Scheduled;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockEvents.FAILED;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockEvents.LOCKED;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.LOCK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.Lock;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.UNLOCK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.Unlock;
-
-/**
- * Raft atomic value service.
- */
-public class AtomixDistributedLockService extends AbstractRaftService {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixDistributedLockOperations.NAMESPACE)
- .register(AtomixDistributedLockEvents.NAMESPACE)
- .register(LockHolder.class)
- .register(ArrayDeque.class)
- .build());
-
- private LockHolder lock;
- private Queue<LockHolder> queue = new ArrayDeque<>();
- private final Map<Long, Scheduled> timers = new HashMap<>();
-
- @Override
- protected void configure(RaftServiceExecutor executor) {
- executor.register(LOCK, SERIALIZER::decode, this::lock);
- executor.register(UNLOCK, SERIALIZER::decode, this::unlock);
- }
-
- @Override
- public void snapshot(SnapshotWriter writer) {
- writer.writeObject(lock, SERIALIZER::encode);
- writer.writeObject(queue, SERIALIZER::encode);
- }
-
- @Override
- public void install(SnapshotReader reader) {
- lock = reader.readObject(SERIALIZER::decode);
- queue = reader.readObject(SERIALIZER::decode);
-
- // After the snapshot is installed, we need to cancel any existing timers and schedule new ones based on the
- // state provided by the snapshot.
- timers.values().forEach(Scheduled::cancel);
- timers.clear();
- for (LockHolder holder : queue) {
- if (holder.expire > 0) {
- timers.put(holder.index,
- scheduler().schedule(Duration.ofMillis(holder.expire - wallClock().getTime().unixTimestamp()),
- () -> {
- timers.remove(holder.index);
- queue.remove(holder);
- RaftSession session = sessions().getSession(holder.session);
- if (session != null && session.getState().active()) {
- session.publish(FAILED, SERIALIZER::encode, new LockEvent(holder.id, holder.index));
- }
- }));
- }
- }
- }
-
- @Override
- public void onExpire(RaftSession session) {
- releaseSession(session);
- }
-
- @Override
- public void onClose(RaftSession session) {
- releaseSession(session);
- }
-
- /**
- * Applies a lock commit.
- *
- * @param commit the lock commit
- */
- protected void lock(Commit<Lock> commit) {
- // If the lock is not already owned, immediately grant the lock to the requester.
- // Note that we still have to publish an event to the session. The event is guaranteed to be received
- // by the client-side primitive after the LOCK response.
- if (lock == null) {
- lock = new LockHolder(
- commit.value().id(),
- commit.index(),
- commit.session().sessionId().id(),
- 0);
- commit.session().publish(
- LOCKED,
- SERIALIZER::encode,
- new LockEvent(commit.value().id(), commit.index()));
- // If the timeout is 0, that indicates this is a tryLock request. Immediately fail the request.
- } else if (commit.value().timeout() == 0) {
- commit.session().publish(FAILED, SERIALIZER::encode, new LockEvent(commit.value().id(), commit.index()));
- // If a timeout exists, add the request to the queue and set a timer. Note that the lock request expiration
- // time is based on the *state machine* time - not the system time - to ensure consistency across servers.
- } else if (commit.value().timeout() > 0) {
- LockHolder holder = new LockHolder(
- commit.value().id(),
- commit.index(),
- commit.session().sessionId().id(),
- wallClock().getTime().unixTimestamp() + commit.value().timeout());
- queue.add(holder);
- timers.put(commit.index(), scheduler().schedule(Duration.ofMillis(commit.value().timeout()), () -> {
- // When the lock request timer expires, remove the request from the queue and publish a FAILED
- // event to the session. Note that this timer is guaranteed to be executed in the same thread as the
- // state machine commands, so there's no need to use a lock here.
- timers.remove(commit.index());
- queue.remove(holder);
- if (commit.session().getState().active()) {
- commit.session().publish(
- FAILED,
- SERIALIZER::encode,
- new LockEvent(commit.value().id(), commit.index()));
- }
- }));
- // If the lock is -1, just add the request to the queue with no expiration.
- } else {
- LockHolder holder = new LockHolder(
- commit.value().id(),
- commit.index(),
- commit.session().sessionId().id(),
- 0);
- queue.add(holder);
- }
- }
-
- /**
- * Applies an unlock commit.
- *
- * @param commit the unlock commit
- */
- protected void unlock(Commit<Unlock> commit) {
- if (lock != null) {
- // If the commit's session does not match the current lock holder, ignore the request.
- if (lock.session != commit.session().sessionId().id()) {
- return;
- }
-
- // If the current lock ID does not match the requested lock ID, ignore the request. This ensures that
- // internal releases of locks that were never acquired by the client-side primitive do not cause
- // legitimate locks to be unlocked.
- if (lock.id != commit.value().id()) {
- return;
- }
-
- // The lock has been released. Populate the lock from the queue.
- lock = queue.poll();
- while (lock != null) {
- // If the waiter has a lock timer, cancel the timer.
- Scheduled timer = timers.remove(lock.index);
- if (timer != null) {
- timer.cancel();
- }
-
- // If the lock session is for some reason inactive, continue on to the next waiter. Otherwise,
- // publish a LOCKED event to the new lock holder's session.
- RaftSession session = sessions().getSession(lock.session);
- if (session == null || !session.getState().active()) {
- lock = queue.poll();
- } else {
- session.publish(
- LOCKED,
- SERIALIZER::encode,
- new LockEvent(lock.id, commit.index()));
- break;
- }
- }
- }
- }
-
- /**
- * Handles a session that has been closed by a client or expired by the cluster.
- * <p>
- * When a session is removed, if the session is the current lock holder then the lock is released and the next
- * session waiting in the queue is granted the lock. Additionally, all pending lock requests for the session
- * are removed from the lock queue.
- *
- * @param session the closed session
- */
- private void releaseSession(RaftSession session) {
- // Remove all instances of the session from the lock queue.
- queue.removeIf(lock -> lock.session == session.sessionId().id());
-
- // If the removed session is the current holder of the lock, nullify the lock and attempt to grant it
- // to the next waiter in the queue.
- if (lock != null && lock.session == session.sessionId().id()) {
- lock = queue.poll();
- while (lock != null) {
- // If the waiter has a lock timer, cancel the timer.
- Scheduled timer = timers.remove(lock.index);
- if (timer != null) {
- timer.cancel();
- }
-
- // If the lock session is inactive, continue on to the next waiter. Otherwise,
- // publish a LOCKED event to the new lock holder's session.
- RaftSession lockSession = sessions().getSession(lock.session);
- if (lockSession == null || !lockSession.getState().active()) {
- lock = queue.poll();
- } else {
- lockSession.publish(
- LOCKED,
- SERIALIZER::encode,
- new LockEvent(lock.id, lock.index));
- break;
- }
- }
- }
- }
-
- private class LockHolder {
- private final int id;
- private final long index;
- private final long session;
- private final long expire;
-
- public LockHolder(int id, long index, long session, long expire) {
- this.id = id;
- this.index = index;
- this.session = session;
- this.expire = expire;
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("id", id)
- .add("index", index)
- .add("session", session)
- .add("expire", expire)
- .toString();
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTree.java
deleted file mode 100644
index 877d3fe..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTree.java
+++ /dev/null
@@ -1,332 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-
-import com.google.common.util.concurrent.MoreExecutors;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GetChildren;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Update;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionRollback;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.IllegalDocumentModificationException;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.BEGIN;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.PREPARE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.PREPARE_AND_COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ROLLBACK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET_CHILDREN;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.UPDATE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeResult.Status.ILLEGAL_MODIFICATION;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeResult.Status.INVALID_PATH;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeResult.Status.OK;
-
-/**
- * Distributed resource providing the {@link AsyncDocumentTree} primitive.
- */
-public class AtomixDocumentTree extends AbstractRaftPrimitive implements AsyncDocumentTree<byte[]> {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixDocumentTreeOperations.NAMESPACE)
- .register(AtomixDocumentTreeEvents.NAMESPACE)
- .build());
-
- private final Map<DocumentTreeListener<byte[]>, InternalListener> eventListeners = new ConcurrentHashMap<>();
-
- public AtomixDocumentTree(RaftProxy proxy) {
- super(proxy);
- proxy.addStateChangeListener(state -> {
- if (state == RaftProxy.State.CONNECTED && isListening()) {
- proxy.invoke(ADD_LISTENER, SERIALIZER::encode, new Listen());
- }
- });
- proxy.addEventListener(CHANGE, SERIALIZER::decode, this::processTreeUpdates);
- }
-
- @Override
- public Type primitiveType() {
- return Type.DOCUMENT_TREE;
- }
-
- @Override
- public CompletableFuture<Void> destroy() {
- return proxy.invoke(CLEAR);
- }
-
- @Override
- public DocumentPath root() {
- return DocumentPath.ROOT;
- }
-
- @Override
- public CompletableFuture<Map<String, Versioned<byte[]>>> getChildren(DocumentPath path) {
- return proxy.<GetChildren, DocumentTreeResult<Map<String, Versioned<byte[]>>>>invoke(
- GET_CHILDREN,
- SERIALIZER::encode,
- new GetChildren(checkNotNull(path)),
- SERIALIZER::decode)
- .thenCompose(result -> {
- if (result.status() == INVALID_PATH) {
- return Tools.exceptionalFuture(new NoSuchDocumentPathException());
- } else if (result.status() == ILLEGAL_MODIFICATION) {
- return Tools.exceptionalFuture(new IllegalDocumentModificationException());
- } else {
- return CompletableFuture.completedFuture(result);
- }
- }).thenApply(result -> result.result());
- }
-
- @Override
- public CompletableFuture<Versioned<byte[]>> get(DocumentPath path) {
- return proxy.invoke(GET, SERIALIZER::encode, new Get(checkNotNull(path)), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Versioned<byte[]>> set(DocumentPath path, byte[] value) {
- return proxy.<Update, DocumentTreeResult<Versioned<byte[]>>>invoke(UPDATE,
- SERIALIZER::encode,
- new Update(checkNotNull(path), Optional.ofNullable(value), Match.any(), Match.any()),
- SERIALIZER::decode)
- .thenCompose(result -> {
- if (result.status() == INVALID_PATH) {
- return Tools.exceptionalFuture(new NoSuchDocumentPathException());
- } else if (result.status() == ILLEGAL_MODIFICATION) {
- return Tools.exceptionalFuture(new IllegalDocumentModificationException());
- } else {
- return CompletableFuture.completedFuture(result);
- }
- }).thenApply(result -> result.result());
- }
-
- @Override
- public CompletableFuture<Boolean> create(DocumentPath path, byte[] value) {
- return createInternal(path, value)
- .thenCompose(status -> {
- if (status == ILLEGAL_MODIFICATION) {
- return Tools.exceptionalFuture(new IllegalDocumentModificationException());
- }
- return CompletableFuture.completedFuture(true);
- });
- }
-
- @Override
- public CompletableFuture<Boolean> createRecursive(DocumentPath path, byte[] value) {
- return createInternal(path, value)
- .thenCompose(status -> {
- if (status == ILLEGAL_MODIFICATION) {
- return createRecursive(path.parent(), null)
- .thenCompose(r -> createInternal(path, value).thenApply(v -> true));
- }
- return CompletableFuture.completedFuture(status == OK);
- });
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, byte[] newValue, long version) {
- return proxy.<Update, DocumentTreeResult<byte[]>>invoke(UPDATE,
- SERIALIZER::encode,
- new Update(checkNotNull(path),
- Optional.ofNullable(newValue),
- Match.any(),
- Match.ifValue(version)), SERIALIZER::decode)
- .thenApply(result -> result.updated());
- }
-
- @Override
- public CompletableFuture<Boolean> replace(DocumentPath path, byte[] newValue, byte[] currentValue) {
- return proxy.<Update, DocumentTreeResult<byte[]>>invoke(UPDATE,
- SERIALIZER::encode,
- new Update(checkNotNull(path),
- Optional.ofNullable(newValue),
- Match.ifValue(currentValue),
- Match.any()),
- SERIALIZER::decode)
- .thenCompose(result -> {
- if (result.status() == INVALID_PATH) {
- return Tools.exceptionalFuture(new NoSuchDocumentPathException());
- } else if (result.status() == ILLEGAL_MODIFICATION) {
- return Tools.exceptionalFuture(new IllegalDocumentModificationException());
- } else {
- return CompletableFuture.completedFuture(result);
- }
- }).thenApply(result -> result.updated());
- }
-
- @Override
- public CompletableFuture<Versioned<byte[]>> removeNode(DocumentPath path) {
- if (path.equals(DocumentPath.from("root"))) {
- return Tools.exceptionalFuture(new IllegalDocumentModificationException());
- }
- return proxy.<Update, DocumentTreeResult<Versioned<byte[]>>>invoke(UPDATE,
- SERIALIZER::encode,
- new Update(checkNotNull(path), null, Match.any(), Match.ifNotNull()),
- SERIALIZER::decode)
- .thenCompose(result -> {
- if (result.status() == INVALID_PATH) {
- return Tools.exceptionalFuture(new NoSuchDocumentPathException());
- } else if (result.status() == ILLEGAL_MODIFICATION) {
- return Tools.exceptionalFuture(new IllegalDocumentModificationException());
- } else {
- return CompletableFuture.completedFuture(result);
- }
- }).thenApply(result -> result.result());
- }
-
- @Override
- public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<byte[]> listener) {
- checkNotNull(path);
- checkNotNull(listener);
- InternalListener internalListener = new InternalListener(path, listener, MoreExecutors.directExecutor());
- // TODO: Support API that takes an executor
- if (!eventListeners.containsKey(listener)) {
- return proxy.invoke(ADD_LISTENER, SERIALIZER::encode, new Listen(path))
- .thenRun(() -> eventListeners.put(listener, internalListener));
- }
- return CompletableFuture.completedFuture(null);
- }
-
- @Override
- public CompletableFuture<Void> removeListener(DocumentTreeListener<byte[]> listener) {
- checkNotNull(listener);
- InternalListener internalListener = eventListeners.remove(listener);
- if (internalListener != null && eventListeners.isEmpty()) {
- return proxy.invoke(REMOVE_LISTENER, SERIALIZER::encode, new Unlisten(internalListener.path))
- .thenApply(v -> null);
- }
- return CompletableFuture.completedFuture(null);
- }
-
- private CompletableFuture<DocumentTreeResult.Status> createInternal(DocumentPath path, byte[] value) {
- return proxy.<Update, DocumentTreeResult<byte[]>>invoke(UPDATE,
- SERIALIZER::encode,
- new Update(checkNotNull(path), Optional.ofNullable(value), Match.any(), Match.ifNull()),
- SERIALIZER::decode)
- .thenApply(result -> result.status());
- }
-
- private boolean isListening() {
- return !eventListeners.isEmpty();
- }
-
- private void processTreeUpdates(List<DocumentTreeEvent<byte[]>> events) {
- events.forEach(event -> eventListeners.values().forEach(listener -> listener.event(event)));
- }
-
- @Override
- public CompletableFuture<Version> begin(TransactionId transactionId) {
- return proxy.<TransactionBegin, Long>invoke(
- BEGIN,
- SERIALIZER::encode,
- new TransactionBegin(transactionId),
- SERIALIZER::decode)
- .thenApply(Version::new);
- }
-
- @Override
- public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<byte[]>> transactionLog) {
- return proxy.<TransactionPrepare, PrepareResult>invoke(
- PREPARE,
- SERIALIZER::encode,
- new TransactionPrepare(transactionLog),
- SERIALIZER::decode)
- .thenApply(v -> v == PrepareResult.OK);
- }
-
- @Override
- public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<byte[]>> transactionLog) {
- return proxy.<TransactionPrepareAndCommit, PrepareResult>invoke(
- PREPARE_AND_COMMIT,
- SERIALIZER::encode,
- new TransactionPrepareAndCommit(transactionLog),
- SERIALIZER::decode)
- .thenApply(v -> v == PrepareResult.OK);
- }
-
- @Override
- public CompletableFuture<Void> commit(TransactionId transactionId) {
- return proxy.<TransactionCommit, CommitResult>invoke(
- COMMIT,
- SERIALIZER::encode,
- new TransactionCommit(transactionId),
- SERIALIZER::decode)
- .thenApply(v -> null);
- }
-
- @Override
- public CompletableFuture<Void> rollback(TransactionId transactionId) {
- return proxy.invoke(
- ROLLBACK,
- SERIALIZER::encode,
- new TransactionRollback(transactionId),
- SERIALIZER::decode)
- .thenApply(v -> null);
- }
-
- private class InternalListener implements DocumentTreeListener<byte[]> {
-
- private final DocumentPath path;
- private final DocumentTreeListener<byte[]> listener;
- private final Executor executor;
-
- public InternalListener(DocumentPath path, DocumentTreeListener<byte[]> listener, Executor executor) {
- this.path = path;
- this.listener = listener;
- this.executor = executor;
- }
-
- @Override
- public void event(DocumentTreeEvent<byte[]> event) {
- if (event.path().isDescendentOf(path)) {
- executor.execute(() -> listener.event(event));
- }
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeEvents.java
deleted file mode 100644
index 47dcd31..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeEvents.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.DocumentTreeEvent;
-
-/**
- * Atomix document tree events.
- */
-public enum AtomixDocumentTreeEvents implements EventType {
- CHANGE;
-
- @Override
- public String id() {
- return name();
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
- .register(DocumentTreeEvent.class)
- .register(DocumentTreeEvent.Type.class)
- .build("AtomixDocumentTreeEvents");
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeOperations.java
deleted file mode 100644
index 6ec6091..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeOperations.java
+++ /dev/null
@@ -1,370 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.LinkedHashMap;
-import java.util.Optional;
-
-import com.google.common.base.MoreObjects;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-/**
- * {@link AtomixDocumentTree} resource state machine operations.
- */
-public enum AtomixDocumentTreeOperations implements OperationId {
- ADD_LISTENER(OperationType.COMMAND),
- REMOVE_LISTENER(OperationType.COMMAND),
- GET(OperationType.QUERY),
- GET_CHILDREN(OperationType.QUERY),
- UPDATE(OperationType.COMMAND),
- CLEAR(OperationType.COMMAND),
- BEGIN(OperationType.COMMAND),
- PREPARE(OperationType.COMMAND),
- PREPARE_AND_COMMIT(OperationType.COMMAND),
- COMMIT(OperationType.COMMAND),
- ROLLBACK(OperationType.COMMAND);
-
- private final OperationType type;
-
- AtomixDocumentTreeOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(LinkedHashMap.class)
- .register(Listen.class)
- .register(Unlisten.class)
- .register(Get.class)
- .register(GetChildren.class)
- .register(Update.class)
- .register(TransactionBegin.class)
- .register(TransactionPrepare.class)
- .register(TransactionPrepareAndCommit.class)
- .register(TransactionCommit.class)
- .register(TransactionRollback.class)
- .register(TransactionId.class)
- .register(TransactionLog.class)
- .register(PrepareResult.class)
- .register(CommitResult.class)
- .register(RollbackResult.class)
- .register(NodeUpdate.class)
- .register(NodeUpdate.Type.class)
- .register(DocumentPath.class)
- .register(Match.class)
- .register(Versioned.class)
- .register(DocumentTreeResult.class)
- .register(DocumentTreeResult.Status.class)
- .build("AtomixDocumentTreeOperations");
-
- /**
- * Base class for document tree operations.
- */
- public abstract static class DocumentTreeOperation {
- }
-
- /**
- * Base class for document tree operations that serialize a {@link DocumentPath}.
- */
- @SuppressWarnings("serial")
- public abstract static class PathOperation extends DocumentTreeOperation {
- private DocumentPath path;
-
- PathOperation(DocumentPath path) {
- this.path = path;
- }
-
- public DocumentPath path() {
- return path;
- }
- }
-
- /**
- * DocumentTree#get query.
- */
- @SuppressWarnings("serial")
- public static class Get extends PathOperation {
- public Get() {
- super(null);
- }
-
- public Get(DocumentPath path) {
- super(path);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("path", path())
- .toString();
- }
- }
-
- /**
- * DocumentTree#getChildren query.
- */
- @SuppressWarnings("serial")
- public static class GetChildren extends PathOperation {
- public GetChildren() {
- super(null);
- }
-
- public GetChildren(DocumentPath path) {
- super(path);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("path", path())
- .toString();
- }
- }
-
- /**
- * DocumentTree update command.
- */
- @SuppressWarnings("serial")
- public static class Update extends PathOperation {
- private Optional<byte[]> value;
- private Match<byte[]> valueMatch;
- private Match<Long> versionMatch;
-
- public Update() {
- super(null);
- this.value = null;
- this.valueMatch = null;
- this.versionMatch = null;
- }
-
- public Update(DocumentPath path, Optional<byte[]> value, Match<byte[]> valueMatch, Match<Long> versionMatch) {
- super(path);
- this.value = value;
- this.valueMatch = valueMatch;
- this.versionMatch = versionMatch;
- }
-
- public Optional<byte[]> value() {
- return value;
- }
-
- public Match<byte[]> valueMatch() {
- return valueMatch;
- }
-
- public Match<Long> versionMatch() {
- return versionMatch;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("path", path())
- .add("value", value)
- .add("valueMatch", valueMatch)
- .add("versionMatch", versionMatch)
- .toString();
- }
- }
-
- /**
- * Change listen.
- */
- @SuppressWarnings("serial")
- public static class Listen extends PathOperation {
- public Listen() {
- this(DocumentPath.from("root"));
- }
-
- public Listen(DocumentPath path) {
- super(path);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("path", path())
- .toString();
- }
- }
-
- /**
- * Change unlisten.
- */
- @SuppressWarnings("serial")
- public static class Unlisten extends PathOperation {
- public Unlisten() {
- this(DocumentPath.from("root"));
- }
-
- public Unlisten(DocumentPath path) {
- super(path);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("path", path())
- .toString();
- }
- }
-
- /**
- * Transaction begin command.
- */
- public static class TransactionBegin extends PathOperation {
- private TransactionId transactionId;
-
- public TransactionBegin() {
- super(null);
- }
-
- public TransactionBegin(TransactionId transactionId) {
- super(DocumentPath.from(transactionId.toString()));
- this.transactionId = transactionId;
- }
-
- public TransactionId transactionId() {
- return transactionId;
- }
- }
-
- /**
- * Transaction prepare command.
- */
- @SuppressWarnings("serial")
- public static class TransactionPrepare extends PathOperation {
- private TransactionLog<NodeUpdate<byte[]>> transactionLog;
-
- public TransactionPrepare() {
- super(null);
- }
-
- public TransactionPrepare(TransactionLog<NodeUpdate<byte[]>> transactionLog) {
- super(DocumentPath.from(transactionLog.transactionId().toString()));
- this.transactionLog = transactionLog;
- }
-
- public TransactionLog<NodeUpdate<byte[]>> transactionLog() {
- return transactionLog;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("transactionLog", transactionLog)
- .toString();
- }
- }
-
- /**
- * Transaction prepareAndCommit command.
- */
- @SuppressWarnings("serial")
- public static class TransactionPrepareAndCommit extends TransactionPrepare {
- public TransactionPrepareAndCommit() {
- }
-
- public TransactionPrepareAndCommit(TransactionLog<NodeUpdate<byte[]>> transactionLog) {
- super(transactionLog);
- }
- }
-
- /**
- * Transaction commit command.
- */
- @SuppressWarnings("serial")
- public static class TransactionCommit extends PathOperation {
- private TransactionId transactionId;
-
- public TransactionCommit() {
- super(null);
- }
-
- public TransactionCommit(TransactionId transactionId) {
- super(DocumentPath.from(transactionId.toString()));
- this.transactionId = transactionId;
- }
-
- /**
- * Returns the transaction identifier.
- * @return transaction id
- */
- public TransactionId transactionId() {
- return transactionId;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("transactionId", transactionId)
- .toString();
- }
- }
-
- /**
- * Transaction rollback command.
- */
- @SuppressWarnings("serial")
- public static class TransactionRollback extends PathOperation {
- private TransactionId transactionId;
-
- public TransactionRollback() {
- super(null);
- }
-
- public TransactionRollback(TransactionId transactionId) {
- super(DocumentPath.from(transactionId.toString()));
- this.transactionId = transactionId;
- }
-
- /**
- * Returns the transaction identifier.
- * @return transaction id
- */
- public TransactionId transactionId() {
- return transactionId;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("transactionId", transactionId)
- .toString();
- }
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeService.java
deleted file mode 100644
index 41a1a9f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeService.java
+++ /dev/null
@@ -1,622 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Queue;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Queues;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.event.EventType;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GetChildren;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Update;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionRollback;
-
-import org.onosproject.store.primitives.resources.impl.DocumentTreeResult.Status;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTree;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeEvent.Type;
-import org.onosproject.store.service.IllegalDocumentModificationException;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkState;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.BEGIN;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.PREPARE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.PREPARE_AND_COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ROLLBACK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET_CHILDREN;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.UPDATE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.REMOVE_LISTENER;
-
-/**
- * State Machine for {@link AtomixDocumentTree} resource.
- */
-public class AtomixDocumentTreeService extends AbstractRaftService {
- private final Serializer serializer = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixDocumentTreeOperations.NAMESPACE)
- .register(AtomixDocumentTreeEvents.NAMESPACE)
- .register(new com.esotericsoftware.kryo.Serializer<Listener>() {
- @Override
- public void write(Kryo kryo, Output output, Listener listener) {
- output.writeLong(listener.session.sessionId().id());
- kryo.writeObject(output, listener.path);
- }
-
- @Override
- public Listener read(Kryo kryo, Input input, Class<Listener> type) {
- return new Listener(sessions().getSession(input.readLong()),
- kryo.readObjectOrNull(input, DocumentPath.class));
- }
- }, Listener.class)
- .register(Versioned.class)
- .register(DocumentPath.class)
- .register(new LinkedHashMap().keySet().getClass())
- .register(TreeMap.class)
- .register(Ordering.class)
- .register(TransactionScope.class)
- .register(TransactionLog.class)
- .register(TransactionId.class)
- .register(SessionListenCommits.class)
- .register(new com.esotericsoftware.kryo.Serializer<DefaultDocumentTree>() {
- @Override
- public void write(Kryo kryo, Output output, DefaultDocumentTree object) {
- kryo.writeObject(output, object.root);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public DefaultDocumentTree read(Kryo kryo, Input input, Class<DefaultDocumentTree> type) {
- return new DefaultDocumentTree(versionCounter::incrementAndGet,
- kryo.readObject(input, DefaultDocumentTreeNode.class));
- }
- }, DefaultDocumentTree.class)
- .register(DefaultDocumentTreeNode.class)
- .build());
-
- private Map<Long, SessionListenCommits> listeners = new HashMap<>();
- private AtomicLong versionCounter = new AtomicLong(0);
- private DocumentTree<byte[]> docTree;
- private Map<TransactionId, TransactionScope> activeTransactions = Maps.newHashMap();
- private Set<DocumentPath> preparedKeys = Sets.newHashSet();
-
- public AtomixDocumentTreeService(Ordering ordering) {
- this.docTree = new DefaultDocumentTree<>(versionCounter::incrementAndGet, ordering);
- }
-
- @Override
- public void snapshot(SnapshotWriter writer) {
- writer.writeLong(versionCounter.get());
- writer.writeObject(listeners, serializer::encode);
- writer.writeObject(docTree, serializer::encode);
- writer.writeObject(preparedKeys, serializer::encode);
- writer.writeObject(activeTransactions, serializer::encode);
- }
-
- @Override
- public void install(SnapshotReader reader) {
- versionCounter = new AtomicLong(reader.readLong());
- listeners = reader.readObject(serializer::decode);
- docTree = reader.readObject(serializer::decode);
- preparedKeys = reader.readObject(serializer::decode);
- activeTransactions = reader.readObject(serializer::decode);
- }
-
- @Override
- protected void configure(RaftServiceExecutor executor) {
- // Listeners
- executor.register(ADD_LISTENER, serializer::decode, this::listen);
- executor.register(REMOVE_LISTENER, serializer::decode, this::unlisten);
- // queries
- executor.register(GET, serializer::decode, this::get, serializer::encode);
- executor.register(GET_CHILDREN, serializer::decode, this::getChildren, serializer::encode);
- // commands
- executor.register(UPDATE, serializer::decode, this::update, serializer::encode);
- executor.register(CLEAR, this::clear);
- executor.register(BEGIN, serializer::decode, this::begin, serializer::encode);
- executor.register(PREPARE, serializer::decode, this::prepare, serializer::encode);
- executor.register(PREPARE_AND_COMMIT, serializer::decode, this::prepareAndCommit, serializer::encode);
- executor.register(COMMIT, serializer::decode, this::commit, serializer::encode);
- executor.register(ROLLBACK, serializer::decode, this::rollback, serializer::encode);
- }
-
- /**
- * Returns a boolean indicating whether the given path is currently locked by a transaction.
- *
- * @param path the path to check
- * @return whether the given path is locked by a running transaction
- */
- private boolean isLocked(DocumentPath path) {
- return preparedKeys.contains(path);
- }
-
- protected void listen(Commit<? extends Listen> commit) {
- Long sessionId = commit.session().sessionId().id();
- listeners.computeIfAbsent(sessionId, k -> new SessionListenCommits())
- .add(new Listener(commit.session(), commit.value().path()));
- }
-
- protected void unlisten(Commit<? extends Unlisten> commit) {
- Long sessionId = commit.session().sessionId().id();
- SessionListenCommits listenCommits = listeners.get(sessionId);
- if (listenCommits != null) {
- listenCommits.remove(commit);
- }
- }
-
- protected Versioned<byte[]> get(Commit<? extends Get> commit) {
- try {
- Versioned<byte[]> value = docTree.get(commit.value().path());
- return value == null ? null : value.map(node -> node);
- } catch (IllegalStateException e) {
- return null;
- }
- }
-
- protected DocumentTreeResult<Map<String, Versioned<byte[]>>> getChildren(Commit<? extends GetChildren> commit) {
- try {
- return DocumentTreeResult.ok(docTree.getChildren(commit.value().path()));
- } catch (NoSuchDocumentPathException e) {
- return DocumentTreeResult.invalidPath();
- }
- }
-
- protected DocumentTreeResult<Versioned<byte[]>> update(Commit<? extends Update> commit) {
- DocumentTreeResult<Versioned<byte[]>> result = null;
- DocumentPath path = commit.value().path();
-
- // If the path is locked by a transaction, return a WRITE_LOCK error.
- if (isLocked(path)) {
- return DocumentTreeResult.writeLock();
- }
-
- Versioned<byte[]> currentValue = docTree.get(path);
- try {
- Match<Long> versionMatch = commit.value().versionMatch();
- Match<byte[]> valueMatch = commit.value().valueMatch();
-
- if (versionMatch.matches(currentValue == null ? null : currentValue.version())
- && valueMatch.matches(currentValue == null ? null : currentValue.value())) {
- if (commit.value().value() == null) {
- Versioned<byte[]> oldValue = docTree.removeNode(path);
- result = new DocumentTreeResult<>(Status.OK, oldValue);
- if (oldValue != null) {
- notifyListeners(new DocumentTreeEvent<>(
- path,
- Type.DELETED,
- Optional.empty(),
- Optional.of(oldValue)));
- }
- } else {
- Versioned<byte[]> oldValue = docTree.set(path, commit.value().value().orElse(null));
- Versioned<byte[]> newValue = docTree.get(path);
- result = new DocumentTreeResult<>(Status.OK, newValue);
- if (oldValue == null) {
- notifyListeners(new DocumentTreeEvent<>(
- path,
- Type.CREATED,
- Optional.of(newValue),
- Optional.empty()));
- } else {
- notifyListeners(new DocumentTreeEvent<>(
- path,
- Type.UPDATED,
- Optional.of(newValue),
- Optional.of(oldValue)));
- }
- }
- } else {
- result = new DocumentTreeResult<>(
- commit.value().value() == null ? Status.INVALID_PATH : Status.NOOP,
- currentValue);
- }
- } catch (IllegalDocumentModificationException e) {
- result = DocumentTreeResult.illegalModification();
- } catch (NoSuchDocumentPathException e) {
- result = DocumentTreeResult.invalidPath();
- } catch (Exception e) {
- logger().error("Failed to apply {} to state machine", commit.value(), e);
- throw new IllegalStateException(e);
- }
- return result;
- }
-
- protected void clear(Commit<Void> commit) {
- Queue<DocumentPath> toClearQueue = Queues.newArrayDeque();
- Map<String, Versioned<byte[]>> topLevelChildren = docTree.getChildren(DocumentPath.from("root"));
- toClearQueue.addAll(topLevelChildren.keySet()
- .stream()
- .map(name -> new DocumentPath(name, DocumentPath.from("root")))
- .collect(Collectors.toList()));
- while (!toClearQueue.isEmpty()) {
- DocumentPath path = toClearQueue.remove();
- Map<String, Versioned<byte[]>> children = docTree.getChildren(path);
- if (children.size() == 0) {
- docTree.removeNode(path);
- } else {
- children.keySet().forEach(name -> toClearQueue.add(new DocumentPath(name, path)));
- toClearQueue.add(path);
- }
- }
- }
-
- /**
- * Handles a begin commit.
- *
- * @param commit transaction begin commit
- * @return transaction state version
- */
- protected long begin(Commit<? extends TransactionBegin> commit) {
- long version = commit.index();
- activeTransactions.put(commit.value().transactionId(), new TransactionScope(version));
- return version;
- }
-
- /**
- * Handles an prepare commit.
- *
- * @param commit transaction prepare commit
- * @return prepare result
- */
- protected PrepareResult prepare(Commit<? extends TransactionPrepare> commit) {
- try {
- TransactionLog<NodeUpdate<byte[]>> transactionLog = commit.value().transactionLog();
- // Iterate through records in the transaction log and perform isolation checks.
- for (NodeUpdate<byte[]> record : transactionLog.records()) {
- DocumentPath path = record.path();
-
- // If the prepared keys already contains the key contained within the record, that indicates a
- // conflict with a concurrent transaction.
- if (preparedKeys.contains(path)) {
- return PrepareResult.CONCURRENT_TRANSACTION;
- }
-
- // Read the existing value from the map.
- Versioned<byte[]> existingValue = docTree.get(path);
-
- // If the update is an UPDATE_NODE or DELETE_NODE, verify that versions match.
- switch (record.type()) {
- case UPDATE_NODE:
- case DELETE_NODE:
- if (existingValue == null || existingValue.version() != record.version()) {
- return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
- }
- default:
- break;
- }
- }
-
- // No violations detected. Mark modified keys locked for transactions.
- transactionLog.records().forEach(record -> {
- preparedKeys.add(record.path());
- });
-
- // Update the transaction scope. If the transaction scope is not set on this node, that indicates the
- // coordinator is communicating with another node. Transactions assume that the client is communicating
- // with a single leader in order to limit the overhead of retaining tombstones.
- TransactionScope transactionScope = activeTransactions.get(transactionLog.transactionId());
- if (transactionScope == null) {
- activeTransactions.put(
- transactionLog.transactionId(),
- new TransactionScope(transactionLog.version(), commit.value().transactionLog()));
- return PrepareResult.PARTIAL_FAILURE;
- } else {
- activeTransactions.put(
- transactionLog.transactionId(),
- transactionScope.prepared(commit));
- return PrepareResult.OK;
- }
- } catch (Exception e) {
- logger().warn("Failure applying {}", commit, e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Handles an prepare and commit commit.
- *
- * @param commit transaction prepare and commit commit
- * @return prepare result
- */
- protected PrepareResult prepareAndCommit(Commit<? extends TransactionPrepareAndCommit> commit) {
- TransactionId transactionId = commit.value().transactionLog().transactionId();
- PrepareResult prepareResult = prepare(commit);
- TransactionScope transactionScope = activeTransactions.remove(transactionId);
- if (prepareResult == PrepareResult.OK) {
- transactionScope = transactionScope.prepared(commit);
- commitTransaction(transactionScope);
- }
- return prepareResult;
- }
-
- /**
- * Applies committed operations to the state machine.
- */
- private CommitResult commitTransaction(TransactionScope transactionScope) {
- TransactionLog<NodeUpdate<byte[]>> transactionLog = transactionScope.transactionLog();
-
- List<DocumentTreeEvent<byte[]>> eventsToPublish = Lists.newArrayList();
- DocumentTreeEvent<byte[]> start = new DocumentTreeEvent<>(
- DocumentPath.from(transactionScope.transactionLog().transactionId().toString()),
- Type.TRANSACTION_START,
- Optional.empty(),
- Optional.empty());
- eventsToPublish.add(start);
-
- for (NodeUpdate<byte[]> record : transactionLog.records()) {
- DocumentPath path = record.path();
- checkState(preparedKeys.remove(path), "path is not prepared");
-
- // FIXME revisit this block, it never respects NodeUpdate type
-
- Versioned<byte[]> previousValue = null;
- try {
- previousValue = docTree.removeNode(path);
- } catch (NoSuchDocumentPathException e) {
- logger().info("Value is being inserted first time");
- }
-
- if (record.value() != null) {
- if (docTree.create(path, record.value())) {
- Versioned<byte[]> newValue = docTree.get(path);
- eventsToPublish.add(new DocumentTreeEvent<>(
- path,
- Optional.ofNullable(newValue),
- Optional.ofNullable(previousValue)));
- }
- } else if (previousValue != null) {
- eventsToPublish.add(new DocumentTreeEvent<>(
- path,
- Optional.empty(),
- Optional.of(previousValue)));
- }
- }
-
- DocumentTreeEvent<byte[]> end = new DocumentTreeEvent<>(
- DocumentPath.from(transactionScope.transactionLog().transactionId().toString()),
- Type.TRANSACTION_END,
- Optional.empty(),
- Optional.empty());
- eventsToPublish.add(end);
- publish(eventsToPublish);
-
- return CommitResult.OK;
- }
-
- /**
- * Handles an commit commit (ha!).
- *
- * @param commit transaction commit commit
- * @return commit result
- */
- protected CommitResult commit(Commit<? extends TransactionCommit> commit) {
- TransactionId transactionId = commit.value().transactionId();
- TransactionScope transactionScope = activeTransactions.remove(transactionId);
- if (transactionScope == null) {
- return CommitResult.UNKNOWN_TRANSACTION_ID;
- }
- try {
- return commitTransaction(transactionScope);
- } catch (Exception e) {
- logger().warn("Failure applying {}", commit, e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Handles an rollback commit (ha!).
- *
- * @param commit transaction rollback commit
- * @return rollback result
- */
- protected RollbackResult rollback(Commit<? extends TransactionRollback> commit) {
- TransactionId transactionId = commit.value().transactionId();
- TransactionScope transactionScope = activeTransactions.remove(transactionId);
- if (transactionScope == null) {
- return RollbackResult.UNKNOWN_TRANSACTION_ID;
- } else if (!transactionScope.isPrepared()) {
- return RollbackResult.OK;
- } else {
- transactionScope.transactionLog().records()
- .forEach(record -> {
- preparedKeys.remove(record.path());
- });
- return RollbackResult.OK;
- }
-
- }
-
- /**
- * Map transaction scope.
- */
- private static final class TransactionScope {
- private final long version;
- private final TransactionLog<NodeUpdate<byte[]>> transactionLog;
-
- private TransactionScope(long version) {
- this(version, null);
- }
-
- private TransactionScope(long version, TransactionLog<NodeUpdate<byte[]>> transactionLog) {
- this.version = version;
- this.transactionLog = transactionLog;
- }
-
- /**
- * Returns the transaction version.
- *
- * @return the transaction version
- */
- long version() {
- return version;
- }
-
- /**
- * Returns whether this is a prepared transaction scope.
- *
- * @return whether this is a prepared transaction scope
- */
- boolean isPrepared() {
- return transactionLog != null;
- }
-
- /**
- * Returns the transaction commit log.
- *
- * @return the transaction commit log
- */
- TransactionLog<NodeUpdate<byte[]>> transactionLog() {
- checkState(isPrepared(), "Transaction not prepared");
- return transactionLog;
- }
-
- /**
- * Returns a new transaction scope with a prepare commit.
- *
- * @param commit the prepare commit
- * @return new transaction scope updated with the prepare commit
- */
- TransactionScope prepared(Commit<? extends TransactionPrepare> commit) {
- return new TransactionScope(version, commit.value().transactionLog());
- }
- }
-
- private void publish(List<DocumentTreeEvent<byte[]>> events) {
- listeners.values().forEach(session -> {
- session.publish(CHANGE, events);
- });
- }
-
- private void notifyListeners(DocumentTreeEvent<byte[]> event) {
- listeners.values()
- .stream()
- .filter(l -> event.path().isDescendentOf(l.leastCommonAncestorPath()))
- .forEach(listener -> listener.publish(CHANGE, Arrays.asList(event)));
- }
-
- @Override
- public void onExpire(RaftSession session) {
- closeListener(session.sessionId().id());
- }
-
- @Override
- public void onClose(RaftSession session) {
- closeListener(session.sessionId().id());
- }
-
- private void closeListener(Long sessionId) {
- listeners.remove(sessionId);
- }
-
- private class SessionListenCommits {
- private final List<Listener> listeners = Lists.newArrayList();
- private DocumentPath leastCommonAncestorPath;
-
- public void add(Listener listener) {
- listeners.add(listener);
- recomputeLeastCommonAncestor();
- }
-
- public void remove(Commit<? extends Unlisten> commit) {
- // Remove the first listen commit with path matching path in unlisten commit
- Iterator<Listener> iterator = listeners.iterator();
- while (iterator.hasNext()) {
- Listener listener = iterator.next();
- if (listener.path().equals(commit.value().path())) {
- iterator.remove();
- }
- }
- recomputeLeastCommonAncestor();
- }
-
- public DocumentPath leastCommonAncestorPath() {
- return leastCommonAncestorPath;
- }
-
- public <M> void publish(EventType topic, M message) {
- listeners.stream().findAny().ifPresent(listener ->
- listener.session().publish(topic, serializer::encode, message));
- }
-
- private void recomputeLeastCommonAncestor() {
- this.leastCommonAncestorPath = DocumentPath.leastCommonAncestor(listeners.stream()
- .map(Listener::path)
- .collect(Collectors.toList()));
- }
- }
-
- private static class Listener {
- private final RaftSession session;
- private final DocumentPath path;
-
- public Listener(RaftSession session, DocumentPath path) {
- this.session = session;
- this.path = path;
- }
-
- public DocumentPath path() {
- return path;
- }
-
- public RaftSession session() {
- return session;
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixIdGenerator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixIdGenerator.java
deleted file mode 100644
index 0fa6c7b..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixIdGenerator.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.AsyncAtomicIdGenerator;
-
-/**
- * {@code AsyncAtomicIdGenerator} implementation backed by Atomix
- * {@link AsyncAtomicCounter}.
- */
-public class AtomixIdGenerator implements AsyncAtomicIdGenerator {
-
- private static final long DEFAULT_BATCH_SIZE = 1000;
- private final AsyncAtomicCounter counter;
- private final long batchSize;
- private CompletableFuture<Long> reserveFuture;
- private long base;
- private final AtomicLong delta = new AtomicLong();
-
- public AtomixIdGenerator(AsyncAtomicCounter counter) {
- this(counter, DEFAULT_BATCH_SIZE);
- }
-
- AtomixIdGenerator(AsyncAtomicCounter counter, long batchSize) {
- this.counter = counter;
- this.batchSize = batchSize;
- }
-
- @Override
- public String name() {
- return counter.name();
- }
-
- @Override
- public synchronized CompletableFuture<Long> nextId() {
- long nextDelta = delta.incrementAndGet();
- if ((base == 0 && reserveFuture == null) || nextDelta > batchSize) {
- delta.set(0);
- long delta = this.delta.incrementAndGet();
- return reserve().thenApply(base -> base + delta);
- } else {
- return reserveFuture.thenApply(base -> base + nextDelta);
- }
- }
-
- private CompletableFuture<Long> reserve() {
- if (reserveFuture == null || reserveFuture.isDone()) {
- reserveFuture = counter.getAndAdd(batchSize);
- } else {
- reserveFuture = reserveFuture.thenCompose(v -> counter.getAndAdd(batchSize));
- }
- reserveFuture = reserveFuture.thenApply(base -> {
- this.base = base;
- return base;
- });
- return reserveFuture;
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElector.java
deleted file mode 100644
index 53c7094..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElector.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Anoint;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GetElectedTopics;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GetLeadership;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Promote;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Run;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Withdraw;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncLeaderElector;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.ANOINT;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.EVICT;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_ALL_LEADERSHIPS;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_ELECTED_TOPICS;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_LEADERSHIP;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.PROMOTE;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.RUN;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.WITHDRAW;
-
-/**
- * Distributed resource providing the {@link AsyncLeaderElector} primitive.
- */
-public class AtomixLeaderElector extends AbstractRaftPrimitive implements AsyncLeaderElector {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.API)
- .register(AtomixLeaderElectorOperations.NAMESPACE)
- .register(AtomixLeaderElectorEvents.NAMESPACE)
- .build());
-
- private final Set<Consumer<Change<Leadership>>> leadershipChangeListeners = Sets.newCopyOnWriteArraySet();
-
- public AtomixLeaderElector(RaftProxy proxy) {
- super(proxy);
- proxy.addStateChangeListener(state -> {
- if (state == RaftProxy.State.CONNECTED && isListening()) {
- proxy.invoke(ADD_LISTENER);
- }
- });
- proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleEvent);
- }
-
- private void handleEvent(List<Change<Leadership>> changes) {
- changes.forEach(change -> leadershipChangeListeners.forEach(l -> l.accept(change)));
- }
-
- @Override
- public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
- return proxy.<Run, Leadership>invoke(RUN, SERIALIZER::encode, new Run(topic, nodeId), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Void> withdraw(String topic) {
- return proxy.invoke(WITHDRAW, SERIALIZER::encode, new Withdraw(topic));
- }
-
- @Override
- public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
- return proxy.<Anoint, Boolean>invoke(ANOINT, SERIALIZER::encode, new Anoint(topic, nodeId), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
- return proxy.<Promote, Boolean>invoke(
- PROMOTE, SERIALIZER::encode, new Promote(topic, nodeId), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Void> evict(NodeId nodeId) {
- return proxy.invoke(EVICT, SERIALIZER::encode, new AtomixLeaderElectorOperations.Evict(nodeId));
- }
-
- @Override
- public CompletableFuture<Leadership> getLeadership(String topic) {
- return proxy.invoke(GET_LEADERSHIP, SERIALIZER::encode, new GetLeadership(topic), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Map<String, Leadership>> getLeaderships() {
- return proxy.invoke(GET_ALL_LEADERSHIPS, SERIALIZER::decode);
- }
-
- public CompletableFuture<Set<String>> getElectedTopics(NodeId nodeId) {
- return proxy.invoke(GET_ELECTED_TOPICS, SERIALIZER::encode, new GetElectedTopics(nodeId), SERIALIZER::decode);
- }
-
- @Override
- public synchronized CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> consumer) {
- if (leadershipChangeListeners.isEmpty()) {
- return proxy.invoke(ADD_LISTENER).thenRun(() -> leadershipChangeListeners.add(consumer));
- } else {
- leadershipChangeListeners.add(consumer);
- return CompletableFuture.completedFuture(null);
- }
- }
-
- @Override
- public synchronized CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
- if (leadershipChangeListeners.remove(consumer) && leadershipChangeListeners.isEmpty()) {
- return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
- }
- return CompletableFuture.completedFuture(null);
- }
-
- private boolean isListening() {
- return !leadershipChangeListeners.isEmpty();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorEvents.java
deleted file mode 100644
index 3c5a2c7..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorEvents.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Atomix leader elector events.
- */
-public enum AtomixLeaderElectorEvents implements EventType {
- CHANGE;
-
- @Override
- public String id() {
- return name();
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
- .build("AtomixLeaderElectorEvents");
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorOperations.java
deleted file mode 100644
index c778cea..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorOperations.java
+++ /dev/null
@@ -1,331 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import com.google.common.base.MoreObjects;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@link AtomixLeaderElector} resource state machine operations.
- */
-public enum AtomixLeaderElectorOperations implements OperationId {
- ADD_LISTENER(OperationType.COMMAND),
- REMOVE_LISTENER(OperationType.COMMAND),
- RUN(OperationType.COMMAND),
- WITHDRAW(OperationType.COMMAND),
- ANOINT(OperationType.COMMAND),
- PROMOTE(OperationType.COMMAND),
- EVICT(OperationType.COMMAND),
- GET_LEADERSHIP(OperationType.QUERY),
- GET_ALL_LEADERSHIPS(OperationType.QUERY),
- GET_ELECTED_TOPICS(OperationType.QUERY);
-
- private final OperationType type;
-
- AtomixLeaderElectorOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.API)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(Run.class)
- .register(Withdraw.class)
- .register(Anoint.class)
- .register(Promote.class)
- .register(Evict.class)
- .register(GetLeadership.class)
- .register(GetElectedTopics.class)
- .build("AtomixLeaderElectorOperations");
-
- /**
- * Abstract election query.
- */
- @SuppressWarnings("serial")
- public abstract static class ElectionOperation {
- }
-
- /**
- * Abstract election topic query.
- */
- @SuppressWarnings("serial")
- public abstract static class TopicOperation extends ElectionOperation {
- String topic;
-
- public TopicOperation() {
- }
-
- public TopicOperation(String topic) {
- this.topic = checkNotNull(topic);
- }
-
- /**
- * Returns the topic.
- * @return topic
- */
- public String topic() {
- return topic;
- }
- }
-
- /**
- * GetLeader query.
- */
- @SuppressWarnings("serial")
- public static class GetLeadership extends TopicOperation {
-
- public GetLeadership() {
- }
-
- public GetLeadership(String topic) {
- super(topic);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("topic", topic)
- .toString();
- }
- }
-
- /**
- * GetElectedTopics query.
- */
- @SuppressWarnings("serial")
- public static class GetElectedTopics extends ElectionOperation {
- private NodeId nodeId;
-
- public GetElectedTopics() {
- }
-
- public GetElectedTopics(NodeId nodeId) {
- checkArgument(nodeId != null, "nodeId cannot be null");
- this.nodeId = nodeId;
- }
-
- /**
- * Returns the nodeId to check.
- *
- * @return The nodeId to check.
- */
- public NodeId nodeId() {
- return nodeId;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("nodeId", nodeId)
- .toString();
- }
- }
-
- /**
- * Enter and run for leadership.
- */
- @SuppressWarnings("serial")
- public static class Run extends ElectionOperation {
- private String topic;
- private NodeId nodeId;
-
- public Run() {
- }
-
- public Run(String topic, NodeId nodeId) {
- this.topic = topic;
- this.nodeId = nodeId;
- }
-
- /**
- * Returns the topic.
- *
- * @return topic
- */
- public String topic() {
- return topic;
- }
-
- /**
- * Returns the nodeId.
- *
- * @return the nodeId
- */
- public NodeId nodeId() {
- return nodeId;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("topic", topic)
- .add("nodeId", nodeId)
- .toString();
- }
- }
-
- /**
- * Withdraw from a leadership contest.
- */
- @SuppressWarnings("serial")
- public static class Withdraw extends ElectionOperation {
- private String topic;
-
- public Withdraw() {
- }
-
- public Withdraw(String topic) {
- this.topic = topic;
- }
-
- /**
- * Returns the topic.
- *
- * @return The topic
- */
- public String topic() {
- return topic;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("topic", topic)
- .toString();
- }
- }
-
- /**
- * Command for administratively changing the leadership state for a node.
- */
- @SuppressWarnings("serial")
- public abstract static class ElectionChangeOperation extends ElectionOperation {
- private String topic;
- private NodeId nodeId;
-
- ElectionChangeOperation() {
- topic = null;
- nodeId = null;
- }
-
- public ElectionChangeOperation(String topic, NodeId nodeId) {
- this.topic = topic;
- this.nodeId = nodeId;
- }
-
- /**
- * Returns the topic.
- *
- * @return The topic
- */
- public String topic() {
- return topic;
- }
-
- /**
- * Returns the nodeId to make leader.
- *
- * @return The nodeId
- */
- public NodeId nodeId() {
- return nodeId;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("topic", topic)
- .add("nodeId", nodeId)
- .toString();
- }
- }
-
- /**
- * Command for administratively anoint a node as leader.
- */
- @SuppressWarnings("serial")
- public static class Anoint extends ElectionChangeOperation {
-
- private Anoint() {
- }
-
- public Anoint(String topic, NodeId nodeId) {
- super(topic, nodeId);
- }
- }
-
- /**
- * Command for administratively promote a node as top candidate.
- */
- @SuppressWarnings("serial")
- public static class Promote extends ElectionChangeOperation {
-
- private Promote() {
- }
-
- public Promote(String topic, NodeId nodeId) {
- super(topic, nodeId);
- }
- }
-
- /**
- * Command for administratively evicting a node from all leadership topics.
- */
- @SuppressWarnings("serial")
- public static class Evict extends ElectionOperation {
- private NodeId nodeId;
-
- public Evict() {
- }
-
- public Evict(NodeId nodeId) {
- this.nodeId = nodeId;
- }
-
- /**
- * Returns the node identifier.
- *
- * @return The nodeId
- */
- public NodeId nodeId() {
- return nodeId;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("nodeId", nodeId)
- .toString();
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java
deleted file mode 100644
index dcddcfc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java
+++ /dev/null
@@ -1,539 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.cluster.Leader;
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Anoint;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Evict;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GetElectedTopics;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GetLeadership;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Promote;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Run;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Withdraw;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.ANOINT;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.EVICT;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_ALL_LEADERSHIPS;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_ELECTED_TOPICS;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_LEADERSHIP;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.PROMOTE;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.RUN;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.WITHDRAW;
-
-/**
- * State machine for {@link AtomixLeaderElector} resource.
- */
-public class AtomixLeaderElectorService extends AbstractRaftService {
-
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(AtomixLeaderElectorOperations.NAMESPACE)
- .register(AtomixLeaderElectorEvents.NAMESPACE)
- .register(ElectionState.class)
- .register(Registration.class)
- .register(new LinkedHashMap<>().keySet().getClass())
- .build());
-
- private Map<String, AtomicLong> termCounters = new HashMap<>();
- private Map<String, ElectionState> elections = new HashMap<>();
- private Map<Long, RaftSession> listeners = new LinkedHashMap<>();
-
- @Override
- public void snapshot(SnapshotWriter writer) {
- writer.writeObject(Sets.newHashSet(listeners.keySet()), SERIALIZER::encode);
- writer.writeObject(termCounters, SERIALIZER::encode);
- writer.writeObject(elections, SERIALIZER::encode);
- logger().debug("Took state machine snapshot");
- }
-
- @Override
- public void install(SnapshotReader reader) {
- listeners = new LinkedHashMap<>();
- for (Long sessionId : reader.<Set<Long>>readObject(SERIALIZER::decode)) {
- listeners.put(sessionId, sessions().getSession(sessionId));
- }
- termCounters = reader.readObject(SERIALIZER::decode);
- elections = reader.readObject(SERIALIZER::decode);
- logger().debug("Reinstated state machine from snapshot");
- }
-
- @Override
- protected void configure(RaftServiceExecutor executor) {
- // Notification
- executor.register(ADD_LISTENER, this::listen);
- executor.register(REMOVE_LISTENER, this::unlisten);
- // Commands
- executor.register(RUN, SERIALIZER::decode, this::run, SERIALIZER::encode);
- executor.register(WITHDRAW, SERIALIZER::decode, this::withdraw);
- executor.register(ANOINT, SERIALIZER::decode, this::anoint, SERIALIZER::encode);
- executor.register(PROMOTE, SERIALIZER::decode, this::promote, SERIALIZER::encode);
- executor.register(EVICT, SERIALIZER::decode, this::evict);
- // Queries
- executor.register(GET_LEADERSHIP, SERIALIZER::decode, this::getLeadership, SERIALIZER::encode);
- executor.register(GET_ALL_LEADERSHIPS, this::allLeaderships, SERIALIZER::encode);
- executor.register(GET_ELECTED_TOPICS, SERIALIZER::decode, this::electedTopics, SERIALIZER::encode);
- }
-
- private void notifyLeadershipChange(Leadership previousLeadership, Leadership newLeadership) {
- notifyLeadershipChanges(Lists.newArrayList(new Change<>(previousLeadership, newLeadership)));
- }
-
- private void notifyLeadershipChanges(List<Change<Leadership>> changes) {
- if (changes.isEmpty()) {
- return;
- }
- listeners.values().forEach(session -> session.publish(CHANGE, SERIALIZER::encode, changes));
- }
-
- /**
- * Applies listen commits.
- *
- * @param commit listen commit
- */
- public void listen(Commit<Void> commit) {
- listeners.put(commit.session().sessionId().id(), commit.session());
- }
-
- /**
- * Applies unlisten commits.
- *
- * @param commit unlisten commit
- */
- public void unlisten(Commit<Void> commit) {
- listeners.remove(commit.session().sessionId().id());
- }
-
- /**
- * Applies an {@link AtomixLeaderElectorOperations.Run} commit.
- * @param commit commit entry
- * @return topic leader. If no previous leader existed this is the node that just entered the race.
- */
- public Leadership run(Commit<? extends Run> commit) {
- try {
- String topic = commit.value().topic();
- Leadership oldLeadership = leadership(topic);
- Registration registration = new Registration(commit.value().nodeId(), commit.session().sessionId().id());
- elections.compute(topic, (k, v) -> {
- if (v == null) {
- return new ElectionState(registration, termCounter(topic)::incrementAndGet);
- } else {
- if (!v.isDuplicate(registration)) {
- return new ElectionState(v).addRegistration(registration, termCounter(topic)::incrementAndGet);
- } else {
- return v;
- }
- }
- });
- Leadership newLeadership = leadership(topic);
-
- if (!Objects.equal(oldLeadership, newLeadership)) {
- notifyLeadershipChange(oldLeadership, newLeadership);
- }
- return newLeadership;
- } catch (Exception e) {
- logger().error("State machine operation failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Applies an {@link AtomixLeaderElectorOperations.Withdraw} commit.
- * @param commit withdraw commit
- */
- public void withdraw(Commit<? extends Withdraw> commit) {
- try {
- String topic = commit.value().topic();
- Leadership oldLeadership = leadership(topic);
- elections.computeIfPresent(topic, (k, v) -> v.cleanup(commit.session(),
- termCounter(topic)::incrementAndGet));
- Leadership newLeadership = leadership(topic);
- if (!Objects.equal(oldLeadership, newLeadership)) {
- notifyLeadershipChange(oldLeadership, newLeadership);
- }
- } catch (Exception e) {
- logger().error("State machine operation failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Applies an {@link AtomixLeaderElectorOperations.Anoint} commit.
- * @param commit anoint commit
- * @return {@code true} if changes were made and the transfer occurred; {@code false} if it did not.
- */
- public boolean anoint(Commit<? extends Anoint> commit) {
- try {
- String topic = commit.value().topic();
- NodeId nodeId = commit.value().nodeId();
- Leadership oldLeadership = leadership(topic);
- ElectionState electionState = elections.computeIfPresent(topic,
- (k, v) -> v.transferLeadership(nodeId, termCounter(topic)));
- Leadership newLeadership = leadership(topic);
- if (!Objects.equal(oldLeadership, newLeadership)) {
- notifyLeadershipChange(oldLeadership, newLeadership);
- }
- return (electionState != null &&
- electionState.leader() != null &&
- commit.value().nodeId().equals(electionState.leader().nodeId()));
- } catch (Exception e) {
- logger().error("State machine operation failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Applies an {@link AtomixLeaderElectorOperations.Promote} commit.
- * @param commit promote commit
- * @return {@code true} if changes desired end state is achieved.
- */
- public boolean promote(Commit<? extends Promote> commit) {
- try {
- String topic = commit.value().topic();
- NodeId nodeId = commit.value().nodeId();
- Leadership oldLeadership = leadership(topic);
- if (oldLeadership == null || !oldLeadership.candidates().contains(nodeId)) {
- return false;
- }
- elections.computeIfPresent(topic, (k, v) -> v.promote(nodeId));
- Leadership newLeadership = leadership(topic);
- if (!Objects.equal(oldLeadership, newLeadership)) {
- notifyLeadershipChange(oldLeadership, newLeadership);
- }
- return true;
- } catch (Exception e) {
- logger().error("State machine operation failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Applies an {@link AtomixLeaderElectorOperations.Evict} commit.
- * @param commit evict commit
- */
- public void evict(Commit<? extends Evict> commit) {
- try {
- List<Change<Leadership>> changes = Lists.newArrayList();
- NodeId nodeId = commit.value().nodeId();
- Set<String> topics = Maps.filterValues(elections, e -> e.candidates().contains(nodeId)).keySet();
- topics.forEach(topic -> {
- Leadership oldLeadership = leadership(topic);
- elections.compute(topic, (k, v) -> v.evict(nodeId, termCounter(topic)::incrementAndGet));
- Leadership newLeadership = leadership(topic);
- if (!Objects.equal(oldLeadership, newLeadership)) {
- changes.add(new Change<>(oldLeadership, newLeadership));
- }
- });
- notifyLeadershipChanges(changes);
- } catch (Exception e) {
- logger().error("State machine operation failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Applies an {@link AtomixLeaderElectorOperations.GetLeadership} commit.
- * @param commit GetLeadership commit
- * @return leader
- */
- public Leadership getLeadership(Commit<? extends GetLeadership> commit) {
- String topic = commit.value().topic();
- try {
- return leadership(topic);
- } catch (Exception e) {
- logger().error("State machine operation failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Applies an {@link AtomixLeaderElectorOperations.GetElectedTopics} commit.
- * @param commit commit entry
- * @return set of topics for which the node is the leader
- */
- public Set<String> electedTopics(Commit<? extends GetElectedTopics> commit) {
- try {
- NodeId nodeId = commit.value().nodeId();
- return ImmutableSet.copyOf(Maps.filterEntries(elections, e -> {
- Leader leader = leadership(e.getKey()).leader();
- return leader != null && leader.nodeId().equals(nodeId);
- }).keySet());
- } catch (Exception e) {
- logger().error("State machine operation failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- /**
- * Applies an {@link AtomixLeaderElectorOperations#GET_ALL_LEADERSHIPS} commit.
- * @param commit GetAllLeaderships commit
- * @return topic to leader mapping
- */
- public Map<String, Leadership> allLeaderships(Commit<Void> commit) {
- Map<String, Leadership> result = new HashMap<>();
- try {
- result.putAll(Maps.transformEntries(elections, (k, v) -> leadership(k)));
- return result;
- } catch (Exception e) {
- logger().error("State machine operation failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- private Leadership leadership(String topic) {
- return new Leadership(topic,
- leader(topic),
- candidates(topic));
- }
-
- private Leader leader(String topic) {
- ElectionState electionState = elections.get(topic);
- return electionState == null ? null : electionState.leader();
- }
-
- private List<NodeId> candidates(String topic) {
- ElectionState electionState = elections.get(topic);
- return electionState == null ? new LinkedList<>() : electionState.candidates();
- }
-
- private void onSessionEnd(RaftSession session) {
- listeners.remove(session.sessionId().id());
- Set<String> topics = ImmutableSet.copyOf(elections.keySet());
- List<Change<Leadership>> changes = Lists.newArrayList();
- for (String topic: topics) {
- Leadership oldLeadership = leadership(topic);
- elections.compute(topic, (k, v) -> v.cleanup(session, termCounter(topic)::incrementAndGet));
- Leadership newLeadership = leadership(topic);
- if (!Objects.equal(oldLeadership, newLeadership)) {
- changes.add(new Change<>(oldLeadership, newLeadership));
- }
- }
- notifyLeadershipChanges(changes);
- }
-
- private static class Registration {
- private final NodeId nodeId;
- private final long sessionId;
-
- public Registration(NodeId nodeId, long sessionId) {
- this.nodeId = nodeId;
- this.sessionId = sessionId;
- }
-
- public NodeId nodeId() {
- return nodeId;
- }
-
- public long sessionId() {
- return sessionId;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("nodeId", nodeId)
- .add("sessionId", sessionId)
- .toString();
- }
- }
-
- private static class ElectionState {
- final Registration leader;
- final long term;
- final long termStartTime;
- final List<Registration> registrations;
-
- public ElectionState(Registration registration, Supplier<Long> termCounter) {
- registrations = Arrays.asList(registration);
- term = termCounter.get();
- termStartTime = System.currentTimeMillis();
- leader = registration;
- }
-
- public ElectionState(ElectionState other) {
- registrations = Lists.newArrayList(other.registrations);
- leader = other.leader;
- term = other.term;
- termStartTime = other.termStartTime;
- }
-
- public ElectionState(List<Registration> registrations,
- Registration leader,
- long term,
- long termStartTime) {
- this.registrations = Lists.newArrayList(registrations);
- this.leader = leader;
- this.term = term;
- this.termStartTime = termStartTime;
- }
-
- public ElectionState cleanup(RaftSession session, Supplier<Long> termCounter) {
- Optional<Registration> registration =
- registrations.stream().filter(r -> r.sessionId() == session.sessionId().id()).findFirst();
- if (registration.isPresent()) {
- List<Registration> updatedRegistrations =
- registrations.stream()
- .filter(r -> r.sessionId() != session.sessionId().id())
- .collect(Collectors.toList());
- if (leader.sessionId() == session.sessionId().id()) {
- if (!updatedRegistrations.isEmpty()) {
- return new ElectionState(updatedRegistrations,
- updatedRegistrations.get(0),
- termCounter.get(),
- System.currentTimeMillis());
- } else {
- return new ElectionState(updatedRegistrations, null, term, termStartTime);
- }
- } else {
- return new ElectionState(updatedRegistrations, leader, term, termStartTime);
- }
- } else {
- return this;
- }
- }
-
- public ElectionState evict(NodeId nodeId, Supplier<Long> termCounter) {
- Optional<Registration> registration =
- registrations.stream().filter(r -> r.nodeId.equals(nodeId)).findFirst();
- if (registration.isPresent()) {
- List<Registration> updatedRegistrations =
- registrations.stream()
- .filter(r -> !r.nodeId().equals(nodeId))
- .collect(Collectors.toList());
- if (leader.nodeId().equals(nodeId)) {
- if (!updatedRegistrations.isEmpty()) {
- return new ElectionState(updatedRegistrations,
- updatedRegistrations.get(0),
- termCounter.get(),
- System.currentTimeMillis());
- } else {
- return new ElectionState(updatedRegistrations, null, term, termStartTime);
- }
- } else {
- return new ElectionState(updatedRegistrations, leader, term, termStartTime);
- }
- } else {
- return this;
- }
- }
-
- public boolean isDuplicate(Registration registration) {
- return registrations.stream().anyMatch(r -> r.sessionId() == registration.sessionId());
- }
-
- public Leader leader() {
- if (leader == null) {
- return null;
- } else {
- NodeId leaderNodeId = leader.nodeId();
- return new Leader(leaderNodeId, term, termStartTime);
- }
- }
-
- public List<NodeId> candidates() {
- return registrations.stream().map(registration -> registration.nodeId()).collect(Collectors.toList());
- }
-
- public ElectionState addRegistration(Registration registration, Supplier<Long> termCounter) {
- if (!registrations.stream().anyMatch(r -> r.sessionId() == registration.sessionId())) {
- List<Registration> updatedRegistrations = new LinkedList<>(registrations);
- updatedRegistrations.add(registration);
- boolean newLeader = leader == null;
- return new ElectionState(updatedRegistrations,
- newLeader ? registration : leader,
- newLeader ? termCounter.get() : term,
- newLeader ? System.currentTimeMillis() : termStartTime);
- }
- return this;
- }
-
- public ElectionState transferLeadership(NodeId nodeId, AtomicLong termCounter) {
- Registration newLeader = registrations.stream()
- .filter(r -> r.nodeId().equals(nodeId))
- .findFirst()
- .orElse(null);
- if (newLeader != null) {
- return new ElectionState(registrations,
- newLeader,
- termCounter.incrementAndGet(),
- System.currentTimeMillis());
- } else {
- return this;
- }
- }
-
- public ElectionState promote(NodeId nodeId) {
- Registration registration = registrations.stream()
- .filter(r -> r.nodeId().equals(nodeId))
- .findFirst()
- .orElse(null);
- List<Registration> updatedRegistrations = Lists.newArrayList();
- updatedRegistrations.add(registration);
- registrations.stream()
- .filter(r -> !r.nodeId().equals(nodeId))
- .forEach(updatedRegistrations::add);
- return new ElectionState(updatedRegistrations,
- leader,
- term,
- termStartTime);
-
- }
- }
-
- @Override
- public void onExpire(RaftSession session) {
- onSessionEnd(session);
- }
-
- @Override
- public void onClose(RaftSession session) {
- onSessionEnd(session);
- }
-
- private AtomicLong termCounter(String topic) {
- return termCounters.computeIfAbsent(topic, k -> new AtomicLong(0));
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixValue.java
deleted file mode 100644
index a4cdd52..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixValue.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AtomicValueEvent;
-import org.onosproject.store.service.AtomicValueEventListener;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.collect.Sets;
-
-/**
- * Implementation of {@link AsyncAtomicValue} backed by {@link AtomixConsistentMap}.
- */
-public class AtomixValue implements AsyncAtomicValue<String> {
-
- private final String name;
- private final AtomixConsistentMap atomixMap;
- private MapEventListener<String, byte[]> mapEventListener;
- private final Set<AtomicValueEventListener<String>> listeners = Sets.newIdentityHashSet();
-
- AtomixValue(String name, AtomixConsistentMap atomixMap) {
- this.name = name;
- this.atomixMap = atomixMap;
- }
-
- @Override
- public CompletableFuture<Boolean> compareAndSet(String expect, String update) {
- return atomixMap.replace(name, Tools.getBytesUtf8(expect), Tools.getBytesUtf8(update));
- }
-
- @Override
- public CompletableFuture<String> get() {
- return atomixMap.get(name)
- .thenApply(v -> v != null ? Tools.toStringUtf8(v.value()) : null);
- }
-
- @Override
- public CompletableFuture<String> getAndSet(String value) {
- return atomixMap.put(name, Tools.getBytesUtf8(value))
- .thenApply(v -> v != null ? Tools.toStringUtf8(v.value()) : null);
- }
-
- @Override
- public CompletableFuture<Void> set(String value) {
- return getAndSet(value).thenApply(v -> null);
- }
-
- @Override
- public CompletableFuture<Void> addListener(AtomicValueEventListener<String> listener) {
- // TODO: synchronization
- if (mapEventListener == null) {
- mapEventListener = event -> {
- Versioned<byte[]> newValue = event.newValue();
- Versioned<byte[]> oldValue = event.oldValue();
- if (Objects.equals(event.key(), name)) {
- listener.event(new AtomicValueEvent<>(name,
- newValue == null ? null : Tools.toStringUtf8(newValue.value()),
- oldValue == null ? null : Tools.toStringUtf8(oldValue.value())));
- }
- };
- return atomixMap.addListener(mapEventListener).whenComplete((r, e) -> {
- if (e == null) {
- listeners.add(listener);
- } else {
- mapEventListener = null;
- }
- });
- } else {
- listeners.add(listener);
- return CompletableFuture.completedFuture(null);
- }
- }
-
- @Override
- public CompletableFuture<Void> removeListener(AtomicValueEventListener<String> listener) {
- // TODO: synchronization
- listeners.remove(listener);
- if (listeners.isEmpty()) {
- return atomixMap.removeListener(mapEventListener);
- } else {
- return CompletableFuture.completedFuture(null);
- }
- }
-
- @Override
- public String name() {
- return null;
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueue.java
deleted file mode 100644
index 73bc802..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueue.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Timer;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.Consumer;
-
-import com.google.common.collect.ImmutableList;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.AbstractAccumulator;
-import org.onlab.util.Accumulator;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Add;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Complete;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Take;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueue;
-import org.onosproject.store.service.WorkQueueStats;
-import org.slf4j.Logger;
-
-import static java.util.concurrent.Executors.newSingleThreadExecutor;
-import static org.onlab.util.Tools.groupedThreads;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueEvents.TASK_AVAILABLE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.COMPLETE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.REGISTER;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.STATS;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.TAKE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.UNREGISTER;
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * Distributed resource providing the {@link WorkQueue} primitive.
- */
-public class AtomixWorkQueue extends AbstractRaftPrimitive implements WorkQueue<byte[]> {
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixWorkQueueOperations.NAMESPACE)
- .register(AtomixWorkQueueEvents.NAMESPACE)
- .build());
-
- private final Logger log = getLogger(getClass());
- private final ExecutorService executor = newSingleThreadExecutor(groupedThreads("AtomixWorkQueue", "%d", log));
- private final AtomicReference<TaskProcessor> taskProcessor = new AtomicReference<>();
- private final Timer timer = new Timer("atomix-work-queue-completer");
- private final AtomicBoolean isRegistered = new AtomicBoolean(false);
-
- public AtomixWorkQueue(RaftProxy proxy) {
- super(proxy);
- proxy.addStateChangeListener(state -> {
- if (state == RaftProxy.State.CONNECTED && isRegistered.get()) {
- proxy.invoke(REGISTER);
- }
- });
- proxy.addEventListener(TASK_AVAILABLE, this::resumeWork);
- }
-
- @Override
- public CompletableFuture<Void> destroy() {
- executor.shutdown();
- timer.cancel();
- return proxy.invoke(CLEAR);
- }
-
- @Override
- public CompletableFuture<Void> addMultiple(Collection<byte[]> items) {
- if (items.isEmpty()) {
- return CompletableFuture.completedFuture(null);
- }
- return proxy.invoke(ADD, SERIALIZER::encode, new Add(items));
- }
-
- @Override
- public CompletableFuture<Collection<Task<byte[]>>> take(int maxTasks) {
- if (maxTasks <= 0) {
- return CompletableFuture.completedFuture(ImmutableList.of());
- }
- return proxy.invoke(TAKE, SERIALIZER::encode, new Take(maxTasks), SERIALIZER::decode);
- }
-
- @Override
- public CompletableFuture<Void> complete(Collection<String> taskIds) {
- if (taskIds.isEmpty()) {
- return CompletableFuture.completedFuture(null);
- }
- return proxy.invoke(COMPLETE, SERIALIZER::encode, new Complete(taskIds));
- }
-
- @Override
- public CompletableFuture<Void> registerTaskProcessor(Consumer<byte[]> callback,
- int parallelism,
- Executor executor) {
- Accumulator<String> completedTaskAccumulator =
- new CompletedTaskAccumulator(timer, 50, 50); // TODO: make configurable
- taskProcessor.set(new TaskProcessor(callback,
- parallelism,
- executor,
- completedTaskAccumulator));
- return register().thenCompose(v -> take(parallelism))
- .thenAccept(taskProcessor.get());
- }
-
- @Override
- public CompletableFuture<Void> stopProcessing() {
- return unregister();
- }
-
- @Override
- public CompletableFuture<WorkQueueStats> stats() {
- return proxy.invoke(STATS, SERIALIZER::decode);
- }
-
- private void resumeWork() {
- TaskProcessor activeProcessor = taskProcessor.get();
- if (activeProcessor == null) {
- return;
- }
- this.take(activeProcessor.headRoom())
- .whenCompleteAsync((tasks, e) -> activeProcessor.accept(tasks), executor);
- }
-
- private CompletableFuture<Void> register() {
- return proxy.invoke(REGISTER).thenRun(() -> isRegistered.set(true));
- }
-
- private CompletableFuture<Void> unregister() {
- return proxy.invoke(UNREGISTER).thenRun(() -> isRegistered.set(false));
- }
-
- // TaskId accumulator for paced triggering of task completion calls.
- private class CompletedTaskAccumulator extends AbstractAccumulator<String> {
- CompletedTaskAccumulator(Timer timer, int maxTasksToBatch, int maxBatchMillis) {
- super(timer, maxTasksToBatch, maxBatchMillis, Integer.MAX_VALUE);
- }
-
- @Override
- public void processItems(List<String> items) {
- complete(items);
- }
- }
-
- private class TaskProcessor implements Consumer<Collection<Task<byte[]>>> {
-
- private final AtomicInteger headRoom;
- private final Consumer<byte[]> backingConsumer;
- private final Executor executor;
- private final Accumulator<String> taskCompleter;
-
- public TaskProcessor(Consumer<byte[]> backingConsumer,
- int parallelism,
- Executor executor,
- Accumulator<String> taskCompleter) {
- this.backingConsumer = backingConsumer;
- this.headRoom = new AtomicInteger(parallelism);
- this.executor = executor;
- this.taskCompleter = taskCompleter;
- }
-
- public int headRoom() {
- return headRoom.get();
- }
-
- @Override
- public void accept(Collection<Task<byte[]>> tasks) {
- if (tasks == null) {
- return;
- }
- headRoom.addAndGet(-1 * tasks.size());
- tasks.forEach(task ->
- executor.execute(() -> {
- try {
- backingConsumer.accept(task.payload());
- taskCompleter.add(task.taskId());
- } catch (Exception e) {
- log.debug("Task execution failed", e);
- } finally {
- headRoom.incrementAndGet();
- resumeWork();
- }
- }));
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueEvents.java
deleted file mode 100644
index 5458300..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueEvents.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Atomix work queue events.
- */
-public enum AtomixWorkQueueEvents implements EventType {
- TASK_AVAILABLE;
-
- @Override
- public String id() {
- return name();
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
- .build("AtomixWorkQueueEvents");
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueOperations.java
deleted file mode 100644
index 2452cf8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueOperations.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-
-import com.google.common.base.MoreObjects;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueueStats;
-
-/**
- * {@link AtomixWorkQueue} resource state machine operations.
- */
-public enum AtomixWorkQueueOperations implements OperationId {
- STATS(OperationType.QUERY),
- REGISTER(OperationType.COMMAND),
- UNREGISTER(OperationType.COMMAND),
- ADD(OperationType.COMMAND),
- TAKE(OperationType.COMMAND),
- COMPLETE(OperationType.COMMAND),
- CLEAR(OperationType.COMMAND);
-
- private final OperationType type;
-
- AtomixWorkQueueOperations(OperationType type) {
- this.type = type;
- }
-
- @Override
- public String id() {
- return name();
- }
-
- @Override
- public OperationType type() {
- return type;
- }
-
- public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
- .register(Add.class)
- .register(Take.class)
- .register(Complete.class)
- .register(Task.class)
- .register(WorkQueueStats.class)
- .build("AtomixWorkQueueOperations");
-
- /**
- * Work queue operation.
- */
- public abstract static class WorkQueueOperation {
- }
-
- /**
- * Command to add a collection of tasks to the queue.
- */
- @SuppressWarnings("serial")
- public static class Add extends WorkQueueOperation {
- private Collection<byte[]> items;
-
- private Add() {
- }
-
- public Add(Collection<byte[]> items) {
- this.items = items;
- }
-
- public Collection<byte[]> items() {
- return items;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("items", items)
- .toString();
- }
- }
-
- /**
- * Command to take a task from the queue.
- */
- @SuppressWarnings("serial")
- public static class Take extends WorkQueueOperation {
- private int maxTasks;
-
- private Take() {
- }
-
- public Take(int maxTasks) {
- this.maxTasks = maxTasks;
- }
-
- public int maxTasks() {
- return maxTasks;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("maxTasks", maxTasks)
- .toString();
- }
- }
-
- @SuppressWarnings("serial")
- public static class Complete extends WorkQueueOperation {
- private Collection<String> taskIds;
-
- private Complete() {
- }
-
- public Complete(Collection<String> taskIds) {
- this.taskIds = taskIds;
- }
-
- public Collection<String> taskIds() {
- return taskIds;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("taskIds", taskIds)
- .toString();
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueService.java
deleted file mode 100644
index 4540aa5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueService.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Queues;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Add;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Complete;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Take;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueueStats;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueEvents.TASK_AVAILABLE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.COMPLETE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.REGISTER;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.STATS;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.TAKE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.UNREGISTER;
-
-/**
- * State machine for {@link AtomixWorkQueue} resource.
- */
-public class AtomixWorkQueueService extends AbstractRaftService {
-
- private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
- .register(KryoNamespaces.BASIC)
- .register(AtomixWorkQueueOperations.NAMESPACE)
- .register(AtomixWorkQueueEvents.NAMESPACE)
- .register(TaskAssignment.class)
- .register(new HashMap().keySet().getClass())
- .register(ArrayDeque.class)
- .build());
-
- private final AtomicLong totalCompleted = new AtomicLong(0);
-
- private Queue<Task<byte[]>> unassignedTasks = Queues.newArrayDeque();
- private Map<String, TaskAssignment> assignments = Maps.newHashMap();
- private Map<Long, RaftSession> registeredWorkers = Maps.newHashMap();
-
- @Override
- public void snapshot(SnapshotWriter writer) {
- writer.writeObject(Sets.newHashSet(registeredWorkers.keySet()), SERIALIZER::encode);
- writer.writeObject(assignments, SERIALIZER::encode);
- writer.writeObject(unassignedTasks, SERIALIZER::encode);
- writer.writeLong(totalCompleted.get());
- }
-
- @Override
- public void install(SnapshotReader reader) {
- registeredWorkers = Maps.newHashMap();
- for (Long sessionId : reader.<Set<Long>>readObject(SERIALIZER::decode)) {
- registeredWorkers.put(sessionId, sessions().getSession(sessionId));
- }
- assignments = reader.readObject(SERIALIZER::decode);
- unassignedTasks = reader.readObject(SERIALIZER::decode);
- totalCompleted.set(reader.readLong());
- }
-
- @Override
- protected void configure(RaftServiceExecutor executor) {
- executor.register(STATS, this::stats, SERIALIZER::encode);
- executor.register(REGISTER, this::register);
- executor.register(UNREGISTER, this::unregister);
- executor.register(ADD, SERIALIZER::decode, this::add);
- executor.register(TAKE, SERIALIZER::decode, this::take, SERIALIZER::encode);
- executor.register(COMPLETE, SERIALIZER::decode, this::complete);
- executor.register(CLEAR, this::clear);
- }
-
- protected WorkQueueStats stats(Commit<Void> commit) {
- return WorkQueueStats.builder()
- .withTotalCompleted(totalCompleted.get())
- .withTotalPending(unassignedTasks.size())
- .withTotalInProgress(assignments.size())
- .build();
- }
-
- protected void clear(Commit<Void> commit) {
- unassignedTasks.clear();
- assignments.clear();
- registeredWorkers.clear();
- totalCompleted.set(0);
- }
-
- protected void register(Commit<Void> commit) {
- registeredWorkers.put(commit.session().sessionId().id(), commit.session());
- }
-
- protected void unregister(Commit<Void> commit) {
- registeredWorkers.remove(commit.session().sessionId().id());
- }
-
- protected void add(Commit<? extends Add> commit) {
- Collection<byte[]> items = commit.value().items();
-
- AtomicInteger itemIndex = new AtomicInteger(0);
- items.forEach(item -> {
- String taskId = String.format("%d:%d:%d", commit.session().sessionId().id(),
- commit.index(),
- itemIndex.getAndIncrement());
- unassignedTasks.add(new Task<>(taskId, item));
- });
-
- // Send an event to all sessions that have expressed interest in task processing
- // and are not actively processing a task.
- registeredWorkers.values().forEach(session -> session.publish(TASK_AVAILABLE));
- // FIXME: This generates a lot of event traffic.
- }
-
- protected Collection<Task<byte[]>> take(Commit<? extends Take> commit) {
- try {
- if (unassignedTasks.isEmpty()) {
- return ImmutableList.of();
- }
- long sessionId = commit.session().sessionId().id();
- int maxTasks = commit.value().maxTasks();
- return IntStream.range(0, Math.min(maxTasks, unassignedTasks.size()))
- .mapToObj(i -> {
- Task<byte[]> task = unassignedTasks.poll();
- String taskId = task.taskId();
- TaskAssignment assignment = new TaskAssignment(sessionId, task);
-
- // bookkeeping
- assignments.put(taskId, assignment);
-
- return task;
- })
- .collect(Collectors.toCollection(ArrayList::new));
- } catch (Exception e) {
- logger().warn("State machine update failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- protected void complete(Commit<? extends Complete> commit) {
- long sessionId = commit.session().sessionId().id();
- try {
- commit.value().taskIds().forEach(taskId -> {
- TaskAssignment assignment = assignments.get(taskId);
- if (assignment != null && assignment.sessionId() == sessionId) {
- assignments.remove(taskId);
- // bookkeeping
- totalCompleted.incrementAndGet();
- }
- });
- } catch (Exception e) {
- logger().warn("State machine update failed", e);
- throw new IllegalStateException(e);
- }
- }
-
- @Override
- public void onExpire(RaftSession session) {
- evictWorker(session.sessionId().id());
- }
-
- @Override
- public void onClose(RaftSession session) {
- evictWorker(session.sessionId().id());
- }
-
- private void evictWorker(long sessionId) {
- registeredWorkers.remove(sessionId);
-
- // TODO: Maintain an index of tasks by session for efficient access.
- Iterator<Map.Entry<String, TaskAssignment>> iter = assignments.entrySet().iterator();
- while (iter.hasNext()) {
- Map.Entry<String, TaskAssignment> entry = iter.next();
- TaskAssignment assignment = entry.getValue();
- if (assignment.sessionId() == sessionId) {
- unassignedTasks.add(assignment.task());
- iter.remove();
- }
- }
- }
-
- private static class TaskAssignment {
- private final long sessionId;
- private final Task<byte[]> task;
-
- public TaskAssignment(long sessionId, Task<byte[]> task) {
- this.sessionId = sessionId;
- this.task = task;
- }
-
- public long sessionId() {
- return sessionId;
- }
-
- public Task<byte[]> task() {
- return task;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("sessionId", sessionId)
- .add("task", task)
- .toString();
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/CommitResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/CommitResult.java
deleted file mode 100644
index 5b1da77..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/CommitResult.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-/**
- * Response enum for two phase commit operation.
- */
-public enum CommitResult {
- /**
- * Signifies a successful commit execution.
- */
- OK,
-
- /**
- * Signifies a failure due to unrecognized transaction identifier.
- */
- UNKNOWN_TRANSACTION_ID,
-
- /**
- * Signifies a failure to get participants to agree to commit (during prepare stage).
- */
- FAILURE_TO_PREPARE,
-
- /**
- * Failure during commit phase.
- */
- FAILURE_DURING_COMMIT
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTree.java
deleted file mode 100644
index 2d5832c..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTree.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTree;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.DocumentTreeNode;
-import org.onosproject.store.service.IllegalDocumentModificationException;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.base.Supplier;
-import com.google.common.collect.Maps;
-
-/**
- * Simple implementation of a {@link DocumentTree}.
- *
- * @param <V> tree node value type
- */
-public class DefaultDocumentTree<V> implements DocumentTree<V> {
-
- private static final DocumentPath ROOT_PATH = DocumentPath.from("root");
- final DefaultDocumentTreeNode<V> root;
- private final Supplier<Long> versionSupplier;
-
- public DefaultDocumentTree() {
- AtomicLong versionCounter = new AtomicLong(0);
- versionSupplier = versionCounter::incrementAndGet;
- root = new DefaultDocumentTreeNode<>(ROOT_PATH, null, versionSupplier.get(), Ordering.NATURAL, null);
- }
-
- public DefaultDocumentTree(Supplier<Long> versionSupplier, Ordering ordering) {
- root = new DefaultDocumentTreeNode<>(ROOT_PATH, null, versionSupplier.get(), ordering, null);
- this.versionSupplier = versionSupplier;
- }
-
- DefaultDocumentTree(Supplier<Long> versionSupplier, DefaultDocumentTreeNode<V> root) {
- this.root = root;
- this.versionSupplier = versionSupplier;
- }
-
- @Override
- public String name() {
- return null;
- }
-
- @Override
- public DocumentPath root() {
- return ROOT_PATH;
- }
-
- @Override
- public Map<String, Versioned<V>> getChildren(DocumentPath path) {
- DocumentTreeNode<V> node = getNode(path);
- if (node != null) {
- Map<String, Versioned<V>> childrenValues = Maps.newLinkedHashMap();
- node.children().forEachRemaining(n -> childrenValues.put(simpleName(n.path()), n.value()));
- return childrenValues;
- }
- throw new NoSuchDocumentPathException();
- }
-
- @Override
- public Versioned<V> get(DocumentPath path) {
- DocumentTreeNode<V> currentNode = getNode(path);
- return currentNode != null ? currentNode.value() : null;
- }
-
- @Override
- public Versioned<V> set(DocumentPath path, V value) {
- checkRootModification(path);
- DefaultDocumentTreeNode<V> node = getNode(path);
- if (node != null) {
- return node.update(value, versionSupplier.get());
- } else {
- create(path, value);
- return null;
- }
- }
-
- @Override
- public boolean create(DocumentPath path, V value) {
- checkRootModification(path);
- DocumentTreeNode<V> node = getNode(path);
- if (node != null) {
- return false;
- }
- DocumentPath parentPath = path.parent();
- DefaultDocumentTreeNode<V> parentNode = getNode(parentPath);
- if (parentNode == null) {
- throw new IllegalDocumentModificationException();
- }
- parentNode.addChild(simpleName(path), value, versionSupplier.get());
- return true;
- }
-
- @Override
- public boolean createRecursive(DocumentPath path, V value) {
- checkRootModification(path);
- DocumentTreeNode<V> node = getNode(path);
- if (node != null) {
- return false;
- }
- DocumentPath parentPath = path.parent();
- if (getNode(parentPath) == null) {
- createRecursive(parentPath, null);
- }
- DefaultDocumentTreeNode<V> parentNode = getNode(parentPath);
- if (parentNode == null) {
- throw new IllegalDocumentModificationException();
- }
- parentNode.addChild(simpleName(path), value, versionSupplier.get());
- return true;
- }
-
- @Override
- public boolean replace(DocumentPath path, V newValue, long version) {
- checkRootModification(path);
- DocumentTreeNode<V> node = getNode(path);
- if (node != null && node.value() != null && node.value().version() == version) {
- set(path, newValue);
- return true;
- }
- return false;
- }
-
- @Override
- public boolean replace(DocumentPath path, V newValue, V expectedValue) {
- checkRootModification(path);
- if (Objects.equals(newValue, expectedValue)) {
- return false;
- }
- DocumentTreeNode<V> node = getNode(path);
- V prevValue = Optional.ofNullable(node)
- .map(DocumentTreeNode::value)
- .map(Versioned::valueOrNull)
- .orElse(null);
- if (Objects.equals(prevValue, expectedValue)) {
- set(path, newValue);
- return true;
- }
- return false;
- }
-
- @Override
- public Versioned<V> removeNode(DocumentPath path) {
- checkRootModification(path);
- DefaultDocumentTreeNode<V> nodeToRemove = getNode(path);
- if (nodeToRemove == null) {
- throw new NoSuchDocumentPathException();
- }
- if (nodeToRemove.hasChildren()) {
- throw new IllegalDocumentModificationException();
- }
- DefaultDocumentTreeNode<V> parent = (DefaultDocumentTreeNode<V>) nodeToRemove.parent();
- parent.removeChild(simpleName(path));
- return nodeToRemove.value();
- }
-
- @Override
- public void addListener(DocumentPath path, DocumentTreeListener<V> listener) {
- // TODO Auto-generated method stub
- }
-
- @Override
- public void removeListener(DocumentTreeListener<V> listener) {
- // TODO Auto-generated method stub
- }
-
- private DefaultDocumentTreeNode<V> getNode(DocumentPath path) {
- Iterator<String> pathElements = path.pathElements().iterator();
- DefaultDocumentTreeNode<V> currentNode = root;
- checkArgument("root".equals(pathElements.next()), "Path should start with root: %s", path);
- while (pathElements.hasNext() && currentNode != null) {
- currentNode = (DefaultDocumentTreeNode<V>) currentNode.child(pathElements.next());
- }
- return currentNode;
- }
-
- private String simpleName(DocumentPath path) {
- return path.pathElements().get(path.pathElements().size() - 1);
- }
-
- private void checkRootModification(DocumentPath path) {
- if (ROOT_PATH.equals(path)) {
- throw new IllegalDocumentModificationException();
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeNode.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeNode.java
deleted file mode 100644
index 0f5d66a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeNode.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Objects;
-
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeNode;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-/**
- * A {@code DocumentTree} node.
- */
-public class DefaultDocumentTreeNode<V> implements DocumentTreeNode<V> {
- private final DocumentPath key;
- private Versioned<V> value;
- private final Map<String, DocumentTreeNode<V>> children;
- private final Ordering ordering;
- private final DocumentTreeNode<V> parent;
-
- public DefaultDocumentTreeNode(DocumentPath key,
- V value,
- long version,
- Ordering ordering,
- DocumentTreeNode<V> parent) {
- this.key = checkNotNull(key);
- this.value = new Versioned<>(value, version);
- this.ordering = ordering;
- this.parent = parent;
-
- switch (ordering) {
- case INSERTION:
- children = Maps.newLinkedHashMap();
- break;
- case NATURAL:
- default:
- children = Maps.newTreeMap();
- break;
- }
- }
-
- @Override
- public DocumentPath path() {
- return key;
- }
-
- @Override
- public Versioned<V> value() {
- return value;
- }
-
- @Override
- public Iterator<DocumentTreeNode<V>> children() {
- return ImmutableList.copyOf(children.values()).iterator();
- }
-
- @Override
- public DocumentTreeNode<V> child(String name) {
- return children.get(name);
- }
-
-
- public DocumentTreeNode<V> parent() {
- return parent;
- }
-
- /**
- * Adds a new child only if one does not exist with the name.
- * @param name relative path name of the child node
- * @param newValue new value to set
- * @param newVersion new version to set
- * @return previous value; can be {@code null} if no child currently exists with that relative path name.
- * a non null return value indicates child already exists and no modification occurred.
- */
- public Versioned<V> addChild(String name, V newValue, long newVersion) {
- DefaultDocumentTreeNode<V> child = (DefaultDocumentTreeNode<V>) children.get(name);
- if (child != null) {
- return child.value();
- }
- children.put(name, new DefaultDocumentTreeNode<>(
- new DocumentPath(name, path()), newValue, newVersion, ordering, this));
- return null;
- }
-
- /**
- * Updates the node value.
- *
- * @param newValue new value to set
- * @param newVersion new version to set
- * @return previous value
- */
- public Versioned<V> update(V newValue, long newVersion) {
- Versioned<V> previousValue = value;
- value = new Versioned<>(newValue, newVersion);
- return previousValue;
- }
-
-
- /**
- * Removes a child node.
- *
- * @param name the name of child node to be removed
- * @return {@code true} if the child set was modified as a result of this call, {@code false} otherwise
- */
- public boolean removeChild(String name) {
- return children.remove(name) != null;
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(this.key);
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj instanceof DefaultDocumentTreeNode) {
- DefaultDocumentTreeNode<V> that = (DefaultDocumentTreeNode<V>) obj;
- if (this.parent.equals(that.parent)) {
- if (this.children.size() == that.children.size()) {
- return Sets.symmetricDifference(this.children.keySet(), that.children.keySet()).isEmpty();
- }
- }
- }
- return false;
- }
-
- @Override
- public String toString() {
- MoreObjects.ToStringHelper helper =
- MoreObjects.toStringHelper(getClass())
- .add("parent", this.parent)
- .add("key", this.key)
- .add("value", this.value);
- for (DocumentTreeNode<V> child : children.values()) {
- helper = helper.add("child", "\n" + child.path().pathElements()
- .get(child.path().pathElements().size() - 1) +
- " : " + child.value());
- }
- return helper.toString();
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DocumentTreeResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DocumentTreeResult.java
deleted file mode 100644
index bf544dd..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DocumentTreeResult.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import com.google.common.base.MoreObjects;
-
-/**
- * Result of a document tree operation.
- *
- * @param <V> value type
- */
-public class DocumentTreeResult<V> {
-
- public enum Status {
- /**
- * Indicates a successful update.
- */
- OK,
-
- /**
- * Indicates a noop i.e. existing and new value are both same.
- */
- NOOP,
-
- /**
- * Indicates a failed update due to a write lock.
- */
- WRITE_LOCK,
-
- /**
- * Indicates a failed update due to a invalid path.
- */
- INVALID_PATH,
-
- /**
- * Indicates a failed update due to a illegal modification attempt.
- */
- ILLEGAL_MODIFICATION,
- }
-
- @SuppressWarnings("unchecked")
- public static final DocumentTreeResult WRITE_LOCK =
- new DocumentTreeResult(Status.WRITE_LOCK, null);
-
- @SuppressWarnings("unchecked")
- public static final DocumentTreeResult INVALID_PATH =
- new DocumentTreeResult(Status.INVALID_PATH, null);
-
- @SuppressWarnings("unchecked")
- public static final DocumentTreeResult ILLEGAL_MODIFICATION =
- new DocumentTreeResult(Status.ILLEGAL_MODIFICATION, null);
-
- /**
- * Returns a successful result.
- *
- * @param result the operation result
- * @param <V> the result value type
- * @return successful result
- */
- public static <V> DocumentTreeResult<V> ok(V result) {
- return new DocumentTreeResult<V>(Status.OK, result);
- }
-
- /**
- * Returns a {@code WRITE_LOCK} error result.
- *
- * @param <V> the result value type
- * @return write lock result
- */
- @SuppressWarnings("unchecked")
- public static <V> DocumentTreeResult<V> writeLock() {
- return WRITE_LOCK;
- }
-
- /**
- * Returns an {@code INVALID_PATH} result.
- *
- * @param <V> the result value type
- * @return invalid path result
- */
- @SuppressWarnings("unchecked")
- public static <V> DocumentTreeResult<V> invalidPath() {
- return INVALID_PATH;
- }
-
- /**
- * Returns an {@code ILLEGAL_MODIFICATION} result.
- *
- * @param <V> the result value type
- * @return illegal modification result
- */
- @SuppressWarnings("unchecked")
- public static <V> DocumentTreeResult<V> illegalModification() {
- return ILLEGAL_MODIFICATION;
- }
-
- private final Status status;
- private final V result;
-
- public DocumentTreeResult(Status status, V result) {
- this.status = status;
- this.result = result;
- }
-
- public Status status() {
- return status;
- }
-
- public V result() {
- return result;
- }
-
- public boolean updated() {
- return status == Status.OK;
- }
-
- public boolean created() {
- return updated() && result == null;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(getClass())
- .add("status", status)
- .add("value", result)
- .toString();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/LockEvent.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/LockEvent.java
deleted file mode 100644
index cddb41f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/LockEvent.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-/**
- * Locked event.
- */
-public class LockEvent {
- private final int id;
- private final long version;
-
- public LockEvent() {
- this(0, 0);
- }
-
- public LockEvent(int id, long version) {
- this.id = id;
- this.version = version;
- }
-
- /**
- * Returns the lock ID.
- *
- * @return The lock ID.
- */
- public int id() {
- return id;
- }
-
- /**
- * Returns the lock version.
- *
- * @return The lock version.
- */
- public long version() {
- return version;
- }
-
- @Override
- public String toString() {
- return toStringHelper(this)
- .add("id", id)
- .add("version", version)
- .toString();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/MapEntryUpdateResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/MapEntryUpdateResult.java
deleted file mode 100644
index 4bc783b..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/MapEntryUpdateResult.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.store.service.Versioned;
-
-/**
- * Result of a map entry update operation.
- * <p>
- * Both old and new values are accessible along with a flag that indicates if the
- * the value was updated. If flag is false, oldValue and newValue both
- * point to the same unmodified value.
- *
- * @param <K> key type
- * @param <V> result type
- */
-public class MapEntryUpdateResult<K, V> {
-
- public enum Status {
-
- /**
- * Indicates a successful update.
- */
- OK,
-
- /**
- * Indicates a noop i.e. existing and new value are both null.
- */
- NOOP,
-
- /**
- * Indicates a failed update due to a write lock.
- */
- WRITE_LOCK,
-
- /**
- * Indicates a failed update due to a precondition check failure.
- */
- PRECONDITION_FAILED
- }
-
- private final Status status;
- private final long version;
- private final K key;
- private final Versioned<V> result;
-
- public MapEntryUpdateResult(Status status, long version, K key, Versioned<V> result) {
- this.status = status;
- this.version = version;
- this.key = key;
- this.result = result;
- }
-
- /**
- * Returns {@code true} if the update was successful.
- * @return {@code true} if yes, {@code false} otherwise
- */
- public boolean updated() {
- return status == Status.OK;
- }
-
- /**
- * Returns the update status.
- * @return update status
- */
- public Status status() {
- return status;
- }
-
- /**
- * Returns the result version.
- * @return result version
- */
- public long version() {
- return version;
- }
-
- /**
- * Returns the value.
- * @return the value associated with key if updated was successful, otherwise current value
- */
- public Versioned<V> result() {
- return result;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(MapEntryUpdateResult.class)
- .add("status", status)
- .add("key", key)
- .add("result", result)
- .toString();
- }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/PrepareResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/PrepareResult.java
deleted file mode 100644
index f4e2bdc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/PrepareResult.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-/**
- * Response enum for two phase commit prepare operation.
- */
-public enum PrepareResult {
- /**
- * Signifies a successful execution of the prepare operation.
- */
- OK,
-
- /**
- * Signifies some participants in a distributed prepare operation failed.
- */
- PARTIAL_FAILURE,
-
- /**
- * Signifies a failure to another transaction locking the underlying state.
- */
- CONCURRENT_TRANSACTION,
-
- /**
- * Signifies a optimistic lock failure. This can happen if underlying state has changed since it was last read.
- */
- OPTIMISTIC_LOCK_FAILURE,
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/RollbackResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/RollbackResult.java
deleted file mode 100644
index 3c6558b..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/RollbackResult.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-/**
- * Response enum for two phase commit rollback operation.
- */
-public enum RollbackResult {
- /**
- * Signifies a successful rollback execution.
- */
- OK,
-
- /**
- * Signifies a failure due to unrecognized transaction identifier.
- */
- UNKNOWN_TRANSACTION_ID,
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/AsyncConsistentMapMock.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/AsyncConsistentMapMock.java
deleted file mode 100644
index 9391f8d..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/AsyncConsistentMapMock.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.onosproject.store.service.AsyncConsistentMapAdapter;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Versioned;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-/*Simple template for asynchronous map
-Serializer must be defined independently
- */
-public class AsyncConsistentMapMock<K, V> extends AsyncConsistentMapAdapter<K, V> {
- private final Map<K, V> baseMap = new HashMap<>();
- private final List<MapEventListener<K, V>> listeners;
-
- Versioned<V> makeVersioned(V v) {
- return new Versioned<>(v, 1, 0);
- }
-
- public AsyncConsistentMapMock() {
- this.listeners = new ArrayList<>();
- }
-
- public CompletableFuture<Integer> size() {
- return CompletableFuture.completedFuture(baseMap.size());
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- return CompletableFuture.completedFuture(baseMap.containsKey(key));
- }
-
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- return CompletableFuture.completedFuture(baseMap.values()
- .stream()
- .anyMatch(v -> {
- if (v instanceof byte[] && value instanceof byte[]) {
- return Arrays.equals((byte[]) v, (byte[]) value);
- } else {
- return Objects.equals(v, value);
- }
- }));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(K key) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.get(key)));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.getOrDefault(key, defaultValue)));
- }
-
- @Override
- public CompletableFuture<Versioned<V>>
- computeIf(K key, Predicate<? super V> condition,
- BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
-
- V value = baseMap.get(key);
-
- if (condition.test(value)) {
- value = baseMap.compute(key, remappingFunction);
- }
- return CompletableFuture.completedFuture(makeVersioned(value));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.put(key, value)));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.put(key, value)));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> remove(K key) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.remove(key)));
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- baseMap.clear();
- return CompletableFuture.allOf();
- }
-
- @Override
- public CompletableFuture<Set<K>> keySet() {
- return CompletableFuture.completedFuture(baseMap.keySet());
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V>>> values() {
- Set<Versioned<V>> valuesAsVersionedCollection =
- baseMap.values().stream().map(this::makeVersioned)
- .collect(Collectors.toSet());
- return CompletableFuture.completedFuture(valuesAsVersionedCollection);
- }
-
- @Override
- public CompletableFuture<Set<Map.Entry<K, Versioned<V>>>> entrySet() {
- Map<K, Versioned<V>> valuesAsVersionedMap = new HashMap<>();
- baseMap.entrySet()
- .forEach(e -> valuesAsVersionedMap.put(e.getKey(),
- makeVersioned(e.getValue())));
- return CompletableFuture.completedFuture(valuesAsVersionedMap.entrySet());
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.putIfAbsent(key, value)));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return CompletableFuture.completedFuture(baseMap.remove(key, value));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, long version) {
- Object value = baseMap.remove(key);
- return CompletableFuture.completedFuture(value != null);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.replace(key, value)));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
- V current = baseMap.get(key);
- if (current instanceof byte[] && oldValue instanceof byte[]) {
- baseMap.put(key, newValue);
- return CompletableFuture.completedFuture(Arrays.equals((byte[]) current, (byte[]) oldValue));
- } else {
- return CompletableFuture.completedFuture(Objects.equals(current, oldValue));
- }
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
- return CompletableFuture.completedFuture(baseMap.replace(key, newValue) != null);
- }
-
- @Override
- public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor e) {
- listeners.add(listener);
- return CompletableFuture.allOf();
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
- listeners.remove(listener);
- return CompletableFuture.allOf();
- }
-}
-
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMapTest.java
deleted file mode 100644
index cdb4ecb..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMapTest.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onosproject.store.primitives.DefaultConsistentMap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapService;
-import org.onosproject.store.primitives.resources.impl.AtomixTestBase;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.ConsistentMap;
-import org.onosproject.store.service.Serializer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-/**
- * Unit tests for cached {@link AtomixConsistentMap}.
- */
-public class CachingAsyncConsistentMapTest extends AtomixTestBase<AtomixConsistentMap> {
-
- @Override
- protected RaftService createService() {
- return new AtomixConsistentMapService();
- }
-
- @Override
- protected AtomixConsistentMap createPrimitive(RaftProxy proxy) {
- return new AtomixConsistentMap(proxy);
- }
-
- /**
- * Tests that reads following events are not stale when cached.
- */
- @Test
- public void testCacheConsistency() throws Throwable {
- Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
-
- ConsistentMap<String, String> map1 = new DefaultConsistentMap<>(
- new CachingAsyncConsistentMap<>(
- new TranscodingAsyncConsistentMap<>(
- newPrimitive("testCacheConsistency"),
- k -> k,
- k -> k,
- v -> serializer.encode(v),
- v -> serializer.decode(v))), 5000);
- ConsistentMap<String, String> map2 = new DefaultConsistentMap<>(
- new CachingAsyncConsistentMap<>(
- new TranscodingAsyncConsistentMap<>(
- newPrimitive("testCacheConsistency"),
- k -> k,
- k -> k,
- v -> serializer.encode(v),
- v -> serializer.decode(v))), 5000);
-
- CountDownLatch latch = new CountDownLatch(1);
- AtomicBoolean failed = new AtomicBoolean();
-
- Executor executor = Executors.newSingleThreadExecutor();
- map1.addListener(event -> {
- // Check only the "baz" value since it's the last one written. If we check for "bar" on the "bar" event,
- // there's a race in the test wherein the cache can legitimately be updated to "baz" before the next read.
- if (event.newValue().value().equals("baz")) {
- try {
- assertEquals(event.newValue().value(), map1.get("foo").value());
- } catch (AssertionError e) {
- failed.set(true);
- }
- latch.countDown();
- }
- }, executor);
-
- map2.put("foo", "bar");
- map2.put("foo", "baz");
-
- latch.await(10, TimeUnit.SECONDS);
- if (latch.getCount() == 1 || failed.get()) {
- fail();
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimapTest.java
deleted file mode 100644
index 84f7595..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimapTest.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Collection;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onosproject.store.primitives.DefaultConsistentMultimap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapService;
-import org.onosproject.store.primitives.resources.impl.AtomixTestBase;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.ConsistentMultimap;
-import org.onosproject.store.service.Serializer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-/**
- * Unit tests for cached {@link AtomixConsistentSetMultimap}.
- */
-public class CachingAsyncConsistentMultimapTest extends AtomixTestBase<AtomixConsistentSetMultimap> {
-
- @Override
- protected RaftService createService() {
- return new AtomixConsistentSetMultimapService();
- }
-
- @Override
- protected AtomixConsistentSetMultimap createPrimitive(RaftProxy proxy) {
- return new AtomixConsistentSetMultimap(proxy);
- }
-
- /**
- * Tests that reads following events are not stale when cached.
- */
- @Test
- public void testCacheConsistency() throws Throwable {
- Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
-
- ConsistentMultimap<String, String> multimap1 = new DefaultConsistentMultimap<>(
- new CachingAsyncConsistentMultimap<>(
- new TranscodingAsyncConsistentMultimap<>(
- newPrimitive("testCacheConsistency"),
- k -> k,
- k -> k,
- v -> serializer.decode(v),
- v -> serializer.encode(v))), 5000);
- ConsistentMultimap<String, String> multimap2 = new DefaultConsistentMultimap<>(
- new CachingAsyncConsistentMultimap<>(
- new TranscodingAsyncConsistentMultimap<>(
- newPrimitive("testCacheConsistency"),
- k -> k,
- k -> k,
- v -> serializer.decode(v),
- v -> serializer.encode(v))), 5000);
-
- CountDownLatch latch = new CountDownLatch(1);
- AtomicBoolean failed = new AtomicBoolean();
-
- Executor executor = Executors.newSingleThreadExecutor();
- multimap1.addListener(event -> {
- if (event.newValue().equals("baz")) {
- Collection<? extends String> values = multimap1.get("foo").value();
- try {
- assertEquals(2, values.size());
- } catch (AssertionError e) {
- failed.set(true);
- }
- latch.countDown();
- }
- }, executor);
-
- multimap2.put("foo", "bar");
- multimap2.put("foo", "baz");
-
- latch.await(10, TimeUnit.SECONDS);
- if (latch.getCount() == 1 || failed.get()) {
- fail();
- }
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTreeTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTreeTest.java
deleted file mode 100644
index 0606e23..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTreeTest.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onosproject.store.primitives.DefaultDocumentTree;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTree;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeService;
-import org.onosproject.store.primitives.resources.impl.AtomixTestBase;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTree;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.Serializer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-/**
- * Unit tests for cached {@link AtomixDocumentTree}.
- */
-public class CachingAsyncDocumentTreeTest extends AtomixTestBase<AtomixDocumentTree> {
-
- @Override
- protected RaftService createService() {
- return new AtomixDocumentTreeService(Ordering.NATURAL);
- }
-
- @Override
- protected AtomixDocumentTree createPrimitive(RaftProxy proxy) {
- return new AtomixDocumentTree(proxy);
- }
-
- /**
- * Tests that reads following events are not stale when cached.
- */
- @Test
- public void testCacheConsistency() throws Throwable {
- Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
-
- DocumentTree<String> tree1 = new DefaultDocumentTree<>(
- new CachingAsyncDocumentTree<>(
- new DefaultDistributedDocumentTree<>(
- "testCacheConsistency",
- newPrimitive("testCacheConsistency"),
- serializer)), 5000);
- DocumentTree<String> tree2 = new DefaultDocumentTree<>(
- new CachingAsyncDocumentTree<>(
- new DefaultDistributedDocumentTree<>(
- "testCacheConsistency",
- newPrimitive("testCacheConsistency"),
- serializer)), 5000);
-
- CountDownLatch latch = new CountDownLatch(1);
- AtomicBoolean failed = new AtomicBoolean();
-
- Executor executor = Executors.newSingleThreadExecutor();
- DocumentPath path = DocumentPath.from("root|foo");
- tree1.addListener(path, event -> executor.execute(() -> {
- // Check only the "baz" value since it's the last one written. If we check for "bar" on the "bar" event,
- // there's a race in the test wherein the cache can legitimately be updated to "baz" before the next read.
- if (event.newValue().get().value().equals("baz")) {
- try {
- assertEquals(event.newValue().get().value(), tree1.get(path).value());
- } catch (AssertionError e) {
- failed.set(true);
- }
- latch.countDown();
- }
- }));
-
- tree2.set(path, "bar");
- tree2.set(path, "baz");
-
- latch.await(10, TimeUnit.SECONDS);
- if (latch.getCount() == 1 || failed.get()) {
- fail();
- }
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValueTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValueTest.java
deleted file mode 100644
index a34cf98..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValueTest.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.onlab.util.Tools;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.utils.MeteringAgent;
-
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-
-
-public class DefaultAsyncAtomicValueTest {
- DefaultAsyncAtomicValue<String> defaultAsyncAtomicValue;
-
- private AsyncConsistentMap<String, byte[]> asyncMap;
-
- private Serializer serializer;
- private MeteringAgent meteringAgent;
-
- private static final String MAPNAME = "map1";
-
-
- private static final String NAME = "atomicValue";
- private static final String NAME1 = "atomicValue1";
- private static final String TEST = "foo";
- private static final String TEST1 = "bar";
- private static final int INTNAME = 20;
- private static final long VERSION1 = 1;
-
- private final byte[] value1 = Tools.getBytesUtf8(NAME);
- private final byte[] value2 = Tools.getBytesUtf8(NAME1);
- private final byte[] value3 = Tools.getBytesUtf8("tester");
- private final byte[] defaultValue = Tools.getBytesUtf8("default");
-
- @Before
- public void setUp() throws Exception {
- asyncMap = new AsyncConsistentMapMock<>();
- serializer = Serializer.using(KryoNamespaces.BASIC);
- meteringAgent = new MeteringAgent(NAME, "*", false);
- defaultAsyncAtomicValue = new DefaultAsyncAtomicValue(MAPNAME, serializer,
- asyncMap, meteringAgent);
- }
-
- @After
- public void tearDown() throws Exception {
- defaultAsyncAtomicValue.destroy();
- }
-
- @Test
- public void testAsyncMapping() {
- assertThat(asyncMap.size().join(), is(0));
- asyncMap.put(TEST, value1);
- asyncMap.put(TEST1, value2);
- asyncMap.put("default", defaultValue);
-
- assertThat(asyncMap.getOrDefault("noMatch", defaultValue).join().value(),
- is(asyncMap.get("default").join().value()));
-
- assertThat(asyncMap.size().join(), is(3));
- assertThat(asyncMap.get(TEST).join().value(), is(value1));
-
- assertThat(asyncMap.getOrDefault(TEST, Tools.getBytesUtf8("newTest")).join().value(),
- is(asyncMap.get(TEST).join().value()));
-
- assertThat(asyncMap.containsKey(TEST).join(), is(true));
-
- asyncMap.put(TEST, value3);
- assertThat(asyncMap.get(TEST).join().value(), is(value3));
- asyncMap.putIfAbsent(TEST, value3);
- assertThat(asyncMap.size().join(), is(3));
-
- asyncMap.replace(TEST, value3, value1);
- assertThat(asyncMap.get(TEST).join().value(), is(value1));
-
- asyncMap.replace(TEST, VERSION1, value3);
- assertThat(asyncMap.get(TEST).join().value(), is(value3));
-
- asyncMap.replace(TEST, value3, defaultValue);
- assertThat(asyncMap.get(TEST).join().value(), is(defaultValue));
- asyncMap.replace(TEST, value1);
- assertThat(asyncMap.get(TEST).join().value(), is(value1));
-
- asyncMap.remove(TEST, value2);
-
- assertThat(asyncMap.size().join(), is(3));
- }
-
- @Test
- public void testAsync() {
- asyncMap.put(TEST, value1);
- asyncMap.put(TEST1, value2);
-
- assertNull(defaultAsyncAtomicValue.get().join());
- defaultAsyncAtomicValue = new DefaultAsyncAtomicValue(NAME, serializer,
- asyncMap, meteringAgent);
- assertThat(defaultAsyncAtomicValue.name(), is(NAME));
- defaultAsyncAtomicValue.set(null).join();
- assertNull(defaultAsyncAtomicValue.get().join());
-
- defaultAsyncAtomicValue.set(NAME).join();
- assertThat(defaultAsyncAtomicValue.get().join(), is(NAME));
-
- defaultAsyncAtomicValue.set(NAME1).join();
- assertThat(defaultAsyncAtomicValue.get().join(), is(NAME1));
- defaultAsyncAtomicValue.compareAndSet(NAME1, NAME).join();
- assertThat(defaultAsyncAtomicValue.get().join(), is(NAME));
-
-
- defaultAsyncAtomicValue.getAndSet(null).join();
- assertNull(defaultAsyncAtomicValue.get().join());
-
- defaultAsyncAtomicValue.set(NAME1).join();
- assertThat(defaultAsyncAtomicValue.getAndSet(NAME).join(), is(NAME1));
- assertThat(defaultAsyncAtomicValue.getAndSet("new").join(), is(NAME));
-
- assertThat(defaultAsyncAtomicValue.get().join(), is("new"));
- }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSetTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSetTest.java
deleted file mode 100644
index 899ff47..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSetTest.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.Serializer;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-public class DefaultAsyncDistributedSetTest {
- DefaultAsyncDistributedSet<String> defaultAsyncDistributedSet;
-
- private static String name;
- private static Serializer serializer;
- private static AsyncConsistentMap<String, Boolean> asyncConsistentMap;
- private static AsyncConsistentMap<String, byte[]> baseMap;
- private static Map<Integer, String> map;
- private static Collection<String> collection;
- private static Collection<String> collection1;
- private static Set<String> set;
-
- private static final Boolean MAGENT = false;
- private static final String TEST1 = "one";
- private static final String TEST2 = "two";
- private static final String TEST3 = "three";
- private static final String TEST4 = "four";
- private static final String TEST5 = "five";
-
- @Before
- public void setUp() throws Exception {
- serializer = Serializer.using(KryoNamespaces.API);
- asyncConsistentMap = new AsyncConsistentMapMock<>();
- name = "Default Name";
- map = new HashMap<>();
- collection = new ArrayList<>();
- set = new HashSet<>();
-
- defaultAsyncDistributedSet = new DefaultAsyncDistributedSet<>(asyncConsistentMap,
- name, MAGENT);
- }
- @After
- public void clear() throws Exception {
- defaultAsyncDistributedSet.clear().join();
- assertThat(defaultAsyncDistributedSet.size().join(), is(0));
- assertTrue(defaultAsyncDistributedSet.isEmpty().join());
- }
- @Test
- public void testProperties() {
- assertThat(defaultAsyncDistributedSet.name(), is(name));
- assertTrue(defaultAsyncDistributedSet.isEmpty().join());
-
- collection.add(TEST1);
- collection.add(TEST2);
- collection.add(TEST3);
- set.add(TEST4);
- set.add(TEST5);
-
- assertThat(defaultAsyncDistributedSet.size().join(), is(0));
- defaultAsyncDistributedSet.add(TEST1).join();
- defaultAsyncDistributedSet.add(TEST2).join();
- assertThat(defaultAsyncDistributedSet.size().join(), is(2));
- defaultAsyncDistributedSet.add(TEST3).join();
- assertThat(defaultAsyncDistributedSet.size().join(), is(3));
-
- defaultAsyncDistributedSet.remove(TEST1);
- assertThat(defaultAsyncDistributedSet.size().join(), is(2));
- assertFalse(defaultAsyncDistributedSet.contains(TEST1).join());
- assertTrue(defaultAsyncDistributedSet.contains(TEST2).join());
-
- defaultAsyncDistributedSet.addAll(collection).join();
- assertTrue(defaultAsyncDistributedSet.containsAll(collection).join());
- assertFalse(defaultAsyncDistributedSet.retainAll(collection).join());
-
- assertThat(defaultAsyncDistributedSet.size().join(), is(3));
-
- defaultAsyncDistributedSet.addAll(set).join();
- assertThat(defaultAsyncDistributedSet.size().join(), is(5));
- assertTrue(defaultAsyncDistributedSet.contains(TEST4).join());
- defaultAsyncDistributedSet.retainAll(set);
-
- assertTrue(defaultAsyncDistributedSet.contains(TEST4).join());
- assertThat(defaultAsyncDistributedSet.contains(TEST1).join(), is(false));
-
- assertThat(defaultAsyncDistributedSet.getAsImmutableSet().join().size(),
- is(2));
- }
-
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/NotNullConsistentMapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/NotNullConsistentMapTest.java
deleted file mode 100644
index 9c9e55d..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/NotNullConsistentMapTest.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.Arrays;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapService;
-import org.onosproject.store.primitives.resources.impl.AtomixTestBase;
-import org.onosproject.store.service.AsyncConsistentMap;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Unit tests for {@link AtomixConsistentMap}.
- */
-public class NotNullConsistentMapTest extends AtomixTestBase<AtomixConsistentMap> {
-
- @Override
- protected RaftService createService() {
- return new AtomixConsistentMapService();
- }
-
- @Override
- protected AtomixConsistentMap createPrimitive(RaftProxy proxy) {
- return new AtomixConsistentMap(proxy);
- }
-
- /**
- * Tests not null values.
- */
- @Test
- public void testNotNullValues() throws Throwable {
- final byte[] rawFooValue = Tools.getBytesUtf8("Hello foo!");
- final byte[] rawBarValue = Tools.getBytesUtf8("Hello bar!");
-
- AsyncConsistentMap<String, byte[]> map =
- DistributedPrimitives.newNotNullMap(newPrimitive("testNotNullValues"));
-
- map.get("foo")
- .thenAccept(v -> assertNull(v)).join();
- map.put("foo", null)
- .thenAccept(v -> assertNull(v)).join();
- map.put("foo", rawFooValue).thenAccept(v -> assertNull(v)).join();
- map.get("foo").thenAccept(v -> {
- assertNotNull(v);
- assertTrue(Arrays.equals(v.value(), rawFooValue));
- }).join();
- map.put("foo", null).thenAccept(v -> {
- assertNotNull(v);
- assertTrue(Arrays.equals(v.value(), rawFooValue));
- }).join();
- map.get("foo").thenAccept(v -> assertNull(v)).join();
- map.replace("foo", rawFooValue, null)
- .thenAccept(replaced -> assertFalse(replaced)).join();
- map.replace("foo", null, rawBarValue)
- .thenAccept(replaced -> assertTrue(replaced)).join();
- map.get("foo").thenAccept(v -> {
- assertNotNull(v);
- assertTrue(Arrays.equals(v.value(), rawBarValue));
- }).join();
- map.replace("foo", rawBarValue, null)
- .thenAccept(replaced -> assertTrue(replaced)).join();
- map.get("foo").thenAccept(v -> assertNull(v)).join();
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMapTest.java
deleted file mode 100644
index 5fc019a..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMapTest.java
+++ /dev/null
@@ -1,300 +0,0 @@
-/*
- * Copyright 2015-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import com.google.common.collect.Lists;
-import com.google.common.hash.Hashing;
-import org.junit.Before;
-import org.junit.Test;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncConsistentMapAdapter;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Versioned;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import static org.hamcrest.Matchers.hasItem;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
-
-
-public class PartitionedAsyncConsistentMapTest {
-
- PartitionedAsyncConsistentMap<String, String> partitionedAsyncConsistentMap;
-
- private AsyncConsistentMapMock asyncMap1;
- private AsyncConsistentMapMock asyncMap2;
- private Map<PartitionId, AsyncConsistentMap<String, String>> partitions;
- private List<PartitionId> sortedMemberPartitionIds;
-
-
- private Map<String, String> baseMap;
- private String partitionName = "PartitionManager";
- private PartitionId pid1;
- private PartitionId pid2;
- private Hasher<String> hasher;
- private Serializer serializer;
-
-
- private final List<String> allKeys = Lists.newArrayList(KEY1, KEY2);
-
- private static final String KEY1 = "AAA";
- private static final String VALUE1 = "one";
- private static final String KEY2 = "BBB";
- private static final String VALUE2 = "two";
- private static final String TEST3 = "CCC";
- private static final String VALUE3 = "three";
- private static final String TEST4 = "DDD";
- private static final String VALUE4 = "four";
-
-
-
- public class AsyncConsistentMapMock<K, V> extends AsyncConsistentMapAdapter<K, V> {
- private final List<MapEventListener<K, V>> listeners;
- Collection<Consumer<Status>> statusChangeListeners = new ArrayList<>();
- private final Map<K, V> baseMap;
-
- Versioned<V> makeVersioned(V v) {
- return new Versioned<>(v, 0, 0);
- }
-
- AsyncConsistentMapMock(Map<K, V> newBaseMap) {
- baseMap = newBaseMap;
- listeners = new ArrayList<>();
- }
-
- public CompletableFuture<Integer> size() {
- return CompletableFuture.completedFuture(baseMap.size());
- }
-
- @Override
- public CompletableFuture<Boolean> containsKey(K key) {
- return CompletableFuture.completedFuture(baseMap.containsKey(key));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> getOrDefault(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.getOrDefault(key, value)));
- }
- @Override
- public CompletableFuture<Boolean> containsValue(V value) {
- return CompletableFuture.completedFuture(baseMap.containsValue(value));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> get(K key) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.get(key)));
- }
-
- @Override
- public CompletableFuture<Versioned<V>>
- computeIf(K key, Predicate<? super V> condition,
- BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- return null;
- }
-
- @Override
- public CompletableFuture<Versioned<V>> put(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.put(key, value)));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.put(key, value)));
- }
-
- @Override
- public CompletableFuture<Versioned<V>> remove(K key) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.remove(key)));
- }
-
- @Override
- public CompletableFuture<Void> clear() {
- baseMap.clear();
- return CompletableFuture.allOf();
- }
-
- @Override
- public CompletableFuture<Set<K>> keySet() {
- return CompletableFuture.completedFuture(baseMap.keySet());
- }
-
- @Override
- public CompletableFuture<Collection<Versioned<V>>> values() {
- Set<Versioned<V>> valuesAsVersionedCollection =
- baseMap.values().stream().map(this::makeVersioned)
- .collect(Collectors.toSet());
- return CompletableFuture.completedFuture(valuesAsVersionedCollection);
- }
-
- @Override
- public CompletableFuture<Set<Map.Entry<K, Versioned<V>>>> entrySet() {
- Map<K, Versioned<V>> valuesAsVersionedMap = new HashMap<>();
- baseMap.entrySet()
- .forEach(e -> valuesAsVersionedMap.put(e.getKey(),
- makeVersioned(e.getValue())));
- return CompletableFuture.completedFuture(valuesAsVersionedMap.entrySet());
- }
-
- @Override
- public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.putIfAbsent(key, value)));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, V value) {
- return CompletableFuture.completedFuture(baseMap.remove(key, value));
- }
-
- @Override
- public CompletableFuture<Boolean> remove(K key, long version) {
- Object value = baseMap.remove(key);
- return CompletableFuture.completedFuture(value != null);
- }
-
- @Override
- public CompletableFuture<Versioned<V>> replace(K key, V value) {
- return CompletableFuture.completedFuture(makeVersioned(baseMap.replace(key, value)));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
- return CompletableFuture.completedFuture(baseMap.replace(key, oldValue, newValue));
- }
-
- @Override
- public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
- return CompletableFuture.completedFuture(baseMap.replace(key, newValue) != null);
- }
-
- @Override
- public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor e) {
- listeners.add(listener);
- return CompletableFuture.allOf();
- }
-
- @Override
- public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
- listeners.remove(listener);
- return CompletableFuture.allOf();
- }
-
- @Override
- public void addStatusChangeListener(Consumer<Status> listener) {
- statusChangeListeners.add(listener);
- }
-
- @Override
- public void removeStatusChangeListener(Consumer<Status> listener) {
- statusChangeListeners.remove(listener);
- }
-
- @Override
- public Collection<Consumer<Status>> statusChangeListeners() {
- return statusChangeListeners;
- }
- }
-
- @Before
- public void setUp() throws Exception {
- baseMap = new HashMap<>();
- asyncMap1 = new AsyncConsistentMapMock<>(baseMap);
- asyncMap2 = new AsyncConsistentMapMock<>(baseMap);
-
- pid1 = PartitionId.from(1);
- pid2 = PartitionId.from(2);
- partitions = new HashMap<>();
- serializer = Serializer.using(KryoNamespaces.BASIC);
-
- asyncMap1.put(KEY1, VALUE1);
- asyncMap2.put(KEY2, VALUE2);
- partitions.put(pid1, asyncMap1);
- partitions.put(pid2, asyncMap2);
-
- sortedMemberPartitionIds = Lists.newArrayList(partitions.keySet());
-
- hasher = key -> {
- int hashCode = Hashing.sha256().hashBytes(serializer.encode(key)).asInt();
- return sortedMemberPartitionIds.get(Math.abs(hashCode) % partitions.size());
- };
-
- partitionedAsyncConsistentMap = new PartitionedAsyncConsistentMap(partitionName,
- partitions, hasher);
-
- }
-
- @Test
- public void tester() {
- assertThat(partitionedAsyncConsistentMap.isEmpty().join(), is(false));
- assertThat(partitionedAsyncConsistentMap.name(), is("PartitionManager"));
- asyncMap1.put(TEST3, VALUE3);
- partitions.put(pid1, asyncMap1);
- assertThat(partitionedAsyncConsistentMap.size().join(), is(6));
-
- assertThat(partitionedAsyncConsistentMap.entrySet().join().size(), is(3));
-
- asyncMap2.put(TEST4, VALUE4);
- partitions.put(pid2, asyncMap2);
- assertThat(partitionedAsyncConsistentMap.size().join(), is(8));
-
-
- assertThat(partitionedAsyncConsistentMap.containsValue(VALUE1).join(), is(true));
- assertThat(partitionedAsyncConsistentMap.containsValue("newValue").join(), is(false));
- assertThat(partitionedAsyncConsistentMap.containsKey(KEY2).join(), is(true));
- assertThat(partitionedAsyncConsistentMap.containsKey("newKey").join(), is(false));
-
- partitionedAsyncConsistentMap.putAndGet(KEY1, "newOne").join();
- assertThat(partitionedAsyncConsistentMap.containsValue("newOne").join(), is(true));
- partitionedAsyncConsistentMap.remove(KEY1).join();
- assertThat(partitionedAsyncConsistentMap.containsKey(KEY1).join(), is(false));
- partitionedAsyncConsistentMap.putIfAbsent(KEY1, "same").join();
- partitionedAsyncConsistentMap.replace(KEY1, "same", "one").join();
- assertThat(partitionedAsyncConsistentMap.containsValue("one").join(), is(true));
- partitionedAsyncConsistentMap.putIfAbsent("EEE", "five");
-
- assertThat(partitionedAsyncConsistentMap.get(KEY2).join().value(), is(VALUE2));
- assertThat(partitionedAsyncConsistentMap.getOrDefault(KEY1, "nil").join().value(),
- is(VALUE1));
-
- assertThat(partitionedAsyncConsistentMap.getOrDefault("newKey", "testDefault").join().value(),
- is("testDefault"));
-
- assertNotNull(partitionedAsyncConsistentMap.keySet().join());
- assertThat(partitionedAsyncConsistentMap.keySet().join().size(), is(5));
- assertThat(partitionedAsyncConsistentMap.keySet().join(), hasItem("CCC"));
-
-
- partitionedAsyncConsistentMap.clear().join();
- assertThat(partitionedAsyncConsistentMap.size().join(), is(0));
- assertThat(partitionedAsyncConsistentMap.isEmpty().join(), is(true));
-
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TransactionManagerTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TransactionManagerTest.java
deleted file mode 100644
index 918eae3..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TransactionManagerTest.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.HashMap;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-
-import com.google.common.collect.Sets;
-import org.junit.Test;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.primitives.PartitionService;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.ConsistentMapBuilder;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.StorageService;
-
-import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.anyString;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.mock;
-import static org.easymock.EasyMock.replay;
-import static org.junit.Assert.assertSame;
-
-/**
- * Transaction manager test.
- */
-public class TransactionManagerTest {
-
- @Test
- public void testTransactionMapCaching() throws Exception {
- AsyncConsistentMap asyncMap = mock(AsyncConsistentMap.class);
- expect(asyncMap.name()).andReturn("foo");
- expect(asyncMap.addListener(anyObject(MapEventListener.class), anyObject(Executor.class)))
- .andReturn(CompletableFuture.completedFuture(null)).anyTimes();
- asyncMap.addStatusChangeListener(anyObject(Consumer.class));
- expectLastCall().anyTimes();
- expect(asyncMap.entrySet()).andReturn(CompletableFuture.completedFuture(new HashMap<>().entrySet())).anyTimes();
-
- ConsistentMapBuilder mapBuilder = mock(ConsistentMapBuilder.class);
- expect(mapBuilder.withName(anyString())).andReturn(mapBuilder).anyTimes();
- expect(mapBuilder.withSerializer(anyObject(Serializer.class))).andReturn(mapBuilder).anyTimes();
- expect(mapBuilder.buildAsyncMap()).andReturn(asyncMap).anyTimes();
-
- DistributedPrimitiveCreator primitiveCreator = mock(DistributedPrimitiveCreator.class);
- expect(primitiveCreator.newAsyncConsistentMap(anyString(), anyObject(Serializer.class)))
- .andReturn(asyncMap).anyTimes();
-
- StorageService storageService = mock(StorageService.class);
- expect(storageService.consistentMapBuilder()).andReturn(mapBuilder);
-
- PartitionService partitionService = mock(PartitionService.class);
- Set<PartitionId> partitionIds = Sets.newHashSet(PartitionId.from(1), PartitionId.from(2), PartitionId.from(3));
- expect(partitionService.getAllPartitionIds())
- .andReturn(partitionIds).anyTimes();
- expect(partitionService.getNumberOfPartitions())
- .andReturn(partitionIds.size()).anyTimes();
- expect(partitionService.getDistributedPrimitiveCreator(anyObject(PartitionId.class)))
- .andReturn(primitiveCreator).anyTimes();
-
- replay(storageService, partitionService, asyncMap, primitiveCreator, mapBuilder);
-
- TransactionManager transactionManager = new TransactionManager(storageService, partitionService, 128);
- TransactionId transactionId = TransactionId.from(UUID.randomUUID().toString());
- TransactionCoordinator transactionCoordinator = new TransactionCoordinator(transactionId, transactionManager);
- Serializer serializer = Serializer.using(KryoNamespaces.API);
-
- PartitionedTransactionalMap<String, String> transactionalMap1 = (PartitionedTransactionalMap)
- transactionManager.getTransactionalMap("foo", serializer, transactionCoordinator);
- PartitionedTransactionalMap<String, String> transactionalMap2 = (PartitionedTransactionalMap)
- transactionManager.getTransactionalMap("foo", serializer, transactionCoordinator);
-
- assertSame(transactionalMap1.partitions.get(PartitionId.from(1)).transaction.transactionalObject,
- transactionalMap2.partitions.get(PartitionId.from(1)).transaction.transactionalObject);
- assertSame(transactionalMap1.partitions.get(PartitionId.from(2)).transaction.transactionalObject,
- transactionalMap2.partitions.get(PartitionId.from(2)).transaction.transactionalObject);
- assertSame(transactionalMap1.partitions.get(PartitionId.from(3)).transaction.transactionalObject,
- transactionalMap2.partitions.get(PartitionId.from(3)).transaction.transactionalObject);
- }
-
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TransactionTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TransactionTest.java
deleted file mode 100644
index 5d9e5d8..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TransactionTest.java
+++ /dev/null
@@ -1,404 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-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.Executor;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Predicate;
-
-import com.google.common.hash.Hashing;
-import org.junit.Test;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.CommitStatus;
-import org.onosproject.store.service.ConsistentMap;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static junit.framework.TestCase.assertNull;
-import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.anyString;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.mock;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.strictMock;
-import static org.easymock.EasyMock.verify;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Transaction test.
- */
-public class TransactionTest {
-
- @Test
- public void testTransaction() throws Exception {
- AsyncConsistentMap<String, String> asyncMap = strictMock(AsyncConsistentMap.class);
- TransactionId transactionId = TransactionId.from("foo");
- List<MapUpdate<String, String>> updates = Collections.singletonList(new MapUpdate<>());
- Transaction<MapUpdate<String, String>> transaction = new Transaction<>(transactionId, asyncMap);
- assertEquals(transactionId, transaction.transactionId());
-
- expect(asyncMap.begin(transactionId))
- .andReturn(CompletableFuture.completedFuture(new Version(1)));
- expect(asyncMap.prepare(new TransactionLog<>(transactionId, 1, updates)))
- .andReturn(CompletableFuture.completedFuture(true));
- expect(asyncMap.commit(transactionId))
- .andReturn(CompletableFuture.completedFuture(null));
- replay(asyncMap);
-
- assertEquals(Transaction.State.ACTIVE, transaction.state());
- assertEquals(1, transaction.begin().join().value());
- assertEquals(Transaction.State.ACTIVE, transaction.state());
- assertTrue(transaction.prepare(updates).join());
- assertEquals(Transaction.State.PREPARED, transaction.state());
- transaction.commit();
- assertEquals(Transaction.State.COMMITTED, transaction.state());
- verify(asyncMap);
- }
-
- @Test
- public void testTransactionFailOnOutOfOrderCalls() throws Exception {
- AsyncConsistentMap<String, String> asyncMap = strictMock(AsyncConsistentMap.class);
- TransactionId transactionId = TransactionId.from("foo");
- List<MapUpdate<String, String>> updates = Collections.singletonList(new MapUpdate<>());
- Transaction<MapUpdate<String, String>> transaction = new Transaction<>(transactionId, asyncMap);
-
- try {
- transaction.prepare(updates);
- fail();
- } catch (IllegalStateException e) {
- }
-
- try {
- transaction.commit();
- fail();
- } catch (IllegalStateException e) {
- }
-
- try {
- transaction.rollback();
- fail();
- } catch (IllegalStateException e) {
- }
-
- expect(asyncMap.begin(transactionId))
- .andReturn(CompletableFuture.completedFuture(new Version(1)));
- expect(asyncMap.prepare(new TransactionLog<>(transactionId, 1, updates)))
- .andReturn(CompletableFuture.completedFuture(true));
- replay(asyncMap);
-
- assertFalse(transaction.isOpen());
- assertEquals(Transaction.State.ACTIVE, transaction.state());
- assertEquals(1, transaction.begin().join().value());
- assertTrue(transaction.isOpen());
- assertEquals(Transaction.State.ACTIVE, transaction.state());
- assertTrue(transaction.prepare(updates).join());
- assertEquals(Transaction.State.PREPARED, transaction.state());
-
- try {
- transaction.begin();
- fail();
- } catch (IllegalStateException e) {
- }
- verify(asyncMap);
- }
-
- @Test
- public void testCoordinatedMapTransaction() throws Exception {
- List<Object> mocks = new ArrayList<>();
-
- Map<PartitionId, DefaultTransactionalMapParticipant<String, String>> participants = new HashMap<>();
- List<PartitionId> sortedParticipants = new ArrayList<>();
- TransactionId transactionId = TransactionId.from(UUID.randomUUID().toString());
- for (int i = 1; i <= 3; i++) {
- AsyncConsistentMap<String, String> asyncMap = mock(AsyncConsistentMap.class);
- mocks.add(asyncMap);
-
- ConsistentMap<String, String> consistentMap = new TestConsistentMap<>();
- Transaction<MapUpdate<String, String>> transaction = new Transaction<>(transactionId, asyncMap);
- PartitionId partitionId = PartitionId.from(i);
- participants.put(partitionId, new DefaultTransactionalMapParticipant<>(consistentMap, transaction));
- sortedParticipants.add(partitionId);
- }
-
- expect(participants.get(PartitionId.from(1)).transaction.transactionalObject
- .begin(anyObject(TransactionId.class)))
- .andReturn(CompletableFuture.completedFuture(new Version(1)));
-
- expect(participants.get(PartitionId.from(1)).transaction.transactionalObject.prepare(
- new TransactionLog<>(transactionId, 1, Arrays.asList(
- MapUpdate.<String, String>newBuilder()
- .withType(MapUpdate.Type.REMOVE_IF_VERSION_MATCH)
- .withKey("foo")
- .withVersion(1)
- .build(),
- MapUpdate.<String, String>newBuilder()
- .withType(MapUpdate.Type.REMOVE_IF_VERSION_MATCH)
- .withKey("baz")
- .withVersion(2)
- .build()
- )))).andReturn(CompletableFuture.completedFuture(true));
-
- expect(participants.get(PartitionId.from(1)).transaction.transactionalObject.commit(transactionId))
- .andReturn(CompletableFuture.completedFuture(null));
-
- expect(participants.get(PartitionId.from(3)).transaction.transactionalObject
- .begin(anyObject(TransactionId.class)))
- .andReturn(CompletableFuture.completedFuture(new Version(1)));
-
- expect(participants.get(PartitionId.from(3)).transaction.transactionalObject.prepare(
- new TransactionLog<>(transactionId, 1, Arrays.asList(
- MapUpdate.<String, String>newBuilder()
- .withType(MapUpdate.Type.PUT_IF_VERSION_MATCH)
- .withKey("bar")
- .withValue("baz")
- .withVersion(1)
- .build()
- )))).andReturn(CompletableFuture.completedFuture(true));
-
- expect(participants.get(PartitionId.from(3)).transaction.transactionalObject.commit(transactionId))
- .andReturn(CompletableFuture.completedFuture(null));
-
- TransactionManager transactionManager = mock(TransactionManager.class);
- expect(transactionManager.updateState(anyObject(TransactionId.class), anyObject(Transaction.State.class)))
- .andReturn(CompletableFuture.completedFuture(null))
- .anyTimes();
- expect(transactionManager.remove(anyObject(TransactionId.class)))
- .andReturn(CompletableFuture.completedFuture(null))
- .anyTimes();
- mocks.add(transactionManager);
-
- TransactionCoordinator transactionCoordinator = new TransactionCoordinator(transactionId, transactionManager);
-
- Hasher<String> hasher = key -> {
- int hashCode = Hashing.sha256().hashBytes(key.getBytes()).asInt();
- return sortedParticipants.get(Math.abs(hashCode) % sortedParticipants.size());
- };
-
- expect(transactionManager.<String, String>getTransactionalMap(anyString(), anyObject(), anyObject()))
- .andReturn(new PartitionedTransactionalMap(participants, hasher));
-
- replay(mocks.toArray());
-
- PartitionedTransactionalMap<String, String> transactionalMap = (PartitionedTransactionalMap)
- transactionCoordinator.getTransactionalMap("foo", Serializer.using(KryoNamespaces.API));
-
- // Sneak a couple entries in the first partition.
- transactionalMap.partitions.get(PartitionId.from(1)).backingMap.put("foo", "bar");
- transactionalMap.partitions.get(PartitionId.from(1)).backingMap.put("baz", "foo");
-
- assertTrue(transactionalMap.containsKey("foo"));
- assertEquals("bar", transactionalMap.remove("foo"));
- assertFalse(transactionalMap.containsKey("bar"));
- assertNull(transactionalMap.put("bar", "baz"));
- assertTrue(transactionalMap.containsKey("bar"));
- assertTrue(transactionalMap.containsKey("baz"));
- assertFalse(transactionalMap.remove("baz", "baz"));
- assertTrue(transactionalMap.remove("baz", "foo"));
- assertFalse(transactionalMap.containsKey("baz"));
-
- assertEquals(CommitStatus.SUCCESS, transactionCoordinator.commit().join());
- verify(mocks.toArray());
- }
-
- private static class TestConsistentMap<K, V> implements ConsistentMap<K, V> {
- private final Map<K, Versioned<V>> map = new HashMap<>();
- private final AtomicLong version = new AtomicLong();
-
- @Override
- public String name() {
- return null;
- }
-
- @Override
- public Type primitiveType() {
- return Type.CONSISTENT_MAP;
- }
-
- private long nextVersion() {
- return version.incrementAndGet();
- }
-
- @Override
- public int size() {
- return map.size();
- }
-
- @Override
- public boolean isEmpty() {
- return map.isEmpty();
- }
-
- @Override
- public boolean containsKey(K key) {
- return map.containsKey(key);
- }
-
- @Override
- public boolean containsValue(V value) {
- return map.containsValue(value);
- }
-
- @Override
- public Versioned<V> get(K key) {
- return map.get(key);
- }
-
- @Override
- public Versioned<V> getOrDefault(K key, V defaultValue) {
- return map.getOrDefault(key, new Versioned<>(defaultValue, 0));
- }
-
- @Override
- public Versioned<V> computeIfAbsent(K key, Function<? super K, ? extends V> mappingFunction) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Versioned<V> compute(K key, BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Versioned<V> computeIfPresent(K key, BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Versioned<V> computeIf(K key,
- Predicate<? super V> condition, BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Versioned<V> put(K key, V value) {
- return map.put(key, new Versioned<>(value, nextVersion()));
- }
-
- @Override
- public Versioned<V> putAndGet(K key, V value) {
- return put(key, value);
- }
-
- @Override
- public Versioned<V> remove(K key) {
- return map.remove(key);
- }
-
- @Override
- public void clear() {
- map.clear();
- }
-
- @Override
- public Set<K> keySet() {
- return map.keySet();
- }
-
- @Override
- public Collection<Versioned<V>> values() {
- return map.values();
- }
-
- @Override
- public Set<Map.Entry<K, Versioned<V>>> entrySet() {
- return map.entrySet();
- }
-
- @Override
- public Versioned<V> putIfAbsent(K key, V value) {
- return map.putIfAbsent(key, new Versioned<>(value, nextVersion()));
- }
-
- @Override
- public boolean remove(K key, V value) {
- return map.remove(key, value);
- }
-
- @Override
- public boolean remove(K key, long version) {
- Versioned<V> value = map.get(key);
- if (value != null && value.version() == version) {
- map.remove(key);
- return true;
- }
- return false;
- }
-
- @Override
- public Versioned<V> replace(K key, V value) {
- return map.replace(key, new Versioned<>(value, nextVersion()));
- }
-
- @Override
- public boolean replace(K key, V oldValue, V newValue) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean replace(K key, long oldVersion, V newValue) {
- Versioned<V> value = map.get(key);
- if (value != null && value.version() == oldVersion) {
- map.put(key, new Versioned<>(newValue, nextVersion()));
- return true;
- }
- return false;
- }
-
- @Override
- public Iterator<Map.Entry<K, Versioned<V>>> iterator() {
- return map.entrySet().iterator();
- }
-
- @Override
- public void addListener(MapEventListener<K, V> listener, Executor executor) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void removeListener(MapEventListener<K, V> listener) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Map<K, V> asJavaMap() {
- throw new UnsupportedOperationException();
- }
- }
-
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMapTest.java
deleted file mode 100644
index 6466c12..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMapTest.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.impl;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.onosproject.net.DeviceId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.Serializer;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Function;
-
-import static org.hamcrest.Matchers.hasItem;
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-public class TranscodingAsyncConsistentMapTest {
-
- private static Serializer serializer;
- private static AsyncConsistentMap<String, DeviceId> transcodingMap;
- private static AsyncConsistentMap<String, byte[]> baseMap;
- private static Map<String, byte[]> map;
-
-
- private static final String KEY1 = "Key1";
- private static final String KEY2 = "Key2";
- private static final String KEY3 = "Key3";
- private static final DeviceId DEV1 = DeviceId.deviceId("Device1");
- private static final DeviceId DEV2 = DeviceId.deviceId("Device2");
- private static final DeviceId DEV3 = DeviceId.deviceId("foo");
- private static final DeviceId DEV4 = DeviceId.deviceId("bar");
-
- @Before
- public void setUp() throws Exception {
- serializer = Serializer.using(KryoNamespaces.API);
- map = new HashMap<>();
- baseMap = new AsyncConsistentMapMock<>();
-
- transcodingMap = DistributedPrimitives.newTranscodingMap(
- baseMap,
- Function.identity(),
- Function.identity(),
- serializer::encode,
- serializer::decode);
- }
-
- @Test
- public void testSize() throws Exception {
-
- assertThat(transcodingMap.size().join(), is(0));
- transcodingMap.put(KEY1, DEV1).join();
- assertThat(transcodingMap.size().join(), is(1));
- transcodingMap.put(KEY1, DEV2).join();
- assertThat(transcodingMap.size().join(), is(1));
-
- transcodingMap.put(KEY2, DEV2).join();
- assertThat(transcodingMap.size().join(), is(2));
- for (int i = 0; i < 20; i++) {
- transcodingMap.put("KEY" + i + 1, DeviceId.deviceId("Device" + i + 1)).join();
- }
- assertThat(transcodingMap.size().join(), is(22));
- }
-
- @Test
- public void testEmpty() throws Exception {
- assertTrue(transcodingMap.isEmpty().join());
- transcodingMap.put(KEY1, DEV1).join();
- assertFalse(transcodingMap.isEmpty().join());
- transcodingMap.remove(KEY1).join();
- assertTrue(transcodingMap.isEmpty().join());
- transcodingMap.put(KEY1, DEV1).join();
-
- transcodingMap.remove(KEY1, DEV1).join();
- transcodingMap.put(KEY2, DEV2).join();
-
- transcodingMap.remove(KEY1, 1).join();
- assertThat(transcodingMap.size().join(), is(1));
-
- transcodingMap.clear().join();
- assertThat(transcodingMap.isEmpty().join(), is(true));
-
- }
-
- @Test
- public void testContains() throws Exception {
- assertFalse(transcodingMap.containsKey(KEY1).join());
- transcodingMap.put(KEY1, DEV1);
- assertTrue(transcodingMap.containsKey(KEY1).join());
- assertTrue(transcodingMap.containsValue(DEV1).join());
-
- transcodingMap.put(KEY2, DEV2);
- assertTrue(transcodingMap.containsValue(DEV2).join());
-
- transcodingMap.remove(KEY1);
- assertFalse(transcodingMap.containsKey(KEY1).join());
- assertFalse(transcodingMap.containsValue(DEV1).join());
- }
-
- @Test
- public void testGet() throws Exception {
- assertNull(transcodingMap.get(KEY1).join().value());
- transcodingMap.put(KEY2, DEV1).join();
- transcodingMap.put(KEY2, DEV3).join();
-
- assertThat(transcodingMap.get(KEY2).join().value(), is(DEV3));
-
- assertThat(transcodingMap.getOrDefault(KEY1, DeviceId.deviceId("bar")).join().value(),
- is(DEV4));
- transcodingMap.put(KEY1, DEV2).join();
- assertThat(transcodingMap.getOrDefault(KEY1, DEV1).join().value(), is(DEV2));
- assertThat(transcodingMap.get(KEY1).join().value(), is(DEV2));
- }
-
- @Test
- public void testSwitch() throws Exception {
- transcodingMap.put(KEY1, DEV1).join();
- transcodingMap.put(KEY2, DEV2).join();
-
- transcodingMap.replace(KEY1, DEV1, DEV3).join();
- assertThat(transcodingMap.containsValue(DEV2).join(), is(true));
- transcodingMap.putAndGet(KEY1, DEV3).join();
- assertThat(transcodingMap.get(KEY1).join().value(), is(DEV3));
-
- transcodingMap.replace(KEY1, DEV1);
- assertThat(transcodingMap.get(KEY1).join().value(), is(DEV1));
-
- transcodingMap.replace(KEY1, 1, DEV4);
- assertThat(transcodingMap.get(KEY1).join().value(), is(DEV4));
-
- assertNull(transcodingMap.remove("keyTest").join().value());
-
- }
-
- @Test
- public void testEntry() throws Exception {
- assertThat(transcodingMap.entrySet().join().size(), is(0));
- transcodingMap.put(KEY2, DEV2).join();
- transcodingMap.put(KEY1, DEV1).join();
- assertThat(transcodingMap.entrySet().join().size(), is(2));
- transcodingMap.put(KEY3, DEV3).join();
- transcodingMap.put(KEY3, DEV4).join();
- assertThat(transcodingMap.entrySet().join().size(), (is(3)));
-
- transcodingMap.put(KEY3, null).join();
- transcodingMap.putIfAbsent(KEY3, DEV3).join();
- assertThat(transcodingMap.entrySet().join().size(), is(3));
- assertThat(transcodingMap.get(KEY3).join().value(), is(DEV3));
- }
-
- @Test
- public void keyTest() throws Exception {
- assertThat(transcodingMap.keySet().join().size(), is(0));
- transcodingMap.putIfAbsent(KEY1, DEV1).join();
- transcodingMap.putIfAbsent(KEY2, DEV2).join();
- transcodingMap.putIfAbsent(KEY3, DEV3).join();
- assertThat(transcodingMap.keySet().join().size(), is(3));
- assertThat(transcodingMap.keySet().join(), hasItem(KEY1));
- assertThat(transcodingMap.keySet().join(), hasItem(KEY2));
- }
-
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapServiceTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapServiceTest.java
deleted file mode 100644
index 9d58758..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapServiceTest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.service.impl.DefaultCommit;
-import io.atomix.protocols.raft.session.impl.RaftSessionContext;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.protocols.raft.storage.snapshot.Snapshot;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotStore;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.storage.StorageLevel;
-import io.atomix.time.WallClockTimestamp;
-import org.junit.Test;
-
-import static org.easymock.EasyMock.mock;
-import static org.junit.Assert.assertEquals;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT;
-
-/**
- * Atomic counter map service test.
- */
-public class AtomixAtomicCounterMapServiceTest {
- @Test
- public void testSnapshot() throws Exception {
- SnapshotStore store = new SnapshotStore(RaftStorage.newBuilder()
- .withPrefix("test")
- .withStorageLevel(StorageLevel.MEMORY)
- .build());
- Snapshot snapshot = store.newSnapshot(2, new WallClockTimestamp());
-
- AtomixAtomicCounterMapService service = new AtomixAtomicCounterMapService();
- service.put(new DefaultCommit<>(
- 2,
- PUT,
- new AtomixAtomicCounterMapOperations.Put("foo", 1),
- mock(RaftSessionContext.class),
- System.currentTimeMillis()));
-
- try (SnapshotWriter writer = snapshot.openWriter()) {
- service.snapshot(writer);
- }
-
- snapshot.complete();
-
- service = new AtomixAtomicCounterMapService();
- try (SnapshotReader reader = snapshot.openReader()) {
- service.install(reader);
- }
-
- long value = service.get(new DefaultCommit<>(
- 2,
- GET,
- new AtomixAtomicCounterMapOperations.Get("foo"),
- mock(RaftSessionContext.class),
- System.currentTimeMillis()));
- assertEquals(1, value);
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapTest.java
deleted file mode 100644
index c40853b..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapTest.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Unit test for {@code AtomixCounterMap}.
- */
-public class AtomixAtomicCounterMapTest extends AtomixTestBase<AtomixAtomicCounterMap> {
-
- @Override
- protected RaftService createService() {
- return new AtomixAtomicCounterMapService();
- }
-
- @Override
- protected AtomixAtomicCounterMap createPrimitive(RaftProxy proxy) {
- return new AtomixAtomicCounterMap(proxy);
- }
-
- /**
- * Tests basic counter map operations.
- */
- @Test
- public void testBasicCounterMapOperations() throws Throwable {
- AtomixAtomicCounterMap map = newPrimitive("testBasicCounterMapOperationMap");
-
- map.isEmpty().thenAccept(isEmpty -> {
- assertTrue(isEmpty);
- }).join();
-
- map.size().thenAccept(size -> {
- assertTrue(size == 0);
- }).join();
-
- map.put("foo", 2).thenAccept(value -> {
- assertTrue(value == 0);
- }).join();
-
- map.incrementAndGet("foo").thenAccept(value -> {
- assertTrue(value == 3);
- }).join();
-
- map.getAndIncrement("foo").thenAccept(value -> {
- assertTrue(value == 3);
- }).join();
-
- map.get("foo").thenAccept(value -> {
- assertTrue(value == 4);
- }).join();
-
- map.getAndDecrement("foo").thenAccept(value -> {
- assertTrue(value == 4);
- }).join();
-
- map.decrementAndGet("foo").thenAccept(value -> {
- assertTrue(value == 2);
- }).join();
-
- map.size().thenAccept(size -> {
- assertTrue(size == 1);
- }).join();
-
- map.isEmpty().thenAccept(isEmpty -> {
- assertFalse(isEmpty);
- }).join();
-
- map.clear().join();
-
- map.isEmpty().thenAccept(isEmpty -> {
- assertTrue(isEmpty);
- }).join();
-
- map.size().thenAccept(size -> {
- assertTrue(size == 0);
- }).join();
-
- map.get("foo").thenAccept(value -> {
- assertTrue(value == 0);
- }).join();
-
- map.incrementAndGet("bar").thenAccept(value -> {
- assertTrue(value == 1);
- }).join();
-
- map.addAndGet("bar", 2).thenAccept(value -> {
- assertTrue(value == 3);
- }).join();
-
- map.getAndAdd("bar", 3).thenAccept(value -> {
- assertTrue(value == 3);
- }).join();
-
- map.get("bar").thenAccept(value -> {
- assertTrue(value == 6);
- }).join();
-
- map.putIfAbsent("bar", 1).thenAccept(value -> {
- assertTrue(value == 6);
- }).join();
-
- map.replace("bar", 6, 1).thenAccept(succeeded -> {
- assertTrue(succeeded);
- }).join();
-
- map.replace("bar", 6, 1).thenAccept(succeeded -> {
- assertFalse(succeeded);
- }).join();
-
- map.size().thenAccept(size -> {
- assertTrue(size == 1);
- }).join();
-
- map.remove("bar").thenAccept(value -> {
- assertTrue(value == 1);
- }).join();
-
- map.size().thenAccept(size -> {
- assertTrue(size == 0);
- }).join();
-
- map.put("baz", 3).thenAccept(value -> {
- assertTrue(value == 0);
- }).join();
-
- map.remove("baz", 2).thenAccept(removed -> {
- assertFalse(removed);
- }).join();
-
- map.put("baz", 2).thenAccept(value -> {
- assertTrue(value == 3);
- }).join();
-
- map.remove("baz", 2).thenAccept(removed -> {
- assertTrue(removed);
- }).join();
-
- map.isEmpty().thenAccept(isEmpty -> {
- assertTrue(isEmpty);
- }).join();
-
- map.replace("baz", 0, 5).thenAccept(replaced -> {
- assertTrue(replaced);
- }).join();
-
- map.get("baz").thenAccept(value -> {
- assertTrue(value == 5);
- }).join();
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapServiceTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapServiceTest.java
deleted file mode 100644
index a7cd94a..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapServiceTest.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.impl.RaftContext;
-import io.atomix.protocols.raft.protocol.RaftServerProtocol;
-import io.atomix.protocols.raft.service.ServiceId;
-import io.atomix.protocols.raft.service.ServiceType;
-import io.atomix.protocols.raft.service.impl.DefaultCommit;
-import io.atomix.protocols.raft.service.impl.DefaultServiceContext;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.session.SessionId;
-import io.atomix.protocols.raft.session.impl.RaftSessionContext;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.protocols.raft.storage.snapshot.Snapshot;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotStore;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.storage.StorageLevel;
-import io.atomix.time.WallClockTimestamp;
-import io.atomix.utils.concurrent.AtomixThreadFactory;
-import io.atomix.utils.concurrent.SingleThreadContextFactory;
-import org.junit.Test;
-import org.onosproject.store.service.Versioned;
-
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.mock;
-import static org.easymock.EasyMock.replay;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Put;
-import static org.onosproject.store.service.DistributedPrimitive.Type.LEADER_ELECTOR;
-
-/**
- * Consistent map service test.
- */
-public class AtomixConsistentMapServiceTest {
- @Test
- @SuppressWarnings("unchecked")
- public void testSnapshot() throws Exception {
- SnapshotStore store = new SnapshotStore(RaftStorage.newBuilder()
- .withPrefix("test")
- .withStorageLevel(StorageLevel.MEMORY)
- .build());
- Snapshot snapshot = store.newSnapshot(2, new WallClockTimestamp());
-
- DefaultServiceContext context = mock(DefaultServiceContext.class);
- expect(context.serviceType()).andReturn(ServiceType.from(LEADER_ELECTOR.name())).anyTimes();
- expect(context.serviceName()).andReturn("test").anyTimes();
- expect(context.serviceId()).andReturn(ServiceId.from(1)).anyTimes();
-
- RaftContext server = mock(RaftContext.class);
- expect(server.getProtocol()).andReturn(mock(RaftServerProtocol.class));
-
- replay(context, server);
-
- RaftSession session = new RaftSessionContext(
- SessionId.from(1),
- MemberId.from("1"),
- "test",
- ServiceType.from(LEADER_ELECTOR.name()),
- ReadConsistency.LINEARIZABLE,
- 100,
- 5000,
- System.currentTimeMillis(),
- context,
- server,
- new SingleThreadContextFactory(new AtomixThreadFactory()));
-
- AtomixConsistentMapService service = new AtomixConsistentMapService();
- service.put(new DefaultCommit<>(
- 2,
- PUT,
- new Put("foo", "Hello world!".getBytes()),
- session,
- System.currentTimeMillis()));
- service.openIterator(new DefaultCommit<>(
- 3,
- OPEN_ITERATOR,
- null,
- session,
- System.currentTimeMillis()));
-
- try (SnapshotWriter writer = snapshot.openWriter()) {
- service.snapshot(writer);
- }
-
- snapshot.complete();
-
- service = new AtomixConsistentMapService();
- try (SnapshotReader reader = snapshot.openReader()) {
- service.install(reader);
- }
-
- Versioned<byte[]> value = service.get(new DefaultCommit<>(
- 2,
- GET,
- new AtomixConsistentMapOperations.Get("foo"),
- mock(RaftSessionContext.class),
- System.currentTimeMillis()));
- assertNotNull(value);
- assertArrayEquals("Hello world!".getBytes(), value.value());
-
- assertEquals(1, service.next(new DefaultCommit<>(
- 4,
- NEXT,
- new AtomixConsistentMapOperations.IteratorPosition(3L, 0),
- session,
- System.currentTimeMillis())).entries().size());
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapTest.java
deleted file mode 100644
index 8753783..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapTest.java
+++ /dev/null
@@ -1,683 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onlab.util.HexString;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.impl.CompatibleValue;
-import org.onosproject.store.primitives.impl.DistributedPrimitives;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.ConcurrentModificationException;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletionException;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import static org.hamcrest.Matchers.is;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Unit tests for {@link AtomixConsistentMap}.
- */
-public class AtomixConsistentMapTest extends AtomixTestBase<AtomixConsistentMap> {
-
- @Override
- protected RaftService createService() {
- return new AtomixConsistentMapService();
- }
-
- @Override
- protected AtomixConsistentMap createPrimitive(RaftProxy proxy) {
- return new AtomixConsistentMap(proxy);
- }
-
- /**
- * Tests various basic map operations.
- */
- @Test
- public void testBasicMapOperations() throws Throwable {
- basicMapOperationTests();
- }
-
- /**
- * Tests various map compute* operations on different cluster sizes.
- */
- @Test
- public void testMapComputeOperations() throws Throwable {
- mapComputeOperationTests();
- }
-
- /**
- * Tests null values.
- */
- @Test
- public void testNullValues() throws Throwable {
- final byte[] rawFooValue = Tools.getBytesUtf8("Hello foo!");
- final byte[] rawBarValue = Tools.getBytesUtf8("Hello bar!");
-
- AtomixConsistentMap map = newPrimitive("testNullValues");
-
- map.get("foo")
- .thenAccept(v -> assertNull(v)).join();
- map.put("foo", null)
- .thenAccept(v -> assertNull(v)).join();
- map.put("foo", rawFooValue).thenAccept(v -> {
- assertNotNull(v);
- assertNull(v.value());
- }).join();
- map.get("foo").thenAccept(v -> {
- assertNotNull(v);
- assertTrue(Arrays.equals(v.value(), rawFooValue));
- }).join();
- map.replace("foo", rawFooValue, null)
- .thenAccept(replaced -> assertTrue(replaced)).join();
- map.get("foo").thenAccept(v -> {
- assertNotNull(v);
- assertNull(v.value());
- }).join();
- map.replace("foo", rawFooValue, rawBarValue)
- .thenAccept(replaced -> assertFalse(replaced)).join();
- map.replace("foo", null, rawBarValue)
- .thenAccept(replaced -> assertTrue(replaced)).join();
- map.get("foo").thenAccept(v -> {
- assertNotNull(v);
- assertTrue(Arrays.equals(v.value(), rawBarValue));
- }).join();
- }
-
- /**
- * Tests map event notifications.
- */
- @Test
- public void testMapListeners() throws Throwable {
- mapListenerTests();
- }
-
- /**
- * Tests map transaction prepare.
- */
- @Test
- public void testTransactionPrepare() throws Throwable {
- transactionPrepareTests();
- }
-
- /**
- * Tests map transaction commit.
- */
- @Test
- public void testTransactionCommit() throws Throwable {
- transactionCommitTests();
- }
-
- /**
- * Tests map transaction rollback.
- */
- @Test
- public void testTransactionRollback() throws Throwable {
- transactionRollbackTests();
- }
-
- protected void basicMapOperationTests() throws Throwable {
- final byte[] rawFooValue = Tools.getBytesUtf8("Hello foo!");
- final byte[] rawBarValue = Tools.getBytesUtf8("Hello bar!");
-
- AtomixConsistentMap map = newPrimitive("testBasicMapOperationMap");
-
- map.isEmpty().thenAccept(result -> {
- assertTrue(result);
- }).join();
-
- map.put("foo", rawFooValue).thenAccept(result -> {
- assertNull(result);
- }).join();
-
- map.size().thenAccept(result -> {
- assertTrue(result == 1);
- }).join();
-
- map.isEmpty().thenAccept(result -> {
- assertFalse(result);
- }).join();
-
- map.putIfAbsent("foo", "Hello foo again!".getBytes()).thenAccept(result -> {
- assertNotNull(result);
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawFooValue));
- }).join();
-
- map.putIfAbsent("bar", rawBarValue).thenAccept(result -> {
- assertNull(result);
- }).join();
-
- map.size().thenAccept(result -> {
- assertTrue(result == 2);
- }).join();
-
- map.keySet().thenAccept(result -> {
- assertTrue(result.size() == 2);
- assertTrue(result.containsAll(Sets.newHashSet("foo", "bar")));
- }).join();
-
- map.values().thenAccept(result -> {
- assertTrue(result.size() == 2);
- List<String> rawValues =
- result.stream().map(v -> Tools.toStringUtf8(v.value())).collect(Collectors.toList());
- assertTrue(rawValues.contains("Hello foo!"));
- assertTrue(rawValues.contains("Hello bar!"));
- }).join();
-
- map.entrySet().thenAccept(result -> {
- assertTrue(result.size() == 2);
- // TODO: check entries
- }).join();
-
- map.get("foo").thenAccept(result -> {
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawFooValue));
- }).join();
-
- map.remove("foo").thenAccept(result -> {
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawFooValue));
- }).join();
-
- map.containsKey("foo").thenAccept(result -> {
- assertFalse(result);
- }).join();
-
- map.get("foo").thenAccept(result -> {
- assertNull(result);
- }).join();
-
- map.get("bar").thenAccept(result -> {
- assertNotNull(result);
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawBarValue));
- }).join();
-
- map.containsKey("bar").thenAccept(result -> {
- assertTrue(result);
- }).join();
-
- map.size().thenAccept(result -> {
- assertTrue(result == 1);
- }).join();
-
- map.containsValue(rawBarValue).thenAccept(result -> {
- assertTrue(result);
- }).join();
-
- map.containsValue(rawFooValue).thenAccept(result -> {
- assertFalse(result);
- }).join();
-
- map.replace("bar", "Goodbye bar!".getBytes()).thenAccept(result -> {
- assertNotNull(result);
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), rawBarValue));
- }).join();
-
- map.replace("foo", "Goodbye foo!".getBytes()).thenAccept(result -> {
- assertNull(result);
- }).join();
-
- // try replace_if_value_match for a non-existent key
- map.replace("foo", "Goodbye foo!".getBytes(), rawFooValue).thenAccept(result -> {
- assertFalse(result);
- }).join();
-
- map.replace("bar", "Goodbye bar!".getBytes(), rawBarValue).thenAccept(result -> {
- assertTrue(result);
- }).join();
-
- map.replace("bar", "Goodbye bar!".getBytes(), rawBarValue).thenAccept(result -> {
- assertFalse(result);
- }).join();
-
- Versioned<byte[]> barValue = map.get("bar").join();
- map.replace("bar", barValue.version(), "Goodbye bar!".getBytes()).thenAccept(result -> {
- assertTrue(result);
- }).join();
-
- map.replace("bar", barValue.version(), rawBarValue).thenAccept(result -> {
- assertFalse(result);
- }).join();
-
- map.clear().join();
-
- map.size().thenAccept(result -> {
- assertTrue(result == 0);
- }).join();
- }
-
- public void mapComputeOperationTests() throws Throwable {
- final byte[] value1 = Tools.getBytesUtf8("value1");
- final byte[] value2 = Tools.getBytesUtf8("value2");
- final byte[] value3 = Tools.getBytesUtf8("value3");
-
- AtomixConsistentMap map = newPrimitive("testMapComputeOperationsMap");
-
- map.computeIfAbsent("foo", k -> value1).thenAccept(result -> {
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value1));
- }).join();
-
- map.computeIfAbsent("foo", k -> value2).thenAccept(result -> {
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value1));
- }).join();
-
- map.computeIfPresent("bar", (k, v) -> value2).thenAccept(result -> {
- assertNull(result);
- }).join();
-
- map.computeIfPresent("foo", (k, v) -> value3).thenAccept(result -> {
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value3));
- }).join();
-
- map.computeIfPresent("foo", (k, v) -> null).thenAccept(result -> {
- assertNull(result);
- }).join();
-
- map.computeIf("foo", v -> v == null, (k, v) -> value1).thenAccept(result -> {
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value1));
- }).join();
-
- map.compute("foo", (k, v) -> value2).thenAccept(result -> {
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value2));
- }).join();
- }
-
- protected void mapListenerTests() throws Throwable {
- final byte[] value1 = Tools.getBytesUtf8("value1");
- final byte[] value2 = Tools.getBytesUtf8("value2");
- final byte[] value3 = Tools.getBytesUtf8("value3");
-
- AtomixConsistentMap map = newPrimitive("testMapListenerMap");
- TestMapEventListener listener = new TestMapEventListener();
-
- // add listener; insert new value into map and verify an INSERT event is received.
- map.addListener(listener).thenCompose(v -> map.put("foo", value1)).join();
- MapEvent<String, byte[]> event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.INSERT, event.type());
- assertTrue(Arrays.equals(value1, event.newValue().value()));
-
- // remove listener and verify listener is not notified.
- map.removeListener(listener).thenCompose(v -> map.put("foo", value2)).join();
- assertFalse(listener.eventReceived());
-
- // add the listener back and verify UPDATE events are received correctly
- map.addListener(listener).thenCompose(v -> map.put("foo", value3)).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.UPDATE, event.type());
- assertTrue(Arrays.equals(value3, event.newValue().value()));
-
- // perform a non-state changing operation and verify no events are received.
- map.putIfAbsent("foo", value1).join();
- assertFalse(listener.eventReceived());
-
- // verify REMOVE events are received correctly.
- map.remove("foo").join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.REMOVE, event.type());
- assertTrue(Arrays.equals(value3, event.oldValue().value()));
-
- // verify compute methods also generate events.
- map.computeIf("foo", v -> v == null, (k, v) -> value1).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.INSERT, event.type());
- assertTrue(Arrays.equals(value1, event.newValue().value()));
-
- map.compute("foo", (k, v) -> value2).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.UPDATE, event.type());
- assertTrue(Arrays.equals(value2, event.newValue().value()));
-
- map.computeIf("foo", v -> Arrays.equals(v, value2), (k, v) -> null).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.REMOVE, event.type());
- assertTrue(Arrays.equals(value2, event.oldValue().value()));
-
- map.removeListener(listener).join();
- }
-
- protected void transactionPrepareTests() throws Throwable {
- AtomixConsistentMap map = newPrimitive("testPrepareTestsMap");
-
- TransactionId transactionId1 = TransactionId.from("tx1");
- TransactionId transactionId2 = TransactionId.from("tx2");
- TransactionId transactionId3 = TransactionId.from("tx3");
- TransactionId transactionId4 = TransactionId.from("tx4");
-
- Version lock1 = map.begin(transactionId1).join();
-
- MapUpdate<String, byte[]> update1 =
- MapUpdate.<String, byte[]>newBuilder()
- .withType(MapUpdate.Type.LOCK)
- .withKey("foo")
- .withVersion(lock1.value())
- .build();
- MapUpdate<String, byte[]> update2 =
- MapUpdate.<String, byte[]>newBuilder()
- .withType(MapUpdate.Type.LOCK)
- .withKey("bar")
- .withVersion(lock1.value())
- .build();
-
- map.prepare(new TransactionLog<>(transactionId1, lock1.value(), Arrays.asList(update1, update2)))
- .thenAccept(result -> {
- assertTrue(result);
- }).join();
-
- Version lock2 = map.begin(transactionId2).join();
-
- MapUpdate<String, byte[]> update3 =
- MapUpdate.<String, byte[]>newBuilder()
- .withType(MapUpdate.Type.LOCK)
- .withKey("foo")
- .withVersion(lock2.value())
- .build();
-
- map.prepare(new TransactionLog<>(transactionId2, lock2.value(), Arrays.asList(update3)))
- .thenAccept(result -> {
- assertFalse(result);
- }).join();
- map.rollback(transactionId2).join();
-
- Version lock3 = map.begin(transactionId3).join();
-
- MapUpdate<String, byte[]> update4 =
- MapUpdate.<String, byte[]>newBuilder()
- .withType(MapUpdate.Type.LOCK)
- .withKey("baz")
- .withVersion(0)
- .build();
-
- map.prepare(new TransactionLog<>(transactionId3, lock3.value(), Arrays.asList(update4)))
- .thenAccept(result -> {
- assertFalse(result);
- }).join();
- map.rollback(transactionId3).join();
-
- Version lock4 = map.begin(transactionId4).join();
-
- MapUpdate<String, byte[]> update5 =
- MapUpdate.<String, byte[]>newBuilder()
- .withType(MapUpdate.Type.LOCK)
- .withKey("baz")
- .withVersion(lock4.value())
- .build();
-
- map.prepare(new TransactionLog<>(transactionId4, lock4.value(), Arrays.asList(update5)))
- .thenAccept(result -> {
- assertTrue(result);
- }).join();
- }
-
- protected void transactionCommitTests() throws Throwable {
- final byte[] value1 = Tools.getBytesUtf8("value1");
- final byte[] value2 = Tools.getBytesUtf8("value2");
-
- AtomixConsistentMap map = newPrimitive("testCommitTestsMap");
- TestMapEventListener listener = new TestMapEventListener();
-
- map.addListener(listener).join();
-
- TransactionId transactionId = TransactionId.from("tx1");
-
- // Begin the transaction.
- Version lock = map.begin(transactionId).join();
-
- // PUT_IF_VERSION_MATCH
- MapUpdate<String, byte[]> update1 =
- MapUpdate.<String, byte[]>newBuilder().withType(MapUpdate.Type.PUT_IF_VERSION_MATCH)
- .withKey("foo")
- .withValue(value1)
- .withVersion(lock.value())
- .build();
-
- map.prepare(new TransactionLog<>(transactionId, lock.value(), Arrays.asList(update1))).thenAccept(result -> {
- assertEquals(true, result);
- }).join();
- // verify changes in Tx is not visible yet until commit
- assertFalse(listener.eventReceived());
-
- map.size().thenAccept(result -> {
- assertTrue(result == 0);
- }).join();
-
- map.get("foo").thenAccept(result -> {
- assertNull(result);
- }).join();
-
- try {
- map.put("foo", value2).join();
- fail("update to map entry in open tx should fail with Exception");
- } catch (CompletionException e) {
- assertEquals(ConcurrentModificationException.class, e.getCause().getClass());
- }
-
- assertFalse(listener.eventReceived());
-
- map.commit(transactionId).join();
- MapEvent<String, byte[]> event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.INSERT, event.type());
- assertTrue(Arrays.equals(value1, event.newValue().value()));
-
- // map should be update-able after commit
- map.put("foo", value2).thenAccept(result -> {
- assertTrue(Arrays.equals(Versioned.valueOrElse(result, null), value1));
- }).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.UPDATE, event.type());
- assertTrue(Arrays.equals(value2, event.newValue().value()));
-
- // REMOVE_IF_VERSION_MATCH
- byte[] currFoo = map.get("foo").get().value();
- long currFooVersion = map.get("foo").get().version();
- MapUpdate<String, byte[]> remove1 =
- MapUpdate.<String, byte[]>newBuilder().withType(MapUpdate.Type.REMOVE_IF_VERSION_MATCH)
- .withKey("foo")
- .withVersion(currFooVersion)
- .build();
-
- transactionId = TransactionId.from("tx2");
-
- // Begin the transaction.
- map.begin(transactionId).join();
-
- map.prepare(new TransactionLog<>(transactionId, lock.value(), Arrays.asList(remove1))).thenAccept(result -> {
- assertTrue("prepare should succeed", result);
- }).join();
- // verify changes in Tx is not visible yet until commit
- assertFalse(listener.eventReceived());
-
- map.size().thenAccept(size -> {
- assertThat(size, is(1));
- }).join();
-
- map.get("foo").thenAccept(result -> {
- assertThat(result.value(), is(currFoo));
- }).join();
-
- map.commit(transactionId).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.REMOVE, event.type());
- assertArrayEquals(currFoo, event.oldValue().value());
-
- map.size().thenAccept(size -> {
- assertThat(size, is(0));
- }).join();
-
- }
-
- protected void transactionRollbackTests() throws Throwable {
- final byte[] value1 = Tools.getBytesUtf8("value1");
- final byte[] value2 = Tools.getBytesUtf8("value2");
-
- AtomixConsistentMap map = newPrimitive("testTransactionRollbackTestsMap");
- TestMapEventListener listener = new TestMapEventListener();
-
- map.addListener(listener).join();
-
- TransactionId transactionId = TransactionId.from("tx1");
-
- Version lock = map.begin(transactionId).join();
-
- MapUpdate<String, byte[]> update1 =
- MapUpdate.<String, byte[]>newBuilder().withType(MapUpdate.Type.PUT_IF_VERSION_MATCH)
- .withKey("foo")
- .withValue(value1)
- .withVersion(lock.value())
- .build();
-
- map.prepare(new TransactionLog<>(transactionId, lock.value(), Arrays.asList(update1))).thenAccept(result -> {
- assertEquals(true, result);
- }).join();
- assertFalse(listener.eventReceived());
-
- map.rollback(transactionId).join();
- assertFalse(listener.eventReceived());
-
- map.get("foo").thenAccept(result -> {
- assertNull(result);
- }).join();
-
- map.put("foo", value2).thenAccept(result -> {
- assertNull(result);
- }).join();
- MapEvent<String, byte[]> event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.INSERT, event.type());
- assertTrue(Arrays.equals(value2, event.newValue().value()));
- }
-
- @Test
- public void testCompatibilityFunction() throws Throwable {
- AtomixConsistentMap atomixMap = newPrimitive("testCompatibilityFunction");
-
- Serializer rawSerializer = Serializer.using(KryoNamespaces.API, CompatibleValue.class);
- Serializer valueSerializer = Serializer.using(KryoNamespaces.BASIC);
-
- // Convert the byte[] value to CompatibleValue<byte[]>
- AsyncConsistentMap<String, CompatibleValue<byte[]>> rawMap = DistributedPrimitives.newTranscodingMap(
- atomixMap,
- key -> HexString.toHexString(rawSerializer.encode(key)),
- string -> rawSerializer.decode(HexString.fromHexString(string)),
- value -> value == null ? null : rawSerializer.encode(value),
- bytes -> rawSerializer.decode(bytes));
-
- // Convert the CompatibleValue<byte[]> value to CompatibleValue<V> using the user-provided serializer.
- AsyncConsistentMap<String, CompatibleValue<String>> compatibleMap =
- DistributedPrimitives.newTranscodingMap(
- rawMap,
- key -> key,
- key -> key,
- value -> value == null ? null :
- new CompatibleValue<byte[]>(valueSerializer.encode(value.value()), value.version()),
- value -> value == null ? null :
- new CompatibleValue<String>(valueSerializer.decode(value.value()), value.version()));
-
- AsyncConsistentMap<String, String> map1 = DistributedPrimitives.newCompatibleMap(
- compatibleMap,
- (value, version) -> version + ":" + value,
- org.onosproject.core.Version.version("1.0.0"));
- AsyncConsistentMap<String, String> map2 = DistributedPrimitives.newCompatibleMap(
- compatibleMap,
- (value, version) -> version + ":" + value,
- org.onosproject.core.Version.version("1.0.1"));
-
- map1.put("foo", "Hello world!").join();
- assertEquals("Hello world!", map1.get("foo").join().value());
- assertEquals("1.0.0:Hello world!", map2.get("foo").join().value());
-
- map2.put("bar", "Hello world again!").join();
- assertEquals("Hello world again!", map2.get("bar").join().value());
- assertEquals("1.0.1:Hello world again!", map1.get("bar").join().value());
- }
-
- @Test
- public void testIterator() throws Exception {
- AtomixConsistentMap map = newPrimitive("testIterator");
- for (int i = 0; i < 100; i++) {
- for (int j = 0; j < 100; j++) {
- map.put(String.valueOf(i), String.valueOf(j).getBytes()).join();
- }
- }
-
- List<Map.Entry<String, Versioned<byte[]>>> entries = new ArrayList<>();
- AsyncIterator<Map.Entry<String, Versioned<byte[]>>> iterator = map.iterator().get(5, TimeUnit.SECONDS);
- while (iterator.hasNext().get(5, TimeUnit.SECONDS)) {
- map.put("foo", UUID.randomUUID().toString().getBytes()).join();
- entries.add(iterator.next().get(5, TimeUnit.SECONDS));
- }
- assertEquals(100, entries.size());
- assertEquals(101, map.asConsistentMap().stream().count());
- }
-
- private static class TestMapEventListener implements MapEventListener<String, byte[]> {
-
- private final BlockingQueue<MapEvent<String, byte[]>> queue = new ArrayBlockingQueue<>(1);
-
- @Override
- public void event(MapEvent<String, byte[]> event) {
- try {
- queue.put(event);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new IllegalStateException(e);
- }
- }
-
- public boolean eventReceived() {
- return !queue.isEmpty();
- }
-
- public MapEvent<String, byte[]> event() throws InterruptedException {
- return queue.take();
- }
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapServiceTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapServiceTest.java
deleted file mode 100644
index 5efdeb3..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapServiceTest.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-import java.util.Collections;
-
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.impl.RaftContext;
-import io.atomix.protocols.raft.protocol.RaftServerProtocol;
-import io.atomix.protocols.raft.service.ServiceId;
-import io.atomix.protocols.raft.service.ServiceType;
-import io.atomix.protocols.raft.service.impl.DefaultCommit;
-import io.atomix.protocols.raft.service.impl.DefaultServiceContext;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.session.SessionId;
-import io.atomix.protocols.raft.session.impl.RaftSessionContext;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.protocols.raft.storage.snapshot.Snapshot;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotStore;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.storage.StorageLevel;
-import io.atomix.time.WallClockTimestamp;
-import io.atomix.utils.concurrent.AtomixThreadFactory;
-import io.atomix.utils.concurrent.SingleThreadContextFactory;
-import org.junit.Test;
-import org.onlab.util.Match;
-import org.onosproject.store.service.Versioned;
-
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.mock;
-import static org.easymock.EasyMock.replay;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT;
-import static org.onosproject.store.service.DistributedPrimitive.Type.LEADER_ELECTOR;
-
-/**
- * Consistent set multimap service test.
- */
-public class AtomixConsistentSetMultimapServiceTest {
- @Test
- @SuppressWarnings("unchecked")
- public void testSnapshot() throws Exception {
- SnapshotStore store = new SnapshotStore(RaftStorage.newBuilder()
- .withPrefix("test")
- .withStorageLevel(StorageLevel.MEMORY)
- .build());
- Snapshot snapshot = store.newSnapshot(2, new WallClockTimestamp());
-
- DefaultServiceContext context = mock(DefaultServiceContext.class);
- expect(context.serviceType()).andReturn(ServiceType.from(LEADER_ELECTOR.name())).anyTimes();
- expect(context.serviceName()).andReturn("test").anyTimes();
- expect(context.serviceId()).andReturn(ServiceId.from(1)).anyTimes();
-
- RaftContext server = mock(RaftContext.class);
- expect(server.getProtocol()).andReturn(mock(RaftServerProtocol.class));
-
- replay(context, server);
-
- RaftSession session = new RaftSessionContext(
- SessionId.from(1),
- MemberId.from("1"),
- "test",
- ServiceType.from(LEADER_ELECTOR.name()),
- ReadConsistency.LINEARIZABLE,
- 100,
- 5000,
- System.currentTimeMillis(),
- context,
- server,
- new SingleThreadContextFactory(new AtomixThreadFactory()));
-
- AtomixConsistentSetMultimapService service = new AtomixConsistentSetMultimapService();
- service.put(new DefaultCommit<>(
- 2,
- PUT,
- new AtomixConsistentSetMultimapOperations.Put(
- "foo", Collections.singletonList("Hello world!".getBytes()), Match.ANY),
- session,
- System.currentTimeMillis()));
- service.openIterator(new DefaultCommit<>(
- 3,
- OPEN_ITERATOR,
- null,
- session,
- System.currentTimeMillis()));
-
- try (SnapshotWriter writer = snapshot.openWriter()) {
- service.snapshot(writer);
- }
-
- snapshot.complete();
-
- service = new AtomixConsistentSetMultimapService();
- try (SnapshotReader reader = snapshot.openReader()) {
- service.install(reader);
- }
-
- Versioned<Collection<? extends byte[]>> value = service.get(new DefaultCommit<>(
- 3,
- GET,
- new AtomixConsistentSetMultimapOperations.Get("foo"),
- session,
- System.currentTimeMillis()));
- assertNotNull(value);
- assertEquals(1, value.value().size());
- assertArrayEquals("Hello world!".getBytes(), value.value().iterator().next());
-
- assertEquals(1, service.next(new DefaultCommit<>(
- 4,
- NEXT,
- new AtomixConsistentSetMultimapOperations.IteratorPosition(3L, 0),
- session,
- System.currentTimeMillis())).entries().size());
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapTest.java
deleted file mode 100644
index 44d09bf..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapTest.java
+++ /dev/null
@@ -1,618 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multiset;
-import com.google.common.collect.TreeMultiset;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.apache.commons.collections.keyvalue.DefaultMapEntry;
-import org.junit.Test;
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.MultimapEventListener;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests the {@link AtomixConsistentSetMultimap}.
- */
-public class AtomixConsistentSetMultimapTest extends AtomixTestBase<AtomixConsistentSetMultimap> {
- private final String keyOne = "hello";
- private final String keyTwo = "goodbye";
- private final String keyThree = "foo";
- private final String keyFour = "bar";
- private final byte[] valueOne = Tools.getBytesUtf8(keyOne);
- private final byte[] valueTwo = Tools.getBytesUtf8(keyTwo);
- private final byte[] valueThree = Tools.getBytesUtf8(keyThree);
- private final byte[] valueFour = Tools.getBytesUtf8(keyFour);
- private final List<String> allKeys = Lists.newArrayList(keyOne, keyTwo,
- keyThree, keyFour);
- private final List<byte[]> allValues = Lists.newArrayList(valueOne,
- valueTwo,
- valueThree,
- valueFour);
-
- @Override
- protected RaftService createService() {
- return new AtomixConsistentSetMultimapService();
- }
-
- @Override
- protected AtomixConsistentSetMultimap createPrimitive(RaftProxy proxy) {
- return new AtomixConsistentSetMultimap(proxy);
- }
-
- /**
- * Test that size behaves correctly (This includes testing of the empty
- * check).
- */
- @Test
- public void testSize() throws Throwable {
- AtomixConsistentSetMultimap map = createResource("testOneMap");
- //Simplest operation case
- map.isEmpty().thenAccept(result -> assertTrue(result));
- map.put(keyOne, valueOne).
- thenAccept(result -> assertTrue(result)).join();
- map.isEmpty().thenAccept(result -> assertFalse(result));
- map.size().thenAccept(result -> assertEquals(1, (int) result))
- .join();
- //Make sure sizing is dependent on values not keys
- map.put(keyOne, valueTwo).
- thenAccept(result -> assertTrue(result)).join();
- map.size().thenAccept(result -> assertEquals(2, (int) result))
- .join();
- //Ensure that double adding has no effect
- map.put(keyOne, valueOne).
- thenAccept(result -> assertFalse(result)).join();
- map.size().thenAccept(result -> assertEquals(2, (int) result))
- .join();
- //Check handling for multiple keys
- map.put(keyTwo, valueOne)
- .thenAccept(result -> assertTrue(result)).join();
- map.put(keyTwo, valueTwo)
- .thenAccept(result -> assertTrue(result)).join();
- map.size().thenAccept(result -> assertEquals(4, (int) result))
- .join();
- //Check size with removal
- map.remove(keyOne, valueOne).
- thenAccept(result -> assertTrue(result)).join();
- map.size().thenAccept(result -> assertEquals(3, (int) result))
- .join();
- //Check behavior under remove of non-existent key
- map.remove(keyOne, valueOne).
- thenAccept(result -> assertFalse(result)).join();
- map.size().thenAccept(result -> assertEquals(3, (int) result))
- .join();
- //Check clearing the entirety of the map
- map.clear().join();
- map.size().thenAccept(result -> assertEquals(0, (int) result))
- .join();
- map.isEmpty().thenAccept(result -> assertTrue(result));
-
- map.destroy().join();
- }
-
- /**
- * Contains tests for value, key and entry.
- */
- @Test
- public void containsTest() throws Throwable {
- AtomixConsistentSetMultimap map = createResource("testTwoMap");
-
- //Populate the maps
- allKeys.forEach(key -> {
- map.putAll(key, allValues)
- .thenAccept(result -> assertTrue(result)).join();
- });
- map.size().thenAccept(result -> assertEquals(16, (int) result)).join();
-
- //Test key contains positive results
- allKeys.forEach(key -> {
- map.containsKey(key)
- .thenAccept(result -> assertTrue(result)).join();
- });
-
- //Test value contains positive results
- allValues.forEach(value -> {
- map.containsValue(value)
- .thenAccept(result -> assertTrue(result)).join();
- });
-
- //Test contains entry for all possible entries
- allKeys.forEach(key -> {
- allValues.forEach(value -> {
- map.containsEntry(key, value)
- .thenAccept(result -> assertTrue(result)).join();
- });
- });
-
- final String[] removedKey = new String[1];
-
- //Test behavior after removals
- allValues.forEach(value -> {
- allKeys.forEach(key -> {
- map.remove(key, value)
- .thenAccept(result -> assertTrue(result)).join();
- map.containsEntry(key, value)
- .thenAccept(result -> assertFalse(result)).join();
- removedKey[0] = key;
- });
- });
-
- //Check that contains key works properly for removed keys
- map.containsKey(removedKey[0])
- .thenAccept(result -> assertFalse(result)).join();
-
- //Check that contains value works correctly for removed values
- allValues.forEach(value -> {
- map.containsValue(value)
- .thenAccept(result -> assertFalse(result)).join();
- });
-
- map.destroy().join();
- }
-
- /**
- * Contains tests for put, putAll, remove, removeAll and replace.
- * @throws Exception
- */
- @Test
- public void addAndRemoveTest() throws Exception {
- AtomixConsistentSetMultimap map = createResource("testThreeMap");
-
- //Test single put
- allKeys.forEach(key -> {
- //Value should actually be added here
- allValues.forEach(value -> {
- map.put(key, value)
- .thenAccept(result -> assertTrue(result)).join();
- //Duplicate values should be ignored here
- map.put(key, value)
- .thenAccept(result -> assertFalse(result)).join();
- });
- });
-
- //Test single remove
- allKeys.forEach(key -> {
- //Value should actually be added here
- allValues.forEach(value -> {
- map.remove(key, value)
- .thenAccept(result -> assertTrue(result)).join();
- //Duplicate values should be ignored here
- map.remove(key, value)
- .thenAccept(result -> assertFalse(result)).join();
- });
- });
-
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
-
- //Test multi put
- allKeys.forEach(key -> {
- map.putAll(key, Lists.newArrayList(allValues.subList(0, 2)))
- .thenAccept(result -> assertTrue(result)).join();
- map.putAll(key, Lists.newArrayList(allValues.subList(0, 2)))
- .thenAccept(result -> assertFalse(result)).join();
- map.putAll(key, Lists.newArrayList(allValues.subList(2, 4)))
- .thenAccept(result -> assertTrue(result)).join();
- map.putAll(key, Lists.newArrayList(allValues.subList(2, 4)))
- .thenAccept(result -> assertFalse(result)).join();
-
- });
-
- //Test multi remove
- allKeys.forEach(key -> {
- //Split the lists to test how multiRemove can work piecewise
- map.removeAll(key, Lists.newArrayList(allValues.subList(0, 2)))
- .thenAccept(result -> assertTrue(result)).join();
- map.removeAll(key, Lists.newArrayList(allValues.subList(0, 2)))
- .thenAccept(result -> assertFalse(result)).join();
- map.removeAll(key, Lists.newArrayList(allValues.subList(2, 4)))
- .thenAccept(result -> assertTrue(result)).join();
- map.removeAll(key, Lists.newArrayList(allValues.subList(2, 4)))
- .thenAccept(result -> assertFalse(result)).join();
- });
-
- allKeys.forEach(key -> {
- map.putAndGet(key, valueOne)
- .thenAccept(result -> assertEquals(1, result.value().size()));
- map.putAndGet(key, valueTwo)
- .thenAccept(result -> assertEquals(2, result.value().size()));
- map.putAndGet(key, valueThree)
- .thenAccept(result -> assertEquals(3, result.value().size()));
- map.putAndGet(key, valueFour)
- .thenAccept(result -> assertEquals(4, result.value().size()));
- });
-
- allKeys.forEach(key -> {
- map.removeAndGet(key, valueOne)
- .thenAccept(result -> assertEquals(3, result.value().size()));
- map.removeAndGet(key, valueTwo)
- .thenAccept(result -> assertEquals(2, result.value().size()));
- map.removeAndGet(key, valueThree)
- .thenAccept(result -> assertEquals(1, result.value().size()));
- map.removeAndGet(key, valueFour)
- .thenAccept(result -> assertEquals(0, result.value().size()));
- });
-
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
-
- //Repopulate for next test
- allKeys.forEach(key -> {
- map.putAll(key, allValues)
- .thenAccept(result -> assertTrue(result)).join();
- });
-
- map.size().thenAccept(result -> assertEquals(16, (int) result)).join();
-
- //Test removeAll of entire entry
- allKeys.forEach(key -> {
- map.removeAll(key).thenAccept(result -> {
- assertTrue(
- byteArrayCollectionIsEqual(allValues, result.value()));
- }).join();
- map.removeAll(key).thenAccept(result -> {
- assertFalse(
- byteArrayCollectionIsEqual(allValues, result.value()));
- }).join();
- });
-
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
-
- //Repopulate for next test
- allKeys.forEach(key -> {
- map.putAll(key, allValues)
- .thenAccept(result -> assertTrue(result)).join();
- });
-
- map.size().thenAccept(result -> assertEquals(16, (int) result)).join();
-
- allKeys.forEach(key -> {
- map.replaceValues(key, allValues)
- .thenAccept(result ->
- assertTrue(byteArrayCollectionIsEqual(allValues,
- result.value())))
- .join();
- map.replaceValues(key, Lists.newArrayList())
- .thenAccept(result ->
- assertTrue(byteArrayCollectionIsEqual(allValues,
- result.value())))
- .join();
- map.replaceValues(key, allValues)
- .thenAccept(result ->
- assertTrue(result.value().isEmpty()))
- .join();
- });
-
-
- //Test replacements of partial sets
- map.size().thenAccept(result -> assertEquals(16, (int) result)).join();
-
- allKeys.forEach(key -> {
- map.remove(key, valueOne)
- .thenAccept(result ->
- assertTrue(result)).join();
- map.replaceValues(key, Lists.newArrayList())
- .thenAccept(result ->
- assertTrue(byteArrayCollectionIsEqual(
- Lists.newArrayList(valueTwo, valueThree,
- valueFour),
- result.value())))
- .join();
- map.replaceValues(key, allValues)
- .thenAccept(result ->
- assertTrue(result.value().isEmpty()))
- .join();
- });
-
- map.destroy().join();
- }
-
- @Test
- public void testStreams() throws Exception {
- AtomixConsistentSetMultimap map = createResource("testStreams");
- for (int i = 0; i < 100; i++) {
- for (int j = 0; j < 100; j++) {
- map.put(String.valueOf(i), String.valueOf(j).getBytes()).join();
- }
- }
-
- List<Map.Entry<String, byte[]>> entries = new ArrayList<>();
- AsyncIterator<Map.Entry<String, byte[]>> iterator = map.iterator().get(5, TimeUnit.SECONDS);
- while (iterator.hasNext().get(5, TimeUnit.SECONDS)) {
- map.put(keyOne, UUID.randomUUID().toString().getBytes()).join();
- entries.add(iterator.next().get(5, TimeUnit.SECONDS));
- }
- assertEquals(10000, entries.size());
- }
-
- /**
- * Tests the get, keySet, keys, values, and entries implementations as well
- * as a trivial test of the asMap functionality (throws error).
- * @throws Exception
- */
- @Test
- public void testAccessors() throws Exception {
- AtomixConsistentSetMultimap map = createResource("testFourMap");
-
- //Populate for full map behavior tests
- allKeys.forEach(key -> {
- map.putAll(key, allValues)
- .thenAccept(result -> assertTrue(result)).join();
- });
-
- map.size().thenAccept(result -> assertEquals(16, (int) result)).join();
-
- allKeys.forEach(key -> {
- map.get(key).thenAccept(result -> {
- assertTrue(byteArrayCollectionIsEqual(allValues,
- result.value()));
- }).join();
- });
-
- //Test that the key set is correct
- map.keySet()
- .thenAccept(result ->
- assertTrue(stringArrayCollectionIsEqual(allKeys,
- result)))
- .join();
- //Test that the correct set and occurrence of values are found in the
- //values result
- map.values().thenAccept(result -> {
- final Multiset<byte[]> set = TreeMultiset.create(
- new ByteArrayComparator());
- for (int i = 0; i < 4; i++) {
- set.addAll(allValues);
- }
- assertEquals(16, result.size());
- result.forEach(value -> assertTrue(set.remove(value)));
- assertTrue(set.isEmpty());
-
- }).join();
-
- //Test that keys returns the right result including the correct number
- //of each item
- map.keys().thenAccept(result -> {
- final Multiset<String> set = TreeMultiset.create();
- for (int i = 0; i < 4; i++) {
- set.addAll(allKeys);
- }
- assertEquals(16, result.size());
- result.forEach(value -> assertTrue(set.remove(value)));
- assertTrue(set.isEmpty());
-
- }).join();
-
- //Test that the right combination of key, value pairs are present
- map.entries().thenAccept(result -> {
- final Multiset<Map.Entry<String, byte[]>> set =
- TreeMultiset.create(new EntryComparator());
- allKeys.forEach(key -> {
- allValues.forEach(value -> {
- set.add(new DefaultMapEntry(key, value));
- });
- });
- assertEquals(16, result.size());
- result.forEach(entry -> assertTrue(set.remove(entry)));
- assertTrue(set.isEmpty());
- }).join();
-
-
- //Testing for empty map behavior
- map.clear().join();
-
- allKeys.forEach(key -> {
- map.get(key).thenAccept(result -> {
- assertTrue(result.value().isEmpty());
- }).join();
- });
-
- map.keySet().thenAccept(result -> assertTrue(result.isEmpty())).join();
- map.values().thenAccept(result -> assertTrue(result.isEmpty())).join();
- map.keys().thenAccept(result -> assertTrue(result.isEmpty())).join();
- map.entries()
- .thenAccept(result -> assertTrue(result.isEmpty())).join();
-
- map.destroy().join();
- }
-
- @Test
- public void testMultimapEvents() throws Throwable {
- final byte[] value1 = Tools.getBytesUtf8("value1");
- final byte[] value2 = Tools.getBytesUtf8("value2");
- final byte[] value3 = Tools.getBytesUtf8("value3");
-
- AtomixConsistentSetMultimap map = createResource("testFourMap");
- TestMultimapEventListener listener = new TestMultimapEventListener();
-
- // add listener; insert new value into map and verify an INSERT event is received.
- map.addListener(listener).thenCompose(v -> map.put("foo", value1)).join();
- MultimapEvent<String, byte[]> event = listener.event();
- assertNotNull(event);
- assertEquals(MultimapEvent.Type.INSERT, event.type());
- assertTrue(Arrays.equals(value1, event.newValue()));
-
- // remove listener and verify listener is not notified.
- map.removeListener(listener).thenCompose(v -> map.put("foo", value2)).join();
- assertFalse(listener.eventReceived());
-
- // add listener; insert new value into map and verify an INSERT event is received.
- map.addListener(listener)
- .thenCompose(v -> map.replaceValues("foo", Arrays.asList(value2, value3))).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MultimapEvent.Type.REMOVE, event.type());
- assertArrayEquals(value1, event.oldValue());
- event = listener.event();
- assertNotNull(event);
- assertEquals(MultimapEvent.Type.INSERT, event.type());
- assertArrayEquals(value3, event.newValue());
-
- // remove listener and verify listener is not notified.
- map.removeListener(listener).thenCompose(v -> map.put("foo", value2)).join();
- assertFalse(listener.eventReceived());
-
- map.removeListener(listener).join();
- }
-
- private AtomixConsistentSetMultimap createResource(String mapName) {
- try {
- AtomixConsistentSetMultimap map = newPrimitive(mapName);
- return map;
- } catch (Throwable e) {
- throw new IllegalStateException(e.toString());
- }
- }
-
- /**
- * Returns two arrays contain the same set of elements,
- * regardless of order.
- * @param o1 first collection
- * @param o2 second collection
- * @return true if they contain the same elements
- */
- private boolean byteArrayCollectionIsEqual(
- Collection<? extends byte[]> o1, Collection<? extends byte[]> o2) {
- if (o1 == null || o2 == null || o1.size() != o2.size()) {
- return false;
- }
- for (byte[] array1 : o1) {
- boolean matched = false;
- for (byte[] array2 : o2) {
- if (Arrays.equals(array1, array2)) {
- matched = true;
- break;
- }
- }
- if (!matched) {
- return false;
- }
- }
- return true;
- }
-
- /**
- * Compares two collections of strings returns true if they contain the
- * same strings, false otherwise.
- * @param s1 string collection one
- * @param s2 string collection two
- * @return true if the two sets contain the same strings
- */
- private boolean stringArrayCollectionIsEqual(
- Collection<? extends String> s1, Collection<? extends String> s2) {
- if (s1 == null || s2 == null || s1.size() != s2.size()) {
- return false;
- }
- for (String string1 : s1) {
- boolean matched = false;
- for (String string2 : s2) {
- if (string1.equals(string2)) {
- matched = true;
- break;
- }
- }
- if (!matched) {
- return false;
- }
- }
- return true;
- }
-
- /**
- * Byte array comparator implementation.
- */
- private class ByteArrayComparator implements Comparator<byte[]> {
-
- @Override
- public int compare(byte[] o1, byte[] o2) {
- if (Arrays.equals(o1, o2)) {
- return 0;
- } else {
- for (int i = 0; i < o1.length && i < o2.length; i++) {
- if (o1[i] < o2[i]) {
- return -1;
- } else if (o1[i] > o2[i]) {
- return 1;
- }
- }
- return o1.length > o2.length ? 1 : -1;
- }
- }
- }
-
- /**
- * Entry comparator, uses both key and value to determine equality,
- * for comparison falls back to the default string comparator.
- */
- private class EntryComparator
- implements Comparator<Map.Entry<String, byte[]>> {
-
- @Override
- public int compare(Map.Entry<String, byte[]> o1,
- Map.Entry<String, byte[]> o2) {
- if (o1 == null || o1.getKey() == null || o2 == null ||
- o2.getKey() == null) {
- throw new IllegalArgumentException();
- }
- if (o1.getKey().equals(o2.getKey()) &&
- Arrays.equals(o1.getValue(), o2.getValue())) {
- return 0;
- } else {
- return o1.getKey().compareTo(o2.getKey());
- }
- }
- }
-
- private static class TestMultimapEventListener implements MultimapEventListener<String, byte[]> {
-
- private final BlockingQueue<MultimapEvent<String, byte[]>> queue = new ArrayBlockingQueue<>(1);
-
- @Override
- public void event(MultimapEvent<String, byte[]> event) {
- try {
- queue.put(event);
- } catch (InterruptedException e) {
- throw new IllegalStateException(e);
- }
- }
-
- public boolean eventReceived() {
- return !queue.isEmpty();
- }
-
- public MultimapEvent<String, byte[]> event() throws InterruptedException {
- return queue.take();
- }
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapTest.java
deleted file mode 100644
index 1548fa5..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapTest.java
+++ /dev/null
@@ -1,594 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import com.google.common.collect.Lists;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Versioned;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Unit tests for {@link AtomixConsistentTreeMap}.
- */
-public class AtomixConsistentTreeMapTest extends AtomixTestBase<AtomixConsistentTreeMap> {
- private final String keyFour = "hello";
- private final String keyThree = "goodbye";
- private final String keyTwo = "foo";
- private final String keyOne = "bar";
- private final byte[] valueOne = Tools.getBytesUtf8(keyOne);
- private final byte[] valueTwo = Tools.getBytesUtf8(keyTwo);
- private final byte[] valueThree = Tools.getBytesUtf8(keyThree);
- private final byte[] valueFour = Tools.getBytesUtf8(keyFour);
- private final byte[] spareValue = Tools.getBytesUtf8("spareValue");
- private final List<String> allKeys = Lists.newArrayList(keyOne, keyTwo,
- keyThree, keyFour);
- private final List<byte[]> allValues = Lists.newArrayList(valueOne,
- valueTwo,
- valueThree,
- valueFour);
-
- @Override
- protected RaftService createService() {
- return new AtomixConsistentTreeMapService();
- }
-
- @Override
- protected AtomixConsistentTreeMap createPrimitive(RaftProxy proxy) {
- return new AtomixConsistentTreeMap(proxy);
- }
-
- /**
- * Tests of the functionality associated with the
- * {@link org.onosproject.store.service.AsyncConsistentMap} interface
- * except transactions and listeners.
- */
- @Test
- public void testBasicMapOperations() throws Throwable {
- //Throughout the test there are isEmpty queries, these are intended to
- //make sure that the previous section has been cleaned up, they serve
- //the secondary purpose of testing isEmpty but that is not their
- //primary purpose.
- AtomixConsistentTreeMap map = createResource("basicTestMap");
- //test size
- map.size().thenAccept(result -> assertEquals(0, (int) result)).join();
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
-
- //test contains key
- allKeys.forEach(key -> map.containsKey(key).
- thenAccept(result -> assertFalse(result)).join());
-
- //test contains value
- allValues.forEach(value -> map.containsValue(value)
- .thenAccept(result -> assertFalse(result)).join());
-
- //test get
- allKeys.forEach(key -> map.get(key).
- thenAccept(result -> assertNull(result)).join());
-
- //test getOrDefault
- allKeys.forEach(key -> map.getOrDefault(key, null).thenAccept(result -> {
- assertEquals(0, result.version());
- assertNull(result.value());
- }).join());
-
- allKeys.forEach(key -> map.getOrDefault(key, "bar".getBytes()).thenAccept(result -> {
- assertEquals(0, result.version());
- assertArrayEquals("bar".getBytes(), result.value());
- }).join());
-
- //populate and redo prior three tests
- allKeys.forEach(key -> map.put(key, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> assertNull(result)).join());
-
- //test contains key
- allKeys.forEach(key -> map.containsKey(key)
- .thenAccept(result -> assertTrue(result)).join());
-
- //test contains value
- allValues.forEach(value -> map.containsValue(value)
- .thenAccept(result -> assertTrue(result)).join());
-
- //test get
- allKeys.forEach(key -> map.get(key).thenAccept(result -> {
- assertArrayEquals(allValues.get(allKeys.indexOf(key)), result.value());
- }).join());
-
- allKeys.forEach(key -> map.getOrDefault(key, null).thenAccept(result -> {
- assertNotEquals(0, result.version());
- assertArrayEquals(allValues.get(allKeys.indexOf(key)), result.value());
- }).join());
-
- //test all compute methods in this section
- allKeys.forEach(key -> map.computeIfAbsent(key, v -> allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> {
- assertArrayEquals(allValues.get(allKeys.indexOf(key)), result.value());
- }).join());
-
- map.size().thenAccept(result -> assertEquals(4, (int) result)).join();
- map.isEmpty().thenAccept(result -> assertFalse(result)).join();
-
- allKeys.forEach(key -> map.computeIfPresent(key, (k, v) -> null).
- thenAccept(result -> assertNull(result)).join());
-
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
-
- allKeys.forEach(key -> map.compute(key, (k, v) -> allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> assertArrayEquals(allValues.get(allKeys.indexOf(key)), result.value())).join());
-
- map.size().thenAccept(result -> assertEquals(4, (int) result)).join();
- map.isEmpty().thenAccept(result -> assertFalse(result)).join();
-
- allKeys.forEach(key -> map.computeIf(key,
- (k) -> allKeys.indexOf(key) < 2, (k, v) -> null).thenAccept(result -> {
- if (allKeys.indexOf(key) < 2) {
- assertNull(result);
- } else {
- assertArrayEquals(allValues.get(allKeys.indexOf(key)), result.value());
- }
- }).join());
-
- map.size().thenAccept(result -> assertEquals(2, (int) result)).join();
- map.isEmpty().thenAccept(result -> assertFalse(result)).join();
-
- //test simple put
- allKeys.forEach(key -> map.put(key, allValues.get(allKeys.indexOf(key))).thenAccept(result -> {
- if (allKeys.indexOf(key) < 2) {
- assertNull(result);
- } else {
- assertArrayEquals(allValues.get(allKeys.indexOf(key)), result.value());
- }
- }).join());
-
- map.size().thenAccept(result -> assertEquals(4, (int) result)).join();
- map.isEmpty().thenAccept(result -> assertFalse(result)).join();
-
- //test put and get for version retrieval
- allKeys.forEach(key -> map.putAndGet(key, allValues.get(allKeys.indexOf(key))).thenAccept(firstResult -> {
- map.putAndGet(key, allValues.get(allKeys.indexOf(key))).thenAccept(secondResult -> {
- assertArrayEquals(allValues.get(allKeys.indexOf(key)), firstResult.value());
- assertArrayEquals(allValues.get(allKeys.indexOf(key)), secondResult.value());
- });
- }).join());
-
- //test removal
- allKeys.forEach(key -> map.remove(key).thenAccept(
- result -> assertArrayEquals(
- allValues.get(allKeys.indexOf(key)), result.value()))
- .join());
- map.isEmpty().thenAccept(result -> assertTrue(result));
-
- //repopulating, this is not mainly for testing
- allKeys.forEach(key -> map.put(key, allValues.get(allKeys.indexOf(key))).thenAccept(result -> {
- assertNull(result);
- }).join());
-
- //Test various collections of keys, values and entries
- map.keySet().thenAccept(keys -> assertTrue(stringArrayCollectionIsEqual(keys, allKeys))).join();
- map.values().thenAccept(values -> assertTrue(
- byteArrayCollectionIsEqual(values.stream().map(v -> v.value())
- .collect(Collectors.toSet()), allValues))).join();
- map.entrySet().thenAccept(entrySet -> {
- entrySet.forEach(entry -> {
- assertTrue(allKeys.contains(entry.getKey()));
- assertTrue(Arrays.equals(entry.getValue().value(),
- allValues.get(allKeys.indexOf(entry.getKey()))));
- });
- }).join();
- map.clear().join();
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
-
- //test conditional put
- allKeys.forEach(key -> map.putIfAbsent(key, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> assertNull(result)).join());
- allKeys.forEach(key -> map.putIfAbsent(key, null).thenAccept(result ->
- assertArrayEquals(result.value(), allValues.get(allKeys.indexOf(key)))
- ).join());
-
- // test alternate removes that specify value or version
- allKeys.forEach(key -> map.remove(key, spareValue).thenAccept(result -> assertFalse(result)).join());
- allKeys.forEach(key -> map.remove(key, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> assertTrue(result)).join());
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
- List<Long> versions = Lists.newArrayList();
-
- //repopulating set for version based removal
- allKeys.forEach(key -> map.putAndGet(key, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> versions.add(result.version())).join());
- allKeys.forEach(key -> map.remove(key, versions.get(0)).thenAccept(result -> {
- assertTrue(result);
- versions.remove(0);
- }).join());
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
-
- //Testing all replace both simple (k, v), and complex that consider
- // previous mapping or version.
- allKeys.forEach(key -> map.put(key, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> assertNull(result)).join());
- allKeys.forEach(key -> map.replace(key, allValues.get(3 - allKeys.indexOf(key)))
- .thenAccept(result -> assertArrayEquals(allValues.get(allKeys.indexOf(key)), result.value()))
- .join());
- allKeys.forEach(key -> map.replace(key, spareValue, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> assertFalse(result)).join());
- allKeys.forEach(key -> map.replace(key, allValues.get(3 - allKeys.indexOf(key)),
- allValues.get(allKeys.indexOf(key))).thenAccept(result -> assertTrue(result)).join());
- map.clear().join();
- map.isEmpty().thenAccept(result -> assertTrue(result)).join();
- versions.clear();
-
- //populate for version based replacement
- allKeys.forEach(key -> map.putAndGet(key, allValues.get(3 - allKeys.indexOf(key)))
- .thenAccept(result -> versions.add(result.version())).join());
- allKeys.forEach(key -> map.replace(key, 0, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> assertFalse(result)).join());
- allKeys.forEach(key -> map.replace(key, versions.get(0), allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> {
- assertTrue(result);
- versions.remove(0);
- }).join());
- }
-
- @Test
- public void mapListenerTests() throws Throwable {
- final byte[] value1 = Tools.getBytesUtf8("value1");
- final byte[] value2 = Tools.getBytesUtf8("value2");
- final byte[] value3 = Tools.getBytesUtf8("value3");
-
- AtomixConsistentTreeMap map = createResource("treeMapListenerTestMap");
- TestMapEventListener listener = new TestMapEventListener();
-
- // add listener; insert new value into map and verify an INSERT event
- // is received.
- map.addListener(listener).thenCompose(v -> map.put("foo", value1))
- .join();
- MapEvent<String, byte[]> event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.INSERT, event.type());
- assertTrue(Arrays.equals(value1, event.newValue().value()));
-
- // remove listener and verify listener is not notified.
- map.removeListener(listener).thenCompose(v -> map.put("foo", value2))
- .join();
- assertFalse(listener.eventReceived());
-
- // add the listener back and verify UPDATE events are received
- // correctly
- map.addListener(listener).thenCompose(v -> map.put("foo", value3))
- .join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.UPDATE, event.type());
- assertTrue(Arrays.equals(value3, event.newValue().value()));
-
- // perform a non-state changing operation and verify no events are
- // received.
- map.putIfAbsent("foo", value1).join();
- assertFalse(listener.eventReceived());
-
- // verify REMOVE events are received correctly.
- map.remove("foo").join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.REMOVE, event.type());
- assertTrue(Arrays.equals(value3, event.oldValue().value()));
-
- // verify compute methods also generate events.
- map.computeIf("foo", v -> v == null, (k, v) -> value1).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.INSERT, event.type());
- assertTrue(Arrays.equals(value1, event.newValue().value()));
-
- map.compute("foo", (k, v) -> value2).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.UPDATE, event.type());
- assertTrue(Arrays.equals(value2, event.newValue().value()));
-
- map.computeIf(
- "foo", v -> Arrays.equals(v, value2), (k, v) -> null).join();
- event = listener.event();
- assertNotNull(event);
- assertEquals(MapEvent.Type.REMOVE, event.type());
- assertTrue(Arrays.equals(value2, event.oldValue().value()));
-
- map.removeListener(listener).join();
- }
-
- /**
- * Tests functionality specified in the {@link AtomixConsistentTreeMap}
- * interface, beyond the functionality provided in
- * {@link org.onosproject.store.service.AsyncConsistentMap}.
- */
- @Test
- public void treeMapFunctionsTest() {
- AtomixConsistentTreeMap map = createResource("treeMapFunctionTestMap");
- //Tests on empty map
- map.firstKey().thenAccept(result -> assertNull(result)).join();
- map.lastKey().thenAccept(result -> assertNull(result)).join();
- map.ceilingEntry(keyOne).thenAccept(result -> assertNull(result))
- .join();
- map.floorEntry(keyOne).thenAccept(result -> assertNull(result)).join();
- map.higherEntry(keyOne).thenAccept(result -> assertNull(result))
- .join();
- map.lowerEntry(keyOne).thenAccept(result -> assertNull(result)).join();
- map.firstEntry().thenAccept(result -> assertNull(result)).join();
- map.lastEntry().thenAccept(result -> assertNull(result)).join();
- map.pollFirstEntry().thenAccept(result -> assertNull(result)).join();
- map.pollLastEntry().thenAccept(result -> assertNull(result)).join();
- map.lowerKey(keyOne).thenAccept(result -> assertNull(result)).join();
- map.floorKey(keyOne).thenAccept(result -> assertNull(result)).join();
- map.ceilingKey(keyOne).thenAccept(result -> assertNull(result))
- .join();
- map.higherKey(keyOne).thenAccept(result -> assertNull(result)).join();
-
- // TODO: delete() is not supported
- //map.delete().join();
-
- allKeys.forEach(key -> map.put(
- key, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> assertNull(result)).join());
- //Note ordering keys are in their proper ordering in ascending order
- //both in naming and in the allKeys list.
-
- map.firstKey().thenAccept(result -> assertEquals(keyOne, result))
- .join();
- map.lastKey().thenAccept(result -> assertEquals(keyFour, result))
- .join();
- map.ceilingEntry(keyOne)
- .thenAccept(result -> {
- assertEquals(keyOne, result.getKey());
- assertArrayEquals(valueOne, result.getValue().value());
- })
- .join();
- //adding an additional letter to make keyOne an unacceptable response
- map.ceilingEntry(keyOne + "a")
- .thenAccept(result -> {
- assertEquals(keyTwo, result.getKey());
- assertArrayEquals(valueTwo, result.getValue().value());
- })
- .join();
- map.ceilingEntry(keyFour + "a")
- .thenAccept(result -> {
- assertNull(result);
- })
- .join();
- map.floorEntry(keyTwo).thenAccept(result -> {
- assertEquals(keyTwo, result.getKey());
- assertArrayEquals(valueTwo, result.getValue().value());
- })
- .join();
- //shorten the key so it itself is not an acceptable reply
- map.floorEntry(keyTwo.substring(0, 2)).thenAccept(result -> {
- assertEquals(keyOne, result.getKey());
- assertArrayEquals(valueOne, result.getValue().value());
- })
- .join();
- // shorten least key so no acceptable response exists
- map.floorEntry(keyOne.substring(0, 1)).thenAccept(
- result -> assertNull(result))
- .join();
-
- map.higherEntry(keyTwo).thenAccept(result -> {
- assertEquals(keyThree, result.getKey());
- assertArrayEquals(valueThree, result.getValue().value());
- })
- .join();
- map.higherEntry(keyFour).thenAccept(result -> assertNull(result))
- .join();
-
- map.lowerEntry(keyFour).thenAccept(result -> {
- assertEquals(keyThree, result.getKey());
- assertArrayEquals(valueThree, result.getValue().value());
- })
- .join();
- map.lowerEntry(keyOne).thenAccept(result -> assertNull(result))
- .join();
- map.firstEntry().thenAccept(result -> {
- assertEquals(keyOne, result.getKey());
- assertArrayEquals(valueOne, result.getValue().value());
- })
- .join();
- map.lastEntry().thenAccept(result -> {
- assertEquals(keyFour, result.getKey());
- assertArrayEquals(valueFour, result.getValue().value());
- })
- .join();
- map.pollFirstEntry().thenAccept(result -> {
- assertEquals(keyOne, result.getKey());
- assertArrayEquals(valueOne, result.getValue().value());
- });
- map.containsKey(keyOne).thenAccept(result -> assertFalse(result))
- .join();
- map.size().thenAccept(result -> assertEquals(3, (int) result)).join();
- map.pollLastEntry().thenAccept(result -> {
- assertEquals(keyFour, result.getKey());
- assertArrayEquals(valueFour, result.getValue().value());
- });
- map.containsKey(keyFour).thenAccept(result -> assertFalse(result))
- .join();
- map.size().thenAccept(result -> assertEquals(2, (int) result)).join();
-
- //repopulate the missing entries
- allKeys.forEach(key -> map.put(
- key, allValues.get(allKeys.indexOf(key)))
- .thenAccept(result -> {
- if (key.equals(keyOne) || key.equals(keyFour)) {
- assertNull(result);
- } else {
- assertArrayEquals(allValues.get(allKeys.indexOf(key)),
- result.value());
- }
- })
- .join());
- map.lowerKey(keyOne).thenAccept(result -> assertNull(result)).join();
- map.lowerKey(keyThree).thenAccept(
- result -> assertEquals(keyTwo, result))
- .join();
- map.floorKey(keyThree).thenAccept(
- result -> assertEquals(keyThree, result))
- .join();
- //shortening the key so there is no acceptable response
- map.floorKey(keyOne.substring(0, 1)).thenAccept(
- result -> assertNull(result))
- .join();
- map.ceilingKey(keyTwo).thenAccept(
- result -> assertEquals(keyTwo, result))
- .join();
- //adding to highest key so there is no acceptable response
- map.ceilingKey(keyFour + "a")
- .thenAccept(result -> assertNull(result))
- .join();
- map.higherKey(keyThree).thenAccept(
- result -> assertEquals(keyFour, result))
- .join();
- map.higherKey(keyFour).thenAccept(
- result -> assertNull(result))
- .join();
-
- // TODO: delete() is not supported
- //map.delete().join();
- }
-
- @Test
- public void testIterator() throws Exception {
- AtomixConsistentTreeMap map = newPrimitive("testIterator");
- for (int i = 0; i < 100; i++) {
- for (int j = 0; j < 100; j++) {
- map.put(String.valueOf(i), String.valueOf(j).getBytes()).join();
- }
- }
-
- List<Map.Entry<String, Versioned<byte[]>>> entries = new ArrayList<>();
- AsyncIterator<Map.Entry<String, Versioned<byte[]>>> iterator = map.iterator().get(5, TimeUnit.SECONDS);
- while (iterator.hasNext().get(5, TimeUnit.SECONDS)) {
- map.put("foo", UUID.randomUUID().toString().getBytes()).join();
- entries.add(iterator.next().get(5, TimeUnit.SECONDS));
- }
- assertEquals(100, entries.size());
- assertEquals(101, map.asConsistentMap().stream().count());
- }
-
- private AtomixConsistentTreeMap createResource(String mapName) {
- try {
- AtomixConsistentTreeMap map = newPrimitive(mapName);
- return map;
- } catch (Throwable e) {
- throw new IllegalStateException(e.toString());
- }
- }
- private static class TestMapEventListener
- implements MapEventListener<String, byte[]> {
-
- private final BlockingQueue<MapEvent<String, byte[]>> queue =
- new ArrayBlockingQueue<>(1);
-
- @Override
- public void event(MapEvent<String, byte[]> event) {
- try {
- queue.put(event);
- } catch (InterruptedException e) {
- Thread.interrupted();
- throw new IllegalStateException(e);
- }
- }
-
- public boolean eventReceived() {
- return !queue.isEmpty();
- }
-
- public MapEvent<String, byte[]> event() throws InterruptedException {
- return queue.take();
- }
- }
-
- /**
- * Returns two arrays contain the same set of elements,
- * regardless of order.
- * @param o1 first collection
- * @param o2 second collection
- * @return true if they contain the same elements
- */
- private boolean byteArrayCollectionIsEqual(
- Collection<? extends byte[]> o1, Collection<? extends byte[]> o2) {
- if (o1 == null || o2 == null || o1.size() != o2.size()) {
- return false;
- }
- for (byte[] array1 : o1) {
- boolean matched = false;
- for (byte[] array2 : o2) {
- if (Arrays.equals(array1, array2)) {
- matched = true;
- break;
- }
- }
- if (!matched) {
- return false;
- }
- }
- return true;
- }
-
- /**
- * Compares two collections of strings returns true if they contain the
- * same strings, false otherwise.
- * @param s1 string collection one
- * @param s2 string collection two
- * @return true if the two sets contain the same strings
- */
- private boolean stringArrayCollectionIsEqual(
- Collection<? extends String> s1, Collection<? extends String> s2) {
- if (s1 == null || s2 == null || s1.size() != s2.size()) {
- return false;
- }
- for (String string1 : s1) {
- boolean matched = false;
- for (String string2 : s2) {
- if (string1.equals(string2)) {
- matched = true;
- break;
- }
- }
- if (!matched) {
- return false;
- }
- }
- return true;
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixCounterServiceTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixCounterServiceTest.java
deleted file mode 100644
index 4dd4148..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixCounterServiceTest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.service.impl.DefaultCommit;
-import io.atomix.protocols.raft.session.impl.RaftSessionContext;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.protocols.raft.storage.snapshot.Snapshot;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotStore;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.storage.StorageLevel;
-import io.atomix.time.WallClockTimestamp;
-import org.junit.Test;
-
-import static org.easymock.EasyMock.mock;
-import static org.junit.Assert.assertEquals;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.SET;
-
-/**
- * Counter service test.
- */
-public class AtomixCounterServiceTest {
- @Test
- public void testSnapshot() throws Exception {
- SnapshotStore store = new SnapshotStore(RaftStorage.newBuilder()
- .withPrefix("test")
- .withStorageLevel(StorageLevel.MEMORY)
- .build());
- Snapshot snapshot = store.newSnapshot(2, new WallClockTimestamp());
-
- AtomixCounterService service = new AtomixCounterService();
- service.set(new DefaultCommit<>(
- 2,
- SET,
- new AtomixCounterOperations.Set(1L),
- mock(RaftSessionContext.class),
- System.currentTimeMillis()));
-
- try (SnapshotWriter writer = snapshot.openWriter()) {
- service.snapshot(writer);
- }
-
- snapshot.complete();
-
- service = new AtomixCounterService();
- try (SnapshotReader reader = snapshot.openReader()) {
- service.install(reader);
- }
-
- long value = service.get(new DefaultCommit<>(
- 2,
- GET,
- null,
- mock(RaftSessionContext.class),
- System.currentTimeMillis()));
- assertEquals(1, value);
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixCounterTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixCounterTest.java
deleted file mode 100644
index 05e07e0..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixCounterTest.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Unit tests for {@link AtomixCounter}.
- */
-public class AtomixCounterTest extends AtomixTestBase<AtomixCounter> {
- @Override
- protected RaftService createService() {
- return new AtomixCounterService();
- }
-
- @Override
- protected AtomixCounter createPrimitive(RaftProxy proxy) {
- return new AtomixCounter(proxy);
- }
-
- @Test
- public void testBasicOperations() throws Throwable {
- basicOperationsTest();
- }
-
- protected void basicOperationsTest() throws Throwable {
- AtomixCounter along = newPrimitive("test-counter-basic-operations");
- assertEquals(0, along.get().join().longValue());
- assertEquals(1, along.incrementAndGet().join().longValue());
- along.set(100).join();
- assertEquals(100, along.get().join().longValue());
- assertEquals(100, along.getAndAdd(10).join().longValue());
- assertEquals(110, along.get().join().longValue());
- assertFalse(along.compareAndSet(109, 111).join());
- assertTrue(along.compareAndSet(110, 111).join());
- assertEquals(100, along.addAndGet(-11).join().longValue());
- assertEquals(100, along.getAndIncrement().join().longValue());
- assertEquals(101, along.get().join().longValue());
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockServiceTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockServiceTest.java
deleted file mode 100644
index cf3e051..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockServiceTest.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.impl.RaftContext;
-import io.atomix.protocols.raft.impl.RaftServiceManager;
-import io.atomix.protocols.raft.operation.OperationType;
-import io.atomix.protocols.raft.protocol.RaftServerProtocol;
-import io.atomix.protocols.raft.service.ServiceId;
-import io.atomix.protocols.raft.service.ServiceType;
-import io.atomix.protocols.raft.service.impl.DefaultCommit;
-import io.atomix.protocols.raft.service.impl.DefaultServiceContext;
-import io.atomix.protocols.raft.session.SessionId;
-import io.atomix.protocols.raft.session.impl.RaftSessionContext;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.protocols.raft.storage.snapshot.Snapshot;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotStore;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.storage.StorageLevel;
-import io.atomix.time.WallClockTimestamp;
-import io.atomix.utils.concurrent.AtomixThreadFactory;
-import io.atomix.utils.concurrent.SingleThreadContextFactory;
-import io.atomix.utils.concurrent.ThreadContext;
-import org.junit.Test;
-
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.mock;
-import static org.easymock.EasyMock.replay;
-import static org.onosproject.store.service.DistributedPrimitive.Type.LEADER_ELECTOR;
-
-/**
- * Distributed lock service test.
- */
-public class AtomixDistributedLockServiceTest {
- @Test
- public void testSnapshot() throws Exception {
- SnapshotStore store = new SnapshotStore(RaftStorage.newBuilder()
- .withPrefix("test")
- .withStorageLevel(StorageLevel.MEMORY)
- .build());
- Snapshot snapshot = store.newSnapshot(2, new WallClockTimestamp());
-
- AtomicLong index = new AtomicLong();
- DefaultServiceContext context = mock(DefaultServiceContext.class);
- expect(context.serviceType()).andReturn(ServiceType.from(LEADER_ELECTOR.name())).anyTimes();
- expect(context.serviceName()).andReturn("test").anyTimes();
- expect(context.serviceId()).andReturn(ServiceId.from(1)).anyTimes();
- expect(context.currentIndex()).andReturn(index.get()).anyTimes();
- expect(context.currentOperation()).andReturn(OperationType.COMMAND).anyTimes();
- expect(context.locked()).andReturn(false).anyTimes();
-
- RaftContext server = mock(RaftContext.class);
- expect(server.getProtocol()).andReturn(mock(RaftServerProtocol.class)).anyTimes();
- RaftServiceManager manager = mock(RaftServiceManager.class);
- expect(manager.executor()).andReturn(mock(ThreadContext.class)).anyTimes();
- expect(server.getServiceManager()).andReturn(manager).anyTimes();
-
- replay(context, server);
-
- AtomixDistributedLockService service = new AtomixDistributedLockService();
- service.init(context);
-
- RaftSessionContext session = new RaftSessionContext(
- SessionId.from(1),
- MemberId.from("1"),
- "test",
- ServiceType.from(LEADER_ELECTOR.name()),
- ReadConsistency.LINEARIZABLE,
- 100,
- 5000,
- System.currentTimeMillis(),
- context,
- server,
- new SingleThreadContextFactory(new AtomixThreadFactory()));
- session.open();
-
- service.lock(new DefaultCommit<>(
- index.incrementAndGet(),
- AtomixDistributedLockOperations.LOCK,
- new AtomixDistributedLockOperations.Lock(1, 0),
- session,
- System.currentTimeMillis()));
-
- try (SnapshotWriter writer = snapshot.openWriter()) {
- service.snapshot(writer);
- }
-
- snapshot.complete();
-
- service = new AtomixDistributedLockService();
- try (SnapshotReader reader = snapshot.openReader()) {
- service.install(reader);
- }
-
- service.unlock(new DefaultCommit<>(
- index.incrementAndGet(),
- AtomixDistributedLockOperations.UNLOCK,
- new AtomixDistributedLockOperations.Unlock(1),
- session,
- System.currentTimeMillis()));
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockTest.java
deleted file mode 100644
index cc32ba9..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockTest.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.time.Duration;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onosproject.store.service.Version;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Raft lock test.
- */
-public class AtomixDistributedLockTest extends AtomixTestBase<AtomixDistributedLock> {
- @Override
- protected RaftService createService() {
- return new AtomixDistributedLockService();
- }
-
- @Override
- protected AtomixDistributedLock createPrimitive(RaftProxy proxy) {
- return new AtomixDistributedLock(proxy);
- }
-
- /**
- * Tests locking and unlocking a lock.
- */
- @Test
- public void testLockUnlock() throws Throwable {
- AtomixDistributedLock lock = newPrimitive("test-lock-unlock");
- lock.lock().join();
- lock.unlock().join();
- }
-
- /**
- * Tests releasing a lock when the client's session is closed.
- */
- @Test
- public void testReleaseOnClose() throws Throwable {
- AtomixDistributedLock lock1 = newPrimitive("test-lock-release-on-close");
- AtomixDistributedLock lock2 = newPrimitive("test-lock-release-on-close");
- lock1.lock().join();
- CompletableFuture<Version> future = lock2.lock();
- lock1.close();
- future.join();
- }
-
- /**
- * Tests attempting to acquire a lock.
- */
- @Test
- public void testTryLockFail() throws Throwable {
- AtomixDistributedLock lock1 = newPrimitive("test-try-lock-fail");
- AtomixDistributedLock lock2 = newPrimitive("test-try-lock-fail");
-
- lock1.lock().join();
-
- assertFalse(lock2.tryLock().join().isPresent());
- }
-
- /**
- * Tests attempting to acquire a lock.
- */
- @Test
- public void testTryLockSucceed() throws Throwable {
- AtomixDistributedLock lock = newPrimitive("test-try-lock-succeed");
- assertTrue(lock.tryLock().join().isPresent());
- }
-
- /**
- * Tests attempting to acquire a lock with a timeout.
- */
- @Test
- public void testTryLockFailWithTimeout() throws Throwable {
- AtomixDistributedLock lock1 = newPrimitive("test-try-lock-fail-with-timeout");
- AtomixDistributedLock lock2 = newPrimitive("test-try-lock-fail-with-timeout");
-
- lock1.lock().join();
-
- assertFalse(lock2.tryLock(Duration.ofSeconds(1)).join().isPresent());
- }
-
- /**
- * Tests attempting to acquire a lock with a timeout.
- */
- @Test
- public void testTryLockSucceedWithTimeout() throws Throwable {
- AtomixDistributedLock lock1 = newPrimitive("test-try-lock-succeed-with-timeout");
- AtomixDistributedLock lock2 = newPrimitive("test-try-lock-succeed-with-timeout");
-
- lock1.lock().join();
-
- CompletableFuture<Optional<Version>> future = lock2.tryLock(Duration.ofSeconds(1));
- lock1.unlock().join();
- assertTrue(future.join().isPresent());
- }
-
- /**
- * Tests unlocking a lock with a blocking call in the event thread.
- */
- @Test
- public void testBlockingUnlock() throws Throwable {
- AtomixDistributedLock lock1 = newPrimitive("test-blocking-unlock");
- AtomixDistributedLock lock2 = newPrimitive("test-blocking-unlock");
-
- lock1.lock().thenRun(() -> {
- lock1.unlock().join();
- }).join();
-
- lock2.lock().join();
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeServiceTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeServiceTest.java
deleted file mode 100644
index 21c5621..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeServiceTest.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Optional;
-
-import io.atomix.protocols.raft.service.impl.DefaultCommit;
-import io.atomix.protocols.raft.session.impl.RaftSessionContext;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.protocols.raft.storage.snapshot.Snapshot;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotStore;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.storage.StorageLevel;
-import io.atomix.time.WallClockTimestamp;
-import org.junit.Test;
-import org.onlab.util.Match;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.Versioned;
-
-import static org.easymock.EasyMock.mock;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.UPDATE;
-
-/**
- * Document tree service test.
- */
-public class AtomixDocumentTreeServiceTest {
-
- @Test
- public void testNaturalOrderedSnapshot() throws Exception {
- testSnapshot(Ordering.NATURAL);
- }
-
- @Test
- public void testInsertionOrderedSnapshot() throws Exception {
- testSnapshot(Ordering.INSERTION);
- }
-
- private void testSnapshot(Ordering ordering) throws Exception {
- SnapshotStore store = new SnapshotStore(RaftStorage.newBuilder()
- .withPrefix("test")
- .withStorageLevel(StorageLevel.MEMORY)
- .build());
- Snapshot snapshot = store.newSnapshot(2, new WallClockTimestamp());
-
- AtomixDocumentTreeService service = new AtomixDocumentTreeService(ordering);
- service.update(new DefaultCommit<>(
- 2,
- UPDATE,
- new AtomixDocumentTreeOperations.Update(
- DocumentPath.from("root|foo"),
- Optional.of("Hello world!".getBytes()),
- Match.any(),
- Match.ifNull()),
- mock(RaftSessionContext.class),
- System.currentTimeMillis()));
-
- try (SnapshotWriter writer = snapshot.openWriter()) {
- service.snapshot(writer);
- }
-
- snapshot.complete();
-
- service = new AtomixDocumentTreeService(ordering);
- try (SnapshotReader reader = snapshot.openReader()) {
- service.install(reader);
- }
-
- Versioned<byte[]> value = service.get(new DefaultCommit<>(
- 2,
- GET,
- new AtomixDocumentTreeOperations.Get(DocumentPath.from("root|foo")),
- mock(RaftSessionContext.class),
- System.currentTimeMillis()));
- assertNotNull(value);
- assertArrayEquals("Hello world!".getBytes(), value.value());
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeTest.java
deleted file mode 100644
index 6abe84f..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeTest.java
+++ /dev/null
@@ -1,497 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import com.google.common.base.Throwables;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.IllegalDocumentModificationException;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Unit tests for {@link AtomixDocumentTree}.
- */
-public class AtomixDocumentTreeTest extends AtomixTestBase<AtomixDocumentTree> {
- private Ordering ordering = Ordering.NATURAL;
-
- @Override
- protected RaftService createService() {
- return new AtomixDocumentTreeService(ordering);
- }
-
- @Override
- protected AtomixDocumentTree createPrimitive(RaftProxy proxy) {
- return new AtomixDocumentTree(proxy);
- }
-
- @Override
- protected AtomixDocumentTree newPrimitive(String name) {
- return newPrimitive(name, Ordering.NATURAL);
- }
-
- protected AtomixDocumentTree newPrimitive(String name, Ordering ordering) {
- this.ordering = ordering;
- return super.newPrimitive(name);
- }
-
- /**
- * Tests queries (get and getChildren).
- */
- @Test
- public void testQueries() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- Versioned<byte[]> root = tree.get(path("root")).join();
- assertEquals(1, root.version());
- assertNull(root.value());
- }
-
- /**
- * Tests create.
- */
- @Test
- public void testCreate() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.create(path("root.a"), "a".getBytes()).join();
- tree.create(path("root.a.b"), "ab".getBytes()).join();
- tree.create(path("root.a.c"), "ac".getBytes()).join();
- Versioned<byte[]> a = tree.get(path("root.a")).join();
- assertArrayEquals("a".getBytes(), a.value());
-
- Versioned<byte[]> ab = tree.get(path("root.a.b")).join();
- assertArrayEquals("ab".getBytes(), ab.value());
-
- Versioned<byte[]> ac = tree.get(path("root.a.c")).join();
- assertArrayEquals("ac".getBytes(), ac.value());
-
- tree.create(path("root.x"), null).join();
- Versioned<byte[]> x = tree.get(path("root.x")).join();
- assertNull(x.value());
- }
-
- /**
- * Tests recursive create.
- */
- @Test
- public void testRecursiveCreate() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.createRecursive(path("root.a.b.c"), "abc".getBytes()).join();
- Versioned<byte[]> a = tree.get(path("root.a")).join();
- assertArrayEquals(null, a.value());
-
- Versioned<byte[]> ab = tree.get(path("root.a.b")).join();
- assertArrayEquals(null, ab.value());
-
- Versioned<byte[]> abc = tree.get(path("root.a.b.c")).join();
- assertArrayEquals("abc".getBytes(), abc.value());
- }
-
- /**
- * Tests child node order.
- */
- @Test
- public void testOrder() throws Throwable {
- AtomixDocumentTree naturalTree = newPrimitive(UUID.randomUUID().toString(), Ordering.NATURAL);
- naturalTree.create(path("root.c"), "foo".getBytes());
- naturalTree.create(path("root.b"), "bar".getBytes());
- naturalTree.create(path("root.a"), "baz".getBytes());
-
- Iterator<Map.Entry<String, Versioned<byte[]>>> naturalIterator = naturalTree.getChildren(path("root"))
- .join().entrySet().iterator();
- assertEquals("a", naturalIterator.next().getKey());
- assertEquals("b", naturalIterator.next().getKey());
- assertEquals("c", naturalIterator.next().getKey());
-
- AtomixDocumentTree insertionTree = newPrimitive(UUID.randomUUID().toString(), Ordering.INSERTION);
- insertionTree.create(path("root.c"), "foo".getBytes());
- insertionTree.create(path("root.b"), "bar".getBytes());
- insertionTree.create(path("root.a"), "baz".getBytes());
-
- Iterator<Map.Entry<String, Versioned<byte[]>>> insertionIterator = insertionTree.getChildren(path("root"))
- .join().entrySet().iterator();
- assertEquals("c", insertionIterator.next().getKey());
- assertEquals("b", insertionIterator.next().getKey());
- assertEquals("a", insertionIterator.next().getKey());
- }
-
- /**
- * Tests set.
- */
- @Test
- public void testSet() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.create(path("root.a"), "a".getBytes()).join();
- tree.create(path("root.a.b"), "ab".getBytes()).join();
- tree.create(path("root.a.c"), "ac".getBytes()).join();
-
- tree.set(path("root.a.d"), "ad".getBytes()).join();
- Versioned<byte[]> ad = tree.get(path("root.a.d")).join();
- assertArrayEquals("ad".getBytes(), ad.value());
-
- tree.set(path("root.a"), "newA".getBytes()).join();
- Versioned<byte[]> newA = tree.get(path("root.a")).join();
- assertArrayEquals("newA".getBytes(), newA.value());
-
- tree.set(path("root.a.b"), "newAB".getBytes()).join();
- Versioned<byte[]> newAB = tree.get(path("root.a.b")).join();
- assertArrayEquals("newAB".getBytes(), newAB.value());
-
- tree.set(path("root.x"), null).join();
- Versioned<byte[]> x = tree.get(path("root.x")).join();
- assertNull(x.value());
- }
-
- /**
- * Tests replace if version matches.
- */
- @Test
- public void testReplaceVersion() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.create(path("root.a"), "a".getBytes()).join();
- tree.create(path("root.a.b"), "ab".getBytes()).join();
- tree.create(path("root.a.c"), "ac".getBytes()).join();
-
- Versioned<byte[]> ab = tree.get(path("root.a.b")).join();
- assertTrue(tree.replace(path("root.a.b"), "newAB".getBytes(), ab.version()).join());
- Versioned<byte[]> newAB = tree.get(path("root.a.b")).join();
- assertArrayEquals("newAB".getBytes(), newAB.value());
-
- assertFalse(tree.replace(path("root.a.b"), "newestAB".getBytes(), ab.version()).join());
- assertArrayEquals("newAB".getBytes(), tree.get(path("root.a.b")).join().value());
-
- assertFalse(tree.replace(path("root.a.d"), "foo".getBytes(), 1).join());
- }
-
- /**
- * Tests replace if value matches.
- */
- @Test
- public void testReplaceValue() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.create(path("root.a"), "a".getBytes()).join();
- tree.create(path("root.a.b"), "ab".getBytes()).join();
- tree.create(path("root.a.c"), "ac".getBytes()).join();
-
- Versioned<byte[]> ab = tree.get(path("root.a.b")).join();
- assertTrue(tree.replace(path("root.a.b"), "newAB".getBytes(), ab.value()).join());
- Versioned<byte[]> newAB = tree.get(path("root.a.b")).join();
- assertArrayEquals("newAB".getBytes(), newAB.value());
-
- assertFalse(tree.replace(path("root.a.b"), "newestAB".getBytes(), ab.value()).join());
- assertArrayEquals("newAB".getBytes(), tree.get(path("root.a.b")).join().value());
-
- assertFalse(tree.replace(path("root.a.d"), "bar".getBytes(), "foo".getBytes()).join());
-
- assertTrue(tree.replace(path("root.x"), "beta".getBytes(), null).join());
-
- }
-
- /**
- * Tests remove.
- */
- @Test
- public void testRemove() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.create(path("root.a"), "a".getBytes()).join();
- tree.create(path("root.a.b"), "ab".getBytes()).join();
- tree.create(path("root.a.c"), "ac".getBytes()).join();
-
- Versioned<byte[]> ab = tree.removeNode(path("root.a.b")).join();
- assertArrayEquals("ab".getBytes(), ab.value());
- assertNull(tree.get(path("root.a.b")).join());
-
- Versioned<byte[]> ac = tree.removeNode(path("root.a.c")).join();
- assertArrayEquals("ac".getBytes(), ac.value());
- assertNull(tree.get(path("root.a.c")).join());
-
- Versioned<byte[]> a = tree.removeNode(path("root.a")).join();
- assertArrayEquals("a".getBytes(), a.value());
- assertNull(tree.get(path("root.a")).join());
-
- tree.create(path("root.x"), null).join();
- Versioned<byte[]> x = tree.removeNode(path("root.x")).join();
- assertNull(x.value());
- assertNull(tree.get(path("root.a.x")).join());
- }
-
- /**
- * Tests invalid removes.
- */
- @Test
- public void testRemoveFailures() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.create(path("root.a"), "a".getBytes()).join();
- tree.create(path("root.a.b"), "ab".getBytes()).join();
- tree.create(path("root.a.c"), "ac".getBytes()).join();
-
- try {
- tree.removeNode(path("root")).join();
- fail();
- } catch (Exception e) {
- assertTrue(Throwables.getRootCause(e) instanceof IllegalDocumentModificationException);
- }
-
- try {
- tree.removeNode(path("root.a")).join();
- fail();
- } catch (Exception e) {
- assertTrue(Throwables.getRootCause(e) instanceof IllegalDocumentModificationException);
- }
-
- try {
- tree.removeNode(path("root.d")).join();
- fail();
- } catch (Exception e) {
- assertTrue(Throwables.getRootCause(e) instanceof NoSuchDocumentPathException);
- }
- }
-
- /**
- * Tests invalid create.
- */
- @Test
- public void testCreateFailures() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- try {
- tree.create(path("root.a.c"), "ac".getBytes()).join();
- fail();
- } catch (Exception e) {
- assertTrue(Throwables.getRootCause(e) instanceof IllegalDocumentModificationException);
- }
- }
-
- /**
- * Tests invalid set.
- */
- @Test
- public void testSetFailures() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- try {
- tree.set(path("root.a.c"), "ac".getBytes()).join();
- fail();
- } catch (Exception e) {
- assertTrue(Throwables.getRootCause(e) instanceof IllegalDocumentModificationException);
- }
- }
-
- /**
- * Tests getChildren.
- */
- @Test
- public void testGetChildren() throws Throwable {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.create(path("root.a"), "a".getBytes()).join();
- tree.create(path("root.a.b"), "ab".getBytes()).join();
- tree.create(path("root.a.c"), "ac".getBytes()).join();
-
- Map<String, Versioned<byte[]>> rootChildren = tree.getChildren(path("root")).join();
- assertEquals(1, rootChildren.size());
- Versioned<byte[]> a = rootChildren.get("a");
- assertArrayEquals("a".getBytes(), a.value());
-
- Map<String, Versioned<byte[]>> children = tree.getChildren(path("root.a")).join();
- assertEquals(2, children.size());
- Versioned<byte[]> ab = children.get("b");
- assertArrayEquals("ab".getBytes(), ab.value());
- Versioned<byte[]> ac = children.get("c");
- assertArrayEquals("ac".getBytes(), ac.value());
-
- assertEquals(0, tree.getChildren(path("root.a.b")).join().size());
- assertEquals(0, tree.getChildren(path("root.a.c")).join().size());
- }
-
- /**
- * Tests destroy.
- */
- @Test
- public void testClear() {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- tree.create(path("root.a"), "a".getBytes()).join();
- tree.create(path("root.a.b"), "ab".getBytes()).join();
- tree.create(path("root.a.c"), "ac".getBytes()).join();
-
- tree.destroy().join();
- assertEquals(0, tree.getChildren(path("root")).join().size());
- }
-
- /**
- * Tests listeners.
- */
- @Test
- public void testNotifications() throws Exception {
- AtomixDocumentTree tree = newPrimitive(UUID.randomUUID().toString());
- TestEventListener listener = new TestEventListener();
-
- // add listener; create a node in the tree and verify an CREATED event is received.
- tree.addListener(listener).thenCompose(v -> tree.set(path("root.a"), "a".getBytes())).join();
- DocumentTreeEvent<byte[]> event = listener.event();
- assertEquals(DocumentTreeEvent.Type.CREATED, event.type());
- assertFalse(event.oldValue().isPresent());
- assertArrayEquals("a".getBytes(), event.newValue().get().value());
- // update a node in the tree and verify an UPDATED event is received.
- tree.set(path("root.a"), "newA".getBytes()).join();
- event = listener.event();
- assertEquals(DocumentTreeEvent.Type.UPDATED, event.type());
- assertArrayEquals("newA".getBytes(), event.newValue().get().value());
- assertArrayEquals("a".getBytes(), event.oldValue().get().value());
- // remove a node in the tree and verify an REMOVED event is received.
- tree.removeNode(path("root.a")).join();
- event = listener.event();
- assertEquals(DocumentTreeEvent.Type.DELETED, event.type());
- assertFalse(event.newValue().isPresent());
- assertArrayEquals("newA".getBytes(), event.oldValue().get().value());
- // recursively create a node and verify CREATED events for all intermediate nodes.
- tree.createRecursive(path("root.x.y"), "xy".getBytes()).join();
- event = listener.event();
- assertEquals(DocumentTreeEvent.Type.CREATED, event.type());
- assertEquals(path("root.x"), event.path());
- event = listener.event();
- assertEquals(DocumentTreeEvent.Type.CREATED, event.type());
- assertEquals(path("root.x.y"), event.path());
- assertArrayEquals("xy".getBytes(), event.newValue().get().value());
- }
-
- @Test
- public void testFilteredNotifications() throws Throwable {
- String treeName = UUID.randomUUID().toString();
- AtomixDocumentTree tree1 = newPrimitive(treeName);
- AtomixDocumentTree tree2 = newPrimitive(treeName);
-
- TestEventListener listener1a = new TestEventListener(3);
- TestEventListener listener1ab = new TestEventListener(2);
- TestEventListener listener2abc = new TestEventListener(1);
-
- tree1.addListener(path("root.a"), listener1a).join();
- tree1.addListener(path("root.a.b"), listener1ab).join();
- tree2.addListener(path("root.a.b.c"), listener2abc).join();
-
- tree1.createRecursive(path("root.a.b.c"), "abc".getBytes()).join();
- DocumentTreeEvent<byte[]> event = listener1a.event();
- assertEquals(path("root.a"), event.path());
- event = listener1a.event();
- assertEquals(path("root.a.b"), event.path());
- event = listener1a.event();
- assertEquals(path("root.a.b.c"), event.path());
- event = listener1ab.event();
- assertEquals(path("root.a.b"), event.path());
- event = listener1ab.event();
- assertEquals(path("root.a.b.c"), event.path());
- event = listener2abc.event();
- assertEquals(path("root.a.b.c"), event.path());
- }
-
- @Test
- public void testTransaction() throws Throwable {
- String treeName = UUID.randomUUID().toString();
- AtomixDocumentTree tree = newPrimitive(treeName);
-
- byte[] value1 = "abc".getBytes();
- byte[] value2 = "def".getBytes();
-
- assertTrue(tree.create(path("root.a"), value1).join());
- assertTrue(tree.create(path("root.b"), value2).join());
-
- long aVersion = tree.get(path("root.a")).join().version();
- long bVersion = tree.get(path("root.b")).join().version();
-
- TransactionId transactionId = TransactionId.from("1");
- Version transactionVersion = tree.begin(transactionId).join();
- List<NodeUpdate<byte[]>> records = Arrays.asList(
- NodeUpdate.<byte[]>newBuilder()
- .withType(NodeUpdate.Type.CREATE_NODE)
- .withPath(path("root.c"))
- .withValue(value1)
- .build(),
- NodeUpdate.<byte[]>newBuilder()
- .withType(NodeUpdate.Type.UPDATE_NODE)
- .withPath(path("root.a"))
- .withValue(value2)
- .withVersion(aVersion)
- .build(),
- NodeUpdate.<byte[]>newBuilder()
- .withType(NodeUpdate.Type.DELETE_NODE)
- .withPath(path("root.b"))
- .withVersion(bVersion)
- .build());
- TransactionLog<NodeUpdate<byte[]>> transactionLog = new TransactionLog<>(
- transactionId,
- transactionVersion.value(),
- records);
- assertTrue(tree.prepare(transactionLog).join());
- tree.commit(transactionId).join();
-
- assertArrayEquals(value2, tree.get(path("root.a")).join().value());
- assertNull(tree.get(path("root.b")).join());
- assertArrayEquals(value1, tree.get(path("root.c")).join().value());
- }
-
- private static class TestEventListener implements DocumentTreeListener<byte[]> {
-
- private final BlockingQueue<DocumentTreeEvent<byte[]>> queue;
-
- public TestEventListener() {
- this(1);
- }
-
- public TestEventListener(int maxEvents) {
- queue = new ArrayBlockingQueue<>(maxEvents);
- }
-
- @Override
- public void event(DocumentTreeEvent<byte[]> event) {
-
- try {
- queue.put(event);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new IllegalStateException(e);
- }
- }
-
- public DocumentTreeEvent<byte[]> event() throws InterruptedException {
- return queue.take();
- }
- }
-
- private static DocumentPath path(String path) {
- return DocumentPath.from(path.replace(".", DocumentPath.DEFAULT_SEPARATOR));
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixIdGeneratorTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixIdGeneratorTest.java
deleted file mode 100644
index 24f9065..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixIdGeneratorTest.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.concurrent.CompletableFuture;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Unit test for {@code AtomixIdGenerator}.
- */
-public class AtomixIdGeneratorTest extends AtomixTestBase<AtomixCounter> {
-
- @Override
- protected RaftService createService() {
- return new AtomixCounterService();
- }
-
- @Override
- protected AtomixCounter createPrimitive(RaftProxy proxy) {
- return new AtomixCounter(proxy);
- }
-
- /**
- * Tests generating IDs.
- */
- @Test
- public void testNextId() throws Throwable {
- AtomixIdGenerator idGenerator1 = new AtomixIdGenerator(newPrimitive("testNextId"));
- AtomixIdGenerator idGenerator2 = new AtomixIdGenerator(newPrimitive("testNextId"));
-
- CompletableFuture<Long> future11 = idGenerator1.nextId();
- CompletableFuture<Long> future12 = idGenerator1.nextId();
- CompletableFuture<Long> future13 = idGenerator1.nextId();
- assertEquals(Long.valueOf(1), future11.join());
- assertEquals(Long.valueOf(2), future12.join());
- assertEquals(Long.valueOf(3), future13.join());
-
- CompletableFuture<Long> future21 = idGenerator1.nextId();
- CompletableFuture<Long> future22 = idGenerator1.nextId();
- CompletableFuture<Long> future23 = idGenerator1.nextId();
- assertEquals(Long.valueOf(6), future23.join());
- assertEquals(Long.valueOf(5), future22.join());
- assertEquals(Long.valueOf(4), future21.join());
-
- CompletableFuture<Long> future31 = idGenerator2.nextId();
- CompletableFuture<Long> future32 = idGenerator2.nextId();
- CompletableFuture<Long> future33 = idGenerator2.nextId();
- assertEquals(Long.valueOf(1001), future31.join());
- assertEquals(Long.valueOf(1002), future32.join());
- assertEquals(Long.valueOf(1003), future33.join());
- }
-
- /**
- * Tests generating IDs.
- */
- @Test
- public void testNextIdBatchRollover() throws Throwable {
- AtomixIdGenerator idGenerator1 = new AtomixIdGenerator(newPrimitive("testNextIdBatchRollover"), 2);
- AtomixIdGenerator idGenerator2 = new AtomixIdGenerator(newPrimitive("testNextIdBatchRollover"), 2);
-
- CompletableFuture<Long> future11 = idGenerator1.nextId();
- CompletableFuture<Long> future12 = idGenerator1.nextId();
- CompletableFuture<Long> future13 = idGenerator1.nextId();
- assertEquals(Long.valueOf(1), future11.join());
- assertEquals(Long.valueOf(2), future12.join());
- assertEquals(Long.valueOf(3), future13.join());
-
- CompletableFuture<Long> future21 = idGenerator2.nextId();
- CompletableFuture<Long> future22 = idGenerator2.nextId();
- CompletableFuture<Long> future23 = idGenerator2.nextId();
- assertEquals(Long.valueOf(5), future21.join());
- assertEquals(Long.valueOf(6), future22.join());
- assertEquals(Long.valueOf(7), future23.join());
-
- CompletableFuture<Long> future14 = idGenerator1.nextId();
- CompletableFuture<Long> future15 = idGenerator1.nextId();
- CompletableFuture<Long> future16 = idGenerator1.nextId();
- assertEquals(Long.valueOf(4), future14.join());
- assertEquals(Long.valueOf(9), future15.join());
- assertEquals(Long.valueOf(10), future16.join());
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorServiceTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorServiceTest.java
deleted file mode 100644
index 4f7aa72..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorServiceTest.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.impl.RaftContext;
-import io.atomix.protocols.raft.protocol.RaftServerProtocol;
-import io.atomix.protocols.raft.service.ServiceId;
-import io.atomix.protocols.raft.service.ServiceType;
-import io.atomix.protocols.raft.service.impl.DefaultCommit;
-import io.atomix.protocols.raft.service.impl.DefaultServiceContext;
-import io.atomix.protocols.raft.session.SessionId;
-import io.atomix.protocols.raft.session.impl.RaftSessionContext;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.protocols.raft.storage.snapshot.Snapshot;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotStore;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.storage.StorageLevel;
-import io.atomix.time.WallClockTimestamp;
-import io.atomix.utils.concurrent.AtomixThreadFactory;
-import io.atomix.utils.concurrent.SingleThreadContextFactory;
-import org.junit.Test;
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.mock;
-import static org.easymock.EasyMock.replay;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_LEADERSHIP;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.RUN;
-import static org.onosproject.store.service.DistributedPrimitive.Type.LEADER_ELECTOR;
-
-/**
- * Leader elector service test.
- */
-public class AtomixLeaderElectorServiceTest {
- @Test
- public void testSnapshot() throws Exception {
- SnapshotStore store = new SnapshotStore(RaftStorage.newBuilder()
- .withPrefix("test")
- .withStorageLevel(StorageLevel.MEMORY)
- .build());
- Snapshot snapshot = store.newSnapshot(2, new WallClockTimestamp());
-
- DefaultServiceContext context = mock(DefaultServiceContext.class);
- expect(context.serviceType()).andReturn(ServiceType.from(LEADER_ELECTOR.name())).anyTimes();
- expect(context.serviceName()).andReturn("test").anyTimes();
- expect(context.serviceId()).andReturn(ServiceId.from(1)).anyTimes();
-
- RaftContext server = mock(RaftContext.class);
- expect(server.getProtocol()).andReturn(mock(RaftServerProtocol.class));
-
- replay(context, server);
-
- AtomixLeaderElectorService service = new AtomixLeaderElectorService();
- service.init(context);
-
- NodeId nodeId = NodeId.nodeId("1");
- service.run(new DefaultCommit<>(
- 2,
- RUN,
- new AtomixLeaderElectorOperations.Run("test", nodeId),
- new RaftSessionContext(
- SessionId.from(1),
- MemberId.from("1"),
- "test",
- ServiceType.from(LEADER_ELECTOR.name()),
- ReadConsistency.LINEARIZABLE,
- 100,
- 5000,
- System.currentTimeMillis(),
- context,
- server,
- new SingleThreadContextFactory(new AtomixThreadFactory())),
- System.currentTimeMillis()));
-
- try (SnapshotWriter writer = snapshot.openWriter()) {
- service.snapshot(writer);
- }
-
- snapshot.complete();
-
- service = new AtomixLeaderElectorService();
- service.init(context);
-
- try (SnapshotReader reader = snapshot.openReader()) {
- service.install(reader);
- }
-
- Leadership value = service.getLeadership(new DefaultCommit<>(
- 2,
- GET_LEADERSHIP,
- new AtomixLeaderElectorOperations.GetLeadership("test"),
- mock(RaftSessionContext.class),
- System.currentTimeMillis()));
- assertNotNull(value);
- assertEquals(value.leader().nodeId(), nodeId);
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorTest.java
deleted file mode 100644
index b0ec22d..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorTest.java
+++ /dev/null
@@ -1,359 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.LinkedList;
-import java.util.Queue;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Unit tests for {@link AtomixLeaderElector}.
- */
-public class AtomixLeaderElectorTest extends AtomixTestBase<AtomixLeaderElector> {
-
- NodeId node1 = new NodeId("node1");
- NodeId node2 = new NodeId("node2");
- NodeId node3 = new NodeId("node3");
-
- @Override
- protected RaftService createService() {
- return new AtomixLeaderElectorService();
- }
-
- @Override
- protected AtomixLeaderElector createPrimitive(RaftProxy proxy) {
- return new AtomixLeaderElector(proxy);
- }
-
- @Test
- public void testRun() throws Throwable {
- leaderElectorRunTests();
- }
-
- private void leaderElectorRunTests() throws Throwable {
- AtomixLeaderElector elector1 = newPrimitive("test-elector-run");
- elector1.run("foo", node1).thenAccept(result -> {
- assertEquals(node1, result.leaderNodeId());
- assertEquals(1, result.leader().term());
- assertEquals(1, result.candidates().size());
- assertEquals(node1, result.candidates().get(0));
- }).join();
-
- AtomixLeaderElector elector2 = newPrimitive("test-elector-run");
- elector2.run("foo", node2).thenAccept(result -> {
- assertEquals(node1, result.leaderNodeId());
- assertEquals(1, result.leader().term());
- assertEquals(2, result.candidates().size());
- assertEquals(node1, result.candidates().get(0));
- assertEquals(node2, result.candidates().get(1));
- }).join();
- }
-
- @Test
- public void testWithdraw() throws Throwable {
- leaderElectorWithdrawTests();
- }
-
- private void leaderElectorWithdrawTests() throws Throwable {
- AtomixLeaderElector elector1 = newPrimitive("test-elector-withdraw");
- elector1.run("foo", node1).join();
- AtomixLeaderElector elector2 = newPrimitive("test-elector-withdraw");
- elector2.run("foo", node2).join();
-
- LeaderEventListener listener1 = new LeaderEventListener();
- elector1.addChangeListener(listener1).join();
-
- LeaderEventListener listener2 = new LeaderEventListener();
- elector2.addChangeListener(listener2).join();
-
- elector1.withdraw("foo").join();
-
- listener1.nextEvent().thenAccept(result -> {
- assertEquals(node2, result.newValue().leaderNodeId());
- assertEquals(2, result.newValue().leader().term());
- assertEquals(1, result.newValue().candidates().size());
- assertEquals(node2, result.newValue().candidates().get(0));
- }).join();
-
- listener2.nextEvent().thenAccept(result -> {
- assertEquals(node2, result.newValue().leaderNodeId());
- assertEquals(2, result.newValue().leader().term());
- assertEquals(1, result.newValue().candidates().size());
- assertEquals(node2, result.newValue().candidates().get(0));
- }).join();
-
- Leadership leadership1 = elector1.getLeadership("foo").join();
- assertEquals(node2, leadership1.leader().nodeId());
- assertEquals(1, leadership1.candidates().size());
-
- Leadership leadership2 = elector2.getLeadership("foo").join();
- assertEquals(node2, leadership2.leader().nodeId());
- assertEquals(1, leadership2.candidates().size());
- }
-
- @Test
- public void testAnoint() throws Throwable {
- leaderElectorAnointTests();
- }
-
- private void leaderElectorAnointTests() throws Throwable {
- AtomixLeaderElector elector1 = newPrimitive("test-elector-anoint");
- AtomixLeaderElector elector2 = newPrimitive("test-elector-anoint");
- AtomixLeaderElector elector3 = newPrimitive("test-elector-anoint");
- elector1.run("foo", node1).join();
- elector2.run("foo", node2).join();
-
- LeaderEventListener listener1 = new LeaderEventListener();
- elector1.addChangeListener(listener1).join();
- LeaderEventListener listener2 = new LeaderEventListener();
- elector2.addChangeListener(listener2);
- LeaderEventListener listener3 = new LeaderEventListener();
- elector3.addChangeListener(listener3).join();
-
- elector3.anoint("foo", node3).thenAccept(result -> {
- assertFalse(result);
- }).join();
- assertFalse(listener1.hasEvent());
- assertFalse(listener2.hasEvent());
- assertFalse(listener3.hasEvent());
-
- elector3.anoint("foo", node2).thenAccept(result -> {
- assertTrue(result);
- }).join();
-
- listener1.nextEvent().thenAccept(result -> {
- assertEquals(node2, result.newValue().leaderNodeId());
- assertEquals(2, result.newValue().candidates().size());
- assertEquals(node1, result.newValue().candidates().get(0));
- assertEquals(node2, result.newValue().candidates().get(1));
- }).join();
- listener2.nextEvent().thenAccept(result -> {
- assertEquals(node2, result.newValue().leaderNodeId());
- assertEquals(2, result.newValue().candidates().size());
- assertEquals(node1, result.newValue().candidates().get(0));
- assertEquals(node2, result.newValue().candidates().get(1));
- }).join();
- listener3.nextEvent().thenAccept(result -> {
- assertEquals(node2, result.newValue().leaderNodeId());
- assertEquals(2, result.newValue().candidates().size());
- assertEquals(node1, result.newValue().candidates().get(0));
- assertEquals(node2, result.newValue().candidates().get(1));
- }).join();
- }
-
- @Test
- public void testPromote() throws Throwable {
- leaderElectorPromoteTests();
- }
-
- private void leaderElectorPromoteTests() throws Throwable {
- AtomixLeaderElector elector1 = newPrimitive("test-elector-promote");
- AtomixLeaderElector elector2 = newPrimitive("test-elector-promote");
- AtomixLeaderElector elector3 = newPrimitive("test-elector-promote");
- elector1.run("foo", node1).join();
- elector2.run("foo", node2).join();
-
- LeaderEventListener listener1 = new LeaderEventListener();
- elector1.addChangeListener(listener1).join();
- LeaderEventListener listener2 = new LeaderEventListener();
- elector2.addChangeListener(listener2).join();
- LeaderEventListener listener3 = new LeaderEventListener();
- elector3.addChangeListener(listener3).join();
-
- elector3.promote("foo", node3).thenAccept(result -> {
- assertFalse(result);
- }).join();
-
- assertFalse(listener1.hasEvent());
- assertFalse(listener2.hasEvent());
- assertFalse(listener3.hasEvent());
-
- elector3.run("foo", node3).join();
-
- listener1.nextEvent().thenAccept(result -> {
- assertEquals(node3, result.newValue().candidates().get(2));
- }).join();
- listener2.nextEvent().thenAccept(result -> {
- assertEquals(node3, result.newValue().candidates().get(2));
- }).join();
- listener3.nextEvent().thenAccept(result -> {
- assertEquals(node3, result.newValue().candidates().get(2));
- }).join();
-
- elector3.promote("foo", node3).thenAccept(result -> {
- assertTrue(result);
- }).join();
-
- listener1.nextEvent().thenAccept(result -> {
- assertEquals(node3, result.newValue().candidates().get(0));
- }).join();
- listener2.nextEvent().thenAccept(result -> {
- assertEquals(node3, result.newValue().candidates().get(0));
- }).join();
- listener3.nextEvent().thenAccept(result -> {
- assertEquals(node3, result.newValue().candidates().get(0));
- }).join();
- }
-
- @Test
- public void testLeaderSessionClose() throws Throwable {
- leaderElectorLeaderSessionCloseTests();
- }
-
- private void leaderElectorLeaderSessionCloseTests() throws Throwable {
- AtomixLeaderElector elector1 = newPrimitive("test-elector-leader-session-close");
- elector1.run("foo", node1).join();
- AtomixLeaderElector elector2 = newPrimitive("test-elector-leader-session-close");
- LeaderEventListener listener = new LeaderEventListener();
- elector2.run("foo", node2).join();
- elector2.addChangeListener(listener).join();
- elector1.proxy.close();
- listener.nextEvent().thenAccept(result -> {
- assertEquals(node2, result.newValue().leaderNodeId());
- assertEquals(1, result.newValue().candidates().size());
- assertEquals(node2, result.newValue().candidates().get(0));
- }).join();
- }
-
- @Test
- public void testNonLeaderSessionClose() throws Throwable {
- leaderElectorNonLeaderSessionCloseTests();
- }
-
- private void leaderElectorNonLeaderSessionCloseTests() throws Throwable {
- AtomixLeaderElector elector1 = newPrimitive("test-elector-non-leader-session-close");
- elector1.run("foo", node1).join();
- AtomixLeaderElector elector2 = newPrimitive("test-elector-non-leader-session-close");
- LeaderEventListener listener = new LeaderEventListener();
- elector2.run("foo", node2).join();
- elector1.addChangeListener(listener).join();
- elector2.proxy.close().join();
- listener.nextEvent().thenAccept(result -> {
- assertEquals(node1, result.newValue().leaderNodeId());
- assertEquals(1, result.newValue().candidates().size());
- assertEquals(node1, result.newValue().candidates().get(0));
- }).join();
- }
-
- @Test
- public void testQueries() throws Throwable {
- leaderElectorQueryTests();
- }
-
- private void leaderElectorQueryTests() throws Throwable {
- AtomixLeaderElector elector1 = newPrimitive("test-elector-query");
- AtomixLeaderElector elector2 = newPrimitive("test-elector-query");
- elector1.run("foo", node1).join();
- elector2.run("foo", node2).join();
- elector2.run("bar", node2).join();
- elector1.getElectedTopics(node1).thenAccept(result -> {
- assertEquals(1, result.size());
- assertTrue(result.contains("foo"));
- }).join();
- elector2.getElectedTopics(node1).thenAccept(result -> {
- assertEquals(1, result.size());
- assertTrue(result.contains("foo"));
- }).join();
- elector1.getLeadership("foo").thenAccept(result -> {
- assertEquals(node1, result.leaderNodeId());
- assertEquals(node1, result.candidates().get(0));
- assertEquals(node2, result.candidates().get(1));
- }).join();
- elector2.getLeadership("foo").thenAccept(result -> {
- assertEquals(node1, result.leaderNodeId());
- assertEquals(node1, result.candidates().get(0));
- assertEquals(node2, result.candidates().get(1));
- }).join();
- elector1.getLeadership("bar").thenAccept(result -> {
- assertEquals(node2, result.leaderNodeId());
- assertEquals(node2, result.candidates().get(0));
- }).join();
- elector2.getLeadership("bar").thenAccept(result -> {
- assertEquals(node2, result.leaderNodeId());
- assertEquals(node2, result.candidates().get(0));
- }).join();
- elector1.getLeaderships().thenAccept(result -> {
- assertEquals(2, result.size());
- Leadership fooLeadership = result.get("foo");
- assertEquals(node1, fooLeadership.leaderNodeId());
- assertEquals(node1, fooLeadership.candidates().get(0));
- assertEquals(node2, fooLeadership.candidates().get(1));
- Leadership barLeadership = result.get("bar");
- assertEquals(node2, barLeadership.leaderNodeId());
- assertEquals(node2, barLeadership.candidates().get(0));
- }).join();
- elector2.getLeaderships().thenAccept(result -> {
- assertEquals(2, result.size());
- Leadership fooLeadership = result.get("foo");
- assertEquals(node1, fooLeadership.leaderNodeId());
- assertEquals(node1, fooLeadership.candidates().get(0));
- assertEquals(node2, fooLeadership.candidates().get(1));
- Leadership barLeadership = result.get("bar");
- assertEquals(node2, barLeadership.leaderNodeId());
- assertEquals(node2, barLeadership.candidates().get(0));
- }).join();
- }
-
- private static class LeaderEventListener implements Consumer<Change<Leadership>> {
- Queue<Change<Leadership>> eventQueue = new LinkedList<>();
- CompletableFuture<Change<Leadership>> pendingFuture;
-
- @Override
- public void accept(Change<Leadership> change) {
- synchronized (this) {
- if (pendingFuture != null) {
- pendingFuture.complete(change);
- pendingFuture = null;
- } else {
- eventQueue.add(change);
- }
- }
- }
-
- public boolean hasEvent() {
- return !eventQueue.isEmpty();
- }
-
- public void clearEvents() {
- eventQueue.clear();
- }
-
- public CompletableFuture<Change<Leadership>> nextEvent() {
- synchronized (this) {
- if (eventQueue.isEmpty()) {
- if (pendingFuture == null) {
- pendingFuture = new CompletableFuture<>();
- }
- return pendingFuture;
- } else {
- return CompletableFuture.completedFuture(eventQueue.poll());
- }
- }
- }
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixTestBase.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixTestBase.java
deleted file mode 100644
index 6f3ddea..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixTestBase.java
+++ /dev/null
@@ -1,327 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.FileVisitResult;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.SimpleFileVisitor;
-import java.nio.file.attribute.BasicFileAttributes;
-import java.time.Instant;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Lists;
-import io.atomix.protocols.raft.RaftClient;
-import io.atomix.protocols.raft.RaftServer;
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.cluster.RaftMember;
-import io.atomix.protocols.raft.proxy.CommunicationStrategy;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.storage.StorageLevel;
-import org.junit.After;
-import org.junit.Before;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.primitives.impl.RaftClientCommunicator;
-import org.onosproject.store.primitives.impl.RaftServerCommunicator;
-import org.onosproject.store.primitives.impl.StorageNamespaces;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Base class for various Atomix tests.
- *
- * @param <T> the Raft primitive type being tested
- */
-public abstract class AtomixTestBase<T extends AbstractRaftPrimitive> {
-
- protected TestClusterCommunicationServiceFactory communicationServiceFactory;
- protected List<RaftMember> members = Lists.newCopyOnWriteArrayList();
- protected List<RaftClient> clients = Lists.newCopyOnWriteArrayList();
- protected List<RaftServer> servers = Lists.newCopyOnWriteArrayList();
- protected int nextId;
-
- /**
- * Creates the primitive service.
- *
- * @return the primitive service
- */
- protected abstract RaftService createService();
-
- /**
- * Creates a new primitive.
- *
- * @param name the primitive name
- * @return the primitive instance
- */
- protected T newPrimitive(String name) {
- RaftClient client = createClient();
- RaftProxy proxy = client.newProxyBuilder()
- .withName(name)
- .withServiceType("test")
- .withReadConsistency(readConsistency())
- .withCommunicationStrategy(communicationStrategy())
- .build()
- .open()
- .join();
- return createPrimitive(proxy);
- }
-
- /**
- * Creates a new primitive instance.
- *
- * @param proxy the primitive proxy
- * @return the primitive instance
- */
- protected abstract T createPrimitive(RaftProxy proxy);
-
- /**
- * Returns the proxy read consistency.
- *
- * @return the primitive read consistency
- */
- protected ReadConsistency readConsistency() {
- return ReadConsistency.LINEARIZABLE;
- }
-
- /**
- * Returns the proxy communication strategy.
- *
- * @return the primitive communication strategy
- */
- protected CommunicationStrategy communicationStrategy() {
- return CommunicationStrategy.LEADER;
- }
-
- @Before
- public void prepare() {
- members.clear();
- clients.clear();
- servers.clear();
- communicationServiceFactory = new TestClusterCommunicationServiceFactory();
- createServers(3);
- }
-
- @After
- public void cleanup() {
- shutdown();
- }
-
- /**
- * Shuts down clients and servers.
- */
- private void shutdown() {
- clients.forEach(c -> {
- try {
- c.close().get(10, TimeUnit.SECONDS);
- } catch (Exception e) {
- }
- });
-
- servers.forEach(s -> {
- try {
- if (s.isRunning()) {
- s.shutdown().get(10, TimeUnit.SECONDS);
- }
- } catch (Exception e) {
- }
- });
-
- Path directory = Paths.get("target/primitives/");
- if (Files.exists(directory)) {
- try {
- Files.walkFileTree(directory, new SimpleFileVisitor<Path>() {
- @Override
- public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
- Files.delete(file);
- return FileVisitResult.CONTINUE;
- }
-
- @Override
- public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException {
- Files.delete(dir);
- return FileVisitResult.CONTINUE;
- }
- });
- } catch (IOException e) {
- }
- }
- }
-
- /**
- * Returns the next unique member identifier.
- *
- * @return The next unique member identifier.
- */
- private MemberId nextMemberId() {
- return MemberId.from(String.valueOf(++nextId));
- }
-
- /**
- * Returns the next server address.
- *
- * @param type The startup member type.
- * @return The next server address.
- */
- private RaftMember nextMember(RaftMember.Type type) {
- return new TestMember(nextMemberId(), type);
- }
-
- /**
- * Creates a set of Raft servers.
- */
- protected List<RaftServer> createServers(int nodes) {
- List<RaftServer> servers = new ArrayList<>();
-
- for (int i = 0; i < nodes; i++) {
- members.add(nextMember(RaftMember.Type.ACTIVE));
- }
-
- CountDownLatch latch = new CountDownLatch(nodes);
- for (int i = 0; i < nodes; i++) {
- RaftServer server = createServer(members.get(i));
- server.bootstrap(members.stream().map(RaftMember::memberId).collect(Collectors.toList()))
- .thenRun(latch::countDown);
- servers.add(server);
- }
-
- try {
- latch.await(30000, TimeUnit.MILLISECONDS);
- } catch (InterruptedException e) {
- throw new IllegalStateException(e);
- }
-
- return servers;
- }
-
- /**
- * Creates a Raft server.
- */
- private RaftServer createServer(RaftMember member) {
- RaftServer.Builder builder = RaftServer.newBuilder(member.memberId())
- .withType(member.getType())
- .withProtocol(new RaftServerCommunicator(
- "partition-1",
- Serializer.using(StorageNamespaces.RAFT_PROTOCOL),
- communicationServiceFactory.newCommunicationService(NodeId.nodeId(member.memberId().id()))))
- .withStorage(RaftStorage.newBuilder()
- .withStorageLevel(StorageLevel.MEMORY)
- .withDirectory(new File(String.format("target/primitives/%s", member.memberId())))
- .withSerializer(new AtomixSerializerAdapter(Serializer.using(StorageNamespaces.RAFT_STORAGE)))
- .withMaxSegmentSize(1024 * 1024)
- .build())
- .addService("test", this::createService);
-
- RaftServer server = builder.build();
- servers.add(server);
- return server;
- }
-
- /**
- * Creates a Raft client.
- */
- private RaftClient createClient() {
- MemberId memberId = nextMemberId();
- RaftClient client = RaftClient.newBuilder()
- .withMemberId(memberId)
- .withProtocol(new RaftClientCommunicator(
- "partition-1",
- Serializer.using(StorageNamespaces.RAFT_PROTOCOL),
- communicationServiceFactory.newCommunicationService(NodeId.nodeId(memberId.id()))))
- .build();
-
- client.connect(members.stream().map(RaftMember::memberId).collect(Collectors.toList())).join();
- clients.add(client);
- return client;
- }
-
- /**
- * Test member.
- */
- public static class TestMember implements RaftMember {
- private final MemberId memberId;
- private final Type type;
-
- public TestMember(MemberId memberId, Type type) {
- this.memberId = memberId;
- this.type = type;
- }
-
- @Override
- public MemberId memberId() {
- return memberId;
- }
-
- @Override
- public int hash() {
- return memberId.hashCode();
- }
-
- @Override
- public Type getType() {
- return type;
- }
-
- @Override
- public void addTypeChangeListener(Consumer<Type> listener) {
-
- }
-
- @Override
- public void removeTypeChangeListener(Consumer<Type> listener) {
-
- }
-
- @Override
- public Instant getLastUpdated() {
- return Instant.now();
- }
- @Override
- public CompletableFuture<Void> promote() {
- return null;
- }
-
- @Override
- public CompletableFuture<Void> promote(Type type) {
- return null;
- }
-
- @Override
- public CompletableFuture<Void> demote() {
- return null;
- }
-
- @Override
- public CompletableFuture<Void> demote(Type type) {
- return null;
- }
-
- @Override
- public CompletableFuture<Void> remove() {
- return null;
- }
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueServiceTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueServiceTest.java
deleted file mode 100644
index fdc1d97..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueServiceTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-import java.util.Collections;
-
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.impl.RaftContext;
-import io.atomix.protocols.raft.protocol.RaftServerProtocol;
-import io.atomix.protocols.raft.service.ServiceId;
-import io.atomix.protocols.raft.service.ServiceType;
-import io.atomix.protocols.raft.service.impl.DefaultCommit;
-import io.atomix.protocols.raft.service.impl.DefaultServiceContext;
-import io.atomix.protocols.raft.session.SessionId;
-import io.atomix.protocols.raft.session.impl.RaftSessionContext;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.protocols.raft.storage.snapshot.Snapshot;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotStore;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.storage.StorageLevel;
-import io.atomix.time.WallClockTimestamp;
-import io.atomix.utils.concurrent.AtomixThreadFactory;
-import io.atomix.utils.concurrent.SingleThreadContextFactory;
-import org.junit.Test;
-import org.onosproject.store.service.Task;
-
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.mock;
-import static org.easymock.EasyMock.replay;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.TAKE;
-import static org.onosproject.store.service.DistributedPrimitive.Type.WORK_QUEUE;
-
-/**
- * Work queue service test.
- */
-public class AtomixWorkQueueServiceTest {
- @Test
- public void testSnapshot() throws Exception {
- SnapshotStore store = new SnapshotStore(RaftStorage.newBuilder()
- .withPrefix("test")
- .withStorageLevel(StorageLevel.MEMORY)
- .build());
- Snapshot snapshot = store.newSnapshot(2, new WallClockTimestamp());
-
- DefaultServiceContext context = mock(DefaultServiceContext.class);
- expect(context.serviceType()).andReturn(ServiceType.from(WORK_QUEUE.name())).anyTimes();
- expect(context.serviceName()).andReturn("test").anyTimes();
- expect(context.serviceId()).andReturn(ServiceId.from(1)).anyTimes();
-
- RaftContext server = mock(RaftContext.class);
- expect(server.getProtocol()).andReturn(mock(RaftServerProtocol.class));
-
- replay(context, server);
-
- RaftSessionContext session = new RaftSessionContext(
- SessionId.from(1),
- MemberId.from("1"),
- "test",
- ServiceType.from(WORK_QUEUE.name()),
- ReadConsistency.LINEARIZABLE,
- 100,
- 5000,
- System.currentTimeMillis(),
- context,
- server,
- new SingleThreadContextFactory(new AtomixThreadFactory()));
-
- AtomixWorkQueueService service = new AtomixWorkQueueService();
- service.init(context);
-
- service.add(new DefaultCommit<>(
- 2,
- ADD,
- new AtomixWorkQueueOperations.Add(Collections.singletonList("Hello world!".getBytes())),
- session,
- System.currentTimeMillis()));
-
- try (SnapshotWriter writer = snapshot.openWriter()) {
- service.snapshot(writer);
- }
-
- snapshot.complete();
-
- service = new AtomixWorkQueueService();
- service.init(context);
-
- try (SnapshotReader reader = snapshot.openReader()) {
- service.install(reader);
- }
-
- Collection<Task<byte[]>> value = service.take(new DefaultCommit<>(
- 2,
- TAKE,
- new AtomixWorkQueueOperations.Take(1),
- session,
- System.currentTimeMillis()));
- assertNotNull(value);
- assertEquals(1, value.size());
- assertArrayEquals("Hello world!".getBytes(), value.iterator().next().payload());
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueTest.java
deleted file mode 100644
index 0553a1e..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueTest.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.UUID;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.util.concurrent.Uninterruptibles;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import io.atomix.protocols.raft.service.RaftService;
-import org.junit.Test;
-import org.onlab.util.Tools;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueueStats;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Unit tests for {@link AtomixWorkQueue}.
- */
-public class AtomixWorkQueueTest extends AtomixTestBase<AtomixWorkQueue> {
- private static final Duration DEFAULT_PROCESSING_TIME = Duration.ofMillis(100);
- private static final byte[] DEFAULT_PAYLOAD = "hello world".getBytes();
-
- @Override
- protected RaftService createService() {
- return new AtomixWorkQueueService();
- }
-
- @Override
- protected AtomixWorkQueue createPrimitive(RaftProxy proxy) {
- return new AtomixWorkQueue(proxy);
- }
-
- @Test
- public void testAdd() throws Throwable {
- String queueName = UUID.randomUUID().toString();
- AtomixWorkQueue queue1 = newPrimitive(queueName);
- byte[] item = DEFAULT_PAYLOAD;
- queue1.addOne(item).join();
-
- AtomixWorkQueue queue2 = newPrimitive(queueName);
- byte[] task2 = DEFAULT_PAYLOAD;
- queue2.addOne(task2).join();
-
- WorkQueueStats stats = queue1.stats().join();
- assertEquals(stats.totalPending(), 2);
- assertEquals(stats.totalInProgress(), 0);
- assertEquals(stats.totalCompleted(), 0);
- }
-
- @Test
- public void testAddMultiple() throws Throwable {
- String queueName = UUID.randomUUID().toString();
- AtomixWorkQueue queue1 = newPrimitive(queueName);
- byte[] item1 = DEFAULT_PAYLOAD;
- byte[] item2 = DEFAULT_PAYLOAD;
- queue1.addMultiple(Arrays.asList(item1, item2)).join();
-
- WorkQueueStats stats = queue1.stats().join();
- assertEquals(stats.totalPending(), 2);
- assertEquals(stats.totalInProgress(), 0);
- assertEquals(stats.totalCompleted(), 0);
- }
-
- @Test
- public void testTakeAndComplete() throws Throwable {
- String queueName = UUID.randomUUID().toString();
- AtomixWorkQueue queue1 = newPrimitive(queueName);
- byte[] item1 = DEFAULT_PAYLOAD;
- queue1.addOne(item1).join();
-
- AtomixWorkQueue queue2 = newPrimitive(queueName);
- Task<byte[]> removedTask = queue2.take().join();
-
- WorkQueueStats stats = queue2.stats().join();
- assertEquals(stats.totalPending(), 0);
- assertEquals(stats.totalInProgress(), 1);
- assertEquals(stats.totalCompleted(), 0);
-
- assertTrue(Arrays.equals(removedTask.payload(), item1));
- queue2.complete(Arrays.asList(removedTask.taskId())).join();
-
- stats = queue1.stats().join();
- assertEquals(stats.totalPending(), 0);
- assertEquals(stats.totalInProgress(), 0);
- assertEquals(stats.totalCompleted(), 1);
-
- // Another take should return null
- assertNull(queue2.take().join());
- }
-
- @Test
- public void testUnexpectedClientClose() throws Throwable {
- String queueName = UUID.randomUUID().toString();
- AtomixWorkQueue queue1 = newPrimitive(queueName);
- byte[] item1 = DEFAULT_PAYLOAD;
- queue1.addOne(item1).join();
-
- AtomixWorkQueue queue2 = newPrimitive(queueName);
- queue2.take().join();
-
- WorkQueueStats stats = queue1.stats().join();
- assertEquals(0, stats.totalPending());
- assertEquals(1, stats.totalInProgress());
- assertEquals(0, stats.totalCompleted());
-
- queue2.proxy.close().join();
-
- stats = queue1.stats().join();
- assertEquals(1, stats.totalPending());
- assertEquals(0, stats.totalInProgress());
- assertEquals(0, stats.totalCompleted());
- }
-
- @Test
- public void testAutomaticTaskProcessing() throws Throwable {
- String queueName = UUID.randomUUID().toString();
- AtomixWorkQueue queue1 = newPrimitive(queueName);
- Executor executor = Executors.newSingleThreadExecutor();
-
- CountDownLatch latch1 = new CountDownLatch(1);
- queue1.registerTaskProcessor(s -> latch1.countDown(), 2, executor);
-
- AtomixWorkQueue queue2 = newPrimitive(queueName);
- byte[] item1 = DEFAULT_PAYLOAD;
- queue2.addOne(item1).join();
-
- assertTrue(Uninterruptibles.awaitUninterruptibly(latch1, 5000, TimeUnit.MILLISECONDS));
- queue1.stopProcessing();
-
- byte[] item2 = DEFAULT_PAYLOAD;
- byte[] item3 = DEFAULT_PAYLOAD;
-
- Tools.delay((int) DEFAULT_PROCESSING_TIME.toMillis());
-
- queue2.addMultiple(Arrays.asList(item2, item3)).join();
-
- WorkQueueStats stats = queue1.stats().join();
- assertEquals(2, stats.totalPending());
- assertEquals(0, stats.totalInProgress());
- assertEquals(1, stats.totalCompleted());
-
- CountDownLatch latch2 = new CountDownLatch(2);
- queue1.registerTaskProcessor(s -> latch2.countDown(), 2, executor);
-
- Uninterruptibles.awaitUninterruptibly(latch2, 500, TimeUnit.MILLISECONDS);
- }
-
- @Test
- public void testDestroy() {
- String queueName = UUID.randomUUID().toString();
- AtomixWorkQueue queue1 = newPrimitive(queueName);
- byte[] item = DEFAULT_PAYLOAD;
- queue1.addOne(item).join();
-
- AtomixWorkQueue queue2 = newPrimitive(queueName);
- byte[] task2 = DEFAULT_PAYLOAD;
- queue2.addOne(task2).join();
-
- WorkQueueStats stats = queue1.stats().join();
- assertEquals(stats.totalPending(), 2);
- assertEquals(stats.totalInProgress(), 0);
- assertEquals(stats.totalCompleted(), 0);
-
- queue2.destroy().join();
-
- stats = queue1.stats().join();
- assertEquals(stats.totalPending(), 0);
- assertEquals(stats.totalInProgress(), 0);
- assertEquals(stats.totalCompleted(), 0);
- }
-
- @Test
- public void testCompleteAttemptWithIncorrectSession() {
- String queueName = UUID.randomUUID().toString();
- AtomixWorkQueue queue1 = newPrimitive(queueName);
- byte[] item = DEFAULT_PAYLOAD;
- queue1.addOne(item).join();
-
- Task<byte[]> task = queue1.take().join();
- String taskId = task.taskId();
-
- // Create another client and get a handle to the same queue.
- AtomixWorkQueue queue2 = newPrimitive(queueName);
-
- // Attempt completing the task with new client and verify task is not completed
- queue2.complete(taskId).join();
-
- WorkQueueStats stats = queue1.stats().join();
- assertEquals(stats.totalPending(), 0);
- assertEquals(stats.totalInProgress(), 1);
- assertEquals(stats.totalCompleted(), 0);
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeTest.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeTest.java
deleted file mode 100644
index 1398300..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeTest.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTree;
-import org.onosproject.store.service.IllegalDocumentModificationException;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.Versioned;
-
-/**
- * Tests for {@code DefaultDocumentTree}.
- */
-public class DefaultDocumentTreeTest {
-
- @Test
- public void testTreeConstructor() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- Assert.assertEquals(tree.root(), path("root"));
- }
-
- @Test
- public void testCreateNodeAtRoot() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- Assert.assertTrue(tree.create(path("root.a"), "bar"));
- Assert.assertFalse(tree.create(path("root.a"), "baz"));
- }
-
- @Test
- public void testCreateNodeAtNonRoot() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- Assert.assertTrue(tree.create(path("root.a.b"), "baz"));
- }
-
- @Test
- public void testCreateRecursive() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.createRecursive(path("root.a.b.c"), "bar");
- Assert.assertEquals(tree.get(path("root.a.b.c")).value(), "bar");
- Assert.assertNull(tree.get(path("root.a.b")).value());
- Assert.assertNull(tree.get(path("root.a")).value());
- }
-
- @Test(expected = IllegalDocumentModificationException.class)
- public void testCreateRecursiveRoot() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.createRecursive(path("root"), "bar");
- }
-
- @Test(expected = IllegalDocumentModificationException.class)
- public void testCreateNodeFailure() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a.b"), "bar");
- }
-
- @Test
- public void testGetRootValue() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.create(path("root.a.b"), "baz");
- Versioned<String> root = tree.get(path("root"));
- Assert.assertNotNull(root);
- Assert.assertNull(root.value());
- }
-
- @Test
- public void testGetInnerNode() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.create(path("root.a.b"), "baz");
- Versioned<String> nodeValue = tree.get(path("root.a"));
- Assert.assertNotNull(nodeValue);
- Assert.assertEquals("bar", nodeValue.value());
- }
-
- @Test
- public void testGetLeafNode() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.create(path("root.a.b"), "baz");
- Versioned<String> nodeValue = tree.get(path("root.a.b"));
- Assert.assertNotNull(nodeValue);
- Assert.assertEquals("baz", nodeValue.value());
- }
-
- @Test
- public void getMissingNode() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.create(path("root.a.b"), "baz");
- Assert.assertNull(tree.get(path("root.x")));
- Assert.assertNull(tree.get(path("root.a.x")));
- Assert.assertNull(tree.get(path("root.a.b.x")));
- }
-
- @Test
- public void testGetChildren() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.create(path("root.a.b"), "alpha");
- tree.create(path("root.a.c"), "beta");
- Assert.assertEquals(2, tree.getChildren(path("root.a")).size());
- Assert.assertEquals(0, tree.getChildren(path("root.a.b")).size());
- }
-
- @Test(expected = NoSuchDocumentPathException.class)
- public void testGetChildrenFailure() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.getChildren(path("root.a.b"));
- }
-
- @Test(expected = IllegalDocumentModificationException.class)
- public void testSetRootFailure() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.set(tree.root(), "bar");
- }
-
- @Test
- public void testSet() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- Assert.assertNull(tree.set(path("root.a.b"), "alpha"));
- Assert.assertEquals("alpha", tree.set(path("root.a.b"), "beta").value());
- Assert.assertEquals("beta", tree.get(path("root.a.b")).value());
- }
-
- @Test(expected = IllegalDocumentModificationException.class)
- public void testSetInvalidNode() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.set(path("root.a.b"), "alpha");
- }
-
- @Test
- public void testReplaceWithVersion() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.create(path("root.a.b"), "alpha");
- Versioned<String> value = tree.get(path("root.a.b"));
- Assert.assertTrue(tree.replace(path("root.a.b"), "beta", value.version()));
- Assert.assertFalse(tree.replace(path("root.a.b"), "beta", value.version()));
- Assert.assertFalse(tree.replace(path("root.x"), "beta", 1));
- }
-
- @Test
- public void testReplaceWithValue() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.create(path("root.a.b"), "alpha");
- Assert.assertTrue(tree.replace(path("root.a.b"), "beta", "alpha"));
- Assert.assertFalse(tree.replace(path("root.a.b"), "beta", "alpha"));
- Assert.assertFalse(tree.replace(path("root.x"), "beta", "bar"));
- Assert.assertTrue(tree.replace(path("root.x"), "beta", null));
- }
-
- @Test(expected = IllegalDocumentModificationException.class)
- public void testRemoveRoot() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.removeNode(tree.root());
- }
-
- @Test
- public void testRemove() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.create(path("root.a"), "bar");
- tree.create(path("root.a.b"), "alpha");
- Assert.assertEquals("alpha", tree.removeNode(path("root.a.b")).value());
- Assert.assertEquals(0, tree.getChildren(path("root.a")).size());
- }
-
- @Test(expected = NoSuchDocumentPathException.class)
- public void testRemoveInvalidNode() {
- DocumentTree<String> tree = new DefaultDocumentTree<>();
- tree.removeNode(path("root.a"));
- }
-
- private static DocumentPath path(String path) {
- return DocumentPath.from(path.replace(".", DocumentPath.DEFAULT_SEPARATOR));
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/TestClusterCommunicationService.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/TestClusterCommunicationService.java
deleted file mode 100644
index b7a1b89..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/TestClusterCommunicationService.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import com.google.common.collect.Maps;
-import io.atomix.utils.concurrent.Futures;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
-import org.onosproject.store.cluster.messaging.MessageSubject;
-import org.onosproject.store.cluster.messaging.MessagingException;
-
-/**
- * Cluster communication service implementation used for testing.
- */
-public class TestClusterCommunicationService implements ClusterCommunicationService {
- private final NodeId localNodeId;
- private final Map<NodeId, TestClusterCommunicationService> nodes;
- private final Map<MessageSubject, Function<byte[], CompletableFuture<byte[]>>> subscribers =
- Maps.newConcurrentMap();
-
- public TestClusterCommunicationService(NodeId localNodeId, Map<NodeId, TestClusterCommunicationService> nodes) {
- this.localNodeId = localNodeId;
- this.nodes = nodes;
- nodes.put(localNodeId, this);
- }
-
- @Override
- public <M> void broadcast(M message, MessageSubject subject, Function<M, byte[]> encoder) {
- nodes.forEach((nodeId, node) -> {
- if (!nodeId.equals(localNodeId)) {
- node.handle(subject, encoder.apply(message));
- }
- });
- }
-
- @Override
- public <M> void broadcastIncludeSelf(M message, MessageSubject subject, Function<M, byte[]> encoder) {
- nodes.values().forEach(node -> node.handle(subject, encoder.apply(message)));
- }
-
- @Override
- public <M> CompletableFuture<Void> unicast(
- M message, MessageSubject subject, Function<M, byte[]> encoder, NodeId toNodeId) {
- TestClusterCommunicationService node = nodes.get(toNodeId);
- if (node != null) {
- node.handle(subject, encoder.apply(message));
- }
- return CompletableFuture.completedFuture(null);
- }
-
- @Override
- public <M> void multicast(M message, MessageSubject subject, Function<M, byte[]> encoder, Set<NodeId> nodeIds) {
- nodes.entrySet().stream()
- .filter(e -> nodeIds.contains(e.getKey()))
- .forEach(e -> e.getValue().handle(subject, encoder.apply(message)));
- }
-
- @Override
- public <M, R> CompletableFuture<R> sendAndReceive(
- M message,
- MessageSubject subject,
- Function<M, byte[]> encoder,
- Function<byte[], R> decoder,
- NodeId toNodeId) {
- TestClusterCommunicationService node = nodes.get(toNodeId);
- if (node == null) {
- return Futures.exceptionalFuture(new MessagingException.NoRemoteHandler());
- }
- return node.handle(subject, encoder.apply(message)).thenApply(decoder);
- }
-
- private CompletableFuture<byte[]> handle(MessageSubject subject, byte[] message) {
- Function<byte[], CompletableFuture<byte[]>> subscriber = subscribers.get(subject);
- if (subscriber != null) {
- return subscriber.apply(message);
- }
- return Futures.exceptionalFuture(new MessagingException.NoRemoteHandler());
- }
-
- private boolean isSubscriber(MessageSubject subject) {
- return subscribers.containsKey(subject);
- }
-
- @Override
- public <M, R> void addSubscriber(
- MessageSubject subject,
- Function<byte[], M> decoder,
- Function<M, R> handler,
- Function<R, byte[]> encoder,
- Executor executor) {
- subscribers.put(subject, message -> {
- CompletableFuture<byte[]> future = new CompletableFuture<>();
- executor.execute(() -> {
- try {
- future.complete(encoder.apply(handler.apply(decoder.apply(message))));
- } catch (Exception e) {
- future.completeExceptionally(new MessagingException.RemoteHandlerFailure());
- }
- });
- return future;
- });
- }
-
- @Override
- public <M, R> void addSubscriber(
- MessageSubject subject,
- Function<byte[], M> decoder,
- Function<M, CompletableFuture<R>> handler,
- Function<R, byte[]> encoder) {
- subscribers.put(subject, message -> {
- CompletableFuture<byte[]> future = new CompletableFuture<>();
- try {
- handler.apply(decoder.apply(message)).whenComplete((result, error) -> {
- if (error == null) {
- future.complete(encoder.apply(result));
- } else {
- future.completeExceptionally(new MessagingException.RemoteHandlerFailure());
- }
- });
- } catch (Exception e) {
- future.completeExceptionally(new MessagingException.RemoteHandlerFailure());
- }
- return future;
- });
- }
-
- @Override
- public <M> void addSubscriber(
- MessageSubject subject,
- Function<byte[], M> decoder,
- Consumer<M> handler,
- Executor executor) {
- subscribers.put(subject, message -> {
- CompletableFuture<byte[]> future = new CompletableFuture<>();
- executor.execute(() -> {
- try {
- handler.accept(decoder.apply(message));
- future.complete(null);
- } catch (Exception e) {
- future.completeExceptionally(new MessagingException.RemoteHandlerFailure());
- }
- });
- return future;
- });
- }
-
- @Override
- public void removeSubscriber(MessageSubject subject) {
- subscribers.remove(subject);
- }
-
- @Override
- public void addSubscriber(MessageSubject subject, ClusterMessageHandler subscriber, ExecutorService executor) {
- throw new UnsupportedOperationException();
- }
-}
diff --git a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/TestClusterCommunicationServiceFactory.java b/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/TestClusterCommunicationServiceFactory.java
deleted file mode 100644
index 611575c..0000000
--- a/core/store/primitives/src/test/java/org/onosproject/store/primitives/resources/impl/TestClusterCommunicationServiceFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * Licensed 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.onosproject.store.primitives.resources.impl;
-
-import java.util.Map;
-
-import com.google.common.collect.Maps;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-
-/**
- * Test cluster communication service factory.
- */
-public class TestClusterCommunicationServiceFactory {
- private final Map<NodeId, TestClusterCommunicationService> nodes = Maps.newConcurrentMap();
-
- /**
- * Creates a new cluster communication service for the given node.
- *
- * @param localNodeId the node for which to create the service
- * @return the communication service for the given node
- */
- public ClusterCommunicationService newCommunicationService(NodeId localNodeId) {
- return new TestClusterCommunicationService(localNodeId, nodes);
- }
-}