Madan Jampani

Refactor change: Moved all distributed primitive implementation classes into ono…

…s-core-primitives bundle

Change-Id: Icd5dbd4133cb2f21bd403bcd598e6012813e6bfd
Showing 63 changed files with 199 additions and 202 deletions
......@@ -13,14 +13,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.impl;
package org.onosproject.store;
import static com.google.common.base.Preconditions.checkArgument;
import java.util.Objects;
import org.onosproject.store.Timestamp;
import com.google.common.base.MoreObjects;
import com.google.common.collect.ComparisonChain;
......
/*
* Copyright 2015 Open Networking Laboratory
*
* 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.
*/
/**
* Implementation of partitioned and distributed store facility capable of
* providing consistent update semantics.
*/
package org.onosproject.store.consistent.impl;
\ No newline at end of file
......@@ -23,8 +23,8 @@ 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.store.LogicalTimestamp;
import org.onosproject.store.Timestamp;
import org.onosproject.store.impl.LogicalTimestamp;
import org.onosproject.store.service.AtomicCounter;
import org.onosproject.store.service.LogicalClockService;
import org.onosproject.store.service.StorageService;
......
/*
* Copyright 2015 Open Networking Laboratory
*
* 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.
*/
/**
* Distributed map with eventually-consistent update semantics and gossip
* based anti-entropy mechanism.
*/
package org.onosproject.store.ecmap;
\ No newline at end of file
......@@ -27,6 +27,7 @@ import org.junit.Ignore;
import org.junit.Test;
import org.onlab.packet.ChassisId;
import org.onlab.packet.IpAddress;
import org.onlab.util.KryoNamespace;
import org.onosproject.cluster.ClusterService;
import org.onosproject.cluster.ControllerNode;
import org.onosproject.cluster.DefaultControllerNode;
......@@ -56,9 +57,10 @@ import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
import org.onosproject.store.cluster.messaging.ClusterMessage;
import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
import org.onosproject.store.cluster.messaging.MessageSubject;
import org.onosproject.store.consistent.impl.DatabaseManager;
import org.onosproject.store.impl.MastershipBasedTimestamp;
import org.onosproject.store.service.EventuallyConsistentMap;
import org.onosproject.store.service.EventuallyConsistentMapBuilder;
import org.onosproject.store.service.StorageService;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
......@@ -71,6 +73,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.BiFunction;
import java.util.function.Function;
import static java.util.Arrays.asList;
......@@ -132,7 +135,9 @@ public class GossipDeviceStoreTest {
new DefaultControllerNode(NID2, IpAddress.valueOf("127.0.0.2"));
private static final List<SparseAnnotations> NO_ANNOTATION = Collections.<SparseAnnotations>emptyList();
EventuallyConsistentMapBuilder ecMapBuilder;
EventuallyConsistentMap ecMap;
StorageService testStorageService;
private TestGossipDeviceStore testGossipDeviceStore;
private GossipDeviceStore gossipDeviceStore;
private DeviceStore deviceStore;
......@@ -161,9 +166,21 @@ public class GossipDeviceStoreTest {
testGossipDeviceStore = new TestGossipDeviceStore(deviceClockService, clusterService, clusterCommunicator);
testGossipDeviceStore.mastershipService = new TestMastershipService();
TestDatabaseManager testDatabaseManager = new TestDatabaseManager();
testDatabaseManager.init(clusterService, clusterCommunicator);
testGossipDeviceStore.storageService = testDatabaseManager;
ecMapBuilder = createNiceMock(EventuallyConsistentMapBuilder.class);
expect(ecMapBuilder.withName(anyObject(String.class))).andReturn(ecMapBuilder).anyTimes();
expect(ecMapBuilder.withSerializer(anyObject(KryoNamespace.Builder.class))).andReturn(ecMapBuilder).anyTimes();
expect(ecMapBuilder.withAntiEntropyPeriod(5, TimeUnit.SECONDS)).andReturn(ecMapBuilder).anyTimes();
expect(ecMapBuilder.withTimestampProvider(anyObject(BiFunction.class))).andReturn(ecMapBuilder).anyTimes();
expect(ecMapBuilder.withTombstonesDisabled()).andReturn(ecMapBuilder).anyTimes();
ecMap = createNiceMock(EventuallyConsistentMap.class);
expect(ecMapBuilder.build()).andReturn(ecMap).anyTimes();
testStorageService = createNiceMock(StorageService.class);
expect(testStorageService.eventuallyConsistentMapBuilder()).andReturn(ecMapBuilder).anyTimes();
replay(testStorageService, ecMapBuilder, ecMap);
testGossipDeviceStore.storageService = testStorageService;
testGossipDeviceStore.deviceClockService = deviceClockService;
gossipDeviceStore = testGossipDeviceStore;
......@@ -897,12 +914,4 @@ public class GossipDeviceStoreTest {
return DID1.equals(deviceId) || DID2.equals(deviceId);
}
}
private class TestDatabaseManager extends DatabaseManager {
void init(ClusterService clusterService,
ClusterCommunicationService clusterCommunicator) {
this.clusterService = clusterService;
this.clusterCommunicator = clusterCommunicator;
}
}
}
......
......@@ -37,6 +37,36 @@
<groupId>org.onosproject</groupId>
<artifactId>onos-api</artifactId>
</dependency>
<dependency>
<groupId>org.onosproject</groupId>
<artifactId>onos-core-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.onosproject</groupId>
<artifactId>onos-core-serializers</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.mapdb</groupId>
<artifactId>mapdb</artifactId>
<version>1.0.8</version>
</dependency>
<dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.onosproject</groupId>
<artifactId>onos-api</artifactId>
<classifier>tests</classifier>
<scope>test</scope>
</dependency>
<!-- for shaded atomix/copycat -->
<dependency>
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import com.google.common.base.MoreObjects;
import com.google.common.collect.ImmutableMap;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.concurrent.CompletableFuture;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static com.google.common.base.MoreObjects.toStringHelper;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.Collection;
import java.util.Map;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.net.URI;
import java.nio.ByteBuffer;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.function.Consumer;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.typesafe.config.ConfigValueFactory;
import net.kuujo.copycat.cluster.ClusterConfig;
......
......@@ -14,16 +14,23 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Multimap;
import com.google.common.collect.Multimaps;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.Futures;
import static org.onosproject.app.ApplicationEvent.Type.APP_DEACTIVATED;
import static org.onosproject.app.ApplicationEvent.Type.APP_UNINSTALLED;
import static org.slf4j.LoggerFactory.getLogger;
import java.util.Collection;
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.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import java.util.stream.Collectors;
import net.kuujo.copycat.CopycatConfig;
import net.kuujo.copycat.cluster.ClusterConfig;
......@@ -58,37 +65,29 @@ import org.onosproject.core.ApplicationId;
import org.onosproject.core.IdGenerator;
import org.onosproject.persistence.PersistenceService;
import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
import org.onosproject.store.ecmap.EventuallyConsistentMapBuilderImpl;
import org.onosproject.store.service.AtomicCounterBuilder;
import org.onosproject.store.service.AtomicValueBuilder;
import org.onosproject.store.service.ConsistentMapBuilder;
import org.onosproject.store.service.ConsistentMapException;
import org.onosproject.store.service.DistributedQueueBuilder;
import org.onosproject.store.service.DistributedSetBuilder;
import org.onosproject.store.service.EventuallyConsistentMapBuilder;
import org.onosproject.store.service.MapInfo;
import org.onosproject.store.service.PartitionInfo;
import org.onosproject.store.service.DistributedSetBuilder;
import org.onosproject.store.service.StorageAdminService;
import org.onosproject.store.service.StorageService;
import org.onosproject.store.service.Transaction;
import org.onosproject.store.service.TransactionContextBuilder;
import org.slf4j.Logger;
import java.util.Collection;
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.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.slf4j.LoggerFactory.getLogger;
import static org.onosproject.app.ApplicationEvent.Type.APP_UNINSTALLED;
import static org.onosproject.app.ApplicationEvent.Type.APP_DEACTIVATED;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Multimap;
import com.google.common.collect.Multimaps;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.Futures;
/**
* Database manager.
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static com.google.common.base.Preconditions.checkState;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onlab.util.Match;
import org.onosproject.store.service.Transaction;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.nio.ByteBuffer;
......@@ -76,7 +76,7 @@ public class DatabaseSerializer extends SerializerConfig {
.register(Result.Status.class)
.register(DefaultTransaction.class)
.register(Transaction.State.class)
.register(org.onosproject.store.consistent.impl.CommitResponse.class)
.register(org.onosproject.store.primitives.impl.CommitResponse.class)
.register(Match.class)
.register(NodeId.class)
.build();
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import net.kuujo.copycat.state.Command;
import net.kuujo.copycat.state.Initializer;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.AsyncAtomicCounter;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.AsyncAtomicValue;
import org.onosproject.store.service.AsyncConsistentMap;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
......@@ -50,8 +50,8 @@ import java.util.function.Predicate;
import java.util.stream.Collectors;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.onosproject.store.consistent.impl.StateMachineUpdate.Target.MAP_UPDATE;
import static org.onosproject.store.consistent.impl.StateMachineUpdate.Target.TX_COMMIT;
import static org.onosproject.store.primitives.impl.StateMachineUpdate.Target.MAP_UPDATE;
import static org.onosproject.store.primitives.impl.StateMachineUpdate.Target.TX_COMMIT;
import static org.slf4j.LoggerFactory.getLogger;
/**
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.Collection;
import java.util.Map;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.AsyncAtomicCounter;
import org.onosproject.store.service.AtomicCounter;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.AsyncAtomicCounter;
import org.onosproject.store.service.AtomicCounter;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.serializers.KryoNamespaces;
import org.onosproject.store.service.AsyncAtomicValue;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.Collection;
import java.util.Map;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.core.ApplicationId;
import org.onosproject.store.service.AsyncConsistentMap;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.google.common.collect.Sets;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.google.common.base.Objects;
import com.google.common.collect.ImmutableList;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.Futures;
......@@ -28,7 +28,7 @@ import java.util.Set;
import java.util.concurrent.CompletableFuture;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.onosproject.store.consistent.impl.StateMachineUpdate.Target.QUEUE_PUSH;
import static org.onosproject.store.primitives.impl.StateMachineUpdate.Target.QUEUE_PUSH;
/**
* DistributedQueue implementation that provides FIFO ordering semantics.
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.DistributedQueue;
import org.onosproject.store.service.DistributedQueueBuilder;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.AsyncDistributedSet;
import org.onosproject.store.service.ConsistentMapException;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.core.ApplicationId;
import org.onosproject.store.service.AsyncDistributedSet;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.List;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.List;
import java.util.Map;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.TransactionContext;
import org.onosproject.store.service.TransactionContextBuilder;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.List;
import java.util.Map;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.service.Serializer;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import org.onlab.util.KryoNamespace;
import org.onosproject.cluster.ClusterService;
......
......@@ -13,14 +13,33 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
import static org.onlab.util.BoundedThreadPool.newFixedThreadPool;
import static org.onlab.util.Tools.groupedThreads;
import static org.onosproject.store.service.EventuallyConsistentMapEvent.Type.PUT;
import static org.onosproject.store.service.EventuallyConsistentMapEvent.Type.REMOVE;
import java.util.Collection;
import java.util.Collections;
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.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
import com.google.common.collect.Collections2;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.commons.lang3.tuple.Pair;
import org.onlab.util.AbstractAccumulator;
import org.onlab.util.KryoNamespace;
......@@ -29,10 +48,10 @@ import org.onosproject.cluster.ClusterService;
import org.onosproject.cluster.ControllerNode;
import org.onosproject.cluster.NodeId;
import org.onosproject.persistence.PersistenceService;
import org.onosproject.store.LogicalTimestamp;
import org.onosproject.store.Timestamp;
import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
import org.onosproject.store.cluster.messaging.MessageSubject;
import org.onosproject.store.impl.LogicalTimestamp;
import org.onosproject.store.serializers.KryoNamespaces;
import org.onosproject.store.serializers.KryoSerializer;
import org.onosproject.store.service.EventuallyConsistentMap;
......@@ -43,30 +62,12 @@ import org.onosproject.store.service.WallClockTimestamp;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
import java.util.Collections;
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.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiFunction;
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;
import static org.onlab.util.BoundedThreadPool.newFixedThreadPool;
import static org.onlab.util.Tools.groupedThreads;
import static org.onosproject.store.service.EventuallyConsistentMapEvent.Type.PUT;
import static org.onosproject.store.service.EventuallyConsistentMapEvent.Type.REMOVE;
import com.google.common.collect.Collections2;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
/**
* Distributed Map implementation which uses optimistic replication and gossip
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import org.mapdb.DB;
import org.mapdb.DBMaker;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import org.onosproject.store.Timestamp;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.Arrays;
import java.util.Collection;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.codahale.metrics.Counter;
import com.codahale.metrics.Timer;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static org.slf4j.LoggerFactory.getLogger;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
/**
* Partitioner is responsible for mapping keys to individual database partitions.
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import java.util.Map;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static com.google.common.base.MoreObjects.toStringHelper;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.List;
......
......@@ -14,7 +14,7 @@
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.List;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static com.google.common.base.MoreObjects.toStringHelper;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static com.google.common.base.Preconditions.checkNotNull;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import static com.google.common.base.Preconditions.checkNotNull;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import java.util.function.Function;
......
/*
* Copyright 2016 Open Networking Laboratory
* Copyright 2015 Open Networking Laboratory
*
* 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,7 @@
*/
/**
* Implementation for distributed state management primitives.
* Implementation of partitioned and distributed store facility capable of
* providing consistent update semantics.
*/
package org.onosproject.store.primitives.impl;
package org.onosproject.store.primitives.impl;
\ No newline at end of file
......
......@@ -13,12 +13,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static java.util.Collections.unmodifiableCollection;
import static java.util.Collections.unmodifiableSet;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import java.util.Collection;
import java.util.Map;
......@@ -29,6 +30,10 @@ import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
import net.kuujo.copycat.Task;
import net.kuujo.copycat.cluster.Cluster;
import net.kuujo.copycat.resource.ResourceState;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
......@@ -44,10 +49,6 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
import net.kuujo.copycat.Task;
import net.kuujo.copycat.cluster.Cluster;
import net.kuujo.copycat.resource.ResourceState;
/**
*
*/
......
......@@ -13,7 +13,22 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import static com.google.common.base.Preconditions.checkArgument;
import static junit.framework.TestCase.assertFalse;
import static org.easymock.EasyMock.anyObject;
import static org.easymock.EasyMock.createMock;
import static org.easymock.EasyMock.eq;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.expectLastCall;
import static org.easymock.EasyMock.replay;
import static org.easymock.EasyMock.reset;
import static org.easymock.EasyMock.verify;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.util.ArrayList;
import java.util.Collection;
......@@ -43,11 +58,11 @@ import org.onosproject.cluster.DefaultControllerNode;
import org.onosproject.cluster.NodeId;
import org.onosproject.event.AbstractEvent;
import org.onosproject.persistence.PersistenceService;
import org.onosproject.store.LogicalTimestamp;
import org.onosproject.store.Timestamp;
import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
import org.onosproject.store.cluster.messaging.ClusterCommunicationServiceAdapter;
import org.onosproject.store.cluster.messaging.MessageSubject;
import org.onosproject.store.impl.LogicalTimestamp;
import org.onosproject.store.persistence.TestPersistenceService;
import org.onosproject.store.serializers.KryoNamespaces;
import org.onosproject.store.serializers.KryoSerializer;
......@@ -61,21 +76,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.util.concurrent.MoreExecutors;
import static com.google.common.base.Preconditions.checkArgument;
import static junit.framework.TestCase.assertFalse;
import static org.easymock.EasyMock.anyObject;
import static org.easymock.EasyMock.createMock;
import static org.easymock.EasyMock.eq;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.expectLastCall;
import static org.easymock.EasyMock.replay;
import static org.easymock.EasyMock.reset;
import static org.easymock.EasyMock.verify;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/**
* Unit tests for EventuallyConsistentMapImpl.
*/
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.ecmap;
package org.onosproject.store.primitives.impl;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
......@@ -21,9 +21,9 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import org.junit.Test;
import org.onosproject.store.LogicalTimestamp;
import org.onosproject.store.Timestamp;
import org.onosproject.store.ecmap.MapValue.Digest;
import org.onosproject.store.impl.LogicalTimestamp;
import org.onosproject.store.primitives.impl.MapValue.Digest;
/**
* Unit tests for MapValue.
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static junit.framework.TestCase.assertEquals;
import static junit.framework.TestCase.assertFalse;
......
......@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.onosproject.store.consistent.impl;
package org.onosproject.store.primitives.impl;
import static junit.framework.TestCase.assertEquals;
import static junit.framework.TestCase.assertNull;
......