Add distributed lock primitive

Change-Id: I60f4581d5acb5fc9b9a21d4191874bb56348af58
diff --git a/core/api/src/main/java/org/onosproject/store/primitives/DefaultDistributedLock.java b/core/api/src/main/java/org/onosproject/store/primitives/DefaultDistributedLock.java
new file mode 100644
index 0000000..8e48771
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/store/primitives/DefaultDistributedLock.java
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+import java.time.Duration;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.onosproject.store.service.AsyncDistributedLock;
+import org.onosproject.store.service.DistributedLock;
+import org.onosproject.store.service.StorageException;
+import org.onosproject.store.service.Synchronous;
+import org.onosproject.store.service.Version;
+
+/**
+ * Default implementation for a {@code DistributedLock} backed by a {@link AsyncDistributedLock}.
+ */
+public class DefaultDistributedLock extends Synchronous<AsyncDistributedLock> implements DistributedLock {
+
+    private final AsyncDistributedLock asyncLock;
+    private final long operationTimeoutMillis;
+
+    public DefaultDistributedLock(AsyncDistributedLock asyncLock, long operationTimeoutMillis) {
+        super(asyncLock);
+        this.asyncLock = asyncLock;
+        this.operationTimeoutMillis = operationTimeoutMillis;
+    }
+
+    @Override
+    public Version lock() {
+        return complete(asyncLock.lock());
+    }
+
+    @Override
+    public Optional<Version> tryLock() {
+        return complete(asyncLock.tryLock());
+    }
+
+    @Override
+    public Optional<Version> tryLock(Duration timeout) {
+        return complete(asyncLock.tryLock(timeout));
+    }
+
+    @Override
+    public void unlock() {
+        complete(asyncLock.unlock());
+    }
+
+    private <T> T complete(CompletableFuture<T> future) {
+        if (operationTimeoutMillis == -1) {
+            return future.join();
+        }
+        try {
+            return future.get(operationTimeoutMillis, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new StorageException.Interrupted();
+        } catch (TimeoutException e) {
+            throw new StorageException.Timeout();
+        } catch (ExecutionException e) {
+            throw new StorageException(e.getCause());
+        }
+    }
+}
diff --git a/core/api/src/main/java/org/onosproject/store/primitives/DistributedPrimitiveCreator.java b/core/api/src/main/java/org/onosproject/store/primitives/DistributedPrimitiveCreator.java
index 5fad35a..1e68bdb 100644
--- a/core/api/src/main/java/org/onosproject/store/primitives/DistributedPrimitiveCreator.java
+++ b/core/api/src/main/java/org/onosproject/store/primitives/DistributedPrimitiveCreator.java
@@ -25,6 +25,7 @@
 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;
@@ -118,6 +119,14 @@
     <E> AsyncDistributedSet<E> newAsyncDistributedSet(String name, Serializer serializer);
 
     /**
+     * Creates a new {@code AsyncDistributedLock}.
+     *
+     * @param name lock name
+     * @return lock
+     */
+    AsyncDistributedLock newAsyncDistributedLock(String name);
+
+    /**
      * Creates a new {@code AsyncLeaderElector}.
      *
      * @param name leader elector name
diff --git a/core/api/src/main/java/org/onosproject/store/service/AsyncDistributedLock.java b/core/api/src/main/java/org/onosproject/store/service/AsyncDistributedLock.java
new file mode 100644
index 0000000..be35b5e
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/store/service/AsyncDistributedLock.java
@@ -0,0 +1,69 @@
+/*
+ * 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.service;
+
+import java.time.Duration;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import org.onosproject.store.primitives.DefaultDistributedLock;
+
+/**
+ * Asynchronous lock primitive.
+ */
+public interface AsyncDistributedLock extends DistributedPrimitive {
+    @Override
+    default Type primitiveType() {
+        return Type.LOCK;
+    }
+
+    /**
+     * Acquires the lock, blocking until it's available.
+     *
+     * @return future to be completed once the lock has been acquired
+     */
+    CompletableFuture<Version> lock();
+
+    /**
+     * Attempts to acquire the lock.
+     *
+     * @return future to be completed with a boolean indicating whether the lock was acquired
+     */
+    CompletableFuture<Optional<Version>> tryLock();
+
+    /**
+     * Attempts to acquire the lock for a specified amount of time.
+     *
+     * @param timeout the timeout after which to give up attempting to acquire the lock
+     * @return future to be completed with a boolean indicating whether the lock was acquired
+     */
+    CompletableFuture<Optional<Version>> tryLock(Duration timeout);
+
+    /**
+     * Unlocks the lock.
+     *
+     * @return future to be completed once the lock has been released
+     */
+    CompletableFuture<Void> unlock();
+
+    default DistributedLock asLock() {
+        return asLock(-1);
+    }
+
+    default DistributedLock asLock(long timeoutMillis) {
+        return new DefaultDistributedLock(this, timeoutMillis);
+    }
+}
diff --git a/core/api/src/main/java/org/onosproject/store/service/DistributedLock.java b/core/api/src/main/java/org/onosproject/store/service/DistributedLock.java
new file mode 100644
index 0000000..cce53ff
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/store/service/DistributedLock.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.service;
+
+import java.time.Duration;
+import java.util.Optional;
+
+/**
+ * Asynchronous lock primitive.
+ */
+public interface DistributedLock extends DistributedPrimitive {
+    @Override
+    default Type primitiveType() {
+        return Type.LOCK;
+    }
+
+    /**
+     * Acquires the lock, blocking until it's available.
+     *
+     * @return the acquired lock version
+     */
+    Version lock();
+
+    /**
+     * Attempts to acquire the lock.
+     *
+     * @return indicates whether the lock was acquired
+     */
+    Optional<Version> tryLock();
+
+    /**
+     * Attempts to acquire the lock for a specified amount of time.
+     *
+     * @param timeout the timeout after which to give up attempting to acquire the lock
+     * @return indicates whether the lock was acquired
+     */
+    Optional<Version> tryLock(Duration timeout);
+
+    /**
+     * Unlocks the lock.
+     */
+    void unlock();
+}
diff --git a/core/api/src/main/java/org/onosproject/store/service/DistributedLockBuilder.java b/core/api/src/main/java/org/onosproject/store/service/DistributedLockBuilder.java
new file mode 100644
index 0000000..7e61c6a
--- /dev/null
+++ b/core/api/src/main/java/org/onosproject/store/service/DistributedLockBuilder.java
@@ -0,0 +1,28 @@
+/*
+ * 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.service;
+
+import org.onosproject.store.primitives.DistributedPrimitiveBuilder;
+
+/**
+ * Builder for DistributedLock.
+ */
+public abstract class DistributedLockBuilder
+    extends DistributedPrimitiveBuilder<DistributedLockBuilder, AsyncDistributedLock> {
+    public DistributedLockBuilder() {
+        super(DistributedPrimitive.Type.LOCK);
+    }
+}
\ No newline at end of file
diff --git a/core/api/src/main/java/org/onosproject/store/service/DistributedPrimitive.java b/core/api/src/main/java/org/onosproject/store/service/DistributedPrimitive.java
index 8a440f3..620e16d 100644
--- a/core/api/src/main/java/org/onosproject/store/service/DistributedPrimitive.java
+++ b/core/api/src/main/java/org/onosproject/store/service/DistributedPrimitive.java
@@ -99,7 +99,12 @@
         /**
          * Transaction Context.
          */
-        TRANSACTION_CONTEXT
+        TRANSACTION_CONTEXT,
+
+        /**
+         * Distributed lock.
+         */
+        LOCK,
     }
 
     /**
diff --git a/core/api/src/main/java/org/onosproject/store/service/StorageService.java b/core/api/src/main/java/org/onosproject/store/service/StorageService.java
index 94843fc..2dd42fa 100644
--- a/core/api/src/main/java/org/onosproject/store/service/StorageService.java
+++ b/core/api/src/main/java/org/onosproject/store/service/StorageService.java
@@ -24,4 +24,201 @@
  * this service for their distributed state management and coordination.
  */
 public interface StorageService extends PrimitiveService {
+
+    /**
+     * Creates a new EventuallyConsistentMapBuilder.
+     *
+     * @param <K> key type
+     * @param <V> value type
+     * @return builder for an eventually consistent map
+     */
+    <K, V> EventuallyConsistentMapBuilder<K, V> eventuallyConsistentMapBuilder();
+
+    /**
+     * Creates a new ConsistentMapBuilder.
+     *
+     * @param <K> key type
+     * @param <V> value type
+     * @return builder for a consistent map
+     */
+    <K, V> ConsistentMapBuilder<K, V> consistentMapBuilder();
+
+    /**
+     * Creates a new ConsistentMapBuilder.
+     *
+     * @param <V> value type
+     * @return builder for a consistent map
+     */
+    <V> DocumentTreeBuilder<V> documentTreeBuilder();
+
+    /**
+     * Creates a new {@code AsyncConsistentTreeMapBuilder}.
+     *
+     * @param <V> value type
+     * @return builder for a async consistent tree map
+     */
+    <V> ConsistentTreeMapBuilder<V> consistentTreeMapBuilder();
+
+    /**
+     * Creates a new {@code AsyncConsistentSetMultimapBuilder}.
+     *
+     * @param <K> key type
+     * @param <V> value type
+     * @return builder for a set based async consistent multimap
+     */
+    <K, V> ConsistentMultimapBuilder<K, V> consistentMultimapBuilder();
+
+    /**
+     * Creates a new {@code AtomicCounterMapBuilder}.
+     *
+     * @param <K> key type
+     * @return builder for an atomic counter map
+     */
+    <K> AtomicCounterMapBuilder<K> atomicCounterMapBuilder();
+
+    /**
+     * Creates a new DistributedSetBuilder.
+     *
+     * @param <E> set element type
+     * @return builder for an distributed set
+     */
+    <E> DistributedSetBuilder<E> setBuilder();
+
+    /**
+     * Creates a new AtomicCounterBuilder.
+     *
+     * @return atomic counter builder
+     */
+    AtomicCounterBuilder atomicCounterBuilder();
+
+    /**
+     * Creates a new AtomicIdGeneratorBuilder.
+     *
+     * @return atomic ID generator builder
+     */
+    AtomicIdGeneratorBuilder atomicIdGeneratorBuilder();
+
+    /**
+     * Creates a new AtomicValueBuilder.
+     *
+     * @param <V> atomic value type
+     * @return atomic value builder
+     */
+    <V> AtomicValueBuilder<V> atomicValueBuilder();
+
+    /**
+     * Creates a new DistributedLockBuilder.
+     *
+     * @return lock builder
+     */
+    DistributedLockBuilder lockBuilder();
+
+    /**
+     * Creates a new LeaderElectorBuilder.
+     *
+     * @return leader elector builder
+     */
+    LeaderElectorBuilder leaderElectorBuilder();
+
+    /**
+     * Creates a new transaction context builder.
+     *
+     * @return a builder for a transaction context.
+     */
+    TransactionContextBuilder transactionContextBuilder();
+
+    /**
+     * Returns an instance of {@code AsyncAtomicCounter} with specified name.
+     * @param name counter name
+     *
+     * @return AsyncAtomicCounter instance
+     */
+    default AsyncAtomicCounter getAsyncAtomicCounter(String name) {
+        return atomicCounterBuilder().withName(name).build();
+    }
+
+    /**
+     * Returns an instance of {@code AsyncAtomicIdGenerator} with specified name.
+     *
+     * @param name ID generator name
+     * @return AsyncAtomicIdGenerator instance
+     */
+    default AsyncAtomicIdGenerator getAsyncAtomicIdGenerator(String name) {
+        return atomicIdGeneratorBuilder().withName(name).build();
+    }
+
+    /**
+     * Returns an instance of {@code AtomicCounter} with specified name.
+     * @param name counter name
+     *
+     * @return AtomicCounter instance
+     */
+    default AtomicCounter getAtomicCounter(String name) {
+        return getAsyncAtomicCounter(name).asAtomicCounter();
+    }
+
+    /**
+     * Returns an instance of {@code AtomicIdGenerator} with specified name.
+     *
+     * @param name ID generator name
+     * @return AtomicIdGenerator instance
+     */
+    default AtomicIdGenerator getAtomicIdGenerator(String name) {
+        return getAsyncAtomicIdGenerator(name).asAtomicIdGenerator();
+    }
+
+    /**
+     * Returns an instance of {@code WorkQueue} with specified name.
+     *
+     * @param <E> work element type
+     * @param name work queue name
+     * @param serializer serializer
+     * @return WorkQueue instance
+     */
+    <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer);
+
+    /**
+     * Returns an instance of {@code AsyncDocumentTree} with specified name.
+     *
+     * @param <V> tree node value type
+     * @param name document tree name
+     * @param serializer serializer
+     * @return AsyncDocumentTree instance
+     */
+    <V> AsyncDocumentTree<V> getDocumentTree(String name, Serializer serializer);
+
+     /** Returns a set backed instance of {@code AsyncConsistentMultimap} with
+     * the specified name.
+     *
+     * @param name the multimap name
+     * @param serializer serializer
+     * @param <K> key type
+     * @param <V> value type
+     * @return set backed {@code AsyncConsistentMultimap} instance
+     */
+    <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(String name,
+                                                             Serializer serializer);
+
+    /**
+     * Returns an instance of {@code AsyncConsistentTreeMap} with the specified
+     * name.
+     *
+     * @param name the treemap name
+     * @param serializer serializer
+     * @param <V> value type
+     * @return set backed {@code AsyncConsistentTreeMap} instance
+     */
+    <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(String name,
+                                                  Serializer serializer);
+
+    /**
+     * Returns an instance of {@code Topic} with specified name.
+     *
+     * @param <T> topic message type
+     * @param name topic name
+     * @param serializer serializer
+     *
+     * @return Topic instance
+     */
+    <T> Topic<T> getTopic(String name, Serializer serializer);
 }
diff --git a/core/api/src/test/java/org/onosproject/store/service/StorageServiceAdapter.java b/core/api/src/test/java/org/onosproject/store/service/StorageServiceAdapter.java
index 14850ce..4d528e2 100644
--- a/core/api/src/test/java/org/onosproject/store/service/StorageServiceAdapter.java
+++ b/core/api/src/test/java/org/onosproject/store/service/StorageServiceAdapter.java
@@ -60,6 +60,11 @@
     }
 
     @Override
+    public DistributedLockBuilder lockBuilder() {
+        return null;
+    }
+
+    @Override
     public LeaderElectorBuilder leaderElectorBuilder() {
         return null;
     }