VOL-1451 Initial checkin of openonu build

Produced docker container capable of building and running
openonu/brcm_openonci_onu.  Copied over current onu code
and resolved all imports by copying into the local source tree.

Change-Id: Ib9785d37afc65b7d32ecf74aee2456352626e2b6
diff --git a/python/common/kvstore/__init__.py b/python/common/kvstore/__init__.py
new file mode 100644
index 0000000..4a82628
--- /dev/null
+++ b/python/common/kvstore/__init__.py
@@ -0,0 +1,13 @@
+# 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.
diff --git a/python/common/kvstore/consul_client.py b/python/common/kvstore/consul_client.py
new file mode 100644
index 0000000..bc14759
--- /dev/null
+++ b/python/common/kvstore/consul_client.py
@@ -0,0 +1,304 @@
+# 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.
+
+from common.kvstore.kv_client import DEFAULT_TIMEOUT, Event, KVClient, KVPair, RETRY_BACKOFF
+from common.utils.asleep import asleep
+from common.utils.deferred_utils import DeferredWithTimeout, TimeOutError
+from consul import ConsulException
+from consul.twisted import Consul
+from structlog import get_logger
+from twisted.internet.defer import inlineCallbacks, returnValue, Deferred
+
+log = get_logger()
+
+class ConsulClient(KVClient):
+
+    def __init__(self, kv_host, kv_port):
+        KVClient.__init__(self, kv_host, kv_port)
+        self.session_id = None
+        self.client = Consul(kv_host, kv_port)
+
+    def watch(self, key, key_change_callback, timeout=DEFAULT_TIMEOUT):
+        self._retriggering_watch(key, key_change_callback, timeout)
+
+    @inlineCallbacks
+    def _retriggering_watch(self, key, key_change_callback, timeout):
+        self.key_watches[key] = ConsulWatch(self.client, key, key_change_callback, timeout)
+        yield self.key_watches[key].start()
+
+    def close_watch(self, key, timeout=DEFAULT_TIMEOUT):
+        if key in self.key_watches:
+            self.key_watches[key].stop()
+
+    @inlineCallbacks
+    def _op_with_retry(self, operation, key, value, timeout, *args, **kw):
+        log.debug('kv-op', operation=operation, key=key, timeout=timeout, args=args, kw=kw)
+        err = None
+        result = None
+        while True:
+            try:
+                if operation == 'GET':
+                    result = yield self._get(key, **kw)
+                elif operation == 'LIST':
+                    result, err = yield self._list(key)
+                elif operation == 'PUT':
+                    # Put returns a boolean response
+                    result = yield self.client.kv.put(key, value)
+                    if not result:
+                        err = 'put-failed'
+                elif operation == 'DELETE':
+                    # Delete returns a boolean response
+                    result = yield self.client.kv.delete(key)
+                    if not result:
+                        err = 'delete-failed'
+                elif operation == 'RESERVE':
+                    result, err = yield self._reserve(key, value, **kw)
+                elif operation == 'RENEW':
+                    result, err = yield self._renew_reservation(key)
+                elif operation == 'RELEASE':
+                    result, err = yield self._release_reservation(key)
+                elif operation == 'RELEASE-ALL':
+                    err = yield self._release_all_reservations()
+                self._clear_backoff()
+                break
+            except ConsulException as ex:
+                if 'ConnectionRefusedError' in ex.message:
+                    log.exception('comms-exception', ex=ex)
+                    yield self._backoff('consul-not-up')
+                else:
+                    log.error('consul-specific-exception', ex=ex)
+                    err = ex
+            except Exception as ex:
+                log.error('consul-exception', ex=ex)
+                err = ex
+
+            if timeout > 0 and self.retry_time > timeout:
+                err = 'operation-timed-out'
+            if err is not None:
+                self._clear_backoff()
+                break
+
+        returnValue((result,err))
+
+    @inlineCallbacks
+    def _get(self, key, **kw):
+        kvp = None
+        index, rec = yield self.client.kv.get(key, **kw)
+        if rec is not None:
+            kvp = KVPair(rec['Key'], rec['Value'], index)
+        returnValue(kvp)
+
+    @inlineCallbacks
+    def _list(self, key):
+        err = None
+        list = []
+        index, recs = yield self.client.kv.get(key, recurse=True)
+        for rec in recs:
+            list.append(KVPair(rec['Key'], rec['Value'], rec['ModifyIndex']))
+        returnValue((list, err))
+
+    @inlineCallbacks
+    def _reserve(self, key, value, **kw):
+        for name, val in kw.items():
+            if name == 'ttl':
+                ttl = val
+                break
+        reserved = False
+        err = 'reservation-failed'
+        owner = None
+
+        # Create a session
+        self.session_id = yield self.client.session.create(behavior='delete',
+                                                           ttl=ttl) # lock_delay=1)
+        log.debug('create-session', id=self.session_id)
+        # Try to acquire the key
+        result = yield self.client.kv.put(key, value, acquire=self.session_id)
+        log.debug('key-acquire', key=key, value=value, sess=self.session_id, result=result)
+
+        # Check if reservation succeeded
+        index, record = yield self.client.kv.get(key)
+        if record is not None and 'Value' in record:
+            owner = record['Value']
+            log.debug('get-key', session=record['Session'], owner=owner)
+            if record['Session'] == self.session_id and owner == value:
+                reserved = True
+                log.debug('key-reserved', key=key, value=value, ttl=ttl)
+                # Add key to reservation list
+                self.key_reservations[key] = self.session_id
+            else:
+                log.debug('reservation-held-by-another', owner=owner)
+
+        if reserved:
+            err = None
+        returnValue((owner, err))
+
+    @inlineCallbacks
+    def _renew_reservation(self, key):
+        result = None
+        err = None
+        if key not in self.key_reservations:
+            err = 'key-not-reserved'
+        else:
+            session_id = self.key_reservations[key]
+            # A successfully renewed session returns an object with fields:
+            # Node, CreateIndex, Name, ModifyIndex, ID, Behavior, TTL,
+            # LockDelay, and Checks
+            result = yield self.client.session.renew(session_id=session_id)
+            log.debug('session-renew', result=result)
+        if result is None:
+            err = 'session-renewal-failed'
+        returnValue((result, err))
+
+    @inlineCallbacks
+    def _release_reservation(self, key):
+        err = None
+        if key not in self.key_reservations:
+            err = 'key-not-reserved'
+        else:
+            session_id = self.key_reservations[key]
+            # A successfully destroyed session returns a boolean result
+            success = yield self.client.session.destroy(session_id)
+            log.debug('session-destroy', result=success)
+            if not success:
+                err = 'session-destroy-failed'
+            self.session_id = None
+            self.key_reservations.pop(key)
+        returnValue((success, err))
+
+    @inlineCallbacks
+    def _release_all_reservations(self):
+        err = None
+        keys_to_delete = []
+        for key in self.key_reservations:
+            session_id = self.key_reservations[key]
+            # A successfully destroyed session returns a boolean result
+            success = yield self.client.session.destroy(session_id)
+            if not success:
+                err = 'session-destroy-failed'
+                log.debug('session-destroy', id=session_id, result=success)
+            self.session_id = None
+            keys_to_delete.append(key)
+        for key in keys_to_delete:
+            self.key_reservations.pop(key)
+        returnValue(err)
+
+
+class ConsulWatch():
+
+    def __init__(self, consul, key, callback, timeout):
+        self.client = consul
+        self.key = key
+        self.index = None
+        self.callback = callback
+        self.timeout = timeout
+        self.period = 60
+        self.running = True
+        self.retries = 0
+        self.retry_time = 0
+
+    @inlineCallbacks
+    def start(self):
+        self.running = True
+        index, rec = yield self._get_with_retry(self.key, None,
+                                              timeout=self.timeout)
+        self.index = str(index)
+
+        @inlineCallbacks
+        def _get(key, deferred):
+            try:
+                index, rec = yield self._get_with_retry(key, None,
+                                                     timeout=self.timeout,
+                                                     index=self.index)
+                self.index = str(index)
+                if not deferred.called:
+                    log.debug('got-result-cancelling-deferred')
+                    deferred.callback((self.index, rec))
+            except Exception as e:
+                log.exception('got-exception', e=e)
+
+        while self.running:
+            try:
+                rcvd = DeferredWithTimeout(timeout=self.period)
+                _get(self.key, rcvd)
+                try:
+                    # Update index for next watch iteration
+                    index, rec = yield rcvd
+                    log.debug('event-received', index=index, rec=rec)
+                    # Notify client of key change event
+                    if rec is None:
+                        # Key has been deleted
+                        self._send_event(Event(Event.DELETE, self.key, None))
+                    else:
+                        self._send_event(Event(Event.PUT, rec['Key'], rec['Value']))
+                except TimeOutError as e:
+                    log.debug('no-events-over-watch-period', key=self.key)
+                except Exception as e:
+                    log.exception('exception', e=e)
+            except Exception as e:
+                log.exception('exception', e=e)
+
+        log.debug('close-watch', key=self.key)
+
+    def stop(self):
+        self.running = False
+        self.callback = None
+
+    @inlineCallbacks
+    def _get_with_retry(self, key, value, timeout, *args, **kw):
+        log.debug('watch-period', key=key, period=self.period, timeout=timeout, args=args, kw=kw)
+        err = None
+        result = None
+        while True:
+            try:
+                result = yield self.client.kv.get(key, **kw)
+                self._clear_backoff()
+                break
+            except ConsulException as ex:
+                err = ex
+                if 'ConnectionRefusedError' in ex.message:
+                    self._send_event(Event(Event.CONNECTION_DOWN, self.key, None))
+                    log.exception('comms-exception', ex=ex)
+                    yield self._backoff('consul-not-up')
+                else:
+                    log.error('consul-specific-exception', ex=ex)
+            except Exception as ex:
+                err = ex
+                log.error('consul-exception', ex=ex)
+
+            if timeout > 0 and self.retry_time > timeout:
+                err = 'operation-timed-out'
+            if err is not None:
+                self._clear_backoff()
+                break
+
+        returnValue(result)
+
+    def _send_event(self, event):
+        if self.callback is not None:
+            self.callback(event)
+
+    def _backoff(self, msg):
+        wait_time = RETRY_BACKOFF[min(self.retries, len(RETRY_BACKOFF) - 1)]
+        self.retry_time += wait_time
+        self.retries += 1
+        log.error(msg, next_retry_in_secs=wait_time,
+                  total_delay_in_secs = self.retry_time,
+                  retries=self.retries)
+        return asleep(wait_time)
+
+    def _clear_backoff(self):
+        if self.retries:
+            log.debug('reconnected-to-kv', after_retries=self.retries)
+            self.retries = 0
+            self.retry_time = 0
diff --git a/python/common/kvstore/etcd_client.py b/python/common/kvstore/etcd_client.py
new file mode 100644
index 0000000..a958b71
--- /dev/null
+++ b/python/common/kvstore/etcd_client.py
@@ -0,0 +1,240 @@
+# 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.
+
+################################################################################
+#
+# Most of the txaioetcd methods provide a timeout parameter. This parameter
+# is likely intended to limit the amount of time spent by any one method
+# waiting for a response from the etcd server. However, if the server is
+# down, the method immediately throws a ConnectionRefusedError exception;
+# it does not perform any retries. The timeout parameter provided by the
+# methods in EtcdClient cover this contingency.
+#
+################################################################################
+
+from common.kvstore.kv_client import DEFAULT_TIMEOUT, Event, KVClient, KVPair
+from structlog import get_logger
+from twisted.internet import reactor
+from twisted.internet.defer import inlineCallbacks, returnValue, Deferred
+from twisted.internet.error import ConnectionRefusedError
+from txaioetcd import Client, CompVersion, Failed, KeySet, OpGet, OpSet, Transaction
+
+log = get_logger()
+
+class EtcdClient(KVClient):
+
+    def __init__(self, kv_host, kv_port):
+        KVClient.__init__(self, kv_host, kv_port)
+        self.url = u'http://' + kv_host + u':' + str(kv_port)
+        self.client = Client(reactor, self.url)
+
+    @inlineCallbacks
+    def watch(self, key, key_change_callback, timeout=DEFAULT_TIMEOUT):
+        self.key_watches[key] = key_change_callback
+        result = yield self._op_with_retry('WATCH', key, None, timeout, callback=self.key_changed)
+        returnValue(result)
+
+    def key_changed(self, kv):
+        key = kv.key
+        value = kv.value
+        log.debug('key-changed', key=key, value=value)
+        # Notify client of key change event
+        if value is not None:
+            evt = Event(Event.PUT, key, value)
+        else:
+            evt = Event(Event.DELETE, key, None)
+        if key in self.key_watches:
+            self.key_watches[key](evt)
+
+    def close_watch(self, key, timeout=DEFAULT_TIMEOUT):
+        log.debug('close-watch', key=key)
+        if key in self.key_watches:
+            self.key_watches.pop(key)
+
+    @inlineCallbacks
+    def _op_with_retry(self, operation, key, value, timeout, *args, **kw):
+        log.debug('kv-op', operation=operation, key=key, timeout=timeout, args=args, kw=kw)
+        err = None
+        result = None
+        if type(key) == str:
+            key = bytes(key)
+        if value is not None:
+           value = bytes(value)
+        while True:
+            try:
+                if operation == 'GET':
+                    result = yield self._get(key)
+                elif operation == 'LIST':
+                    result, err = yield self._list(key)
+                elif operation == 'PUT':
+                    # Put returns an object of type Revision
+                    result = yield self.client.set(key, value, **kw)
+                elif operation == 'DELETE':
+                    # Delete returns an object of type Deleted
+                    result = yield self.client.delete(key)
+                elif operation == 'RESERVE':
+                    result, err = yield self._reserve(key, value, **kw)
+                elif operation == 'RENEW':
+                    result, err = yield self._renew_reservation(key)
+                elif operation == 'RELEASE':
+                    result, err = yield self._release_reservation(key)
+                elif operation == 'RELEASE-ALL':
+                    err = yield self._release_all_reservations()
+                elif operation == 'WATCH':
+                    for name, val in kw.items():
+                        if name == 'callback':
+                            callback = val
+                            break
+                    result = self.client.watch([KeySet(key, prefix=True)], callback)
+                self._clear_backoff()
+                break
+            except ConnectionRefusedError as ex:
+                log.error('comms-exception', ex=ex)
+                yield self._backoff('etcd-not-up')
+            except Exception as ex:
+                log.error('etcd-exception', ex=ex)
+                err = ex
+
+            if timeout > 0 and self.retry_time > timeout:
+                err = 'operation-timed-out'
+            if err is not None:
+                self._clear_backoff()
+                break
+
+        returnValue((result, err))
+
+    @inlineCallbacks
+    def _get(self, key):
+        kvp = None
+        resp = yield self.client.get(key)
+        if resp.kvs is not None and len(resp.kvs) == 1:
+            kv = resp.kvs[0]
+            kvp = KVPair(kv.key, kv.value, kv.mod_revision)
+        returnValue(kvp)
+
+    @inlineCallbacks
+    def _list(self, key):
+        err = None
+        list = []
+        resp = yield self.client.get(KeySet(key, prefix=True))
+        if resp.kvs is not None and len(resp.kvs) > 0:
+            for kv in resp.kvs:
+                list.append(KVPair(kv.key, kv.value, kv.mod_revision))
+        returnValue((list, err))
+
+    @inlineCallbacks
+    def _reserve(self, key, value, **kw):
+        for name, val in kw.items():
+            if name == 'ttl':
+                ttl = val
+                break
+        reserved = False
+        err = 'reservation-failed'
+        owner = None
+
+        # Create a lease
+        lease = yield self.client.lease(ttl)
+
+        # Create a transaction
+        txn = Transaction(
+            compare=[ CompVersion(key, '==', 0) ],
+            success=[ OpSet(key, bytes(value), lease=lease) ],
+            failure=[ OpGet(key) ]
+        )
+        newly_acquired = False
+        try:
+            result = yield self.client.submit(txn)
+        except Failed as failed:
+            log.debug('key-already-present', key=key)
+            if len(failed.responses) > 0:
+                response = failed.responses[0]
+                if response.kvs is not None and len(response.kvs) > 0:
+                    kv = response.kvs[0]
+                    log.debug('key-already-present', value=kv.value)
+                    if kv.value == value:
+                        reserved = True
+                        log.debug('key-already-reserved', key = kv.key, value=kv.value)
+        else:
+            newly_acquired = True
+            log.debug('key-was-absent', key=key, result=result)
+
+        # Check if reservation succeeded
+        resp = yield self.client.get(key)
+        if resp.kvs is not None and len(resp.kvs) == 1:
+            owner = resp.kvs[0].value
+            if owner == value:
+                if newly_acquired:
+                    log.debug('key-reserved', key=key, value=value, ttl=ttl,
+                             lease_id=lease.lease_id)
+                    reserved = True
+                    # Add key to reservation list
+                    self.key_reservations[key] = lease
+                else:
+                    log.debug("reservation-still-held")
+            else:
+                log.debug('reservation-held-by-another', value=owner)
+
+        if reserved:
+            err = None
+        returnValue((owner, err))
+
+    @inlineCallbacks
+    def _renew_reservation(self, key):
+        result = None
+        err = None
+        if key not in self.key_reservations:
+            err = 'key-not-reserved'
+        else:
+            lease = self.key_reservations[key]
+            # A successfully refreshed lease returns an object of type Header
+            result = yield lease.refresh()
+        if result is None:
+            err = 'lease-refresh-failed'
+        returnValue((result, err))
+
+    @inlineCallbacks
+    def _release_reservation(self, key):
+        err = None
+        if key not in self.key_reservations:
+            err = 'key-not-reserved'
+        else:
+            lease = self.key_reservations[key]
+            time_left = yield lease.remaining()
+            # A successfully revoked lease returns an object of type Header
+            log.debug('release-reservation', key=key, lease_id=lease.lease_id,
+                      time_left_in_secs=time_left)
+            result = yield lease.revoke()
+            if result is None:
+                err = 'lease-revoke-failed'
+            self.key_reservations.pop(key)
+        returnValue((result, err))
+
+    @inlineCallbacks
+    def _release_all_reservations(self):
+        err = None
+        keys_to_delete = []
+        for key in self.key_reservations:
+            lease = self.key_reservations[key]
+            time_left = yield lease.remaining()
+            # A successfully revoked lease returns an object of type Header
+            log.debug('release-reservation', key=key, lease_id=lease.lease_id,
+                      time_left_in_secs=time_left)
+            result = yield lease.revoke()
+            if result is None:
+                err = 'lease-revoke-failed'
+                log.debug('lease-revoke', result=result)
+            keys_to_delete.append(key)
+        for key in keys_to_delete:
+            self.key_reservations.pop(key)
+        returnValue(err)
diff --git a/python/common/kvstore/kv_client.py b/python/common/kvstore/kv_client.py
new file mode 100644
index 0000000..69a6480
--- /dev/null
+++ b/python/common/kvstore/kv_client.py
@@ -0,0 +1,206 @@
+# 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.
+
+from common.utils.asleep import asleep
+from structlog import get_logger
+from twisted.internet.defer import inlineCallbacks, returnValue
+
+log = get_logger()
+
+class KVPair():
+    def __init__(self, key, value, index):
+        self.key = key
+        self.value = value
+        self.index = index
+
+class Event():
+    PUT = 0
+    DELETE = 1
+    CONNECTION_DOWN = 2
+
+    def __init__(self, event_type, key, value):
+        self.event_type = event_type
+        self.key = key
+        self.value = value
+
+RETRY_BACKOFF = [0.05, 0.1, 0.2, 0.5, 1, 2, 5]
+DEFAULT_TIMEOUT = 0.0
+for i in range(len(RETRY_BACKOFF)):
+    DEFAULT_TIMEOUT += RETRY_BACKOFF[i]
+
+class KVClient():
+
+    def __init__(self, kv_host, kv_port):
+        self.host = kv_host
+        self.port = kv_port
+        self.key_reservations = {}
+        self.key_watches = {}
+        self.retries = 0
+        self.retry_time = 0
+
+    @inlineCallbacks
+    def get(self, key, timeout=DEFAULT_TIMEOUT):
+        '''
+        This method returns the value of the given key in KV store.
+
+        :param key: The key whose value is requested
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: (KVPair, error) where KVPair is None if an error occurred
+        '''
+        result = yield self._op_with_retry('GET', key, None, timeout)
+        returnValue(result)
+
+    @inlineCallbacks
+    def list(self, key, timeout=DEFAULT_TIMEOUT):
+        '''
+        The list method returns an array of key-value pairs all of which
+        share the same key prefix.
+
+        :param key: The key prefix
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: ([]KVPair, error) where []KVPair is a list of KVPair objects
+        '''
+        result = yield self._op_with_retry('LIST', key, None, timeout)
+        returnValue(result)
+
+    @inlineCallbacks
+    def put(self, key, value, timeout=DEFAULT_TIMEOUT):
+        '''
+        The put method writes a value to the given key in KV store.
+        Do NOT modify a reserved key in an etcd store; doing so seems
+        to nullify the TTL of the key. In other words, the key lasts
+        forever.
+
+        :param key: The key to be written to
+        :param value: The value of the key
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: error, which is set to None for a successful write
+        '''
+        _, err = yield self._op_with_retry('PUT', key, value, timeout)
+        returnValue(err)
+
+    @inlineCallbacks
+    def delete(self, key, timeout=DEFAULT_TIMEOUT):
+        '''
+        The delete method removes a key from the KV store.
+
+        :param key: The key to be deleted
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: error, which is set to None for a successful deletion
+        '''
+        _, err = yield self._op_with_retry('DELETE', key, None, timeout)
+        returnValue(err)
+
+    @inlineCallbacks
+    def reserve(self, key, value, ttl, timeout=DEFAULT_TIMEOUT):
+        '''
+        This method acts essentially like a semaphore. The underlying mechanism
+        differs depending on the KV store: etcd uses a test-and-set transaction;
+        consul uses an acquire lock. If using etcd, do NOT write to the key
+        subsequent to the initial reservation; the TTL functionality may become
+        impaired (i.e. the reservation never expires).
+
+        :param key: The key under reservation
+        :param value: The reservation owner
+        :param ttl: The time-to-live (TTL) for the reservation. The key is unreserved
+        by the KV store when the TTL expires.
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: (key_value, error) If the key is acquired, then the value returned will
+        be the value passed in.  If the key is already acquired, then the value assigned
+        to that key will be returned.
+        '''
+        result = yield self._op_with_retry('RESERVE', key, value, timeout, ttl=ttl)
+        returnValue(result)
+
+    @inlineCallbacks
+    def renew_reservation(self, key, timeout=DEFAULT_TIMEOUT):
+        '''
+        This method renews the reservation for a given key. A reservation expires
+        after the TTL (Time To Live) period specified when reserving the key.
+
+        :param key: The reserved key
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: error, which is set to None for a successful renewal
+        '''
+        result, err = yield self._op_with_retry('RENEW', key, None, timeout)
+        returnValue(err)
+
+    @inlineCallbacks
+    def release_reservation(self, key, timeout=DEFAULT_TIMEOUT):
+        '''
+        The release_reservation method cancels the reservation for a given key.
+
+        :param key: The reserved key
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: error, which is set to None for a successful cancellation
+        '''
+        result, err = yield self._op_with_retry('RELEASE', key, None, timeout)
+        returnValue(err)
+
+    @inlineCallbacks
+    def release_all_reservations(self, timeout=DEFAULT_TIMEOUT):
+        '''
+        This method cancels all key reservations made previously
+        using the reserve API.
+
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: error, which is set to None for a successful cancellation
+        '''
+        result, err = yield self._op_with_retry('RELEASE-ALL', None, None, timeout)
+        returnValue(err)
+
+    @inlineCallbacks
+    def watch(self, key, key_change_callback, timeout=DEFAULT_TIMEOUT):
+        '''
+        This method provides a watch capability for the given key. If the value of the key
+        changes or the key is deleted, then an event indicating the change is passed to
+        the given callback function.
+
+        :param key: The key to be watched
+        :param key_change_callback: The function invoked whenever the key changes
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: There is no return; key change events are passed to the callback function
+        '''
+        raise NotImplementedError('Method not implemented')
+
+    @inlineCallbacks
+    def close_watch(self, key, timeout=DEFAULT_TIMEOUT):
+        '''
+        This method closes the watch on the given key. Once the watch is closed, key
+        change events are no longer passed to the key change callback function.
+
+        :param key: The key under watch
+        :param timeout: The length of time in seconds the method will wait for a response
+        :return: There is no return
+        '''
+        raise NotImplementedError('Method not implemented')
+
+    @inlineCallbacks
+    def _op_with_retry(self, operation, key, value, timeout, *args, **kw):
+        raise NotImplementedError('Method not implemented')
+
+    def _backoff(self, msg):
+        wait_time = RETRY_BACKOFF[min(self.retries, len(RETRY_BACKOFF) - 1)]
+        self.retry_time += wait_time
+        self.retries += 1
+        log.error(msg, next_retry_in_secs=wait_time,
+                  total_delay_in_secs = self.retry_time,
+                  retries=self.retries)
+        return asleep(wait_time)
+
+    def _clear_backoff(self):
+        if self.retries:
+            log.debug('reset-backoff', after_retries=self.retries)
+            self.retries = 0
+            self.retry_time = 0
\ No newline at end of file
diff --git a/python/common/kvstore/kvstore.py b/python/common/kvstore/kvstore.py
new file mode 100644
index 0000000..662b34d
--- /dev/null
+++ b/python/common/kvstore/kvstore.py
@@ -0,0 +1,31 @@
+# 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.
+
+from common.kvstore.consul_client import ConsulClient
+from common.kvstore.etcd_client import EtcdClient
+
+def create_kv_client(kv_store, host, port):
+    '''
+    Factory for creating a client interface to a KV store
+
+    :param kv_store: Specify either 'etcd' or 'consul'
+    :param host: Name or IP address of host serving the KV store
+    :param port: Port number (integer) of the KV service
+    :return: Reference to newly created client interface
+    '''
+    if kv_store == 'etcd':
+        return EtcdClient(host, port)
+    elif kv_store == 'consul':
+        return ConsulClient(host, port)
+    return None